You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2017/09/25 01:58:43 UTC

[01/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Repository: hbase
Updated Branches:
  refs/heads/master 154048313 -> a5f84430a


http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
index 0487bf4..acf555b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.fail;
 
@@ -46,7 +47,6 @@ 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.mob.MobUtils;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 
 /**
  * Utility class for HFile-related testing.
@@ -144,7 +144,7 @@ public class HFileTestUtil {
         writer.append(kv);
       }
     } finally {
-      writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis()));
+      writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis()));
       writer.close();
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
index d4d22b1..14c22bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+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;
@@ -58,10 +59,9 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -254,7 +254,7 @@ public class TestCoprocessorScanPolicy {
           family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions,
           newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(),
           family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), family.isNewVersionBehavior());
-      return new StoreScanner(store, scanInfo,
+      return new StoreScanner((HStore) store, scanInfo,
           newVersions == null ? OptionalInt.empty() : OptionalInt.of(newVersions.intValue()),
           scanners, ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(),
           HConstants.OLDEST_TIMESTAMP);
@@ -274,7 +274,7 @@ public class TestCoprocessorScanPolicy {
           newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(),
           family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(),
           family.isNewVersionBehavior());
-      return new StoreScanner(store, scanInfo,
+      return new StoreScanner((HStore) store, scanInfo,
           newVersions == null ? OptionalInt.empty() : OptionalInt.of(newVersions.intValue()),
           scanners, scanType, store.getSmallestReadPoint(), earliestPutTs);
     }
@@ -292,8 +292,9 @@ public class TestCoprocessorScanPolicy {
         ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(),
             family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions,
             newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(),
-            family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), family.isNewVersionBehavior());
-        return new StoreScanner(store, scanInfo, scan, targetCols, readPt);
+            family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(),
+            family.isNewVersionBehavior());
+        return new StoreScanner((HStore) store, scanInfo, scan, targetCols, readPt);
       } else {
         return s;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
index b9efe5e..a2b9c1a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java
@@ -42,9 +42,10 @@ 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.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -140,10 +141,10 @@ public class TestHBaseFsckEncryption {
 
   private List<Path> findStorefilePaths(TableName tableName) throws Exception {
     List<Path> paths = new ArrayList<>();
-    for (Region region:
-        TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) {
-      for (Store store: region.getStores()) {
-        for (StoreFile storefile: store.getStorefiles()) {
+    for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName)
+        .getOnlineRegions(htd.getTableName())) {
+      for (HStore store : ((HRegion) region).getStores()) {
+        for (HStoreFile storefile : store.getStorefiles()) {
           paths.add(storefile.getPath());
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
index 07f34b5..d7e3f4f 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.io.compress.Compression
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding
 import org.apache.hadoop.hbase.io.hfile.{HFile, CacheConfig, HFileContextBuilder, HFileWriterImpl}
-import org.apache.hadoop.hbase.regionserver.{HStore, StoreFile, StoreFileWriter, BloomType}
+import org.apache.hadoop.hbase.regionserver.{HStore, HStoreFile, StoreFileWriter, BloomType}
 import org.apache.hadoop.hbase.util.Bytes
 import org.apache.hadoop.mapred.JobConf
 import org.apache.spark.broadcast.Broadcast
@@ -1075,13 +1075,13 @@ class HBaseContext(@transient sc: SparkContext,
                                previousRow: Array[Byte],
                                compactionExclude: Boolean): Unit = {
     if (w != null) {
-      w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY,
+      w.appendFileInfo(HStoreFile.BULKLOAD_TIME_KEY,
         Bytes.toBytes(System.currentTimeMillis()))
-      w.appendFileInfo(StoreFile.BULKLOAD_TASK_KEY,
+      w.appendFileInfo(HStoreFile.BULKLOAD_TASK_KEY,
         Bytes.toBytes(regionSplitPartitioner.getPartition(previousRow)))
-      w.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY,
+      w.appendFileInfo(HStoreFile.MAJOR_COMPACTION_KEY,
         Bytes.toBytes(true))
-      w.appendFileInfo(StoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY,
+      w.appendFileInfo(HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY,
         Bytes.toBytes(compactionExclude))
       w.appendTrackedTimestampsToMetadata()
       w.close()


[07/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
index 8966bd4..737e1a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java
@@ -39,11 +39,11 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ConcatenatedLists;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
@@ -100,19 +100,19 @@ public class StripeStoreFileManager
      * same index, except the last one. Inside each list, the files are in reverse order by
      * seqNum. Note that the length of this is one higher than that of stripeEndKeys.
      */
-    public ArrayList<ImmutableList<StoreFile>> stripeFiles = new ArrayList<>();
+    public ArrayList<ImmutableList<HStoreFile>> stripeFiles = new ArrayList<>();
     /** Level 0. The files are in reverse order by seqNum. */
-    public ImmutableList<StoreFile> level0Files = ImmutableList.<StoreFile>of();
+    public ImmutableList<HStoreFile> level0Files = ImmutableList.of();
 
     /** Cached list of all files in the structure, to return from some calls */
-    public ImmutableList<StoreFile> allFilesCached = ImmutableList.<StoreFile>of();
-    private ImmutableList<StoreFile> allCompactedFilesCached = ImmutableList.<StoreFile>of();
+    public ImmutableList<HStoreFile> allFilesCached = ImmutableList.of();
+    private ImmutableList<HStoreFile> allCompactedFilesCached = ImmutableList.of();
   }
   private State state = null;
 
   /** Cached file metadata (or overrides as the case may be) */
-  private HashMap<StoreFile, byte[]> fileStarts = new HashMap<>();
-  private HashMap<StoreFile, byte[]> fileEnds = new HashMap<>();
+  private HashMap<HStoreFile, byte[]> fileStarts = new HashMap<>();
+  private HashMap<HStoreFile, byte[]> fileEnds = new HashMap<>();
   /** Normally invalid key is null, but in the map null is the result for "no key"; so use
    * the following constant value in these maps instead. Note that this is a constant and
    * we use it to compare by reference when we read from the map. */
@@ -132,17 +132,17 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public void loadFiles(List<StoreFile> storeFiles) {
+  public void loadFiles(List<HStoreFile> storeFiles) {
     loadUnclassifiedStoreFiles(storeFiles);
   }
 
   @Override
-  public Collection<StoreFile> getStorefiles() {
+  public Collection<HStoreFile> getStorefiles() {
     return state.allFilesCached;
   }
 
   @Override
-  public Collection<StoreFile> getCompactedfiles() {
+  public Collection<HStoreFile> getCompactedfiles() {
     return state.allCompactedFilesCached;
   }
 
@@ -152,7 +152,7 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public void insertNewFiles(Collection<StoreFile> sfs) throws IOException {
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
     CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(true);
     // Passing null does not cause NPE??
     cmc.mergeResults(null, sfs);
@@ -160,8 +160,8 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public ImmutableCollection<StoreFile> clearFiles() {
-    ImmutableCollection<StoreFile> result = state.allFilesCached;
+  public ImmutableCollection<HStoreFile> clearFiles() {
+    ImmutableCollection<HStoreFile> result = state.allFilesCached;
     this.state = new State();
     this.fileStarts.clear();
     this.fileEnds.clear();
@@ -169,8 +169,8 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public ImmutableCollection<StoreFile> clearCompactedFiles() {
-    ImmutableCollection<StoreFile> result = state.allCompactedFilesCached;
+  public ImmutableCollection<HStoreFile> clearCompactedFiles() {
+    ImmutableCollection<HStoreFile> result = state.allCompactedFilesCached;
     this.state = new State();
     return result;
   }
@@ -183,7 +183,7 @@ public class StripeStoreFileManager
   /** See {@link StoreFileManager#getCandidateFilesForRowKeyBefore(KeyValue)}
    * for details on this methods. */
   @Override
-  public Iterator<StoreFile> getCandidateFilesForRowKeyBefore(final KeyValue targetKey) {
+  public Iterator<HStoreFile> getCandidateFilesForRowKeyBefore(final KeyValue targetKey) {
     KeyBeforeConcatenatedLists result = new KeyBeforeConcatenatedLists();
     // Order matters for this call.
     result.addSublist(state.level0Files);
@@ -200,14 +200,14 @@ public class StripeStoreFileManager
    * {@link StoreFileManager#updateCandidateFilesForRowKeyBefore(Iterator, KeyValue, Cell)}
    * for details on this methods. */
   @Override
-  public Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
-      Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
+  public Iterator<HStoreFile> updateCandidateFilesForRowKeyBefore(
+      Iterator<HStoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
     KeyBeforeConcatenatedLists.Iterator original =
         (KeyBeforeConcatenatedLists.Iterator)candidateFiles;
     assert original != null;
-    ArrayList<List<StoreFile>> components = original.getComponents();
+    ArrayList<List<HStoreFile>> components = original.getComponents();
     for (int firstIrrelevant = 0; firstIrrelevant < components.size(); ++firstIrrelevant) {
-      StoreFile sf = components.get(firstIrrelevant).get(0);
+      HStoreFile sf = components.get(firstIrrelevant).get(0);
       byte[] endKey = endOf(sf);
       // Entries are ordered as such: L0, then stripes in reverse order. We never remove
       // level 0; we remove the stripe, and all subsequent ones, as soon as we find the
@@ -221,12 +221,6 @@ public class StripeStoreFileManager
     return original;
   }
 
-  private byte[] getSplitPoint(Collection<StoreFile> sfs) throws IOException {
-    Optional<StoreFile> largestFile = StoreUtils.getLargestFile(sfs);
-    return largestFile.isPresent()
-        ? StoreUtils.getFileSplitPoint(largestFile.get(), cellComparator).orElse(null) : null;
-  }
-
   /**
    * Override of getSplitPoint that determines the split point as the boundary between two
    * stripes, unless it causes significant imbalance between split sides' sizes. In that
@@ -235,8 +229,10 @@ public class StripeStoreFileManager
    * @return The split point, or null if no split is possible.
    */
   @Override
-  public byte[] getSplitPoint() throws IOException {
-    if (this.getStorefileCount() == 0) return null;
+  public Optional<byte[]> getSplitPoint() throws IOException {
+    if (this.getStorefileCount() == 0) {
+      return Optional.empty();
+    }
     if (state.stripeFiles.size() <= 1) {
       return getSplitPointFromAllFiles();
     }
@@ -265,7 +261,9 @@ public class StripeStoreFileManager
     if (ratio < 1) {
       ratio = 1 / ratio;
     }
-    if (config.getMaxSplitImbalance() > ratio) return state.stripeEndRows[leftIndex];
+    if (config.getMaxSplitImbalance() > ratio) {
+      return Optional.of(state.stripeEndRows[leftIndex]);
+    }
 
     // If the difference between the sides is too large, we could get the proportional key on
     // the a stripe to equalize the difference, but there's no proportional key method at the
@@ -278,18 +276,21 @@ public class StripeStoreFileManager
     if (newRatio < 1) {
       newRatio = 1 / newRatio;
     }
-    if (newRatio >= ratio)  return state.stripeEndRows[leftIndex];
+    if (newRatio >= ratio) {
+      return Optional.of(state.stripeEndRows[leftIndex]);
+    }
     LOG.debug("Splitting the stripe - ratio w/o split " + ratio + ", ratio with split "
         + newRatio + " configured ratio " + config.getMaxSplitImbalance());
-    // Ok, we may get better ratio, get it.
-    return getSplitPoint(state.stripeFiles.get(isRightLarger ? rightIndex : leftIndex));
+    // OK, we may get better ratio, get it.
+    return StoreUtils.getSplitPoint(state.stripeFiles.get(isRightLarger ? rightIndex : leftIndex),
+      cellComparator);
   }
 
-  private byte[] getSplitPointFromAllFiles() throws IOException {
-    ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
+  private Optional<byte[]> getSplitPointFromAllFiles() throws IOException {
+    ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>();
     sfs.addSublist(state.level0Files);
     sfs.addAllSublists(state.stripeFiles);
-    return getSplitPoint(sfs);
+    return StoreUtils.getSplitPoint(sfs, cellComparator);
   }
 
   private double getMidStripeSplitRatio(long smallerSize, long largerSize, long lastLargerSize) {
@@ -297,7 +298,7 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public Collection<StoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
+  public Collection<HStoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
       byte[] stopRow, boolean includeStopRow) {
     if (state.stripeFiles.isEmpty()) {
       return state.level0Files; // There's just L0.
@@ -313,7 +314,7 @@ public class StripeStoreFileManager
       return state.allFilesCached; // We need to read all files.
     }
 
-    ConcatenatedLists<StoreFile> result = new ConcatenatedLists<>();
+    ConcatenatedLists<HStoreFile> result = new ConcatenatedLists<>();
     result.addAllSublists(state.stripeFiles.subList(firstStripe, lastStripe + 1));
     result.addSublist(state.level0Files);
     return result;
@@ -321,7 +322,7 @@ public class StripeStoreFileManager
 
   @Override
   public void addCompactionResults(
-    Collection<StoreFile> compactedFiles, Collection<StoreFile> results) throws IOException {
+    Collection<HStoreFile> compactedFiles, Collection<HStoreFile> results) throws IOException {
     // See class comment for the assumptions we make here.
     LOG.debug("Attempting to merge compaction results: " + compactedFiles.size()
         + " files replaced by " + results.size());
@@ -337,14 +338,14 @@ public class StripeStoreFileManager
   // Let a background thread close the actual reader on these compacted files and also
   // ensure to evict the blocks from block cache so that they are no longer in
   // cache
-  private void markCompactedAway(Collection<StoreFile> compactedFiles) {
-    for (StoreFile file : compactedFiles) {
+  private void markCompactedAway(Collection<HStoreFile> compactedFiles) {
+    for (HStoreFile file : compactedFiles) {
       file.markCompactedAway();
     }
   }
 
   @Override
-  public void removeCompactedFiles(Collection<StoreFile> compactedFiles) throws IOException {
+  public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) throws IOException {
     // See class comment for the assumptions we make here.
     LOG.debug("Attempting to delete compaction results: " + compactedFiles.size());
     // In order to be able to fail in the middle of the operation, we'll operate on lazy
@@ -378,7 +379,7 @@ public class StripeStoreFileManager
    */
   private long getStripeFilesSize(int stripeIndex) {
     long result = 0;
-    for (StoreFile sf : state.stripeFiles.get(stripeIndex)) {
+    for (HStoreFile sf : state.stripeFiles.get(stripeIndex)) {
       result += sf.getReader().length();
     }
     return result;
@@ -391,13 +392,13 @@ public class StripeStoreFileManager
    * go to level 0.
    * @param storeFiles Store files to add.
    */
-  private void loadUnclassifiedStoreFiles(List<StoreFile> storeFiles) {
+  private void loadUnclassifiedStoreFiles(List<HStoreFile> storeFiles) {
     LOG.debug("Attempting to load " + storeFiles.size() + " store files.");
-    TreeMap<byte[], ArrayList<StoreFile>> candidateStripes = new TreeMap<>(MAP_COMPARATOR);
-    ArrayList<StoreFile> level0Files = new ArrayList<>();
+    TreeMap<byte[], ArrayList<HStoreFile>> candidateStripes = new TreeMap<>(MAP_COMPARATOR);
+    ArrayList<HStoreFile> level0Files = new ArrayList<>();
     // Separate the files into tentative stripes; then validate. Currently, we rely on metadata.
     // If needed, we could dynamically determine the stripes in future.
-    for (StoreFile sf : storeFiles) {
+    for (HStoreFile sf : storeFiles) {
       byte[] startRow = startOf(sf), endRow = endOf(sf);
       // Validate the range and put the files into place.
       if (isInvalid(startRow) || isInvalid(endRow)) {
@@ -410,7 +411,7 @@ public class StripeStoreFileManager
         insertFileIntoStripe(level0Files, sf); // Bad metadata - goes to L0 also.
         ensureLevel0Metadata(sf);
       } else {
-        ArrayList<StoreFile> stripe = candidateStripes.get(endRow);
+        ArrayList<HStoreFile> stripe = candidateStripes.get(endRow);
         if (stripe == null) {
           stripe = new ArrayList<>();
           candidateStripes.put(endRow, stripe);
@@ -423,14 +424,14 @@ public class StripeStoreFileManager
 
     boolean hasOverlaps = false;
     byte[] expectedStartRow = null; // first stripe can start wherever
-    Iterator<Map.Entry<byte[], ArrayList<StoreFile>>> entryIter =
+    Iterator<Map.Entry<byte[], ArrayList<HStoreFile>>> entryIter =
         candidateStripes.entrySet().iterator();
     while (entryIter.hasNext()) {
-      Map.Entry<byte[], ArrayList<StoreFile>> entry = entryIter.next();
-      ArrayList<StoreFile> files = entry.getValue();
+      Map.Entry<byte[], ArrayList<HStoreFile>> entry = entryIter.next();
+      ArrayList<HStoreFile> files = entry.getValue();
       // Validate the file start rows, and remove the bad ones to level 0.
       for (int i = 0; i < files.size(); ++i) {
-        StoreFile sf = files.get(i);
+        HStoreFile sf = files.get(i);
         byte[] startRow = startOf(sf);
         if (expectedStartRow == null) {
           expectedStartRow = startRow; // ensure that first stripe is still consistent
@@ -439,7 +440,7 @@ public class StripeStoreFileManager
           LOG.warn("Store file doesn't fit into the tentative stripes - expected to start at ["
               + Bytes.toString(expectedStartRow) + "], but starts at [" + Bytes.toString(startRow)
               + "], to L0 it goes");
-          StoreFile badSf = files.remove(i);
+          HStoreFile badSf = files.remove(i);
           insertFileIntoStripe(level0Files, badSf);
           ensureLevel0Metadata(badSf);
           --i;
@@ -459,7 +460,7 @@ public class StripeStoreFileManager
     // as open keys anyway, and log the message.
     // If there were errors, we'll play it safe and dump everything into L0.
     if (!candidateStripes.isEmpty()) {
-      StoreFile firstFile = candidateStripes.firstEntry().getValue().get(0);
+      HStoreFile firstFile = candidateStripes.firstEntry().getValue().get(0);
       boolean isOpen = isOpen(startOf(firstFile)) && isOpen(candidateStripes.lastKey());
       if (!isOpen) {
         LOG.warn("The range of the loaded files does not cover full key space: from ["
@@ -470,8 +471,8 @@ public class StripeStoreFileManager
           ensureEdgeStripeMetadata(candidateStripes.lastEntry().getValue(), false);
         } else {
           LOG.warn("Inconsistent files, everything goes to L0.");
-          for (ArrayList<StoreFile> files : candidateStripes.values()) {
-            for (StoreFile sf : files) {
+          for (ArrayList<HStoreFile> files : candidateStripes.values()) {
+            for (HStoreFile sf : files) {
               insertFileIntoStripe(level0Files, sf);
               ensureLevel0Metadata(sf);
             }
@@ -486,9 +487,9 @@ public class StripeStoreFileManager
     state.level0Files = ImmutableList.copyOf(level0Files);
     state.stripeFiles = new ArrayList<>(candidateStripes.size());
     state.stripeEndRows = new byte[Math.max(0, candidateStripes.size() - 1)][];
-    ArrayList<StoreFile> newAllFiles = new ArrayList<>(level0Files);
+    ArrayList<HStoreFile> newAllFiles = new ArrayList<>(level0Files);
     int i = candidateStripes.size() - 1;
-    for (Map.Entry<byte[], ArrayList<StoreFile>> entry : candidateStripes.entrySet()) {
+    for (Map.Entry<byte[], ArrayList<HStoreFile>> entry : candidateStripes.entrySet()) {
       state.stripeFiles.add(ImmutableList.copyOf(entry.getValue()));
       newAllFiles.addAll(entry.getValue());
       if (i > 0) {
@@ -501,14 +502,14 @@ public class StripeStoreFileManager
     debugDumpState("Files loaded");
   }
 
-  private void ensureEdgeStripeMetadata(ArrayList<StoreFile> stripe, boolean isFirst) {
-    HashMap<StoreFile, byte[]> targetMap = isFirst ? fileStarts : fileEnds;
-    for (StoreFile sf : stripe) {
+  private void ensureEdgeStripeMetadata(ArrayList<HStoreFile> stripe, boolean isFirst) {
+    HashMap<HStoreFile, byte[]> targetMap = isFirst ? fileStarts : fileEnds;
+    for (HStoreFile sf : stripe) {
       targetMap.put(sf, OPEN_KEY);
     }
   }
 
-  private void ensureLevel0Metadata(StoreFile sf) {
+  private void ensureLevel0Metadata(HStoreFile sf) {
     if (!isInvalid(startOf(sf))) this.fileStarts.put(sf, INVALID_KEY_IN_MAP);
     if (!isInvalid(endOf(sf))) this.fileEnds.put(sf, INVALID_KEY_IN_MAP);
   }
@@ -616,7 +617,7 @@ public class StripeStoreFileManager
   }
 
 
-  private byte[] startOf(StoreFile sf) {
+  private byte[] startOf(HStoreFile sf) {
     byte[] result = fileStarts.get(sf);
 
     // result and INVALID_KEY_IN_MAP are compared _only_ by reference on purpose here as the latter
@@ -627,7 +628,7 @@ public class StripeStoreFileManager
              : result == INVALID_KEY_IN_MAP ? INVALID_KEY : result;
   }
 
-  private byte[] endOf(StoreFile sf) {
+  private byte[] endOf(HStoreFile sf) {
     byte[] result = fileEnds.get(sf);
 
     // result and INVALID_KEY_IN_MAP are compared _only_ by reference on purpose here as the latter
@@ -643,7 +644,7 @@ public class StripeStoreFileManager
    * @param stripe Stripe copy to insert into.
    * @param sf File to insert.
    */
-  private static void insertFileIntoStripe(ArrayList<StoreFile> stripe, StoreFile sf) {
+  private static void insertFileIntoStripe(ArrayList<HStoreFile> stripe, HStoreFile sf) {
     // The only operation for which sorting of the files matters is KeyBefore. Therefore,
     // we will store the file in reverse order by seqNum from the outset.
     for (int insertBefore = 0; ; ++insertBefore) {
@@ -664,20 +665,20 @@ public class StripeStoreFileManager
    * but will quickly cut down on them as it finds something in the more likely ones; thus,
    * the above allow us to avoid unnecessary copying of a bunch of lists.
    */
-  private static class KeyBeforeConcatenatedLists extends ConcatenatedLists<StoreFile> {
+  private static class KeyBeforeConcatenatedLists extends ConcatenatedLists<HStoreFile> {
     @Override
-    public java.util.Iterator<StoreFile> iterator() {
+    public java.util.Iterator<HStoreFile> iterator() {
       return new Iterator();
     }
 
-    public class Iterator extends ConcatenatedLists<StoreFile>.Iterator {
-      public ArrayList<List<StoreFile>> getComponents() {
+    public class Iterator extends ConcatenatedLists<HStoreFile>.Iterator {
+      public ArrayList<List<HStoreFile>> getComponents() {
         return components;
       }
 
       public void removeComponents(int startIndex) {
-        List<List<StoreFile>> subList = components.subList(startIndex, components.size());
-        for (List<StoreFile> entry : subList) {
+        List<List<HStoreFile>> subList = components.subList(startIndex, components.size());
+        for (List<HStoreFile> entry : subList) {
           size -= entry.size();
         }
         assert size >= 0;
@@ -690,7 +691,7 @@ public class StripeStoreFileManager
           throw new IllegalStateException("No element to remove");
         }
         this.nextWasCalled = false;
-        List<StoreFile> src = components.get(currentComponent);
+        List<HStoreFile> src = components.get(currentComponent);
         if (src instanceof ImmutableList<?>) {
           src = new ArrayList<>(src);
           components.set(currentComponent, src);
@@ -711,14 +712,14 @@ public class StripeStoreFileManager
    * then creates a new state object and puts it in place.
    */
   private class CompactionOrFlushMergeCopy {
-    private ArrayList<List<StoreFile>> stripeFiles = null;
-    private ArrayList<StoreFile> level0Files = null;
+    private ArrayList<List<HStoreFile>> stripeFiles = null;
+    private ArrayList<HStoreFile> level0Files = null;
     private ArrayList<byte[]> stripeEndRows = null;
 
-    private Collection<StoreFile> compactedFiles = null;
-    private Collection<StoreFile> results = null;
+    private Collection<HStoreFile> compactedFiles = null;
+    private Collection<HStoreFile> results = null;
 
-    private List<StoreFile> l0Results = new ArrayList<>();
+    private List<HStoreFile> l0Results = new ArrayList<>();
     private final boolean isFlush;
 
     public CompactionOrFlushMergeCopy(boolean isFlush) {
@@ -727,14 +728,14 @@ public class StripeStoreFileManager
       this.isFlush = isFlush;
     }
 
-    private void mergeResults(Collection<StoreFile> compactedFiles, Collection<StoreFile> results)
+    private void mergeResults(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> results)
         throws IOException {
       assert this.compactedFiles == null && this.results == null;
       this.compactedFiles = compactedFiles;
       this.results = results;
       // Do logical processing.
       if (!isFlush) removeCompactedFiles();
-      TreeMap<byte[], StoreFile> newStripes = processResults();
+      TreeMap<byte[], HStoreFile> newStripes = processResults();
       if (newStripes != null) {
         processNewCandidateStripes(newStripes);
       }
@@ -744,7 +745,7 @@ public class StripeStoreFileManager
       updateMetadataMaps();
     }
 
-    private void deleteResults(Collection<StoreFile> compactedFiles) throws IOException {
+    private void deleteResults(Collection<HStoreFile> compactedFiles) throws IOException {
       this.compactedFiles = compactedFiles;
       // Create new state and update parent.
       State state = createNewState(true);
@@ -762,13 +763,13 @@ public class StripeStoreFileManager
       newState.stripeEndRows = (this.stripeEndRows == null) ? oldState.stripeEndRows
           : this.stripeEndRows.toArray(new byte[this.stripeEndRows.size()][]);
       newState.stripeFiles = new ArrayList<>(this.stripeFiles.size());
-      for (List<StoreFile> newStripe : this.stripeFiles) {
+      for (List<HStoreFile> newStripe : this.stripeFiles) {
         newState.stripeFiles.add(newStripe instanceof ImmutableList<?>
-            ? (ImmutableList<StoreFile>)newStripe : ImmutableList.copyOf(newStripe));
+            ? (ImmutableList<HStoreFile>)newStripe : ImmutableList.copyOf(newStripe));
       }
 
-      List<StoreFile> newAllFiles = new ArrayList<>(oldState.allFilesCached);
-      List<StoreFile> newAllCompactedFiles = new ArrayList<>(oldState.allCompactedFilesCached);
+      List<HStoreFile> newAllFiles = new ArrayList<>(oldState.allFilesCached);
+      List<HStoreFile> newAllCompactedFiles = new ArrayList<>(oldState.allCompactedFilesCached);
       if (!isFlush) {
         newAllFiles.removeAll(compactedFiles);
         if (delCompactedFiles) {
@@ -788,13 +789,13 @@ public class StripeStoreFileManager
     private void updateMetadataMaps() {
       StripeStoreFileManager parent = StripeStoreFileManager.this;
       if (!isFlush) {
-        for (StoreFile sf : this.compactedFiles) {
+        for (HStoreFile sf : this.compactedFiles) {
           parent.fileStarts.remove(sf);
           parent.fileEnds.remove(sf);
         }
       }
       if (this.l0Results != null) {
-        for (StoreFile sf : this.l0Results) {
+        for (HStoreFile sf : this.l0Results) {
           parent.ensureLevel0Metadata(sf);
         }
       }
@@ -804,14 +805,14 @@ public class StripeStoreFileManager
      * @param index Index of the stripe we need.
      * @return A lazy stripe copy from current stripes.
      */
-    private final ArrayList<StoreFile> getStripeCopy(int index) {
-      List<StoreFile> stripeCopy = this.stripeFiles.get(index);
-      ArrayList<StoreFile> result = null;
+    private final ArrayList<HStoreFile> getStripeCopy(int index) {
+      List<HStoreFile> stripeCopy = this.stripeFiles.get(index);
+      ArrayList<HStoreFile> result = null;
       if (stripeCopy instanceof ImmutableList<?>) {
         result = new ArrayList<>(stripeCopy);
         this.stripeFiles.set(index, result);
       } else {
-        result = (ArrayList<StoreFile>)stripeCopy;
+        result = (ArrayList<HStoreFile>)stripeCopy;
       }
       return result;
     }
@@ -819,7 +820,7 @@ public class StripeStoreFileManager
     /**
      * @return A lazy L0 copy from current state.
      */
-    private final ArrayList<StoreFile> getLevel0Copy() {
+    private final ArrayList<HStoreFile> getLevel0Copy() {
       if (this.level0Files == null) {
         this.level0Files = new ArrayList<>(StripeStoreFileManager.this.state.level0Files);
       }
@@ -831,9 +832,9 @@ public class StripeStoreFileManager
      * or to the list of new candidate stripes.
      * @return New candidate stripes.
      */
-    private TreeMap<byte[], StoreFile> processResults() throws IOException {
-      TreeMap<byte[], StoreFile> newStripes = null;
-      for (StoreFile sf : this.results) {
+    private TreeMap<byte[], HStoreFile> processResults() throws IOException {
+      TreeMap<byte[], HStoreFile> newStripes = null;
+      for (HStoreFile sf : this.results) {
         byte[] startRow = startOf(sf), endRow = endOf(sf);
         if (isInvalid(endRow) || isInvalid(startRow)) {
           if (!isFlush) {
@@ -856,7 +857,7 @@ public class StripeStoreFileManager
         if (newStripes == null) {
           newStripes = new TreeMap<>(MAP_COMPARATOR);
         }
-        StoreFile oldSf = newStripes.put(endRow, sf);
+        HStoreFile oldSf = newStripes.put(endRow, sf);
         if (oldSf != null) {
           throw new IOException("Compactor has produced multiple files for the stripe ending in ["
               + Bytes.toString(endRow) + "], found " + sf.getPath() + " and " + oldSf.getPath());
@@ -870,9 +871,9 @@ public class StripeStoreFileManager
      * @param compactedFiles Compacted files.
      */
     private void removeCompactedFiles() throws IOException {
-      for (StoreFile oldFile : this.compactedFiles) {
+      for (HStoreFile oldFile : this.compactedFiles) {
         byte[] oldEndRow = endOf(oldFile);
-        List<StoreFile> source = null;
+        List<HStoreFile> source = null;
         if (isInvalid(oldEndRow)) {
           source = getLevel0Copy();
         } else {
@@ -895,7 +896,7 @@ public class StripeStoreFileManager
      * @param newStripes  New stripes - files by end row.
      */
     private void processNewCandidateStripes(
-        TreeMap<byte[], StoreFile> newStripes) throws IOException {
+        TreeMap<byte[], HStoreFile> newStripes) throws IOException {
       // Validate that the removed and added aggregate ranges still make for a full key space.
       boolean hasStripes = !this.stripeFiles.isEmpty();
       this.stripeEndRows = new ArrayList<>(Arrays.asList(StripeStoreFileManager.this.state.stripeEndRows));
@@ -907,7 +908,7 @@ public class StripeStoreFileManager
       }
 
       boolean canAddNewStripes = true;
-      Collection<StoreFile> filesForL0 = null;
+      Collection<HStoreFile> filesForL0 = null;
       if (hasStripes) {
         // Determine which stripes will need to be removed because they conflict with new stripes.
         // The new boundaries should match old stripe boundaries, so we should get exact matches.
@@ -921,7 +922,7 @@ public class StripeStoreFileManager
         int removeTo = findStripeIndexByEndRow(lastEndRow);
         if (removeTo < 0) throw new IOException("Compaction is trying to add a bad range.");
         // See if there are files in the stripes we are trying to replace.
-        ArrayList<StoreFile> conflictingFiles = new ArrayList<>();
+        ArrayList<HStoreFile> conflictingFiles = new ArrayList<>();
         for (int removeIndex = removeTo; removeIndex >= removeFrom; --removeIndex) {
           conflictingFiles.addAll(this.stripeFiles.get(removeIndex));
         }
@@ -942,7 +943,7 @@ public class StripeStoreFileManager
             filesForL0 = conflictingFiles;
           }
           if (filesForL0 != null) {
-            for (StoreFile sf : filesForL0) {
+            for (HStoreFile sf : filesForL0) {
               insertFileIntoStripe(getLevel0Copy(), sf);
             }
             l0Results.addAll(filesForL0);
@@ -966,7 +967,7 @@ public class StripeStoreFileManager
       // Now, insert new stripes. The total ranges match, so we can insert where we removed.
       byte[] previousEndRow = null;
       int insertAt = removeFrom;
-      for (Map.Entry<byte[], StoreFile> newStripe : newStripes.entrySet()) {
+      for (Map.Entry<byte[], HStoreFile> newStripe : newStripes.entrySet()) {
         if (previousEndRow != null) {
           // Validate that the ranges are contiguous.
           assert !isOpen(previousEndRow);
@@ -977,7 +978,7 @@ public class StripeStoreFileManager
           }
         }
         // Add the new stripe.
-        ArrayList<StoreFile> tmp = new ArrayList<>();
+        ArrayList<HStoreFile> tmp = new ArrayList<>();
         tmp.add(newStripe.getValue());
         stripeFiles.add(insertAt, tmp);
         previousEndRow = newStripe.getKey();
@@ -990,7 +991,7 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public List<StoreFile> getLevel0Files() {
+  public List<HStoreFile> getLevel0Files() {
     return this.state.level0Files;
   }
 
@@ -1005,7 +1006,7 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public ArrayList<ImmutableList<StoreFile>> getStripes() {
+  public ArrayList<ImmutableList<HStoreFile>> getStripes() {
     return this.state.stripeFiles;
   }
 
@@ -1015,22 +1016,22 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public Collection<StoreFile> getUnneededFiles(long maxTs, List<StoreFile> filesCompacting) {
+  public Collection<HStoreFile> getUnneededFiles(long maxTs, List<HStoreFile> filesCompacting) {
     // 1) We can never get rid of the last file which has the maximum seqid in a stripe.
     // 2) Files that are not the latest can't become one due to (1), so the rest are fair game.
     State state = this.state;
-    Collection<StoreFile> expiredStoreFiles = null;
-    for (ImmutableList<StoreFile> stripe : state.stripeFiles) {
+    Collection<HStoreFile> expiredStoreFiles = null;
+    for (ImmutableList<HStoreFile> stripe : state.stripeFiles) {
       expiredStoreFiles = findExpiredFiles(stripe, maxTs, filesCompacting, expiredStoreFiles);
     }
     return findExpiredFiles(state.level0Files, maxTs, filesCompacting, expiredStoreFiles);
   }
 
-  private Collection<StoreFile> findExpiredFiles(ImmutableList<StoreFile> stripe, long maxTs,
-      List<StoreFile> filesCompacting, Collection<StoreFile> expiredStoreFiles) {
+  private Collection<HStoreFile> findExpiredFiles(ImmutableList<HStoreFile> stripe, long maxTs,
+      List<HStoreFile> filesCompacting, Collection<HStoreFile> expiredStoreFiles) {
     // Order by seqnum is reversed.
     for (int i = 1; i < stripe.size(); ++i) {
-      StoreFile sf = stripe.get(i);
+      HStoreFile sf = stripe.get(i);
       synchronized (sf) {
         long fileTs = sf.getReader().getMaxTimestamp();
         if (fileTs < maxTs && !filesCompacting.contains(sf)) {
@@ -1061,7 +1062,7 @@ public class StripeStoreFileManager
     // we flush data to stripe directly.
     int delta = stateLocal.level0Files.isEmpty() ? 0 : 1;
     double max = 0.0;
-    for (ImmutableList<StoreFile> stripeFile : stateLocal.stripeFiles) {
+    for (ImmutableList<HStoreFile> stripeFile : stateLocal.stripeFiles) {
       int stripeFileCount = stripeFile.size();
       double normCount =
           (double) (stripeFileCount + delta - config.getStripeCompactMinFiles())
@@ -1079,7 +1080,7 @@ public class StripeStoreFileManager
   }
 
   @Override
-  public Comparator<StoreFile> getStoreFileComparator() {
+  public Comparator<HStoreFile> getStoreFileComparator() {
     return StoreFileComparators.SEQ_ID;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
index b11b384..bc4d624 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java
@@ -26,13 +26,13 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
@@ -47,7 +47,7 @@ public class StripeStoreFlusher extends StoreFlusher {
   private final StripeCompactionPolicy policy;
   private final StripeCompactionPolicy.StripeInformationProvider stripes;
 
-  public StripeStoreFlusher(Configuration conf, Store store,
+  public StripeStoreFlusher(Configuration conf, HStore store,
       StripeCompactionPolicy policy, StripeStoreFileManager stripes) {
     super(conf, store);
     this.policy = policy;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
index 1c81d31..c5ef127 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java
@@ -24,13 +24,13 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter;
 import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter.WriterFactory;
+import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Base class for implementing a Compactor which will generate multiple output files after
@@ -42,7 +42,7 @@ public abstract class AbstractMultiOutputCompactor<T extends AbstractMultiFileWr
 
   private static final Log LOG = LogFactory.getLog(AbstractMultiOutputCompactor.class);
 
-  public AbstractMultiOutputCompactor(Configuration conf, Store store) {
+  public AbstractMultiOutputCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java
index 4d219b1..d0b0731 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionContext.java
@@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.regionserver.compactions;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
 
 
 /**
@@ -44,7 +44,7 @@ public abstract class CompactionContext {
    * @param filesCompacting files currently compacting
    * @return the list of files that can theoretically be compacted.
    */
-  public abstract List<StoreFile> preSelect(final List<StoreFile> filesCompacting);
+  public abstract List<HStoreFile> preSelect(List<HStoreFile> filesCompacting);
 
   /**
    * Called to select files for compaction. Must fill in the request field if successful.
@@ -54,9 +54,8 @@ public abstract class CompactionContext {
    * @param forceMajor Whether to force major compaction.
    * @return Whether the selection succeeded. Selection may be empty and lead to no compaction.
    */
-  public abstract boolean select(
-      final List<StoreFile> filesCompacting, final boolean isUserCompaction,
-      final boolean mayUseOffPeak, final boolean forceMajor) throws IOException;
+  public abstract boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
+      boolean mayUseOffPeak, boolean forceMajor) throws IOException;
 
   /**
    * Forces external selection to be applied for this compaction.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionPolicy.java
index db4e0f0..a2778d3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionPolicy.java
@@ -22,10 +22,10 @@ package org.apache.hadoop.hbase.regionserver.compactions;
 import java.io.IOException;
 import java.util.Collection;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * A compaction policy determines how to select files for compaction,
@@ -45,8 +45,8 @@ public abstract class CompactionPolicy {
    * @param filesToCompact Files to compact. Can be null.
    * @return True if we should run a major compaction.
    */
-  public abstract boolean shouldPerformMajorCompaction(
-    final Collection<StoreFile> filesToCompact) throws IOException;
+  public abstract boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
+      throws IOException;
 
   /**
    * @param compactionSize Total size of some compaction

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
index e05c165..da35bfc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java
@@ -21,8 +21,8 @@ package org.apache.hadoop.hbase.regionserver.compactions;
 import java.util.Collection;
 import java.util.stream.Collectors;
 
+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.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -40,7 +40,7 @@ public class CompactionRequest {
   private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR }
   private DisplayCompactionType isMajor = DisplayCompactionType.MINOR;
   private int priority = Store.NO_PRIORITY;
-  private Collection<StoreFile> filesToCompact;
+  private Collection<HStoreFile> filesToCompact;
 
   // CompactRequest object creation time.
   private long selectionTime;
@@ -51,19 +51,19 @@ public class CompactionRequest {
   private long totalSize = -1L;
   private CompactionLifeCycleTracker tracker = CompactionLifeCycleTracker.DUMMY;
 
-  public CompactionRequest(Collection<StoreFile> files) {
+  public CompactionRequest(Collection<HStoreFile> files) {
     this.selectionTime = EnvironmentEdgeManager.currentTime();
     this.timeInNanos = System.nanoTime();
     this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null");
     recalculateSize();
   }
 
-  public void updateFiles(Collection<StoreFile> files) {
+  public void updateFiles(Collection<HStoreFile> files) {
     this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null");
     recalculateSize();
   }
 
-  public Collection<StoreFile> getFiles() {
+  public Collection<HStoreFile> getFiles() {
     return this.filesToCompact;
   }
 
@@ -152,7 +152,7 @@ public class CompactionRequest {
    * @param files files that should be included in the compaction
    */
   private void recalculateSize() {
-    this.totalSize = filesToCompact.stream().map(StoreFile::getReader)
+    this.totalSize = filesToCompact.stream().map(HStoreFile::getReader)
         .mapToLong(r -> r != null ? r.length() : 0L).sum();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java
----------------------------------------------------------------------
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 e6d1935..2c9a519 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
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hbase.regionserver.compactions;
 
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -33,19 +36,17 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
 import org.apache.hadoop.hbase.regionserver.CellSink;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 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.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
@@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.io.Closeables;
 
@@ -70,7 +72,7 @@ public abstract class Compactor<T extends CellSink> {
   protected static final long COMPACTION_PROGRESS_LOG_INTERVAL = 60 * 1000;
   protected volatile CompactionProgress progress;
   protected final Configuration conf;
-  protected final Store store;
+  protected final HStore store;
 
   protected final int compactionKVMax;
   protected final Compression.Algorithm compactionCompression;
@@ -88,7 +90,7 @@ public abstract class Compactor<T extends CellSink> {
   private boolean dropCacheMinor;
 
   //TODO: depending on Store is not good but, realistically, all compactors currently do.
-  Compactor(final Configuration conf, final Store store) {
+  Compactor(Configuration conf, HStore store) {
     this.conf = conf;
     this.store = store;
     this.compactionKVMax =
@@ -137,12 +139,12 @@ public abstract class Compactor<T extends CellSink> {
    * @return The result.
    */
   protected FileDetails getFileDetails(
-      Collection<StoreFile> filesToCompact, boolean allFiles) throws IOException {
+      Collection<HStoreFile> filesToCompact, boolean allFiles) throws IOException {
     FileDetails fd = new FileDetails();
     long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() - 
       (1000L * 60 * 60 * 24 * this.keepSeqIdPeriod);  
 
-    for (StoreFile file : filesToCompact) {
+    for (HStoreFile file : filesToCompact) {
       if(allFiles && (file.getModificationTimeStamp() < oldestHFileTimeStampToKeepMVCC)) {
         // when isAllFiles is true, all files are compacted so we can calculate the smallest 
         // MVCC value to keep
@@ -184,7 +186,7 @@ public abstract class Compactor<T extends CellSink> {
       // This is used to remove family delete marker during compaction.
       long earliestPutTs = 0;
       if (allFiles) {
-        tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
+        tmp = fileInfo.get(EARLIEST_PUT_TS);
         if (tmp == null) {
           // There's a file with no information, must be an old one
           // assume we have very old puts
@@ -194,7 +196,7 @@ public abstract class Compactor<T extends CellSink> {
           fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
         }
       }
-      tmp = fileInfo.get(StoreFile.TIMERANGE_KEY);
+      tmp = fileInfo.get(TIMERANGE_KEY);
       TimeRangeTracker trt = TimeRangeTracker.getTimeRangeTracker(tmp);
       fd.latestPutTs = trt == null? HConstants.LATEST_TIMESTAMP: trt.getMax();
       if (LOG.isDebugEnabled()) {
@@ -215,7 +217,7 @@ public abstract class Compactor<T extends CellSink> {
    * @param filesToCompact Files.
    * @return Scanners.
    */
-  protected List<StoreFileScanner> createFileScanners(Collection<StoreFile> filesToCompact,
+  protected List<StoreFileScanner> createFileScanners(Collection<HStoreFile> filesToCompact,
       long smallestReadPoint, boolean useDropBehind) throws IOException {
     return StoreFileScanner.getScannersForCompaction(filesToCompact, useDropBehind,
       smallestReadPoint);
@@ -498,7 +500,7 @@ public abstract class Compactor<T extends CellSink> {
    * @param earliestPutTs Earliest put across all files.
    * @return A compaction scanner.
    */
-  protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+  protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
       ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
     return new StoreScanner(store, store.getScanInfo(), OptionalInt.empty(), scanners, scanType,
         smallestReadPoint, earliestPutTs);
@@ -513,7 +515,7 @@ public abstract class Compactor<T extends CellSink> {
    * @param dropDeletesToRow Drop deletes ending with this row, exclusive. Can be null.
    * @return A compaction scanner.
    */
-  protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+  protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
       long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
       byte[] dropDeletesToRow) throws IOException {
     return new StoreScanner(store, store.getScanInfo(), OptionalInt.empty(), scanners,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
index 6cfe207..a4cc65c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java
@@ -30,14 +30,14 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterators;
@@ -98,9 +98,9 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
    */
   @Override
   @VisibleForTesting
-  public boolean needsCompaction(final Collection<StoreFile> storeFiles,
-      final List<StoreFile> filesCompacting) {
-    ArrayList<StoreFile> candidates = new ArrayList<>(storeFiles);
+  public boolean needsCompaction(Collection<HStoreFile> storeFiles,
+      List<HStoreFile> filesCompacting) {
+    ArrayList<HStoreFile> candidates = new ArrayList<>(storeFiles);
     try {
       return !selectMinorCompaction(candidates, false, true).getFiles().isEmpty();
     } catch (Exception e) {
@@ -109,8 +109,8 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
     }
   }
 
-  public boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
-    throws IOException {
+  public boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
+      throws IOException {
     long mcTime = getNextMajorCompactTime(filesToCompact);
     if (filesToCompact == null || mcTime == 0) {
       if (LOG.isDebugEnabled()) {
@@ -135,7 +135,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
     List<Long> boundaries = getCompactBoundariesForMajor(filesToCompact, now);
     boolean[] filesInWindow = new boolean[boundaries.size()];
 
-    for (StoreFile file: filesToCompact) {
+    for (HStoreFile file: filesToCompact) {
       OptionalLong minTimestamp = file.getMinimumTimestamp();
       long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE;
       if (cfTTL != Long.MAX_VALUE && oldest >= cfTTL) {
@@ -186,7 +186,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
   }
 
   @Override
-  protected CompactionRequest createCompactionRequest(ArrayList<StoreFile> candidateSelection,
+  protected CompactionRequest createCompactionRequest(ArrayList<HStoreFile> candidateSelection,
     boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
     CompactionRequest result = tryingMajor ? selectMajorCompaction(candidateSelection)
       : selectMinorCompaction(candidateSelection, mayUseOffPeak, mayBeStuck);
@@ -196,7 +196,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
     return result;
   }
 
-  public CompactionRequest selectMajorCompaction(ArrayList<StoreFile> candidateSelection) {
+  public CompactionRequest selectMajorCompaction(ArrayList<HStoreFile> candidateSelection) {
     long now = EnvironmentEdgeManager.currentTime();
     return new DateTieredCompactionRequest(candidateSelection,
       this.getCompactBoundariesForMajor(candidateSelection, now));
@@ -210,15 +210,15 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
    * by seqId and maxTimestamp in descending order and build the time windows. All the out-of-order
    * data into the same compaction windows, guaranteeing contiguous compaction based on sequence id.
    */
-  public CompactionRequest selectMinorCompaction(ArrayList<StoreFile> candidateSelection,
+  public CompactionRequest selectMinorCompaction(ArrayList<HStoreFile> candidateSelection,
       boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
     long now = EnvironmentEdgeManager.currentTime();
     long oldestToCompact = getOldestToCompact(comConf.getDateTieredMaxStoreFileAgeMillis(), now);
 
-    List<Pair<StoreFile, Long>> storefileMaxTimestampPairs =
+    List<Pair<HStoreFile, Long>> storefileMaxTimestampPairs =
         Lists.newArrayListWithCapacity(candidateSelection.size());
     long maxTimestampSeen = Long.MIN_VALUE;
-    for (StoreFile storeFile : candidateSelection) {
+    for (HStoreFile storeFile : candidateSelection) {
       // if there is out-of-order data,
       // we put them in the same window as the last file in increasing order
       maxTimestampSeen =
@@ -229,7 +229,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
 
     CompactionWindow window = getIncomingWindow(now);
     int minThreshold = comConf.getDateTieredIncomingWindowMin();
-    PeekingIterator<Pair<StoreFile, Long>> it =
+    PeekingIterator<Pair<HStoreFile, Long>> it =
         Iterators.peekingIterator(storefileMaxTimestampPairs.iterator());
     while (it.hasNext()) {
       if (window.compareToTimestamp(oldestToCompact) < 0) {
@@ -242,7 +242,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
         minThreshold = comConf.getMinFilesToCompact();
       } else {
         // The file is within the target window
-        ArrayList<StoreFile> fileList = Lists.newArrayList();
+        ArrayList<HStoreFile> fileList = Lists.newArrayList();
         // Add all files in the same window. For incoming window
         // we tolerate files with future data although it is sub-optimal
         while (it.hasNext() && window.compareToTimestamp(it.peek().getSecond()) <= 0) {
@@ -261,10 +261,10 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
       }
     }
     // A non-null file list is expected by HStore
-    return new CompactionRequest(Collections.<StoreFile> emptyList());
+    return new CompactionRequest(Collections.emptyList());
   }
 
-  private DateTieredCompactionRequest generateCompactionRequest(ArrayList<StoreFile> storeFiles,
+  private DateTieredCompactionRequest generateCompactionRequest(ArrayList<HStoreFile> storeFiles,
       CompactionWindow window, boolean mayUseOffPeak, boolean mayBeStuck, int minThreshold)
       throws IOException {
     // The files has to be in ascending order for ratio-based compaction to work right
@@ -273,7 +273,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
 
     // Compact everything in the window if have more files than comConf.maxBlockingFiles
     compactionPolicyPerWindow.setMinThreshold(minThreshold);
-    ArrayList<StoreFile> storeFileSelection = mayBeStuck ? storeFiles
+    ArrayList<HStoreFile> storeFileSelection = mayBeStuck ? storeFiles
       : compactionPolicyPerWindow.applyCompactionPolicy(storeFiles, mayUseOffPeak, false);
     if (storeFileSelection != null && !storeFileSelection.isEmpty()) {
       // If there is any file in the window excluded from compaction,
@@ -291,7 +291,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
   /**
    * Return a list of boundaries for multiple compaction output in ascending order.
    */
-  private List<Long> getCompactBoundariesForMajor(Collection<StoreFile> filesToCompact, long now) {
+  private List<Long> getCompactBoundariesForMajor(Collection<HStoreFile> filesToCompact, long now) {
     long minTimestamp =
         filesToCompact.stream().mapToLong(f -> f.getMinimumTimestamp().orElse(Long.MAX_VALUE)).min()
             .orElse(Long.MAX_VALUE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java
index b33663f..5fddf33 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java
@@ -21,14 +21,14 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_DOESNT_OVERRIDE_EQUALS",
   justification="It is intended to use the same equal method as superclass")
 public class DateTieredCompactionRequest extends CompactionRequest {
   private List<Long> boundaries;
 
-  public DateTieredCompactionRequest(Collection<StoreFile> files, List<Long> boundaryList) {
+  public DateTieredCompactionRequest(Collection<HStoreFile> files, List<Long> boundaryList) {
     super(files);
     boundaries = boundaryList;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
index 9c3f576..ba0caa4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java
@@ -24,13 +24,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter;
+import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * This compactor will generate StoreFile for different time ranges.
@@ -40,7 +40,7 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTiered
 
   private static final Log LOG = LogFactory.getLog(DateTieredCompactor.class);
 
-  public DateTieredCompactor(Configuration conf, Store store) {
+  public DateTieredCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
index af30f36..08951b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java
@@ -25,14 +25,14 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 public class DefaultCompactor extends Compactor<StoreFileWriter> {
   private static final Log LOG = LogFactory.getLog(DefaultCompactor.class);
 
-  public DefaultCompactor(final Configuration conf, final Store store) {
+  public DefaultCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 
@@ -76,7 +76,7 @@ public class DefaultCompactor extends Compactor<StoreFileWriter> {
    *         made it through the compaction.
    * @throws IOException
    */
-  public List<Path> compactForTesting(final Collection<StoreFile> filesToCompact, boolean isMajor)
+  public List<Path> compactForTesting(Collection<HStoreFile> filesToCompact, boolean isMajor)
       throws IOException {
     CompactionRequest cr = new CompactionRequest(filesToCompact);
     cr.setIsMajor(isMajor, isMajor);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java
index 842d6e8..b0942f6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java
@@ -26,9 +26,9 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Class to pick which files if any to compact together.
@@ -51,21 +51,20 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
   }
 
   @Override
-  protected final ArrayList<StoreFile> applyCompactionPolicy(final ArrayList<StoreFile> candidates,
-    final boolean mayUseOffPeak, final boolean mightBeStuck) throws IOException {
-    return new ArrayList<>(applyCompactionPolicy(candidates, mightBeStuck,
-        mayUseOffPeak, comConf.getMinFilesToCompact(), comConf.getMaxFilesToCompact()));
+  protected final ArrayList<HStoreFile> applyCompactionPolicy(ArrayList<HStoreFile> candidates,
+      boolean mayUseOffPeak, boolean mightBeStuck) throws IOException {
+    return new ArrayList<>(applyCompactionPolicy(candidates, mightBeStuck, mayUseOffPeak,
+      comConf.getMinFilesToCompact(), comConf.getMaxFilesToCompact()));
   }
 
-  public List<StoreFile> applyCompactionPolicy(final List<StoreFile> candidates,
-       boolean mightBeStuck, boolean mayUseOffPeak, int minFiles, int maxFiles) {
-
+  public List<HStoreFile> applyCompactionPolicy(List<HStoreFile> candidates, boolean mightBeStuck,
+      boolean mayUseOffPeak, int minFiles, int maxFiles) {
     final double currentRatio = mayUseOffPeak
         ? comConf.getCompactionRatioOffPeak() : comConf.getCompactionRatio();
 
     // Start off choosing nothing.
-    List<StoreFile> bestSelection = new ArrayList<>(0);
-    List<StoreFile> smallest = mightBeStuck ? new ArrayList<>(0) : null;
+    List<HStoreFile> bestSelection = new ArrayList<>(0);
+    List<HStoreFile> smallest = mightBeStuck ? new ArrayList<>(0) : null;
     long bestSize = 0;
     long smallestSize = Long.MAX_VALUE;
 
@@ -75,7 +74,7 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
       // Consider every different sub list permutation in between start and end with min files.
       for (int currentEnd = start + minFiles - 1;
           currentEnd < candidates.size(); currentEnd++) {
-        List<StoreFile> potentialMatchFiles = candidates.subList(start, currentEnd + 1);
+        List<HStoreFile> potentialMatchFiles = candidates.subList(start, currentEnd + 1);
 
         // Sanity checks
         if (potentialMatchFiles.size() < minFiles) {
@@ -125,8 +124,8 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
     return new ArrayList<>(bestSelection);
   }
 
-  private boolean isBetterSelection(List<StoreFile> bestSelection,
-      long bestSize, List<StoreFile> selection, long size, boolean mightBeStuck) {
+  private boolean isBetterSelection(List<HStoreFile> bestSelection, long bestSize,
+      List<HStoreFile> selection, long size, boolean mightBeStuck) {
     if (mightBeStuck && bestSize > 0 && size > 0) {
       // Keep the selection that removes most files for least size. That penaltizes adding
       // large files to compaction, but not small files, so we don't become totally inefficient
@@ -146,13 +145,8 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
    * @param potentialMatchFiles StoreFile list.
    * @return Sum of StoreFile.getReader().length();
    */
-  private long getTotalStoreSize(final List<StoreFile> potentialMatchFiles) {
-    long size = 0;
-
-    for (StoreFile s:potentialMatchFiles) {
-      size += s.getReader().length();
-    }
-    return size;
+  private long getTotalStoreSize(List<HStoreFile> potentialMatchFiles) {
+    return potentialMatchFiles.stream().mapToLong(sf -> sf.getReader().length()).sum();
   }
 
   /**
@@ -163,14 +157,14 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
    * @param currentRatio The ratio to use.
    * @return a boolean if these files satisfy the ratio constraints.
    */
-  private boolean filesInRatio(final List<StoreFile> files, final double currentRatio) {
+  private boolean filesInRatio(List<HStoreFile> files, double currentRatio) {
     if (files.size() < 2) {
       return true;
     }
 
     long totalFileSize = getTotalStoreSize(files);
 
-    for (StoreFile file : files) {
+    for (HStoreFile file : files) {
       long singleFileSize = file.getReader().length();
       long sumAllOtherFileSizes = totalFileSize - singleFileSize;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java
index 5eb15cf..a0609bc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java
@@ -26,11 +26,11 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * 
@@ -55,10 +55,9 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
   }
 
   @Override
-  public CompactionRequest selectCompaction(Collection<StoreFile> candidateFiles,
-      List<StoreFile> filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak,
+  public CompactionRequest selectCompaction(Collection<HStoreFile> candidateFiles,
+      List<HStoreFile> filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak,
       boolean forceMajor) throws IOException {
-    
     if(forceMajor){
       LOG.warn("Major compaction is not supported for FIFO compaction policy. Ignore the flag.");
     }
@@ -70,13 +69,13 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
     }
     
     // Nothing to compact
-    Collection<StoreFile> toCompact = getExpiredStores(candidateFiles, filesCompacting);
+    Collection<HStoreFile> toCompact = getExpiredStores(candidateFiles, filesCompacting);
     CompactionRequest result = new CompactionRequest(toCompact);
     return result;
   }
 
   @Override
-  public boolean shouldPerformMajorCompaction(Collection<StoreFile> filesToCompact)
+  public boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
     throws IOException {
     boolean isAfterSplit = StoreUtils.hasReferences(filesToCompact);
     if(isAfterSplit){
@@ -87,8 +86,8 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
   }
 
   @Override
-  public boolean needsCompaction(Collection<StoreFile> storeFiles, 
-      List<StoreFile> filesCompacting) {  
+  public boolean needsCompaction(Collection<HStoreFile> storeFiles,
+      List<HStoreFile> filesCompacting) {
     boolean isAfterSplit = StoreUtils.hasReferences(storeFiles);
     if(isAfterSplit){
       LOG.info("Split detected, delegate to the parent policy.");
@@ -97,9 +96,9 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
     return hasExpiredStores(storeFiles);
   }
 
-  private  boolean hasExpiredStores(Collection<StoreFile> files) {
+  private boolean hasExpiredStores(Collection<HStoreFile> files) {
     long currentTime = EnvironmentEdgeManager.currentTime();
-    for(StoreFile sf: files){
+    for(HStoreFile sf: files){
       // Check MIN_VERSIONS is in HStore removeUnneededFiles
       long maxTs = sf.getReader().getMaxTimestamp();
       long maxTtl = storeConfigInfo.getStoreFileTtl();
@@ -113,11 +112,11 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
     return false;
   }
 
-  private  Collection<StoreFile> getExpiredStores(Collection<StoreFile> files,
-    Collection<StoreFile> filesCompacting) {
+  private Collection<HStoreFile> getExpiredStores(Collection<HStoreFile> files,
+      Collection<HStoreFile> filesCompacting) {
     long currentTime = EnvironmentEdgeManager.currentTime();
-    Collection<StoreFile> expiredStores = new ArrayList<>();
-    for(StoreFile sf: files){
+    Collection<HStoreFile> expiredStores = new ArrayList<>();
+    for(HStoreFile sf: files){
       // Check MIN_VERSIONS is in HStore removeUnneededFiles
       long maxTs = sf.getReader().getMaxTimestamp();
       long maxTtl = storeConfigInfo.getStoreFileTtl();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
index 9e7f869..172f17a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java
@@ -28,13 +28,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * The default algorithm for selecting files for compaction.
@@ -55,7 +55,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
    * @return True if we should run a major compaction.
    */
   @Override
-  public boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
+  public boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
     throws IOException {
     boolean result = false;
     long mcTime = getNextMajorCompactTime(filesToCompact);
@@ -79,7 +79,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
       }
       if (filesToCompact.size() == 1) {
         // Single file
-        StoreFile sf = filesToCompact.iterator().next();
+        HStoreFile sf = filesToCompact.iterator().next();
         OptionalLong minTimestamp = sf.getMinimumTimestamp();
         long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE;
         if (sf.isMajorCompactionResult() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) {
@@ -113,7 +113,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
   }
 
   @Override
-  protected CompactionRequest createCompactionRequest(ArrayList<StoreFile>
+  protected CompactionRequest createCompactionRequest(ArrayList<HStoreFile>
     candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck)
     throws IOException {
     if (!tryingMajor) {
@@ -155,7 +155,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
     * @param candidates pre-filtrate
     * @return filtered subset
     */
-  protected ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
+  protected ArrayList<HStoreFile> applyCompactionPolicy(ArrayList<HStoreFile> candidates,
     boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
     if (candidates.isEmpty()) {
       return candidates;
@@ -174,7 +174,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
     long[] fileSizes = new long[countOfFiles];
     long[] sumSize = new long[countOfFiles];
     for (int i = countOfFiles - 1; i >= 0; --i) {
-      StoreFile file = candidates.get(i);
+      HStoreFile file = candidates.get(i);
       fileSizes[i] = file.getReader().length();
       // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo
       int tooFar = i + comConf.getMaxFilesToCompact() - 1;
@@ -209,8 +209,8 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
    * @param filesCompacting files being scheduled to compact.
    * @return true to schedule a request.
    */
-  public boolean needsCompaction(final Collection<StoreFile> storeFiles,
-      final List<StoreFile> filesCompacting) {
+  public boolean needsCompaction(Collection<HStoreFile> storeFiles,
+      List<HStoreFile> filesCompacting) {
     int numCandidates = storeFiles.size() - filesCompacting.size();
     return numCandidates >= comConf.getMinFilesToCompact();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java
index 5f22d1d..facc161 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java
@@ -20,10 +20,10 @@ import java.util.Random;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
@@ -42,8 +42,8 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
     super(conf, storeConfigInfo);
   }
 
-  public List<StoreFile> preSelectCompactionForCoprocessor(final Collection<StoreFile> candidates,
-      final List<StoreFile> filesCompacting) {
+  public List<HStoreFile> preSelectCompactionForCoprocessor(Collection<HStoreFile> candidates,
+      List<HStoreFile> filesCompacting) {
     return getCurrentEligibleFiles(new ArrayList<>(candidates), filesCompacting);
   }
 
@@ -53,11 +53,11 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
    *   on seqId for data consistency.
    * @return subset copy of candidate list that meets compaction criteria
    */
-  public CompactionRequest selectCompaction(Collection<StoreFile> candidateFiles,
-      final List<StoreFile> filesCompacting, final boolean isUserCompaction,
-      final boolean mayUseOffPeak, final boolean forceMajor) throws IOException {
+  public CompactionRequest selectCompaction(Collection<HStoreFile> candidateFiles,
+      List<HStoreFile> filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak,
+      boolean forceMajor) throws IOException {
     // Preliminary compaction subject to filters
-    ArrayList<StoreFile> candidateSelection = new ArrayList<>(candidateFiles);
+    ArrayList<HStoreFile> candidateSelection = new ArrayList<>(candidateFiles);
     // Stuck and not compacting enough (estimate). It is not guaranteed that we will be
     // able to compact more if stuck and compacting, because ratio policy excludes some
     // non-compacting files from consideration during compaction (see getCurrentEligibleFiles).
@@ -88,7 +88,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
     CompactionRequest result = createCompactionRequest(candidateSelection,
       isTryingMajor || isAfterSplit, mayUseOffPeak, mayBeStuck);
 
-    ArrayList<StoreFile> filesToCompact = Lists.newArrayList(result.getFiles());
+    ArrayList<HStoreFile> filesToCompact = Lists.newArrayList(result.getFiles());
     removeExcessFiles(filesToCompact, isUserCompaction, isTryingMajor);
     result.updateFiles(filesToCompact);
 
@@ -99,16 +99,16 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
     return result;
   }
 
-  protected abstract CompactionRequest createCompactionRequest(ArrayList<StoreFile>
-    candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck)
-    throws IOException;
+  protected abstract CompactionRequest createCompactionRequest(
+      ArrayList<HStoreFile> candidateSelection, boolean tryingMajor, boolean mayUseOffPeak,
+      boolean mayBeStuck) throws IOException;
 
-  /*
+  /**
    * @param filesToCompact Files to compact. Can be null.
    * @return True if we should run a major compaction.
    */
-  public abstract boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
-    throws IOException;
+  public abstract boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
+      throws IOException;
 
   /**
    * Used calculation jitter
@@ -119,7 +119,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
    * @param filesToCompact
    * @return When to run next major compaction
    */
-  public long getNextMajorCompactTime(Collection<StoreFile> filesToCompact) {
+  public long getNextMajorCompactTime(Collection<HStoreFile> filesToCompact) {
     // default = 24hrs
     long period = comConf.getMajorCompactionPeriod();
     if (period <= 0) {
@@ -154,16 +154,16 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
     return compactionSize > comConf.getThrottlePoint();
   }
 
-  public abstract boolean needsCompaction(final Collection<StoreFile> storeFiles,
-    final List<StoreFile> filesCompacting);
+  public abstract boolean needsCompaction(Collection<HStoreFile> storeFiles,
+      List<HStoreFile> filesCompacting);
 
-  protected ArrayList<StoreFile> getCurrentEligibleFiles(ArrayList<StoreFile> candidateFiles,
-      final List<StoreFile> filesCompacting) {
+  protected ArrayList<HStoreFile> getCurrentEligibleFiles(ArrayList<HStoreFile> candidateFiles,
+      final List<HStoreFile> filesCompacting) {
     // candidates = all storefiles not already in compaction queue
     if (!filesCompacting.isEmpty()) {
       // exclude all files older than the newest file we're currently
       // compacting. this allows us to preserve contiguity (HBASE-2856)
-      StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
+      HStoreFile last = filesCompacting.get(filesCompacting.size() - 1);
       int idx = candidateFiles.indexOf(last);
       Preconditions.checkArgument(idx != -1);
       candidateFiles.subList(0, idx + 1).clear();
@@ -176,7 +176,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
    * @return filtered subset exclude all files above maxCompactSize
    *   Also save all references. We MUST compact them
    */
-  protected ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates,
+  protected ArrayList<HStoreFile> skipLargeFiles(ArrayList<HStoreFile> candidates,
     boolean mayUseOffpeak) {
     int pos = 0;
     while (pos < candidates.size() && !candidates.get(pos).isReference()
@@ -195,10 +195,10 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
    * @param candidates pre-filtrate
    * @return filtered subset exclude all bulk load files if configured
    */
-  protected ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
-    candidates.removeAll(Collections2.filter(candidates, new Predicate<StoreFile>() {
+  protected ArrayList<HStoreFile> filterBulk(ArrayList<HStoreFile> candidates) {
+    candidates.removeAll(Collections2.filter(candidates, new Predicate<HStoreFile>() {
       @Override
-      public boolean apply(StoreFile input) {
+      public boolean apply(HStoreFile input) {
         return input.excludeFromMinorCompaction();
       }
     }));
@@ -208,7 +208,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
   /**
    * @param candidates pre-filtrate
    */
-  protected void removeExcessFiles(ArrayList<StoreFile> candidates,
+  protected void removeExcessFiles(ArrayList<HStoreFile> candidates,
       boolean isUserCompaction, boolean isMajorCompaction) {
     int excess = candidates.size() - comConf.getMaxFilesToCompact();
     if (excess > 0) {
@@ -227,12 +227,12 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
    * @param candidates pre-filtrate
    * @return filtered subset forget the compactionSelection if we don't have enough files
    */
-  protected ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates,
-    int minFiles) {
+  protected ArrayList<HStoreFile> checkMinFilesCriteria(ArrayList<HStoreFile> candidates,
+      int minFiles) {
     if (candidates.size() < minFiles) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Not compacting files because we only have " + candidates.size()
-            + " files ready for compaction. Need " + minFiles + " to initiate.");
+        LOG.debug("Not compacting files because we only have " + candidates.size() +
+            " files ready for compaction. Need " + minFiles + " to initiate.");
       }
       candidates.clear();
     }


[06/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java
index e677ed1..9dc8246 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java
@@ -30,9 +30,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
 import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher;
@@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ConcatenatedLists;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 
@@ -63,12 +63,12 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     stripePolicy = new ExploringCompactionPolicy(conf, storeConfigInfo);
   }
 
-  public List<StoreFile> preSelectFilesForCoprocessor(StripeInformationProvider si,
-      List<StoreFile> filesCompacting) {
+  public List<HStoreFile> preSelectFilesForCoprocessor(StripeInformationProvider si,
+      List<HStoreFile> filesCompacting) {
     // We sincerely hope nobody is messing with us with their coprocessors.
     // If they do, they are very likely to shoot themselves in the foot.
     // We'll just exclude all the filesCompacting from the list.
-    ArrayList<StoreFile> candidateFiles = new ArrayList<>(si.getStorefiles());
+    ArrayList<HStoreFile> candidateFiles = new ArrayList<>(si.getStorefiles());
     candidateFiles.removeAll(filesCompacting);
     return candidateFiles;
   }
@@ -102,7 +102,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
   }
 
   public StripeCompactionRequest selectCompaction(StripeInformationProvider si,
-      List<StoreFile> filesCompacting, boolean isOffpeak) throws IOException {
+      List<HStoreFile> filesCompacting, boolean isOffpeak) throws IOException {
     // TODO: first cut - no parallel compactions. To have more fine grained control we
     //       probably need structure more sophisticated than a list.
     if (!filesCompacting.isEmpty()) {
@@ -116,7 +116,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
 
     // This can happen due to region split. We can skip it later; for now preserve
     // compact-all-things behavior.
-    Collection<StoreFile> allFiles = si.getStorefiles();
+    Collection<HStoreFile> allFiles = si.getStorefiles();
     if (StoreUtils.hasReferences(allFiles)) {
       LOG.debug("There are references in the store; compacting all files");
       long targetKvs = estimateTargetKvs(allFiles, config.getInitialCount()).getFirst();
@@ -127,7 +127,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     }
 
     int stripeCount = si.getStripeCount();
-    List<StoreFile> l0Files = si.getLevel0Files();
+    List<HStoreFile> l0Files = si.getLevel0Files();
 
     // See if we need to make new stripes.
     boolean shouldCompactL0 = (this.config.getLevel0MinFiles() <= l0Files.size());
@@ -157,7 +157,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     return selectSingleStripeCompaction(si, false, canDropDeletesNoL0, isOffpeak);
   }
 
-  public boolean needsCompactions(StripeInformationProvider si, List<StoreFile> filesCompacting) {
+  public boolean needsCompactions(StripeInformationProvider si, List<HStoreFile> filesCompacting) {
     // Approximation on whether we need compaction.
     return filesCompacting.isEmpty()
         && (StoreUtils.hasReferences(si.getStorefiles())
@@ -166,7 +166,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
   }
 
   @Override
-  public boolean shouldPerformMajorCompaction(Collection<StoreFile> filesToCompact)
+  public boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
     throws IOException {
     return false; // there's never a major compaction!
   }
@@ -182,7 +182,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
    */
   protected boolean needsSingleStripeCompaction(StripeInformationProvider si) {
     int minFiles = this.config.getStripeCompactMinFiles();
-    for (List<StoreFile> stripe : si.getStripes()) {
+    for (List<HStoreFile> stripe : si.getStripes()) {
       if (stripe.size() >= minFiles) return true;
     }
     return false;
@@ -190,20 +190,20 @@ public class StripeCompactionPolicy extends CompactionPolicy {
 
   protected StripeCompactionRequest selectSingleStripeCompaction(StripeInformationProvider si,
       boolean includeL0, boolean canDropDeletesWithoutL0, boolean isOffpeak) throws IOException {
-    ArrayList<ImmutableList<StoreFile>> stripes = si.getStripes();
+    ArrayList<ImmutableList<HStoreFile>> stripes = si.getStripes();
 
     int bqIndex = -1;
-    List<StoreFile> bqSelection = null;
+    List<HStoreFile> bqSelection = null;
     int stripeCount = stripes.size();
     long bqTotalSize = -1;
     for (int i = 0; i < stripeCount; ++i) {
       // If we want to compact L0 to drop deletes, we only want whole-stripe compactions.
       // So, pass includeL0 as 2nd parameter to indicate that.
-      List<StoreFile> selection = selectSimpleCompaction(stripes.get(i),
+      List<HStoreFile> selection = selectSimpleCompaction(stripes.get(i),
           !canDropDeletesWithoutL0 && includeL0, isOffpeak);
       if (selection.isEmpty()) continue;
       long size = 0;
-      for (StoreFile sf : selection) {
+      for (HStoreFile sf : selection) {
         size += sf.getReader().length();
       }
       if (bqSelection == null || selection.size() > bqSelection.size() ||
@@ -217,7 +217,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
       LOG.debug("No good compaction is possible in any stripe");
       return null;
     }
-    List<StoreFile> filesToCompact = new ArrayList<>(bqSelection);
+    List<HStoreFile> filesToCompact = new ArrayList<>(bqSelection);
     // See if we can, and need to, split this stripe.
     int targetCount = 1;
     long targetKvs = Long.MAX_VALUE;
@@ -244,9 +244,9 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     StripeCompactionRequest req;
     if (includeL0) {
       assert hasAllFiles;
-      List<StoreFile> l0Files = si.getLevel0Files();
+      List<HStoreFile> l0Files = si.getLevel0Files();
       LOG.debug("Adding " + l0Files.size() + " files to compaction to be able to drop deletes");
-      ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
+      ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>();
       sfs.addSublist(filesToCompact);
       sfs.addSublist(l0Files);
       req = new BoundaryStripeCompactionRequest(sfs, si.getStripeBoundaries());
@@ -267,33 +267,16 @@ public class StripeCompactionPolicy extends CompactionPolicy {
    * @param allFilesOnly Whether a compaction of all-or-none files is needed.
    * @return The resulting selection.
    */
-  private List<StoreFile> selectSimpleCompaction(
-      List<StoreFile> sfs, boolean allFilesOnly, boolean isOffpeak) {
+  private List<HStoreFile> selectSimpleCompaction(
+      List<HStoreFile> sfs, boolean allFilesOnly, boolean isOffpeak) {
     int minFilesLocal = Math.max(
         allFilesOnly ? sfs.size() : 0, this.config.getStripeCompactMinFiles());
     int maxFilesLocal = Math.max(this.config.getStripeCompactMaxFiles(), minFilesLocal);
     return stripePolicy.applyCompactionPolicy(sfs, false, isOffpeak, minFilesLocal, maxFilesLocal);
   }
 
-  /**
-   * Selects the compaction that compacts all files (to be removed later).
-   * @param si StoreFileManager.
-   * @param targetStripeCount Target stripe count.
-   * @param targetSize Target stripe size.
-   * @return The compaction.
-   */
-  private StripeCompactionRequest selectCompactionOfAllFiles(StripeInformationProvider si,
-      int targetStripeCount, long targetSize) {
-    Collection<StoreFile> allFiles = si.getStorefiles();
-    SplitStripeCompactionRequest request = new SplitStripeCompactionRequest(
-        allFiles, OPEN_KEY, OPEN_KEY, targetStripeCount, targetSize);
-    request.setMajorRangeFull();
-    LOG.debug("Selecting a compaction that includes all " + allFiles.size() + " files");
-    return request;
-  }
-
   private StripeCompactionRequest selectNewStripesCompaction(StripeInformationProvider si) {
-    List<StoreFile> l0Files = si.getLevel0Files();
+    List<HStoreFile> l0Files = si.getLevel0Files();
     Pair<Long, Integer> kvsAndCount = estimateTargetKvs(l0Files, config.getInitialCount());
     LOG.debug("Creating " + kvsAndCount.getSecond() + " initial stripes with "
         + kvsAndCount.getFirst() + " kvs each via L0 compaction of " + l0Files.size() + " files");
@@ -312,9 +295,9 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     long timestampCutoff = EnvironmentEdgeManager.currentTime() - cfTtl;
     // Merge the longest sequence of stripes where all files have expired, if any.
     int start = -1, bestStart = -1, length = 0, bestLength = 0;
-    ArrayList<ImmutableList<StoreFile>> stripes = si.getStripes();
+    ArrayList<ImmutableList<HStoreFile>> stripes = si.getStripes();
     OUTER: for (int i = 0; i < stripes.size(); ++i) {
-      for (StoreFile storeFile : stripes.get(i)) {
+      for (HStoreFile storeFile : stripes.get(i)) {
         if (storeFile.getReader().getMaxTimestamp() < timestampCutoff) continue;
         // Found non-expired file, this stripe has to stay.
         if (length > bestLength) {
@@ -345,7 +328,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     }
     LOG.debug("Merging " + bestLength + " stripes to delete expired store files");
     int endIndex = bestStart + bestLength - 1;
-    ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
+    ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>();
     sfs.addAllSublists(stripes.subList(bestStart, endIndex + 1));
     SplitStripeCompactionRequest result = new SplitStripeCompactionRequest(sfs,
         si.getStartRow(bestStart), si.getEndRow(endIndex), 1, Long.MAX_VALUE);
@@ -355,23 +338,23 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     return result;
   }
 
-  private static long getTotalKvCount(final Collection<StoreFile> candidates) {
+  private static long getTotalKvCount(final Collection<HStoreFile> candidates) {
     long totalSize = 0;
-    for (StoreFile storeFile : candidates) {
+    for (HStoreFile storeFile : candidates) {
       totalSize += storeFile.getReader().getEntries();
     }
     return totalSize;
   }
 
-  public static long getTotalFileSize(final Collection<StoreFile> candidates) {
+  public static long getTotalFileSize(final Collection<HStoreFile> candidates) {
     long totalSize = 0;
-    for (StoreFile storeFile : candidates) {
+    for (HStoreFile storeFile : candidates) {
       totalSize += storeFile.getReader().length();
     }
     return totalSize;
   }
 
-  private Pair<Long, Integer> estimateTargetKvs(Collection<StoreFile> files, double splitCount) {
+  private Pair<Long, Integer> estimateTargetKvs(Collection<HStoreFile> files, double splitCount) {
     // If the size is larger than what we target, we don't want to split into proportionally
     // larger parts and then have to split again very soon. So, we will increase the multiplier
     // by one until we get small enough parts. E.g. 5Gb stripe that should have been split into
@@ -452,7 +435,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
       this.targetBoundaries = targetBoundaries;
     }
 
-    public BoundaryStripeCompactionRequest(Collection<StoreFile> files,
+    public BoundaryStripeCompactionRequest(Collection<HStoreFile> files,
         List<byte[]> targetBoundaries) {
       this(new CompactionRequest(files), targetBoundaries);
     }
@@ -494,16 +477,11 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     }
 
     public SplitStripeCompactionRequest(
-        CompactionRequest request, byte[] startRow, byte[] endRow, long targetKvs) {
-      this(request, startRow, endRow, Integer.MAX_VALUE, targetKvs);
-    }
-
-    public SplitStripeCompactionRequest(
-        Collection<StoreFile> files, byte[] startRow, byte[] endRow, long targetKvs) {
+        Collection<HStoreFile> files, byte[] startRow, byte[] endRow, long targetKvs) {
       this(files, startRow, endRow, Integer.MAX_VALUE, targetKvs);
     }
 
-    public SplitStripeCompactionRequest(Collection<StoreFile> files,
+    public SplitStripeCompactionRequest(Collection<HStoreFile> files,
         byte[] startRow, byte[] endRow, int targetCount, long targetKvs) {
       this(new CompactionRequest(files), startRow, endRow, targetCount, targetKvs);
     }
@@ -524,7 +502,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
 
   /** The information about stripes that the policy needs to do its stuff */
   public static interface StripeInformationProvider {
-    public Collection<StoreFile> getStorefiles();
+    public Collection<HStoreFile> getStorefiles();
 
     /**
      * Gets the start row for a given stripe.
@@ -543,7 +521,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     /**
      * @return Level 0 files.
      */
-    public List<StoreFile> getLevel0Files();
+    public List<HStoreFile> getLevel0Files();
 
     /**
      * @return All stripe boundaries; including the open ones on both ends.
@@ -553,7 +531,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
     /**
      * @return The stripes.
      */
-    public ArrayList<ImmutableList<StoreFile>> getStripes();
+    public ArrayList<ImmutableList<HStoreFile>> getStripes();
 
     /**
      * @return Stripe count.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
index c75b24c..f552f96 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java
@@ -24,15 +24,15 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.ScanType;
-import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
 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.yetus.audience.InterfaceAudience;
 
 /**
  * This is the placeholder for stripe compactor. The implementation, as well as the proper javadoc,
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 public class StripeCompactor extends AbstractMultiOutputCompactor<StripeMultiFileWriter> {
   private static final Log LOG = LogFactory.getLog(StripeCompactor.class);
 
-  public StripeCompactor(Configuration conf, Store store) {
+  public StripeCompactor(Configuration conf, HStore store) {
     super(conf, store);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
index 615d492..b371b3e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java
@@ -37,16 +37,21 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
@@ -54,10 +59,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.Threads;
 
 /**
  * Utility class to help read/write the Snapshot Manifest.
@@ -228,20 +229,20 @@ public final class SnapshotManifest {
     // 2. iterate through all the stores in the region
     LOG.debug("Creating references for hfiles");
 
-    for (Store store : region.getStores()) {
+    for (HStore store : region.getStores()) {
       // 2.1. build the snapshot reference for the store
       Object familyData = visitor.familyOpen(regionData,
           store.getColumnFamilyDescriptor().getName());
       monitor.rethrowException();
 
-      List<StoreFile> storeFiles = new ArrayList<>(store.getStorefiles());
+      List<HStoreFile> storeFiles = new ArrayList<>(store.getStorefiles());
       if (LOG.isDebugEnabled()) {
         LOG.debug("Adding snapshot references for " + storeFiles  + " hfiles");
       }
 
       // 2.2. iterate through all the store's files and create "references".
       for (int i = 0, sz = storeFiles.size(); i < sz; i++) {
-        StoreFile storeFile = storeFiles.get(i);
+        HStoreFile storeFile = storeFiles.get(i);
         monitor.rethrowException();
 
         // create "reference" to this store file.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
index 08a1512..c457e22 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java
@@ -34,6 +34,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -702,7 +703,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
       Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item, final Table table,
       final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
     Path hfilePath = item.getFilePath();
-    byte[] first, last;
+    Optional<byte[]> first, last;
     try (HFile.Reader hfr = HFile.createReader(hfilePath.getFileSystem(getConf()), hfilePath,
       new CacheConfig(getConf()), true, getConf())) {
       hfr.loadFileInfo();
@@ -713,19 +714,19 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
       return new Pair<>(null, hfilePath.getName());
     }
 
-    LOG.info("Trying to load hfile=" + hfilePath + " first=" + Bytes.toStringBinary(first) +
-        " last=" + Bytes.toStringBinary(last));
-    if (first == null || last == null) {
-      assert first == null && last == null;
+    LOG.info("Trying to load hfile=" + hfilePath + " first=" + first.map(Bytes::toStringBinary) +
+        " last=" + last.map(Bytes::toStringBinary));
+    if (!first.isPresent() || !last.isPresent()) {
+      assert !first.isPresent() && !last.isPresent();
       // TODO what if this is due to a bad HFile?
       LOG.info("hfile " + hfilePath + " has no entries, skipping");
       return null;
     }
-    if (Bytes.compareTo(first, last) > 0) {
-      throw new IllegalArgumentException(
-          "Invalid range: " + Bytes.toStringBinary(first) + " > " + Bytes.toStringBinary(last));
+    if (Bytes.compareTo(first.get(), last.get()) > 0) {
+      throw new IllegalArgumentException("Invalid range: " + Bytes.toStringBinary(first.get()) +
+          " > " + Bytes.toStringBinary(last.get()));
     }
-    int idx = Arrays.binarySearch(startEndKeys.getFirst(), first, Bytes.BYTES_COMPARATOR);
+    int idx = Arrays.binarySearch(startEndKeys.getFirst(), first.get(), Bytes.BYTES_COMPARATOR);
     if (idx < 0) {
       // not on boundary, returns -(insertion index). Calculate region it
       // would be in.
@@ -753,7 +754,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
           "Please use hbck tool to fix it first.");
     }
 
-    boolean lastKeyInRange = Bytes.compareTo(last, startEndKeys.getSecond()[idx]) < 0 ||
+    boolean lastKeyInRange = Bytes.compareTo(last.get(), startEndKeys.getSecond()[idx]) < 0 ||
         Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY);
     if (!lastKeyInRange) {
       List<LoadQueueItem> lqis = splitStoreFile(item, table,
@@ -834,8 +835,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
                 " for family " + builder.getNameAsString());
           }
           reader.loadFileInfo();
-          byte[] first = reader.getFirstRowKey();
-          byte[] last = reader.getLastRowKey();
+          byte[] first = reader.getFirstRowKey().get();
+          byte[] last = reader.getLastRowKey().get();
 
           LOG.info("Trying to figure out region boundaries hfile=" + hfile + " first=" +
               Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
index d4c6254..e8ee3ff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -16,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.util;
 
 import java.io.DataInput;
@@ -26,17 +24,17 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellComparator;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
 import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterBase;
 import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Handles Bloom filter initialization based on configuration and serialized
- * metadata in the reader and writer of {@link org.apache.hadoop.hbase.regionserver.StoreFile}.
+ * Handles Bloom filter initialization based on configuration and serialized metadata in the reader
+ * and writer of {@link org.apache.hadoop.hbase.regionserver.HStoreFile}.
  */
 @InterfaceAudience.Private
 public final class BloomFilterFactory {
@@ -155,7 +153,7 @@ public final class BloomFilterFactory {
 
   /**
    * Creates a new general (Row or RowCol) Bloom filter at the time of
-   * {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing.
+   * {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing.
    *
    * @param conf
    * @param cacheConf
@@ -203,7 +201,7 @@ public final class BloomFilterFactory {
 
   /**
    * Creates a new Delete Family Bloom filter at the time of
-   * {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing.
+   * {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing.
    * @param conf
    * @param cacheConf
    * @param maxKeys an estimate of the number of keys we expect to insert.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 53160e6..0c95e7e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -50,6 +50,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
@@ -858,13 +859,13 @@ public class HBaseFsck extends Configured implements Closeable {
                 new CacheConfig(getConf()), true, getConf());
               if ((reader.getFirstKey() != null)
                   && ((storeFirstKey == null) || (comparator.compare(storeFirstKey,
-                      ((KeyValue.KeyOnlyKeyValue) reader.getFirstKey()).getKey()) > 0))) {
-                storeFirstKey = ((KeyValue.KeyOnlyKeyValue)reader.getFirstKey()).getKey();
+                      ((KeyValue.KeyOnlyKeyValue) reader.getFirstKey().get()).getKey()) > 0))) {
+                storeFirstKey = ((KeyValue.KeyOnlyKeyValue)reader.getFirstKey().get()).getKey();
               }
               if ((reader.getLastKey() != null)
                   && ((storeLastKey == null) || (comparator.compare(storeLastKey,
-                      ((KeyValue.KeyOnlyKeyValue)reader.getLastKey()).getKey())) < 0)) {
-                storeLastKey = ((KeyValue.KeyOnlyKeyValue)reader.getLastKey()).getKey();
+                      ((KeyValue.KeyOnlyKeyValue)reader.getLastKey().get()).getKey())) < 0)) {
+                storeLastKey = ((KeyValue.KeyOnlyKeyValue)reader.getLastKey().get()).getKey();
               }
               reader.close();
             }
@@ -961,10 +962,10 @@ public class HBaseFsck extends Configured implements Closeable {
           CacheConfig cacheConf = new CacheConfig(getConf());
           hf = HFile.createReader(fs, hfile.getPath(), cacheConf, true, getConf());
           hf.loadFileInfo();
-          Cell startKv = hf.getFirstKey();
-          start = CellUtil.cloneRow(startKv);
-          Cell endKv = hf.getLastKey();
-          end = CellUtil.cloneRow(endKv);
+          Optional<Cell> startKv = hf.getFirstKey();
+          start = CellUtil.cloneRow(startKv.get());
+          Optional<Cell> endKv = hf.getLastKey();
+          end = CellUtil.cloneRow(endKv.get());
         } catch (IOException ioe) {
           LOG.warn("Problem reading orphan file " + hfile + ", skipping");
           continue;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
index 34fd6f7..369aed7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java
@@ -17,14 +17,15 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY;
+
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Handles ROW bloom related context. It works with both ByteBufferedCell and byte[] backed cells
@@ -39,7 +40,7 @@ public class RowBloomContext extends BloomContext {
   public void addLastBloomKey(Writer writer) throws IOException {
     if (this.getLastCell() != null) {
       byte[] key = CellUtil.copyRow(this.getLastCell());
-      writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key);
+      writer.appendFileInfo(LAST_BLOOM_KEY, key);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java
index 9ead570..90cbcb0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java
@@ -17,14 +17,15 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY;
+
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Handles ROWCOL bloom related context. It can work with both BytebufferdCells
@@ -43,7 +44,7 @@ public class RowColBloomContext extends BloomContext {
       Cell firstOnRow = CellUtil.createFirstOnRowCol(this.getLastCell());
       // This copy happens only once when the writer is closed
       byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(firstOnRow);
-      writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key);
+      writer.appendFileInfo(LAST_BLOOM_KEY, key);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
index 04aafa7..4b25635 100644
--- a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp
@@ -95,7 +95,7 @@
      List<? extends Store> stores = region.getStores();
      for (Store store : stores) {
        String cf = store.getColumnFamilyName();
-       Collection<StoreFile> storeFiles = store.getStorefiles(); %>
+       Collection<? extends StoreFile> storeFiles = store.getStorefiles(); %>
 
        <h3>Column Family: <%= cf %></h2>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
index 85f65e8..6e7bf7d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hbase;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
@@ -41,16 +39,14 @@ import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -59,6 +55,10 @@ import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
+
 /**
  * Test for the case where a regionserver going down has enough cycles to do damage to regions that
  * have actually been assigned elsehwere.
@@ -201,7 +201,7 @@ public class TestIOFencing {
     }
 
     @Override
-    protected void completeCompaction(Collection<StoreFile> compactedFiles) throws IOException {
+    protected void completeCompaction(Collection<HStoreFile> compactedFiles) throws IOException {
       try {
         r.compactionsWaiting.countDown();
         r.compactionsBlocked.await();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
index dfdd11e..9c100a2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java
@@ -49,12 +49,10 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
 import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.exceptions.MergeRegionException;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
 import org.apache.hadoop.hbase.master.LoadBalancer;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -69,6 +67,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
+
 /**
  * Class to test HBaseAdmin.
  * Spins up the minicluster once at test start and then takes it down afterward.
@@ -1320,8 +1321,8 @@ public class TestAdmin1 {
 
       List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(tableName);
       for (HRegion r : regions) {
-        Store store = r.getStore(Bytes.toBytes(fn));
-        for (StoreFile sf : store.getStorefiles()) {
+        HStore store = r.getStore(Bytes.toBytes(fn));
+        for (HStoreFile sf : store.getStorefiles()) {
           assertTrue(sf.toString().contains(fn));
           assertTrue("Column family " + fn + " should have 3 copies",
             FSUtils.getDefaultReplication(TEST_UTIL.getTestFileSystem(), sf.getPath()) == (sf
@@ -1329,7 +1330,7 @@ public class TestAdmin1 {
         }
 
         store = r.getStore(Bytes.toBytes(fn1));
-        for (StoreFile sf : store.getStorefiles()) {
+        for (HStoreFile sf : store.getStorefiles()) {
           assertTrue(sf.toString().contains(fn1));
           assertTrue("Column family " + fn1 + " should have only 1 copy", 1 == sf.getFileInfo()
               .getFileStatus().getReplication());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
index ac404bb..061d097 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CachedBlock;
+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.Region;
@@ -253,14 +254,16 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
   }
 
   public static class CompactorRegionObserver implements RegionObserver {
+
     @Override
     public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
-        Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
-        InternalScanner s, CompactionLifeCycleTracker request, long readPoint) throws IOException {
-      return createCompactorScanner(store, scanners, scanType, earliestPutTs);
+        Store store, List<? extends KeyValueScanner> scanners, ScanType scanType,
+        long earliestPutTs, InternalScanner s, CompactionLifeCycleTracker request, long readPoint)
+        throws IOException {
+      return createCompactorScanner((HStore) store, scanners, scanType, earliestPutTs);
     }
 
-    private InternalScanner createCompactorScanner(Store store,
+    private InternalScanner createCompactorScanner(HStore store,
         List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs)
         throws IOException {
       return new CompactorStoreScanner(store, store.getScanInfo(), OptionalInt.empty(), scanners,
@@ -270,7 +273,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
 
   private static class CompactorStoreScanner extends StoreScanner {
 
-    public CompactorStoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions,
+    public CompactorStoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions,
         List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint,
         long earliestPutTs) throws IOException {
       super(store, scanInfo, maxVersions, scanners, scanType, smallestReadPoint, earliestPutTs);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index ae4f8a8..aaddd34 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -54,6 +54,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.CompareOperator;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -69,7 +70,6 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.ClusterStatus.Option;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
@@ -105,6 +105,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateR
 import org.apache.hadoop.hbase.regionserver.DelegatingKeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -554,7 +555,7 @@ public class TestFromClientSide {
     }
 
     class MyStoreScanner extends StoreScanner {
-      public MyStoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
+      public MyStoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
           long readPt) throws IOException {
         super(store, scanInfo, scan, columns, readPt);
       }
@@ -588,7 +589,7 @@ public class TestFromClientSide {
     public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
         Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s,
         final long readPt) throws IOException {
-      return new MyStoreScanner(store, store.getScanInfo(), scan, targetCols, readPt);
+      return new MyStoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, readPt);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
index 9938c18..ba1e222 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
@@ -19,6 +19,11 @@
 
 package org.apache.hadoop.hbase.coprocessor;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -57,16 +62,12 @@ import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
 
 /**
  * A sample region observer that tests the RegionObserver interface.
@@ -202,13 +203,13 @@ public class SimpleRegionObserver implements RegionObserver {
 
   @Override
   public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      List<StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {
+      List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {
     ctPreCompactSelect.incrementAndGet();
   }
 
   @Override
   public void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      ImmutableList<StoreFile> selected, CompactionLifeCycleTracker tracker) {
+      ImmutableList<? extends StoreFile> selected, CompactionLifeCycleTracker tracker) {
     ctPostCompactSelect.incrementAndGet();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
index 9c06c3e..afeb763 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java
@@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@@ -116,7 +115,7 @@ public class TestRegionObserverScannerOpenHook {
         Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt)
         throws IOException {
       scan.setFilter(new NoDataFilter());
-      return new StoreScanner(store, store.getScanInfo(), scan, targetCols, readPt);
+      return new StoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, readPt);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
index 0fd3cdb..37d6b8f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java
@@ -96,7 +96,7 @@ public class TestHalfStoreFileReader {
 
     HFile.Reader r = HFile.createReader(fs, p, cacheConf, true, conf);
     r.loadFileInfo();
-    Cell midKV = r.midkey();
+    Cell midKV = r.midKey().get();
     byte[] midkey = CellUtil.cloneRow(midKV);
 
     // System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey));
@@ -155,7 +155,7 @@ public class TestHalfStoreFileReader {
 
     HFile.Reader r = HFile.createReader(fs, p, cacheConf, true, conf);
     r.loadFileInfo();
-    Cell midKV = r.midkey();
+    Cell midKV = r.midKey().get();
     byte[] midkey = CellUtil.cloneRow(midKV);
 
     Reference bottom = new Reference(midkey, Reference.Range.bottom);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index 62a7c48..13589fb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -164,8 +164,8 @@ public class TestHFile  {
     w.close();
     Reader r = HFile.createReader(fs, f, cacheConf, true, conf);
     r.loadFileInfo();
-    assertNull(r.getFirstKey());
-    assertNull(r.getLastKey());
+    assertFalse(r.getFirstKey().isPresent());
+    assertFalse(r.getLastKey().isPresent());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
index 3d1af90..82c0eca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
@@ -571,7 +571,7 @@ public class TestHFileBlockIndex {
    boolean hasArrayIndexOutOfBoundsException = false;
    try {
      // get the mid-key.
-     reader.midkey();
+     reader.midKey();
    } catch (ArrayIndexOutOfBoundsException e) {
      hasArrayIndexOutOfBoundsException = true;
    } finally {
@@ -649,8 +649,8 @@ public class TestHFileBlockIndex {
       assertEquals(expectedNumLevels,
           reader.getTrailer().getNumDataIndexLevels());
 
-      assertTrue(Bytes.equals(keys[0], ((KeyValue)reader.getFirstKey()).getKey()));
-      assertTrue(Bytes.equals(keys[NUM_KV - 1], ((KeyValue)reader.getLastKey()).getKey()));
+      assertTrue(Bytes.equals(keys[0], ((KeyValue)reader.getFirstKey().get()).getKey()));
+      assertTrue(Bytes.equals(keys[NUM_KV - 1], ((KeyValue)reader.getLastKey().get()).getKey()));
       LOG.info("Last key: " + Bytes.toStringBinary(keys[NUM_KV - 1]));
 
       for (boolean pread : new boolean[] { false, true }) {
@@ -706,7 +706,7 @@ public class TestHFileBlockIndex {
       // Validate the mid-key.
       assertEquals(
           Bytes.toStringBinary(blockKeys.get((blockKeys.size() - 1) / 2)),
-          reader.midkey());
+          reader.midKey());
 
       assertEquals(UNCOMPRESSED_INDEX_SIZES[testI],
           reader.getTrailer().getUncompressedDataIndexSize());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
index 5cc2580..5f5cb74 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java
@@ -186,8 +186,8 @@ public class TestHFileSeek extends TestCase {
     Reader reader = HFile.createReaderFromStream(path, fsdis,
         fs.getFileStatus(path).getLen(), new CacheConfig(conf), conf);
     reader.loadFileInfo();
-    KeySampler kSampler = new KeySampler(rng, ((KeyValue) reader.getFirstKey()).getKey(),
-        ((KeyValue) reader.getLastKey()).getKey(), keyLenGen);
+    KeySampler kSampler = new KeySampler(rng, ((KeyValue) reader.getFirstKey().get()).getKey(),
+        ((KeyValue) reader.getLastKey().get()).getKey(), keyLenGen);
     HFileScanner scanner = reader.getScanner(false, USE_PREAD);
     BytesWritable key = new BytesWritable();
     timer.reset();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
----------------------------------------------------------------------
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
index 12aed50..8f0c5d6 100644
--- 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
@@ -723,8 +723,8 @@ public class TestMobCompactor {
 
     @Override
     public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-        List<StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {
-
+        List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker)
+        throws IOException {
       int count = candidates.size();
       if (count >= 2) {
         for (int i = 0; i < count - 2; i++) {
@@ -815,7 +815,7 @@ public class TestMobCompactor {
       Assert.assertTrue(hasFiles);
       Path path = files[0].getPath();
       CacheConfig cacheConf = new CacheConfig(conf);
-      StoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf,
+      HStoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf,
         BloomType.NONE, true);
       sf.initReader();
       HFile.Reader reader = sf.getReader().getHFileReader();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
index b8e1204..6681a96 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.mob.compactions;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -71,7 +72,6 @@ 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.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
@@ -81,7 +81,6 @@ 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.Assert;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
@@ -494,12 +493,12 @@ public class TestPartitionedMobCompactor {
         PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
 
         // Make sure that there is no del Partitions
-        Assert.assertTrue(request.getDelPartitions().size() == 0);
+        assertTrue(request.getDelPartitions().size() == 0);
 
         // Make sure that when there is no startKey/endKey for partition.
         for (CompactionPartition p : request.getCompactionPartitions()) {
-          Assert.assertTrue(p.getStartKey() == null);
-          Assert.assertTrue(p.getEndKey() == null);
+          assertTrue(p.getStartKey() == null);
+          assertTrue(p.getEndKey() == null);
         }
         return null;
       }
@@ -530,18 +529,18 @@ public class TestPartitionedMobCompactor {
       }
       PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
 
-      Assert.assertTrue(request.getDelPartitions().size() == delPartitionSize);
+      assertTrue(request.getDelPartitions().size() == delPartitionSize);
       if (request.getDelPartitions().size() > 0) {
         for (CompactionPartition p : request.getCompactionPartitions()) {
-          Assert.assertTrue(p.getStartKey() != null);
-          Assert.assertTrue(p.getEndKey() != null);
+          assertTrue(p.getStartKey() != null);
+          assertTrue(p.getEndKey() != null);
         }
       }
 
       try {
         for (CompactionDelPartition delPartition : request.getDelPartitions()) {
           for (Path newDelPath : delPartition.listDelFiles()) {
-            StoreFile sf =
+            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.
@@ -553,11 +552,11 @@ public class TestPartitionedMobCompactor {
         // Make sure that CompactionDelPartitions does not overlap
         CompactionDelPartition prevDelP = null;
         for (CompactionDelPartition delP : request.getDelPartitions()) {
-          Assert.assertTrue(
+          assertTrue(
               Bytes.compareTo(delP.getId().getStartKey(), delP.getId().getEndKey()) <= 0);
 
           if (prevDelP != null) {
-            Assert.assertTrue(
+            assertTrue(
                 Bytes.compareTo(prevDelP.getId().getEndKey(), delP.getId().getStartKey()) < 0);
           }
         }
@@ -567,7 +566,7 @@ public class TestPartitionedMobCompactor {
         // 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<StoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
+          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(
@@ -575,23 +574,23 @@ public class TestPartitionedMobCompactor {
                     .getEndKey(), partition.getStartKey()) < 0))) {
 
               if (delFiles.size() > 0) {
-                Assert.assertTrue(delFiles.size() == 1);
+                assertTrue(delFiles.size() == 1);
                 affectedPartitions += delFiles.size();
-                Assert.assertTrue(Bytes.compareTo(partition.getStartKey(),
-                    CellUtil.cloneRow(delFiles.get(0).getLastKey())) <= 0);
-                Assert.assertTrue(Bytes.compareTo(partition.getEndKey(),
-                    CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey())) >= 0);
+                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
-        Assert.assertTrue(affectedPartitions == PartitionsIncludeDelFiles);
+        assertTrue(affectedPartitions == PartitionsIncludeDelFiles);
       } finally {
         for (CompactionDelPartition delPartition : request.getDelPartitions()) {
-          for (StoreFile storeFile : delPartition.getStoreFiles()) {
+          for (HStoreFile storeFile : delPartition.getStoreFiles()) {
             try {
-              storeFile.closeReader(true);
+              storeFile.closeStoreFile(true);
             } catch (IOException e) {
               LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
             }
@@ -679,19 +678,19 @@ public class TestPartitionedMobCompactor {
         // 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()) {
-            Assert.assertTrue(p.getStartKey() == null);
-            Assert.assertTrue(p.getEndKey() == null);
+            assertTrue(p.getStartKey() == null);
+            assertTrue(p.getEndKey() == null);
           }
         }
 
         // Make sure that CompactionDelPartitions does not overlap
         CompactionDelPartition prevDelP = null;
         for (CompactionDelPartition delP : request.getDelPartitions()) {
-          Assert.assertTrue(Bytes.compareTo(delP.getId().getStartKey(),
+          assertTrue(Bytes.compareTo(delP.getId().getStartKey(),
               delP.getId().getEndKey()) <= 0);
 
           if (prevDelP != null) {
-            Assert.assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(),
+            assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(),
                 delP.getId().getStartKey()) < 0);
           }
         }
@@ -699,25 +698,24 @@ public class TestPartitionedMobCompactor {
         // 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<StoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
+          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) {
-                Assert.assertTrue(Bytes
-                    .compareTo(partition.getStartKey(), delFiles.get(0).getFirstKey().getRowArray())
-                    >= 0);
-                Assert.assertTrue(Bytes.compareTo(partition.getEndKey(),
-                    delFiles.get(delFiles.size() - 1).getLastKey().getRowArray()) <= 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
-        Assert.assertEquals(type, request.type);
+        assertEquals(type, request.type);
         // assert get the right partitions
         compareCompactedPartitions(expected, request.compactionPartitions);
         // assert get the right del files
@@ -750,8 +748,8 @@ public class TestPartitionedMobCompactor {
         }
         List<Path> newDelPaths = compactDelFiles(request, delFilePaths);
         // assert the del files are merged.
-        Assert.assertEquals(expectedFileCount, newDelPaths.size());
-        Assert.assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths));
+        assertEquals(expectedFileCount, newDelPaths.size());
+        assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths));
         return null;
       }
     };
@@ -784,9 +782,9 @@ public class TestPartitionedMobCompactor {
     }
     Collections.sort(expected);
     Collections.sort(actualKeys);
-    Assert.assertEquals(expected.size(), actualKeys.size());
+    assertEquals(expected.size(), actualKeys.size());
     for (int i = 0; i < expected.size(); i++) {
-      Assert.assertEquals(expected.get(i), actualKeys.get(i));
+      assertEquals(expected.get(i), actualKeys.get(i));
     }
   }
 
@@ -802,7 +800,7 @@ public class TestPartitionedMobCompactor {
       }
     }
     for (Path f : delFiles) {
-      Assert.assertTrue(delMap.containsKey(f));
+      assertTrue(delMap.containsKey(f));
     }
   }
 
@@ -874,10 +872,10 @@ public class TestPartitionedMobCompactor {
    * @return the cell size
    */
   private int countDelCellsInDelFiles(List<Path> paths) throws IOException {
-    List<StoreFile> sfs = new ArrayList<>();
+    List<HStoreFile> sfs = new ArrayList<>();
     int size = 0;
     for (Path path : paths) {
-      StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
+      HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
       sfs.add(sf);
     }
     List<KeyValueScanner> scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
index 68c5d19..86df39f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java
@@ -38,16 +38,15 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.Predicate;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.junit.rules.TestName;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.HashMultimap;
@@ -435,10 +434,9 @@ public class SpaceQuotaHelperForTests {
     @Override
     public boolean evaluate() throws Exception {
       for (HRegion region : cluster.getRegions(tn)) {
-        for (Store store : region.getStores()) {
-          HStore hstore = (HStore) store;
-          Collection<StoreFile> files =
-              hstore.getStoreEngine().getStoreFileManager().getCompactedfiles();
+        for (HStore store : region.getStores()) {
+          Collection<HStoreFile> files =
+              store.getStoreEngine().getStoreFileManager().getCompactedfiles();
           if (null != files && !files.isEmpty()) {
             LOG.debug(region.getRegionInfo().getEncodedName() + " still has compacted files");
             return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java
index b33b45d..58691c4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java
@@ -19,9 +19,6 @@ package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -32,9 +29,12 @@ import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequ
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
 public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy {
 
-  protected ArrayList<StoreFile> sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes)
+  protected ArrayList<HStoreFile> sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes)
       throws IOException {
     ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
     EnvironmentEdgeManager.injectEdge(timeMachine);
@@ -45,17 +45,17 @@ public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy
       ageInDisk.add(0L);
     }
 
-    ArrayList<StoreFile> ret = Lists.newArrayList();
+    ArrayList<HStoreFile> ret = Lists.newArrayList();
     for (int i = 0; i < sizes.length; i++) {
-      MockStoreFile msf =
-          new MockStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i);
+      MockHStoreFile msf =
+          new MockHStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i);
       msf.setTimeRangeTracker(new TimeRangeTracker(minTimestamps[i], maxTimestamps[i]));
       ret.add(msf);
     }
     return ret;
   }
 
-  protected void compactEquals(long now, ArrayList<StoreFile> candidates, long[] expectedFileSizes,
+  protected void compactEquals(long now, ArrayList<HStoreFile> candidates, long[] expectedFileSizes,
       long[] expectedBoundaries, boolean isMajor, boolean toCompact) throws IOException {
     ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
     EnvironmentEdgeManager.injectEdge(timeMachine);
@@ -64,17 +64,17 @@ public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy
     DateTieredCompactionPolicy policy =
         (DateTieredCompactionPolicy) store.storeEngine.getCompactionPolicy();
     if (isMajor) {
-      for (StoreFile file : candidates) {
-        ((MockStoreFile) file).setIsMajor(true);
+      for (HStoreFile file : candidates) {
+        ((MockHStoreFile) file).setIsMajor(true);
       }
       assertEquals(toCompact, policy.shouldPerformMajorCompaction(candidates));
       request = (DateTieredCompactionRequest) policy.selectMajorCompaction(candidates);
     } else {
-      assertEquals(toCompact, policy.needsCompaction(candidates, ImmutableList.<StoreFile> of()));
+      assertEquals(toCompact, policy.needsCompaction(candidates, ImmutableList.of()));
       request =
           (DateTieredCompactionRequest) policy.selectMinorCompaction(candidates, false, false);
     }
-    List<StoreFile> actual = Lists.newArrayList(request.getFiles());
+    List<HStoreFile> actual = Lists.newArrayList(request.getFiles());
     assertEquals(Arrays.toString(expectedFileSizes), Arrays.toString(getSizes(actual)));
     assertEquals(Arrays.toString(expectedBoundaries),
       Arrays.toString(request.getBoundaries().toArray()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
index 2635e2d..5f85826 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
@@ -592,7 +592,7 @@ public class DataBlockEncodingTool {
     Path path = new Path(hfilePath);
     CacheConfig cacheConf = new CacheConfig(conf);
     FileSystem fs = FileSystem.get(conf);
-    StoreFile hsf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
+    HStoreFile hsf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
     hsf.initReader();
     StoreFileReader reader = hsf.getReader();
     reader.loadFileInfo();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
index bde0934..82e1755 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
@@ -59,7 +59,7 @@ public class EncodedSeekPerformanceTest {
     List<Cell> allKeyValues = new ArrayList<>();
 
     // read all of the key values
-    StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
+    HStoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
         path, configuration, cacheConf, BloomType.NONE, true);
     storeFile.initReader();
     StoreFileReader reader = storeFile.getReader();
@@ -71,7 +71,7 @@ public class EncodedSeekPerformanceTest {
       allKeyValues.add(current);
     }
 
-    storeFile.closeReader(cacheConf.shouldEvictOnClose());
+    storeFile.closeStoreFile(cacheConf.shouldEvictOnClose());
 
     // pick seeks by random
     List<Cell> seeks = new ArrayList<>();
@@ -89,7 +89,7 @@ public class EncodedSeekPerformanceTest {
   private void runTest(Path path, DataBlockEncoding blockEncoding,
       List<Cell> seeks) throws IOException {
     // read all of the key values
-    StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
+    HStoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
       path, configuration, cacheConf, BloomType.NONE, true);
     storeFile.initReader();
     long totalSize = 0;
@@ -132,7 +132,7 @@ public class EncodedSeekPerformanceTest {
     double seeksPerSec = (seeks.size() * NANOSEC_IN_SEC) /
         (finishSeeksTime - startSeeksTime);
 
-    storeFile.closeReader(cacheConf.shouldEvictOnClose());
+    storeFile.closeStoreFile(cacheConf.shouldEvictOnClose());
     clearBlockCache();
 
     System.out.println(blockEncoding);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockHStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockHStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockHStoreFile.java
new file mode 100644
index 0000000..78b1ef6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockHStoreFile.java
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.TreeMap;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HDFSBlocksDistribution;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/** A mock used so our tests don't deal with actual StoreFiles */
+@InterfaceAudience.Private
+public class MockHStoreFile extends HStoreFile {
+  long length = 0;
+  boolean isRef = false;
+  long ageInDisk;
+  long sequenceid;
+  private Map<byte[], byte[]> metadata = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+  byte[] splitPoint = null;
+  TimeRangeTracker timeRangeTracker;
+  long entryCount;
+  boolean isMajor;
+  HDFSBlocksDistribution hdfsBlocksDistribution;
+  long modificationTime;
+  boolean compactedAway;
+
+  MockHStoreFile(HBaseTestingUtility testUtil, Path testPath,
+      long length, long ageInDisk, boolean isRef, long sequenceid) throws IOException {
+    super(testUtil.getTestFileSystem(), testPath, testUtil.getConfiguration(),
+        new CacheConfig(testUtil.getConfiguration()), BloomType.NONE, true);
+    this.length = length;
+    this.isRef = isRef;
+    this.ageInDisk = ageInDisk;
+    this.sequenceid = sequenceid;
+    this.isMajor = false;
+    hdfsBlocksDistribution = new HDFSBlocksDistribution();
+    hdfsBlocksDistribution.addHostsAndBlockWeight(
+      new String[] { RSRpcServices.getHostname(testUtil.getConfiguration(), false) }, 1);
+    modificationTime = EnvironmentEdgeManager.currentTime();
+  }
+
+  void setLength(long newLen) {
+    this.length = newLen;
+  }
+
+  @Override
+  public long getMaxSequenceId() {
+    return sequenceid;
+  }
+
+  @Override
+  public boolean isMajorCompactionResult() {
+    return isMajor;
+  }
+
+  public void setIsMajor(boolean isMajor) {
+    this.isMajor = isMajor;
+  }
+
+  @Override
+  public boolean isReference() {
+    return this.isRef;
+  }
+
+  @Override
+  public boolean isBulkLoadResult() {
+    return false;
+  }
+
+  @Override
+  public byte[] getMetadataValue(byte[] key) {
+    return this.metadata.get(key);
+  }
+
+  public void setMetadataValue(byte[] key, byte[] value) {
+    this.metadata.put(key, value);
+  }
+
+  void setTimeRangeTracker(TimeRangeTracker timeRangeTracker) {
+    this.timeRangeTracker = timeRangeTracker;
+  }
+
+  void setEntries(long entryCount) {
+    this.entryCount = entryCount;
+  }
+
+  public OptionalLong getMinimumTimestamp() {
+    return timeRangeTracker == null ? OptionalLong.empty()
+        : OptionalLong.of(timeRangeTracker.getMin());
+  }
+
+  public OptionalLong getMaximumTimestamp() {
+    return timeRangeTracker == null ? OptionalLong.empty()
+        : OptionalLong.of(timeRangeTracker.getMax());
+  }
+
+  @Override
+  public void markCompactedAway() {
+    this.compactedAway = true;
+  }
+
+  @Override
+  public boolean isCompactedAway() {
+    return compactedAway;
+  }
+
+  @Override
+  public long getModificationTimeStamp() {
+    return modificationTime;
+  }
+
+  @Override
+  public HDFSBlocksDistribution getHDFSBlockDistribution() {
+    return hdfsBlocksDistribution;
+  }
+
+  @Override
+  public void initReader() throws IOException {
+  }
+
+  @Override
+  public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
+      boolean canOptimizeForNonNullColumn) {
+    return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder,
+      canOptimizeForNonNullColumn);
+  }
+
+  @Override
+  public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
+      boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
+      throws IOException {
+    return getReader().getStoreFileScanner(cacheBlocks, false, isCompaction, readPt, scannerOrder,
+      canOptimizeForNonNullColumn);
+  }
+
+  @Override
+  public StoreFileReader getReader() {
+    final long len = this.length;
+    final TimeRangeTracker timeRangeTracker = this.timeRangeTracker;
+    final long entries = this.entryCount;
+    return new StoreFileReader() {
+      @Override
+      public long length() {
+        return len;
+      }
+
+      @Override
+      public long getMaxTimestamp() {
+        return timeRange == null? Long.MAX_VALUE: timeRangeTracker.getMax();
+      }
+
+      @Override
+      public long getEntries() {
+        return entries;
+      }
+
+      @Override
+      public void close(boolean evictOnClose) throws IOException {
+        // no-op
+      }
+
+      @Override
+      public Optional<Cell> getLastKey() {
+        if (splitPoint != null) {
+          return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
+              .setType(KeyValue.Type.Put.getCode())
+              .setRow(Arrays.copyOf(splitPoint, splitPoint.length + 1)).build());
+        } else {
+          return Optional.empty();
+        }
+      }
+
+      @Override
+      public Optional<Cell> midKey() throws IOException {
+        if (splitPoint != null) {
+          return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
+              .setType(KeyValue.Type.Put.getCode()).setRow(splitPoint).build());
+        } else {
+          return Optional.empty();
+        }
+      }
+
+      @Override
+      public Optional<Cell> getFirstKey() {
+        if (splitPoint != null) {
+          return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
+              .setType(KeyValue.Type.Put.getCode()).setRow(splitPoint, 0, splitPoint.length - 1)
+              .build());
+        } else {
+          return Optional.empty();
+        }
+      }
+    };
+  }
+
+  @Override
+  public OptionalLong getBulkLoadTimestamp() {
+    // we always return false for isBulkLoadResult so we do not have a bulk load timestamp
+    return OptionalLong.empty();
+  }
+}


[08/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
index b7e83bf..6cece0f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java
@@ -28,8 +28,6 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
@@ -40,10 +38,10 @@ import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 /**
  * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or
@@ -63,9 +61,9 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
   // General Accessors
   CellComparator getComparator();
 
-  Collection<StoreFile> getStorefiles();
+  Collection<? extends StoreFile> getStorefiles();
 
-  Collection<StoreFile> getCompactedFiles();
+  Collection<? extends StoreFile> getCompactedFiles();
 
   /**
    * Close all the readers We don't need to worry about subsequent requests because the Region
@@ -73,7 +71,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
    * @return the {@link StoreFile StoreFiles} that were previously being used.
    * @throws IOException on failure
    */
-  Collection<StoreFile> close() throws IOException;
+  Collection<? extends StoreFile> close() throws IOException;
 
   /**
    * Return a scanner for both the memstore and the HStore files. Assumes we are not in a
@@ -86,105 +84,6 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
   KeyValueScanner getScanner(Scan scan, final NavigableSet<byte[]> targetCols, long readPt)
       throws IOException;
 
-  /**
-   * Get all scanners with no filtering based on TTL (that happens further down the line).
-   * @param cacheBlocks cache the blocks or not
-   * @param usePread true to use pread, false if not
-   * @param isCompaction true if the scanner is created for compaction
-   * @param matcher the scan query matcher
-   * @param startRow the start row
-   * @param stopRow the stop row
-   * @param readPt the read point of the current scan
-   * @return all scanners for this store
-   */
-  default List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet, boolean usePread,
-      boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt)
-      throws IOException {
-    return getScanners(cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, false,
-      readPt);
-  }
-
-  /**
-   * Get all scanners with no filtering based on TTL (that happens further down the line).
-   * @param cacheBlocks cache the blocks or not
-   * @param usePread true to use pread, false if not
-   * @param isCompaction true if the scanner is created for compaction
-   * @param matcher the scan query matcher
-   * @param startRow the start row
-   * @param includeStartRow true to include start row, false if not
-   * @param stopRow the stop row
-   * @param includeStopRow true to include stop row, false if not
-   * @param readPt the read point of the current scan
-   * @return all scanners for this store
-   */
-  List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean usePread, boolean isCompaction,
-      ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow,
-      boolean includeStopRow, long readPt) throws IOException;
-
-  /**
-   * Recreates the scanners on the current list of active store file scanners
-   * @param currentFileScanners the current set of active store file scanners
-   * @param cacheBlocks cache the blocks or not
-   * @param usePread use pread or not
-   * @param isCompaction is the scanner for compaction
-   * @param matcher the scan query matcher
-   * @param startRow the scan's start row
-   * @param includeStartRow should the scan include the start row
-   * @param stopRow the scan's stop row
-   * @param includeStopRow should the scan include the stop row
-   * @param readPt the read point of the current scane
-   * @param includeMemstoreScanner whether the current scanner should include memstorescanner
-   * @return list of scanners recreated on the current Scanners
-   * @throws IOException
-   */
-  List<KeyValueScanner> recreateScanners(List<KeyValueScanner> currentFileScanners,
-      boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
-      byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
-      boolean includeMemstoreScanner) throws IOException;
-
-  /**
-   * Create scanners on the given files and if needed on the memstore with no filtering based on TTL
-   * (that happens further down the line).
-   * @param files the list of files on which the scanners has to be created
-   * @param cacheBlocks cache the blocks or not
-   * @param usePread true to use pread, false if not
-   * @param isCompaction true if the scanner is created for compaction
-   * @param matcher the scan query matcher
-   * @param startRow the start row
-   * @param stopRow the stop row
-   * @param readPt the read point of the current scan
-   * @param includeMemstoreScanner true if memstore has to be included
-   * @return scanners on the given files and on the memstore if specified
-   */
-  default List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks,
-      boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
-      byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner)
-      throws IOException {
-    return getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow,
-      false, readPt, includeMemstoreScanner);
-  }
-
-  /**
-   * Create scanners on the given files and if needed on the memstore with no filtering based on TTL
-   * (that happens further down the line).
-   * @param files the list of files on which the scanners has to be created
-   * @param cacheBlocks ache the blocks or not
-   * @param usePread true to use pread, false if not
-   * @param isCompaction true if the scanner is created for compaction
-   * @param matcher the scan query matcher
-   * @param startRow the start row
-   * @param includeStartRow true to include start row, false if not
-   * @param stopRow the stop row
-   * @param includeStopRow true to include stop row, false if not
-   * @param readPt the read point of the current scan
-   * @param includeMemstoreScanner true if memstore has to be included
-   * @return scanners on the given files and on the memstore if specified
-   */
-  List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks, boolean usePread,
-      boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow,
-      byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner)
-      throws IOException;
-
   ScanInfo getScanInfo();
 
   /**
@@ -194,7 +93,6 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
 
   FileSystem getFileSystem();
 
-
   /**
    * @param maxKeyCount
    * @param compression Compression algorithm to use
@@ -269,10 +167,10 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
    * @deprecated see compact(CompactionContext, ThroughputController, User)
    */
   @Deprecated
-  List<StoreFile> compact(CompactionContext compaction,
+  List<? extends StoreFile> compact(CompactionContext compaction,
       ThroughputController throughputController) throws IOException;
 
-  List<StoreFile> compact(CompactionContext compaction,
+  List<? extends StoreFile> compact(CompactionContext compaction,
     ThroughputController throughputController, User user) throws IOException;
 
   /**
@@ -297,10 +195,9 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
   boolean canSplit();
 
   /**
-   * Determines if Store should be split
-   * @return byte[] if store should be split, null otherwise.
+   * Determines if Store should be split.
    */
-  byte[] getSplitPoint();
+  Optional<byte[]> getSplitPoint();
 
   // General accessors into the state of the store
   // TODO abstract some of this out into a metrics class

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
index fb5f0e4..60b3c3d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java
@@ -22,13 +22,13 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy;
 import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * StoreEngine is a factory that can create the objects necessary for HStore to operate.
@@ -84,7 +84,7 @@ public abstract class StoreEngine<SF extends StoreFlusher,
    * @param filesCompacting Files currently compacting
    * @return whether a compaction selection is possible
    */
-  public abstract boolean needsCompaction(List<StoreFile> filesCompacting);
+  public abstract boolean needsCompaction(List<HStoreFile> filesCompacting);
 
   /**
    * Creates an instance of a compaction context specific to this engine.
@@ -97,13 +97,13 @@ public abstract class StoreEngine<SF extends StoreFlusher,
    * Create the StoreEngine's components.
    */
   protected abstract void createComponents(
-      Configuration conf, Store store, CellComparator kvComparator) throws IOException;
+      Configuration conf, HStore store, CellComparator cellComparator) throws IOException;
 
   private void createComponentsOnce(
-      Configuration conf, Store store, CellComparator kvComparator) throws IOException {
+      Configuration conf, HStore store, CellComparator cellComparator) throws IOException {
     assert compactor == null && compactionPolicy == null
         && storeFileManager == null && storeFlusher == null;
-    createComponents(conf, store, kvComparator);
+    createComponents(conf, store, cellComparator);
     assert compactor != null && compactionPolicy != null
         && storeFileManager != null && storeFlusher != null;
   }
@@ -113,16 +113,16 @@ public abstract class StoreEngine<SF extends StoreFlusher,
    * @param store The store. An unfortunate dependency needed due to it
    *              being passed to coprocessors via the compactor.
    * @param conf Store configuration.
-   * @param kvComparator KVComparator for storeFileManager.
+   * @param cellComparator CellComparator for storeFileManager.
    * @return StoreEngine to use.
    */
   public static StoreEngine<?, ?, ?, ?> create(
-      Store store, Configuration conf, CellComparator kvComparator) throws IOException {
+      HStore store, Configuration conf, CellComparator cellComparator) throws IOException {
     String className = conf.get(STORE_ENGINE_CLASS_KEY, DEFAULT_STORE_ENGINE_CLASS.getName());
     try {
       StoreEngine<?,?,?,?> se = ReflectionUtils.instantiateWithCustomCtor(
           className, new Class[] { }, new Object[] { });
-      se.createComponentsOnce(conf, store, kvComparator);
+      se.createComponentsOnce(conf, store, cellComparator);
       return se;
     } catch (Exception e) {
       throw new IOException("Unable to load configured store engine '" + className + "'", e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
index d5e51ed..0097bd7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.Comparator;
+import java.util.Optional;
 import java.util.OptionalLong;
 
 import org.apache.hadoop.fs.Path;
@@ -27,73 +28,38 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * An interface to describe a store data file.
+ * <p>
+ * <strong>NOTICE: </strong>this interface is mainly designed for coprocessor, so it will not expose
+ * all the internal APIs for a 'store file'. If you are implementing something inside HBase, i.e,
+ * not a coprocessor hook, usually you should use {@link HStoreFile} directly as it is the only
+ * implementation of this interface.
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 @InterfaceStability.Evolving
 public interface StoreFile {
 
-  static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
-
-  // Keys for fileinfo values in HFile
-
-  /** Max Sequence ID in FileInfo */
-  static final byte[] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
-
-  /** Major compaction flag in FileInfo */
-  static final byte[] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
-
-  /** Minor compaction flag in FileInfo */
-  static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
-      Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
-
-  /** Bloom filter Type in FileInfo */
-  static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
-
-  /** Delete Family Count in FileInfo */
-  static final byte[] DELETE_FAMILY_COUNT = Bytes.toBytes("DELETE_FAMILY_COUNT");
-
-  /** Last Bloom filter key in FileInfo */
-  static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
-
-  /** Key for Timerange information in metadata */
-  static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
-
-  /** Key for timestamp of earliest-put in metadata */
-  static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
-
-  /** Key for the number of mob cells in metadata */
-  static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
-
-  /** Meta key set when store file is a result of a bulk load */
-  static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK");
-  static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP");
-
   /**
-   * Key for skipping resetting sequence id in metadata. For bulk loaded hfiles, the scanner resets
-   * the cell seqId with the latest one, if this metadata is set as true, the reset is skipped.
+   * Get the first key in this store file.
    */
-  static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
-
-  CacheConfig getCacheConf();
+  Optional<Cell> getFirstKey();
 
-  Cell getFirstKey();
-
-  Cell getLastKey();
+  /**
+   * Get the last key in this store file.
+   */
+  Optional<Cell> getLastKey();
 
+  /**
+   * Get the comparator for comparing two cells.
+   */
   Comparator<Cell> getComparator();
 
-  long getMaxMemstoreTS();
-
   /**
-   * @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a
-   *         reference.
+   * Get max of the MemstoreTS in the KV's in this store file.
    */
-  StoreFileInfo getFileInfo();
+  long getMaxMemstoreTS();
 
   /**
    * @return Path or null if this StoreFile was made with a Stream.
@@ -130,14 +96,11 @@ public interface StoreFile {
    */
   long getMaxSequenceId();
 
-  long getModificationTimeStamp() throws IOException;
-
   /**
-   * Only used by the Striped Compaction Policy
-   * @param key
-   * @return value associated with the metadata key
+   * Get the modification time of this store file. Usually will access the file system so throws
+   * IOException.
    */
-  byte[] getMetadataValue(byte[] key);
+  long getModificationTimeStamp() throws IOException;
 
   /**
    * Check if this storefile was created by bulk load. When a hfile is bulk loaded into HBase, we
@@ -149,13 +112,6 @@ public interface StoreFile {
    */
   boolean isBulkLoadResult();
 
-  boolean isCompactedAway();
-
-  /**
-   * @return true if the file is still used in reads
-   */
-  boolean isReferencedInReads();
-
   /**
    * Return the timestamp at which this bulk load file was generated.
    */
@@ -168,49 +124,17 @@ public interface StoreFile {
   HDFSBlocksDistribution getHDFSBlockDistribution();
 
   /**
-   * Initialize the reader used for pread.
-   */
-  void initReader() throws IOException;
-
-  /**
-   * Must be called after initReader.
-   */
-  StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
-      boolean canOptimizeForNonNullColumn);
-
-  StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
-      boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
-      throws IOException;
-
-  /**
-   * @return Current reader. Must call initReader first else returns null.
-   * @see #initReader()
-   */
-  StoreFileReader getReader();
-
-  /**
-   * @param evictOnClose whether to evict blocks belonging to this file
-   * @throws IOException
-   */
-  void closeReader(boolean evictOnClose) throws IOException;
-
-  /**
-   * Marks the status of the file as compactedAway.
+   * @return a length description of this StoreFile, suitable for debug output
    */
-  void markCompactedAway();
+  String toStringDetailed();
 
   /**
-   * Delete this file
-   * @throws IOException
+   * Get the min timestamp of all the cells in the store file.
    */
-  void deleteReader() throws IOException;
+  OptionalLong getMinimumTimestamp();
 
   /**
-   * @return a length description of this StoreFile, suitable for debug output
+   * Get the max timestamp of all the cells in the store file.
    */
-  String toStringDetailed();
-
-  OptionalLong getMinimumTimestamp();
-
   OptionalLong getMaximumTimestamp();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java
index e8ec9fd..cd265e2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java
@@ -26,17 +26,17 @@ import java.util.Comparator;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Useful comparators for comparing StoreFiles.
+ * Useful comparators for comparing store files.
  */
 @InterfaceAudience.Private
 final class StoreFileComparators {
   /**
-   * Comparator that compares based on the Sequence Ids of the the StoreFiles. Bulk loads that did
+   * Comparator that compares based on the Sequence Ids of the the store files. Bulk loads that did
    * not request a seq ID are given a seq id of -1; thus, they are placed before all non- bulk
    * loads, and bulk loads with sequence Id. Among these files, the size is used to determine the
    * ordering, then bulkLoadTime. If there are ties, the path name is used as a tie-breaker.
    */
-  public static final Comparator<StoreFile> SEQ_ID =
+  public static final Comparator<HStoreFile> SEQ_ID =
       Ordering.compound(ImmutableList.of(Ordering.natural().onResultOf(new GetSeqId()),
         Ordering.natural().onResultOf(new GetFileSize()).reverse(),
         Ordering.natural().onResultOf(new GetBulkTime()),
@@ -46,23 +46,23 @@ final class StoreFileComparators {
    * Comparator for time-aware compaction. SeqId is still the first ordering criterion to maintain
    * MVCC.
    */
-  public static final Comparator<StoreFile> SEQ_ID_MAX_TIMESTAMP =
+  public static final Comparator<HStoreFile> SEQ_ID_MAX_TIMESTAMP =
       Ordering.compound(ImmutableList.of(Ordering.natural().onResultOf(new GetSeqId()),
         Ordering.natural().onResultOf(new GetMaxTimestamp()),
         Ordering.natural().onResultOf(new GetFileSize()).reverse(),
         Ordering.natural().onResultOf(new GetBulkTime()),
         Ordering.natural().onResultOf(new GetPathName())));
 
-  private static class GetSeqId implements Function<StoreFile, Long> {
+  private static class GetSeqId implements Function<HStoreFile, Long> {
     @Override
-    public Long apply(StoreFile sf) {
+    public Long apply(HStoreFile sf) {
       return sf.getMaxSequenceId();
     }
   }
 
-  private static class GetFileSize implements Function<StoreFile, Long> {
+  private static class GetFileSize implements Function<HStoreFile, Long> {
     @Override
-    public Long apply(StoreFile sf) {
+    public Long apply(HStoreFile sf) {
       if (sf.getReader() != null) {
         return sf.getReader().length();
       } else {
@@ -73,23 +73,23 @@ final class StoreFileComparators {
     }
   }
 
-  private static class GetBulkTime implements Function<StoreFile, Long> {
+  private static class GetBulkTime implements Function<HStoreFile, Long> {
     @Override
-    public Long apply(StoreFile sf) {
+    public Long apply(HStoreFile sf) {
       return sf.getBulkLoadTimestamp().orElse(Long.MAX_VALUE);
     }
   }
 
-  private static class GetPathName implements Function<StoreFile, String> {
+  private static class GetPathName implements Function<HStoreFile, String> {
     @Override
-    public String apply(StoreFile sf) {
+    public String apply(HStoreFile sf) {
       return sf.getPath().getName();
     }
   }
 
-  private static class GetMaxTimestamp implements Function<StoreFile, Long> {
+  private static class GetMaxTimestamp implements Function<HStoreFile, Long> {
     @Override
-    public Long apply(StoreFile sf) {
+    public Long apply(HStoreFile sf) {
       return sf.getMaximumTimestamp().orElse(Long.MAX_VALUE);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
index c774080..67ef4de 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java
@@ -23,13 +23,14 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
+import java.util.Optional;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
+
 /**
  * Manages the store files and basic metadata about that that determines the logical structure
  * (e.g. what files to return for scan, how to determine split point, and such).
@@ -45,13 +46,13 @@ public interface StoreFileManager {
    * Loads the initial store files into empty StoreFileManager.
    * @param storeFiles The files to load.
    */
-  void loadFiles(List<StoreFile> storeFiles);
+  void loadFiles(List<HStoreFile> storeFiles);
 
   /**
    * Adds new files, either for from MemStore flush or bulk insert, into the structure.
    * @param sfs New store files.
    */
-  void insertNewFiles(Collection<StoreFile> sfs) throws IOException;
+  void insertNewFiles(Collection<HStoreFile> sfs) throws IOException;
 
   /**
    * Adds only the new compaction results into the structure.
@@ -59,34 +60,34 @@ public interface StoreFileManager {
    * @param results The resulting files for the compaction.
    */
   void addCompactionResults(
-      Collection<StoreFile> compactedFiles, Collection<StoreFile> results) throws IOException;
+      Collection<HStoreFile> compactedFiles, Collection<HStoreFile> results) throws IOException;
 
   /**
    * Remove the compacted files
    * @param compactedFiles the list of compacted files
    * @throws IOException
    */
-  void removeCompactedFiles(Collection<StoreFile> compactedFiles) throws IOException;
+  void removeCompactedFiles(Collection<HStoreFile> compactedFiles) throws IOException;
 
   /**
    * Clears all the files currently in use and returns them.
    * @return The files previously in use.
    */
-  ImmutableCollection<StoreFile> clearFiles();
+  ImmutableCollection<HStoreFile> clearFiles();
 
   /**
    * Clears all the compacted files and returns them. This method is expected to be
    * accessed single threaded.
    * @return The files compacted previously.
    */
-  Collection<StoreFile> clearCompactedFiles();
+  Collection<HStoreFile> clearCompactedFiles();
 
   /**
    * Gets the snapshot of the store files currently in use. Can be used for things like metrics
    * and checks; should not assume anything about relations between store files in the list.
    * @return The list of StoreFiles.
    */
-  Collection<StoreFile> getStorefiles();
+  Collection<HStoreFile> getStorefiles();
 
   /**
    * List of compacted files inside this store that needs to be excluded in reads
@@ -95,7 +96,7 @@ public interface StoreFileManager {
    * compacted files are done.
    * @return the list of compacted files
    */
-  Collection<StoreFile> getCompactedfiles();
+  Collection<HStoreFile> getCompactedfiles();
 
   /**
    * Returns the number of files currently in use.
@@ -115,7 +116,7 @@ public interface StoreFileManager {
    * @param stopRow Stop row of the request.
    * @return The list of files that are to be read for this request.
    */
-  Collection<StoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow, byte[] stopRow,
+  Collection<HStoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow, byte[] stopRow,
       boolean includeStopRow);
 
   /**
@@ -124,9 +125,7 @@ public interface StoreFileManager {
    * @return The files that may have the key less than or equal to targetKey, in reverse
    *         order of new-ness, and preference for target key.
    */
-  Iterator<StoreFile> getCandidateFilesForRowKeyBefore(
-    KeyValue targetKey
-  );
+  Iterator<HStoreFile> getCandidateFilesForRowKeyBefore(KeyValue targetKey);
 
   /**
    * Updates the candidate list for finding row key before. Based on the list of candidates
@@ -139,17 +138,16 @@ public interface StoreFileManager {
    * @param candidate The current best candidate found.
    * @return The list to replace candidateFiles.
    */
-  Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
-    Iterator<StoreFile> candidateFiles, KeyValue targetKey, Cell candidate
-  );
+  Iterator<HStoreFile> updateCandidateFilesForRowKeyBefore(Iterator<HStoreFile> candidateFiles,
+      KeyValue targetKey, Cell candidate);
 
 
   /**
    * Gets the split point for the split of this set of store files (approx. middle).
-   * @return The mid-point, or null if no split is possible.
+   * @return The mid-point if possible.
    * @throws IOException
    */
-  byte[] getSplitPoint() throws IOException;
+  Optional<byte[]> getSplitPoint() throws IOException;
 
   /**
    * @return The store compaction priority.
@@ -161,7 +159,7 @@ public interface StoreFileManager {
    * @param filesCompacting Files that are currently compacting.
    * @return The files which don't have any necessary data according to TTL and other criteria.
    */
-  Collection<StoreFile> getUnneededFiles(long maxTs, List<StoreFile> filesCompacting);
+  Collection<HStoreFile> getUnneededFiles(long maxTs, List<HStoreFile> filesCompacting);
 
   /**
    * @return the compaction pressure used for compaction throughput tuning.
@@ -171,7 +169,7 @@ public interface StoreFileManager {
 
   /**
    * @return the comparator used to sort storefiles. Usually, the
-   *         {@link StoreFile#getMaxSequenceId()} is the first priority.
+   *         {@link HStoreFile#getMaxSequenceId()} is the first priority.
    */
-  Comparator<StoreFile> getStoreFileComparator();
+  Comparator<HStoreFile> getStoreFileComparator();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
index e3f97a2..67b8fbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -18,9 +17,14 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY;
+
 import java.io.DataInput;
 import java.io.IOException;
 import java.util.Map;
+import java.util.Optional;
 import java.util.SortedSet;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -414,40 +418,40 @@ public class StoreFileReader {
    * @return true if there is overlap, false otherwise
    */
   public boolean passesKeyRangeFilter(Scan scan) {
-    if (this.getFirstKey() == null || this.getLastKey() == null) {
+    Optional<Cell> firstKeyKV = this.getFirstKey();
+    Optional<Cell> lastKeyKV = this.getLastKey();
+    if (!firstKeyKV.isPresent() || !lastKeyKV.isPresent()) {
       // the file is empty
       return false;
     }
-    if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW)
-        && Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
+    if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW) &&
+        Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
       return true;
     }
     byte[] smallestScanRow = scan.isReversed() ? scan.getStopRow() : scan.getStartRow();
     byte[] largestScanRow = scan.isReversed() ? scan.getStartRow() : scan.getStopRow();
-    Cell firstKeyKV = this.getFirstKey();
-    Cell lastKeyKV = this.getLastKey();
-    boolean nonOverLapping = (getComparator().compareRows(firstKeyKV,
-        largestScanRow, 0, largestScanRow.length) > 0
-        && !Bytes
-        .equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),
-            HConstants.EMPTY_END_ROW))
-        || getComparator().compareRows(lastKeyKV, smallestScanRow, 0, smallestScanRow.length) < 0;
+    boolean nonOverLapping = (getComparator()
+        .compareRows(firstKeyKV.get(), largestScanRow, 0, largestScanRow.length) > 0 &&
+        !Bytes.equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),
+          HConstants.EMPTY_END_ROW)) ||
+        getComparator().compareRows(lastKeyKV.get(), smallestScanRow, 0,
+          smallestScanRow.length) < 0;
     return !nonOverLapping;
   }
 
   public Map<byte[], byte[]> loadFileInfo() throws IOException {
     Map<byte [], byte []> fi = reader.loadFileInfo();
 
-    byte[] b = fi.get(StoreFile.BLOOM_FILTER_TYPE_KEY);
+    byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
     if (b != null) {
       bloomFilterType = BloomType.valueOf(Bytes.toString(b));
     }
 
-    lastBloomKey = fi.get(StoreFile.LAST_BLOOM_KEY);
+    lastBloomKey = fi.get(LAST_BLOOM_KEY);
     if(bloomFilterType == BloomType.ROWCOL) {
       lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length);
     }
-    byte[] cnt = fi.get(StoreFile.DELETE_FAMILY_COUNT);
+    byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
     if (cnt != null) {
       deleteFamilyCnt = Bytes.toLong(cnt);
     }
@@ -537,16 +541,16 @@ public class StoreFileReader {
     this.deleteFamilyBloomFilter = null;
   }
 
-  public Cell getLastKey() {
+  public Optional<Cell> getLastKey() {
     return reader.getLastKey();
   }
 
-  public byte[] getLastRowKey() {
+  public Optional<byte[]> getLastRowKey() {
     return reader.getLastRowKey();
   }
 
-  public Cell midkey() throws IOException {
-    return reader.midkey();
+  public Optional<Cell> midKey() throws IOException {
+    return reader.midKey();
   }
 
   public long length() {
@@ -565,7 +569,7 @@ public class StoreFileReader {
     return deleteFamilyCnt;
   }
 
-  public Cell getFirstKey() {
+  public Optional<Cell> getFirstKey() {
     return reader.getFirstKey();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
index 08111dc..f21b30b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Optional;
 import java.util.PriorityQueue;
 import java.util.concurrent.atomic.LongAdder;
 
@@ -102,7 +103,7 @@ public class StoreFileScanner implements KeyValueScanner {
   /**
    * Return an array of scanners corresponding to the given set of store files.
    */
-  public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
+  public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files,
       boolean cacheBlocks, boolean usePread, long readPt) throws IOException {
     return getScannersForStoreFiles(files, cacheBlocks, usePread, false, false, readPt);
   }
@@ -110,7 +111,7 @@ public class StoreFileScanner implements KeyValueScanner {
   /**
    * Return an array of scanners corresponding to the given set of store files.
    */
-  public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
+  public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files,
       boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean useDropBehind,
       long readPt) throws IOException {
     return getScannersForStoreFiles(files, cacheBlocks, usePread, isCompaction, useDropBehind, null,
@@ -121,7 +122,7 @@ public class StoreFileScanner implements KeyValueScanner {
    * Return an array of scanners corresponding to the given set of store files, And set the
    * ScanQueryMatcher for each store file scanner for further optimization
    */
-  public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
+  public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files,
       boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop,
       ScanQueryMatcher matcher, long readPt) throws IOException {
     if (files.isEmpty()) {
@@ -129,15 +130,15 @@ public class StoreFileScanner implements KeyValueScanner {
     }
     List<StoreFileScanner> scanners = new ArrayList<>(files.size());
     boolean canOptimizeForNonNullColumn = matcher != null ? !matcher.hasNullColumnInQuery() : false;
-    PriorityQueue<StoreFile> sortedFiles =
+    PriorityQueue<HStoreFile> sortedFiles =
         new PriorityQueue<>(files.size(), StoreFileComparators.SEQ_ID);
-    for (StoreFile file : files) {
+    for (HStoreFile file : files) {
       // The sort function needs metadata so we need to open reader first before sorting the list.
       file.initReader();
       sortedFiles.add(file);
     }
     for (int i = 0, n = files.size(); i < n; i++) {
-      StoreFile sf = sortedFiles.remove();
+      HStoreFile sf = sortedFiles.remove();
       StoreFileScanner scanner;
       if (usePread) {
         scanner = sf.getPreadScanner(cacheBlocks, readPt, i, canOptimizeForNonNullColumn);
@@ -154,10 +155,10 @@ public class StoreFileScanner implements KeyValueScanner {
    * Get scanners for compaction. We will create a separated reader for each store file to avoid
    * contention with normal read request.
    */
-  public static List<StoreFileScanner> getScannersForCompaction(Collection<StoreFile> files,
+  public static List<StoreFileScanner> getScannersForCompaction(Collection<HStoreFile> files,
       boolean canUseDropBehind, long readPt) throws IOException {
     List<StoreFileScanner> scanners = new ArrayList<>(files.size());
-    List<StoreFile> sortedFiles = new ArrayList<>(files);
+    List<HStoreFile> sortedFiles = new ArrayList<>(files);
     Collections.sort(sortedFiles, StoreFileComparators.SEQ_ID);
     boolean succ = false;
     try {
@@ -537,12 +538,11 @@ public class StoreFileScanner implements KeyValueScanner {
 
   @Override
   public boolean seekToLastRow() throws IOException {
-    byte[] lastRow = reader.getLastRowKey();
-    if (lastRow == null) {
+    Optional<byte[]> lastRow = reader.getLastRowKey();
+    if (!lastRow.isPresent()) {
       return false;
     }
-    Cell seekKey = CellUtil
-        .createFirstOnRow(lastRow, 0, (short) lastRow.length);
+    Cell seekKey = CellUtil.createFirstOnRow(lastRow.get());
     if (seek(seekKey)) {
       return true;
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
----------------------------------------------------------------------
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 25b9aa1..4dbe280 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
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * 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
@@ -18,7 +17,13 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT;
+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.TIMERANGE_KEY;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -33,10 +38,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
+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;
@@ -48,6 +52,9 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.RowBloomContext;
 import org.apache.hadoop.hbase.util.RowColBloomContext;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 
 /**
  * A StoreFile writer.  Use this to read/write HBase Store Files. It is package
@@ -185,10 +192,9 @@ public class StoreFileWriter implements CellSink, ShipperListener {
    * @throws IOException problem writing to FS
    */
   public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
-  throws IOException {
-    writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
-    writer.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY,
-        Bytes.toBytes(majorCompaction));
+      throws IOException {
+    writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
+    writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction));
     appendTrackedTimestampsToMetadata();
   }
 
@@ -202,9 +208,9 @@ public class StoreFileWriter implements CellSink, ShipperListener {
    */
   public void appendMetadata(final long maxSequenceId, final boolean majorCompaction,
       final long mobCellsCount) throws IOException {
-    writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
-    writer.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction));
-    writer.appendFileInfo(StoreFile.MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount));
+    writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
+    writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction));
+    writer.appendFileInfo(MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount));
     appendTrackedTimestampsToMetadata();
   }
 
@@ -212,8 +218,8 @@ public class StoreFileWriter implements CellSink, ShipperListener {
    * Add TimestampRange and earliest put timestamp to Metadata
    */
   public void appendTrackedTimestampsToMetadata() throws IOException {
-    appendFileInfo(StoreFile.TIMERANGE_KEY, WritableUtils.toByteArray(timeRangeTracker));
-    appendFileInfo(StoreFile.EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
+    appendFileInfo(TIMERANGE_KEY, WritableUtils.toByteArray(timeRangeTracker));
+    appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
   }
 
   /**
@@ -310,8 +316,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
     // add the general Bloom filter writer and append file info
     if (hasGeneralBloom) {
       writer.addGeneralBloomFilter(generalBloomFilterWriter);
-      writer.appendFileInfo(StoreFile.BLOOM_FILTER_TYPE_KEY,
-          Bytes.toBytes(bloomType.toString()));
+      writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString()));
       bloomContext.addLastBloomKey(writer);
     }
     return hasGeneralBloom;
@@ -327,8 +332,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
 
     // append file info about the number of delete family kvs
     // even if there is no delete family Bloom.
-    writer.appendFileInfo(StoreFile.DELETE_FAMILY_COUNT,
-        Bytes.toBytes(this.deleteFamilyCnt));
+    writer.appendFileInfo(DELETE_FAMILY_COUNT, Bytes.toBytes(this.deleteFamilyCnt));
 
     return hasDeleteFamilyBloom;
   }
@@ -501,7 +505,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
       }
 
       // set block storage policy for temp path
-      String policyName = this.conf.get(HColumnDescriptor.STORAGE_POLICY);
+      String policyName = this.conf.get(ColumnFamilyDescriptorBuilder.STORAGE_POLICY);
       if (null == policyName) {
         policyName = this.conf.get(HStore.BLOCK_STORAGE_POLICY_KEY);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java
index f670ade..bc5a7cf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java
@@ -29,10 +29,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Store flusher interface. Turns a snapshot of memstore into a set of store files (usually one).
@@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 @InterfaceAudience.Private
 abstract class StoreFlusher {
   protected Configuration conf;
-  protected Store store;
+  protected HStore store;
 
-  public StoreFlusher(Configuration conf, Store store) {
+  public StoreFlusher(Configuration conf, HStore store) {
     this.conf = conf;
     this.store = store;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 3a98479..dd68d28 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -16,7 +16,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
@@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.IsolationLevel;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.executor.ExecutorService;
@@ -51,6 +49,7 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
 import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
@@ -67,7 +66,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     implements KeyValueScanner, InternalScanner, ChangedReadersObserver {
   private static final Log LOG = LogFactory.getLog(StoreScanner.class);
   // In unit tests, the store could be null
-  protected final Optional<Store> store;
+  protected final Optional<HStore> store;
   private ScanQueryMatcher matcher;
   protected KeyValueHeap heap;
   private boolean cacheBlocks;
@@ -147,7 +146,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   // Indicates whether there was flush during the course of the scan
   private volatile boolean flushed = false;
   // generally we get one file from a flush
-  private final List<StoreFile> flushedStoreFiles = new ArrayList<>(1);
+  private final List<HStoreFile> flushedStoreFiles = new ArrayList<>(1);
   // Since CompactingMemstore is now default, we get three memstore scanners from a flush
   private final List<KeyValueScanner> memStoreScannersAfterFlush = new ArrayList<>(3);
   // The current list of scanners
@@ -160,7 +159,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   private boolean topChanged = false;
 
   /** An internal constructor. */
-  private StoreScanner(Optional<Store> store, Scan scan, ScanInfo scanInfo,
+  private StoreScanner(Optional<HStore> store, Scan scan, ScanInfo scanInfo,
       int numColumns, long readPt, boolean cacheBlocks, ScanType scanType) {
     this.readPt = readPt;
     this.store = store;
@@ -223,7 +222,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
    * @param columns which columns we are scanning
    * @throws IOException
    */
-  public StoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
+  public StoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
       long readPt) throws IOException {
     this(Optional.of(store), scan, scanInfo, columns != null ? columns.size() : 0, readPt,
         scan.getCacheBlocks(), ScanType.USER_SCAN);
@@ -275,7 +274,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
    * @param scanners ancillary scanners
    * @param smallestReadPoint the readPoint that we should use for tracking versions
    */
-  public StoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions,
+  public StoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions,
       List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint,
       long earliestPutTs) throws IOException {
     this(store, scanInfo, maxVersions, scanners, scanType, smallestReadPoint, earliestPutTs, null,
@@ -292,20 +291,20 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
    * @param dropDeletesFromRow The inclusive left bound of the range; can be EMPTY_START_ROW.
    * @param dropDeletesToRow The exclusive right bound of the range; can be EMPTY_END_ROW.
    */
-  public StoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions,
+  public StoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions,
       List<? extends KeyValueScanner> scanners, long smallestReadPoint, long earliestPutTs,
       byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException {
     this(store, scanInfo, maxVersions, scanners, ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint,
         earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
   }
 
-  private StoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions,
+  private StoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions,
       List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint,
       long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException {
     this(Optional.of(store),
         maxVersions.isPresent() ? new Scan().readVersions(maxVersions.getAsInt())
             : SCAN_FOR_COMPACTION,
-        scanInfo, 0, ((HStore) store).getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED),
+        scanInfo, 0, store.getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED),
         false, scanType);
     assert scanType != ScanType.USER_SCAN;
     matcher =
@@ -844,9 +843,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
 
   // Implementation of ChangedReadersObserver
   @Override
-  public void updateReaders(List<StoreFile> sfs, List<KeyValueScanner> memStoreScanners) throws IOException {
-    if (CollectionUtils.isEmpty(sfs)
-      && CollectionUtils.isEmpty(memStoreScanners)) {
+  public void updateReaders(List<HStoreFile> sfs, List<KeyValueScanner> memStoreScanners)
+      throws IOException {
+    if (CollectionUtils.isEmpty(sfs) && CollectionUtils.isEmpty(memStoreScanners)) {
       return;
     }
     flushLock.lock();
@@ -868,7 +867,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
    */
   protected final boolean reopenAfterFlush() throws IOException {
     // here we can make sure that we have a Store instance.
-    Store store = this.store.get();
+    HStore store = this.store.get();
     Cell lastTop = heap.peek();
     // When we have the scan object, should we not pass it to getScanners() to get a limited set of
     // scanners? We did so in the constructor and we could have done it now by storing the scan
@@ -996,7 +995,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     List<KeyValueScanner> newCurrentScanners;
     KeyValueHeap newHeap;
     // We must have a store instance here
-    Store store = this.store.get();
+    HStore store = this.store.get();
     try {
       // recreate the scanners on the current file scanners
       fileScanners = store.recreateScanners(scannersToClose, cacheBlocks, false, false,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
index 9104546..0abaffd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java
@@ -41,7 +41,7 @@ public class StoreUtils {
   /**
    * Creates a deterministic hash code for store file collection.
    */
-  public static OptionalInt getDeterministicRandomSeed(Collection<StoreFile> files) {
+  public static OptionalInt getDeterministicRandomSeed(Collection<HStoreFile> files) {
     return files.stream().mapToInt(f -> f.getPath().getName().hashCode()).findFirst();
   }
 
@@ -49,24 +49,17 @@ public class StoreUtils {
    * Determines whether any files in the collection are references.
    * @param files The files.
    */
-  public static boolean hasReferences(final Collection<StoreFile> files) {
-    if (files != null) {
-      for (StoreFile hsf: files) {
-        if (hsf.isReference()) {
-          return true;
-        }
-      }
-    }
-    return false;
+  public static boolean hasReferences(Collection<HStoreFile> files) {
+    // TODO: make sure that we won't pass null here in the future.
+    return files != null ? files.stream().anyMatch(HStoreFile::isReference) : false;
   }
 
   /**
    * Gets lowest timestamp from candidate StoreFiles
    */
-  public static long getLowestTimestamp(final Collection<StoreFile> candidates)
-    throws IOException {
+  public static long getLowestTimestamp(Collection<HStoreFile> candidates) throws IOException {
     long minTs = Long.MAX_VALUE;
-    for (StoreFile storeFile : candidates) {
+    for (HStoreFile storeFile : candidates) {
       minTs = Math.min(minTs, storeFile.getModificationTimeStamp());
     }
     return minTs;
@@ -77,7 +70,7 @@ public class StoreUtils {
    * @param candidates The files to choose from.
    * @return The largest file; null if no file has a reader.
    */
-  static Optional<StoreFile> getLargestFile(Collection<StoreFile> candidates) {
+  static Optional<HStoreFile> getLargestFile(Collection<HStoreFile> candidates) {
     return candidates.stream().filter(f -> f.getReader() != null)
         .max((f1, f2) -> Long.compare(f1.getReader().length(), f2.getReader().length()));
   }
@@ -89,29 +82,19 @@ public class StoreUtils {
    * @return 0 if no non-bulk-load files are provided or, this is Store that does not yet have any
    *         store files.
    */
-  public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
-    long max = 0;
-    for (StoreFile sf : sfs) {
-      if (!sf.isBulkLoadResult()) {
-        max = Math.max(max, sf.getMaxMemstoreTS());
-      }
-    }
-    return max;
+  public static long getMaxMemstoreTSInList(Collection<HStoreFile> sfs) {
+    return sfs.stream().filter(sf -> !sf.isBulkLoadResult()).mapToLong(HStoreFile::getMaxMemstoreTS)
+        .max().orElse(0L);
   }
 
   /**
-   * Return the highest sequence ID found across all storefiles in
-   * the given list.
+   * Return the highest sequence ID found across all storefiles in the given list.
    * @param sfs
-   * @return 0 if no non-bulk-load files are provided or, this is Store that
-   * does not yet have any store files.
+   * @return 0 if no non-bulk-load files are provided or, this is Store that does not yet have any
+   *         store files.
    */
-  public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
-    long max = 0;
-    for (StoreFile sf : sfs) {
-      max = Math.max(max, sf.getMaxSequenceId());
-    }
-    return max;
+  public static long getMaxSequenceIdInList(Collection<HStoreFile> sfs) {
+    return sfs.stream().mapToLong(HStoreFile::getMaxSequenceId).max().orElse(0L);
   }
 
   /**
@@ -120,7 +103,7 @@ public class StoreUtils {
    * @param comparator Comparator used to compare KVs.
    * @return The split point row, or null if splitting is not possible, or reader is null.
    */
-  static Optional<byte[]> getFileSplitPoint(StoreFile file, CellComparator comparator)
+  static Optional<byte[]> getFileSplitPoint(HStoreFile file, CellComparator comparator)
       throws IOException {
     StoreFileReader reader = file.getReader();
     if (reader == null) {
@@ -130,20 +113,31 @@ public class StoreUtils {
     // Get first, last, and mid keys. Midkey is the key that starts block
     // in middle of hfile. Has column and timestamp. Need to return just
     // the row we want to split on as midkey.
-    Cell midkey = reader.midkey();
-    if (midkey != null) {
-      Cell firstKey = reader.getFirstKey();
-      Cell lastKey = reader.getLastKey();
-      // if the midkey is the same as the first or last keys, we cannot (ever) split this region.
-      if (comparator.compareRows(midkey, firstKey) == 0 ||
-          comparator.compareRows(midkey, lastKey) == 0) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("cannot split because midkey is the same as first or last row");
-        }
-        return Optional.empty();
+    Optional<Cell> optionalMidKey = reader.midKey();
+    if (!optionalMidKey.isPresent()) {
+      return Optional.empty();
+    }
+    Cell midKey = optionalMidKey.get();
+    Cell firstKey = reader.getFirstKey().get();
+    Cell lastKey = reader.getLastKey().get();
+    // if the midkey is the same as the first or last keys, we cannot (ever) split this region.
+    if (comparator.compareRows(midKey, firstKey) == 0 ||
+        comparator.compareRows(midKey, lastKey) == 0) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("cannot split because midkey is the same as first or last row");
       }
-      return Optional.of(CellUtil.cloneRow(midkey));
+      return Optional.empty();
     }
-    return Optional.empty();
+    return Optional.of(CellUtil.cloneRow(midKey));
+  }
+
+  /**
+   * Gets the mid point of the largest file passed in as split point.
+   */
+  static Optional<byte[]> getSplitPoint(Collection<HStoreFile> storefiles,
+      CellComparator comparator) throws IOException {
+    Optional<HStoreFile> largestFile = StoreUtils.getLargestFile(storefiles);
+    return largestFile.isPresent() ? StoreUtils.getFileSplitPoint(largestFile.get(), comparator)
+        : Optional.empty();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
index b14b0d0..39f142f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java
@@ -47,7 +47,7 @@ public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
   private StripeStoreConfig config;
 
   @Override
-  public boolean needsCompaction(List<StoreFile> filesCompacting) {
+  public boolean needsCompaction(List<HStoreFile> filesCompacting) {
     return this.compactionPolicy.needsCompactions(this.storeFileManager, filesCompacting);
   }
 
@@ -58,7 +58,7 @@ public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
 
   @Override
   protected void createComponents(
-      Configuration conf, Store store, CellComparator comparator) throws IOException {
+      Configuration conf, HStore store, CellComparator comparator) throws IOException {
     this.config = new StripeStoreConfig(conf, store);
     this.compactionPolicy = new StripeCompactionPolicy(conf, store, config);
     this.storeFileManager = new StripeStoreFileManager(comparator, conf, this.config);
@@ -74,12 +74,12 @@ public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
     private StripeCompactionPolicy.StripeCompactionRequest stripeRequest = null;
 
     @Override
-    public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
+    public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
       return compactionPolicy.preSelectFilesForCoprocessor(storeFileManager, filesCompacting);
     }
 
     @Override
-    public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
+    public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
         boolean mayUseOffPeak, boolean forceMajor) throws IOException {
       this.stripeRequest = compactionPolicy.selectCompaction(
           storeFileManager, filesCompacting, mayUseOffPeak);


[05/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
deleted file mode 100644
index 6fa951e..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java
+++ /dev/null
@@ -1,220 +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.regionserver;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Map;
-import java.util.OptionalLong;
-import java.util.TreeMap;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-
-/** A mock used so our tests don't deal with actual StoreFiles */
-public class MockStoreFile extends HStoreFile {
-  long length = 0;
-  boolean isRef = false;
-  long ageInDisk;
-  long sequenceid;
-  private Map<byte[], byte[]> metadata = new TreeMap<>(Bytes.BYTES_COMPARATOR);
-  byte[] splitPoint = null;
-  TimeRangeTracker timeRangeTracker;
-  long entryCount;
-  boolean isMajor;
-  HDFSBlocksDistribution hdfsBlocksDistribution;
-  long modificationTime;
-  boolean compactedAway;
-
-  MockStoreFile(HBaseTestingUtility testUtil, Path testPath,
-      long length, long ageInDisk, boolean isRef, long sequenceid) throws IOException {
-    super(testUtil.getTestFileSystem(), testPath, testUtil.getConfiguration(),
-        new CacheConfig(testUtil.getConfiguration()), BloomType.NONE, true);
-    this.length = length;
-    this.isRef = isRef;
-    this.ageInDisk = ageInDisk;
-    this.sequenceid = sequenceid;
-    this.isMajor = false;
-    hdfsBlocksDistribution = new HDFSBlocksDistribution();
-    hdfsBlocksDistribution.addHostsAndBlockWeight(
-      new String[] { RSRpcServices.getHostname(testUtil.getConfiguration(), false) }, 1);
-    modificationTime = EnvironmentEdgeManager.currentTime();
-  }
-
-  void setLength(long newLen) {
-    this.length = newLen;
-  }
-
-  @Override
-  public long getMaxSequenceId() {
-    return sequenceid;
-  }
-
-  @Override
-  public boolean isMajorCompactionResult() {
-    return isMajor;
-  }
-
-  public void setIsMajor(boolean isMajor) {
-    this.isMajor = isMajor;
-  }
-
-  @Override
-  public boolean isReference() {
-    return this.isRef;
-  }
-
-  @Override
-  public boolean isBulkLoadResult() {
-    return false;
-  }
-
-  @Override
-  public byte[] getMetadataValue(byte[] key) {
-    return this.metadata.get(key);
-  }
-
-  public void setMetadataValue(byte[] key, byte[] value) {
-    this.metadata.put(key, value);
-  }
-
-  void setTimeRangeTracker(TimeRangeTracker timeRangeTracker) {
-    this.timeRangeTracker = timeRangeTracker;
-  }
-
-  void setEntries(long entryCount) {
-    this.entryCount = entryCount;
-  }
-
-  public OptionalLong getMinimumTimestamp() {
-    return timeRangeTracker == null ? OptionalLong.empty()
-        : OptionalLong.of(timeRangeTracker.getMin());
-  }
-
-  public OptionalLong getMaximumTimestamp() {
-    return timeRangeTracker == null ? OptionalLong.empty()
-        : OptionalLong.of(timeRangeTracker.getMax());
-  }
-
-  @Override
-  public void markCompactedAway() {
-    this.compactedAway = true;
-  }
-
-  @Override
-  public boolean isCompactedAway() {
-    return compactedAway;
-  }
-
-  @Override
-  public long getModificationTimeStamp() {
-    return modificationTime;
-  }
-
-  @Override
-  public HDFSBlocksDistribution getHDFSBlockDistribution() {
-    return hdfsBlocksDistribution;
-  }
-
-  @Override
-  public void initReader() throws IOException {
-  }
-
-  @Override
-  public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
-      boolean canOptimizeForNonNullColumn) {
-    return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder,
-      canOptimizeForNonNullColumn);
-  }
-
-  @Override
-  public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
-      boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
-      throws IOException {
-    return getReader().getStoreFileScanner(cacheBlocks, false, isCompaction, readPt, scannerOrder,
-      canOptimizeForNonNullColumn);
-  }
-
-  @Override
-  public StoreFileReader getReader() {
-    final long len = this.length;
-    final TimeRangeTracker timeRangeTracker = this.timeRangeTracker;
-    final long entries = this.entryCount;
-    return new StoreFileReader() {
-      @Override
-      public long length() {
-        return len;
-      }
-
-      @Override
-      public long getMaxTimestamp() {
-        return timeRange == null? Long.MAX_VALUE: timeRangeTracker.getMax();
-      }
-
-      @Override
-      public long getEntries() {
-        return entries;
-      }
-
-      @Override
-      public void close(boolean evictOnClose) throws IOException {
-        // no-op
-      }
-
-      @Override
-      public Cell getLastKey() {
-        if (splitPoint != null) {
-          return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length + 1));
-        } else {
-          return null;
-        }
-      }
-
-      @Override
-      public Cell midkey() throws IOException {
-        if (splitPoint != null) {
-          return CellUtil.createCell(splitPoint);
-        } else {
-          return null;
-        }
-      }
-
-      @Override
-      public Cell getFirstKey() {
-        if (splitPoint != null) {
-          return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length - 1));
-        } else {
-          return null;
-        }
-      }
-    };
-  }
-
-  @Override
-  public OptionalLong getBulkLoadTimestamp() {
-    // we always return false for isBulkLoadResult so we do not have a bulk load timestamp
-    return OptionalLong.empty();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
index 36c2e19..eecc069 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 
 /**
  * RegionObserver that just reimplements the default behavior,
@@ -51,7 +50,7 @@ public class NoOpScanPolicyObserver implements RegionObserver {
     ScanInfo oldSI = store.getScanInfo();
     ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),
         oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
-    return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners,
+    return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners,
         ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP);
   }
 
@@ -67,7 +66,7 @@ public class NoOpScanPolicyObserver implements RegionObserver {
     ScanInfo oldSI = store.getScanInfo();
     ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),
         oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
-    return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, scanType,
+    return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, scanType,
         store.getSmallestReadPoint(), earliestPutTs);
   }
 
@@ -76,10 +75,10 @@ public class NoOpScanPolicyObserver implements RegionObserver {
       Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPoint)
       throws IOException {
     Region r = c.getEnvironment().getRegion();
-    return scan.isReversed() ? new ReversedStoreScanner(store,
-        store.getScanInfo(), scan, targetCols, r.getReadPoint(scan
-            .getIsolationLevel())) : new StoreScanner(store,
-        store.getScanInfo(), scan, targetCols, r.getReadPoint(scan
-            .getIsolationLevel()));
+    return scan.isReversed()
+        ? new ReversedStoreScanner((HStore) store, store.getScanInfo(), scan, targetCols,
+            r.getReadPoint(scan.getIsolationLevel()))
+        : new StoreScanner((HStore) store, store.getScanInfo(), scan, targetCols,
+            r.getReadPoint(scan.getIsolationLevel()));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index 8fad157..2d08e50 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -218,7 +218,7 @@ public class TestCacheOnWriteInSchema {
   private void readStoreFile(Path path) throws IOException {
     CacheConfig cacheConf = store.getCacheConfig();
     BlockCache cache = cacheConf.getBlockCache();
-    StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
+    HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
     sf.initReader();
     HFile.Reader reader = sf.getReader().getHFileReader();
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index 3649823..356054e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
 import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES;
 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
@@ -226,7 +227,7 @@ public class TestCompaction {
 
   private int count() throws IOException {
     int count = 0;
-    for (StoreFile f: this.r.stores.
+    for (HStoreFile f: this.r.stores.
         get(COLUMN_FAMILY_TEXT).getStorefiles()) {
       HFileScanner scanner = f.getReader().getScanner(false, false);
       if (!scanner.seekTo()) {
@@ -255,9 +256,9 @@ public class TestCompaction {
     for (int i = 0; i < nfiles; i++) {
       createStoreFile(r);
     }
-    HStore store = (HStore) r.getStore(COLUMN_FAMILY);
+    HStore store = r.getStore(COLUMN_FAMILY);
 
-    Collection<StoreFile> storeFiles = store.getStorefiles();
+    Collection<HStoreFile> storeFiles = store.getStorefiles();
     DefaultCompactor tool = (DefaultCompactor)store.storeEngine.getCompactor();
     tool.compactForTesting(storeFiles, false);
 
@@ -276,8 +277,8 @@ public class TestCompaction {
     } catch (Exception e) {
       // The complete compaction should fail and the corrupt file should remain
       // in the 'tmp' directory;
-      assert (fs.exists(origPath));
-      assert (!fs.exists(dstPath));
+      assertTrue(fs.exists(origPath));
+      assertFalse(fs.exists(dstPath));
       System.out.println("testCompactionWithCorruptResult Passed");
       return;
     }
@@ -389,8 +390,8 @@ public class TestCompaction {
   }
 
   private class StoreMockMaker extends StatefulStoreMockMaker {
-    public ArrayList<StoreFile> compacting = new ArrayList<>();
-    public ArrayList<StoreFile> notCompacting = new ArrayList<>();
+    public ArrayList<HStoreFile> compacting = new ArrayList<>();
+    public ArrayList<HStoreFile> notCompacting = new ArrayList<>();
     private ArrayList<Integer> results;
 
     public StoreMockMaker(ArrayList<Integer> results) {
@@ -398,19 +399,21 @@ public class TestCompaction {
     }
 
     public class TestCompactionContext extends CompactionContext {
-      private List<StoreFile> selectedFiles;
-      public TestCompactionContext(List<StoreFile> selectedFiles) {
+
+      private List<HStoreFile> selectedFiles;
+
+      public TestCompactionContext(List<HStoreFile> selectedFiles) {
         super();
         this.selectedFiles = selectedFiles;
       }
 
       @Override
-      public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
+      public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
         return new ArrayList<>();
       }
 
       @Override
-      public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
+      public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
           boolean mayUseOffPeak, boolean forceMajor) throws IOException {
         this.request = new CompactionRequest(selectedFiles);
         this.request.setPriority(getPriority());
@@ -445,7 +448,7 @@ public class TestCompaction {
       notCompacting.addAll(ctx.selectedFiles);
     }
 
-    public synchronized void finishCompaction(List<StoreFile> sfs) {
+    public synchronized void finishCompaction(List<HStoreFile> sfs) {
       if (sfs.isEmpty()) return;
       synchronized (results) {
         results.add(sfs.size());
@@ -466,7 +469,9 @@ public class TestCompaction {
       public volatile boolean isInCompact = false;
 
       public void unblock() {
-        synchronized (this) { this.notifyAll(); }
+        synchronized (this) {
+          this.notifyAll();
+        }
       }
 
       @Override
@@ -484,12 +489,12 @@ public class TestCompaction {
       }
 
       @Override
-      public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
+      public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
         return new ArrayList<>();
       }
 
       @Override
-      public boolean select(List<StoreFile> f, boolean i, boolean m, boolean e)
+      public boolean select(List<HStoreFile> f, boolean i, boolean m, boolean e)
           throws IOException {
         this.request = new CompactionRequest(new ArrayList<>());
         return true;
@@ -673,14 +678,14 @@ public class TestCompaction {
   }
 
   public static class DummyCompactor extends DefaultCompactor {
-    public DummyCompactor(Configuration conf, Store store) {
+    public DummyCompactor(Configuration conf, HStore store) {
       super(conf, store);
       this.keepSeqIdPeriod = 0;
     }
   }
 
-  private static StoreFile createFile() throws Exception {
-    StoreFile sf = mock(StoreFile.class);
+  private static HStoreFile createFile() throws Exception {
+    HStoreFile sf = mock(HStoreFile.class);
     when(sf.getPath()).thenReturn(new Path("file"));
     StoreFileReader r = mock(StoreFileReader.class);
     when(r.length()).thenReturn(10L);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
index d68f07e..2a2602c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
@@ -89,7 +88,7 @@ public class TestCompactionArchiveConcurrentClose {
     HTableDescriptor htd = new HTableDescriptor(tableName);
     htd.addFamily(new HColumnDescriptor(fam));
     HRegionInfo info = new HRegionInfo(tableName, null, null, false);
-    Region region = initHRegion(htd, info);
+    HRegion region = initHRegion(htd, info);
     RegionServerServices rss = mock(RegionServerServices.class);
     List<Region> regions = new ArrayList<>();
     regions.add(region);
@@ -112,12 +111,12 @@ public class TestCompactionArchiveConcurrentClose {
       region.flush(true);
     }
 
-    Store store = region.getStore(fam);
+    HStore store = region.getStore(fam);
     assertEquals(fileCount, store.getStorefilesCount());
 
-    Collection<StoreFile> storefiles = store.getStorefiles();
+    Collection<HStoreFile> storefiles = store.getStorefiles();
     // None of the files should be in compacted state.
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       assertFalse(file.isCompactedAway());
     }
     // Do compaction
@@ -157,7 +156,7 @@ public class TestCompactionArchiveConcurrentClose {
     }
   }
 
-  private Region initHRegion(HTableDescriptor htd, HRegionInfo info)
+  private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info)
       throws IOException {
     Configuration conf = testUtil.getConfiguration();
     Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
@@ -185,7 +184,7 @@ public class TestCompactionArchiveConcurrentClose {
     }
 
     @Override
-    public void removeStoreFiles(String familyName, Collection<StoreFile> storeFiles)
+    public void removeStoreFiles(String familyName, Collection<HStoreFile> storeFiles)
         throws IOException {
       super.removeStoreFiles(familyName, storeFiles);
       archived.set(true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
index 3f1613c..1d976e9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java
@@ -120,24 +120,24 @@ public class TestCompactionArchiveIOException {
       region.flush(true);
     }
 
-    HStore store = (HStore) region.getStore(fam);
+    HStore store = region.getStore(fam);
     assertEquals(fileCount, store.getStorefilesCount());
 
-    Collection<StoreFile> storefiles = store.getStorefiles();
+    Collection<HStoreFile> storefiles = store.getStorefiles();
     // None of the files should be in compacted state.
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       assertFalse(file.isCompactedAway());
     }
 
     StoreFileManager fileManager = store.getStoreEngine().getStoreFileManager();
-    Collection<StoreFile> initialCompactedFiles = fileManager.getCompactedfiles();
+    Collection<HStoreFile> initialCompactedFiles = fileManager.getCompactedfiles();
     assertTrue(initialCompactedFiles == null || initialCompactedFiles.isEmpty());
 
     // Do compaction
     region.compact(true);
 
     // all prior store files should now be compacted
-    Collection<StoreFile> compactedFilesPreClean = fileManager.getCompactedfiles();
+    Collection<HStoreFile> compactedFilesPreClean = fileManager.getCompactedfiles();
     assertNotNull(compactedFilesPreClean);
     assertTrue(compactedFilesPreClean.size() > 0);
 
@@ -148,17 +148,17 @@ public class TestCompactionArchiveIOException {
     out.writeInt(1);
     out.close();
 
-    StoreFile errStoreFile = new MockStoreFile(testUtil, errFile, 1, 0, false, 1);
+    HStoreFile errStoreFile = new MockHStoreFile(testUtil, errFile, 1, 0, false, 1);
     fileManager.addCompactionResults(
-        ImmutableList.of(errStoreFile), ImmutableList.<StoreFile>of());
+        ImmutableList.of(errStoreFile), ImmutableList.of());
 
     // cleanup compacted files
     cleaner.chore();
 
     // make sure the compacted files are cleared
-    Collection<StoreFile> compactedFilesPostClean = fileManager.getCompactedfiles();
+    Collection<HStoreFile> compactedFilesPostClean = fileManager.getCompactedfiles();
     assertEquals(1, compactedFilesPostClean.size());
-    for (StoreFile origFile : compactedFilesPreClean) {
+    for (HStoreFile origFile : compactedFilesPreClean) {
       assertFalse(compactedFilesPostClean.contains(origFile));
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
index 0e4c4f9..6ae10ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -46,6 +44,8 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
 public class TestCompactionPolicy {
   private final static Log LOG = LogFactory.getLog(TestCompactionPolicy.class);
   protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -141,7 +141,7 @@ public class TestCompactionPolicy {
     return result;
   }
 
-  List<StoreFile> sfCreate(long... sizes) throws IOException {
+  List<HStoreFile> sfCreate(long... sizes) throws IOException {
     ArrayList<Long> ageInDisk = new ArrayList<>();
     for (int i = 0; i < sizes.length; i++) {
       ageInDisk.add(0L);
@@ -149,11 +149,11 @@ public class TestCompactionPolicy {
     return sfCreate(toArrayList(sizes), ageInDisk);
   }
 
-  List<StoreFile> sfCreate(ArrayList<Long> sizes, ArrayList<Long> ageInDisk) throws IOException {
+  List<HStoreFile> sfCreate(ArrayList<Long> sizes, ArrayList<Long> ageInDisk) throws IOException {
     return sfCreate(false, sizes, ageInDisk);
   }
 
-  List<StoreFile> sfCreate(boolean isReference, long... sizes) throws IOException {
+  List<HStoreFile> sfCreate(boolean isReference, long... sizes) throws IOException {
     ArrayList<Long> ageInDisk = new ArrayList<>(sizes.length);
     for (int i = 0; i < sizes.length; i++) {
       ageInDisk.add(0L);
@@ -161,17 +161,17 @@ public class TestCompactionPolicy {
     return sfCreate(isReference, toArrayList(sizes), ageInDisk);
   }
 
-  List<StoreFile> sfCreate(boolean isReference, ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
+  List<HStoreFile> sfCreate(boolean isReference, ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
       throws IOException {
-    List<StoreFile> ret = Lists.newArrayList();
+    List<HStoreFile> ret = Lists.newArrayList();
     for (int i = 0; i < sizes.size(); i++) {
-      ret.add(new MockStoreFile(TEST_UTIL, TEST_FILE, sizes.get(i), ageInDisk.get(i), isReference,
+      ret.add(new MockHStoreFile(TEST_UTIL, TEST_FILE, sizes.get(i), ageInDisk.get(i), isReference,
           i));
     }
     return ret;
   }
 
-  long[] getSizes(List<StoreFile> sfList) {
+  long[] getSizes(List<HStoreFile> sfList) {
     long[] aNums = new long[sfList.size()];
     for (int i = 0; i < sfList.size(); ++i) {
       aNums[i] = sfList.get(i).getReader().length();
@@ -179,23 +179,23 @@ public class TestCompactionPolicy {
     return aNums;
   }
 
-  void compactEquals(List<StoreFile> candidates, long... expected) throws IOException {
+  void compactEquals(List<HStoreFile> candidates, long... expected) throws IOException {
     compactEquals(candidates, false, false, expected);
   }
 
-  void compactEquals(List<StoreFile> candidates, boolean forcemajor, long... expected)
+  void compactEquals(List<HStoreFile> candidates, boolean forcemajor, long... expected)
       throws IOException {
     compactEquals(candidates, forcemajor, false, expected);
   }
 
-  void compactEquals(List<StoreFile> candidates, boolean forcemajor, boolean isOffPeak,
+  void compactEquals(List<HStoreFile> candidates, boolean forcemajor, boolean isOffPeak,
       long... expected) throws IOException {
     store.forceMajor = forcemajor;
     // Test Default compactions
     CompactionRequest result =
         ((RatioBasedCompactionPolicy) store.storeEngine.getCompactionPolicy()).selectCompaction(
           candidates, new ArrayList<>(), false, isOffPeak, forcemajor);
-    List<StoreFile> actual = new ArrayList<>(result.getFiles());
+    List<HStoreFile> actual = new ArrayList<>(result.getFiles());
     if (isOffPeak && !forcemajor) {
       Assert.assertTrue(result.isOffPeak());
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
index 26172f5..9c33d28 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
@@ -200,7 +200,7 @@ public class TestCompoundBloomFilter {
 
   private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
       Path sfPath) throws IOException {
-    StoreFile sf = new HStoreFile(fs, sfPath, conf, cacheConf, bt, true);
+    HStoreFile sf = new HStoreFile(fs, sfPath, conf, cacheConf, bt, true);
     sf.initReader();
     StoreFileReader r = sf.getReader();
     final boolean pread = true; // does not really matter

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
index 059b850..3689cf7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java
@@ -98,7 +98,7 @@ public class TestDefaultCompactSelection extends TestCompactionPolicy {
       // timestamp a bit to make sure that now - lowestModTime is greater than major compaction
       // period(1ms).
       // trigger an aged major compaction
-      List<StoreFile> candidates = sfCreate(50, 25, 12, 12);
+      List<HStoreFile> candidates = sfCreate(50, 25, 12, 12);
       edge.increment(2);
       compactEquals(candidates, 50, 25, 12, 12);
       // major sure exceeding maxCompactSize also downgrades aged minors
@@ -164,10 +164,10 @@ public class TestDefaultCompactSelection extends TestCompactionPolicy {
         oldScanInfo.getTimeToPurgeDeletes(), oldScanInfo.getComparator(), oldScanInfo.isNewVersionBehavior());
     store.setScanInfo(newScanInfo);
     // Do not compact empty store file
-    List<StoreFile> candidates = sfCreate(0);
-    for (StoreFile file : candidates) {
-      if (file instanceof MockStoreFile) {
-        MockStoreFile mockFile = (MockStoreFile) file;
+    List<HStoreFile> candidates = sfCreate(0);
+    for (HStoreFile file : candidates) {
+      if (file instanceof MockHStoreFile) {
+        MockHStoreFile mockFile = (MockHStoreFile) file;
         mockFile.setTimeRangeTracker(new TimeRangeTracker(-1, -1));
         mockFile.setEntries(0);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
index b9982aa..df5e97a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
@@ -22,10 +22,10 @@ package org.apache.hadoop.hbase.regionserver;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
+import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
-import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -34,13 +34,13 @@ import org.mockito.Mockito;
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestDefaultStoreEngine {
   public static class DummyStoreFlusher extends DefaultStoreFlusher {
-    public DummyStoreFlusher(Configuration conf, Store store) {
+    public DummyStoreFlusher(Configuration conf, HStore store) {
       super(conf, store);
     }
   }
 
   public static class DummyCompactor extends DefaultCompactor {
-    public DummyCompactor(Configuration conf, Store store) {
+    public DummyCompactor(Configuration conf, HStore store) {
       super(conf, store);
     }
   }
@@ -59,7 +59,7 @@ public class TestDefaultStoreEngine {
         DummyCompactionPolicy.class.getName());
     conf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,
         DummyStoreFlusher.class.getName());
-    Store mockStore = Mockito.mock(Store.class);
+    HStore mockStore = Mockito.mock(HStore.class);
     StoreEngine<?, ?, ?, ?> se = StoreEngine.create(mockStore, conf, CellComparator.COMPARATOR);
     Assert.assertTrue(se instanceof DefaultStoreEngine);
     Assert.assertTrue(se.getCompactionPolicy() instanceof DummyCompactionPolicy);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
index 9acf244..f58d19a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java
@@ -210,14 +210,14 @@ public class TestEncryptionKeyRotation {
     boolean compacted = false;
     for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName)
         .getOnlineRegions(tableName)) {
-      for (Store store : region.getStores()) {
+      for (HStore store : ((HRegion) region).getStores()) {
         compacted = false;
         while (!compacted) {
           if (store.getStorefiles() != null) {
             while (store.getStorefilesCount() != 1) {
               Thread.sleep(100);
             }
-            for (StoreFile storefile : store.getStorefiles()) {
+            for (HStoreFile storefile : store.getStorefiles()) {
               if (!storefile.isCompactedAway()) {
                 compacted = true;
                 break;
@@ -234,10 +234,10 @@ public class TestEncryptionKeyRotation {
 
   private static List<Path> findStorefilePaths(TableName tableName) throws Exception {
     List<Path> paths = new ArrayList<>();
-    for (Region region:
-        TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(tableName)) {
-      for (Store store: region.getStores()) {
-        for (StoreFile storefile: store.getStorefiles()) {
+    for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName)
+        .getOnlineRegions(tableName)) {
+      for (HStore store : ((HRegion) region).getStores()) {
+        for (HStoreFile storefile : store.getStorefiles()) {
           paths.add(storefile.getPath());
         }
       }
@@ -247,13 +247,13 @@ public class TestEncryptionKeyRotation {
 
   private static List<Path> findCompactedStorefilePaths(TableName tableName) throws Exception {
     List<Path> paths = new ArrayList<>();
-    for (Region region:
-        TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(tableName)) {
-      for (Store store : region.getStores()) {
-        Collection<StoreFile> compactedfiles =
-            ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
+    for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName)
+        .getOnlineRegions(tableName)) {
+      for (HStore store : ((HRegion) region).getStores()) {
+        Collection<HStoreFile> compactedfiles =
+            store.getStoreEngine().getStoreFileManager().getCompactedfiles();
         if (compactedfiles != null) {
-          for (StoreFile storefile : compactedfiles) {
+          for (HStoreFile storefile : compactedfiles) {
             paths.add(storefile.getPath());
           }
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
index 3837e94..75c752c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java
@@ -55,8 +55,8 @@ public class TestEncryptionRandomKeying {
     List<Path> paths = new ArrayList<>();
     for (Region region:
         TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) {
-      for (Store store: region.getStores()) {
-        for (StoreFile storefile: store.getStorefiles()) {
+      for (HStore store : ((HRegion) region).getStores()) {
+        for (HStoreFile storefile : store.getStorefiles()) {
           paths.add(storefile.getPath());
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
index daddb5c..48081bd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java
@@ -94,7 +94,7 @@ public class TestFSErrorsExposed {
     TestHStoreFile.writeStoreFile(
         writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
 
-    StoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
+    HStoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
         BloomType.NONE, true);
     sf.initReader();
     StoreFileReader reader = sf.getReader();
@@ -144,12 +144,12 @@ public class TestFSErrorsExposed {
     TestHStoreFile.writeStoreFile(
         writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
 
-    StoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
+    HStoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
         BloomType.NONE, true);
 
     List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(
         Collections.singletonList(sf), false, true, false, false,
-        // 0 is passed as readpoint because this test operates on StoreFile directly
+        // 0 is passed as readpoint because this test operates on HStoreFile directly
         0);
     KeyValueScanner scanner = scanners.get(0);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 603203a..fc0659f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -518,7 +518,7 @@ public class TestHMobStore {
     this.store.add(new KeyValue(row, family, qf6, 1, value), null);
     flush(2);
 
-    Collection<StoreFile> storefiles = this.store.getStorefiles();
+    Collection<HStoreFile> storefiles = this.store.getStorefiles();
     checkMobHFileEncrytption(storefiles);
 
     // Scan the values
@@ -547,8 +547,8 @@ public class TestHMobStore {
     checkMobHFileEncrytption(this.store.getStorefiles());
   }
 
-  private void checkMobHFileEncrytption(Collection<StoreFile> storefiles) {
-    StoreFile storeFile = storefiles.iterator().next();
+  private void checkMobHFileEncrytption(Collection<HStoreFile> storefiles) {
+    HStoreFile storeFile = storefiles.iterator().next();
     HFile.Reader reader = storeFile.getReader().getHFileReader();
     byte[] encryptionKey = reader.getTrailer().getEncryptionKey();
     Assert.assertTrue(null != encryptionKey);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 334df17..0b0d651 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -18,6 +18,51 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
+import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -84,25 +129,14 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.regionserver.Region.RowLock;
-import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem;
+import org.apache.hadoop.hbase.regionserver.TestHStore.FaultyFileSystem;
 import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler;
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
@@ -116,6 +150,7 @@ import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.FaultyFSLog;
 import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALProvider;
@@ -136,50 +171,16 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
-import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
 
 /**
  * Basic stand-alone testing of HRegion.  No clusters!
@@ -897,7 +898,7 @@ public class TestHRegion {
       // this will create a region with 3 files
       assertEquals(3, region.getStore(family).getStorefilesCount());
       List<Path> storeFiles = new ArrayList<>(3);
-      for (StoreFile sf : region.getStore(family).getStorefiles()) {
+      for (HStoreFile sf : region.getStore(family).getStorefiles()) {
         storeFiles.add(sf.getPath());
       }
 
@@ -958,8 +959,8 @@ public class TestHRegion {
       }
 
       // now check whether we have only one store file, the compacted one
-      Collection<StoreFile> sfs = region.getStore(family).getStorefiles();
-      for (StoreFile sf : sfs) {
+      Collection<HStoreFile> sfs = region.getStore(family).getStorefiles();
+      for (HStoreFile sf : sfs) {
         LOG.info(sf.getPath());
       }
       if (!mismatchedRegionName) {
@@ -1011,7 +1012,7 @@ public class TestHRegion {
       // this will create a region with 3 files from flush
       assertEquals(3, region.getStore(family).getStorefilesCount());
       List<String> storeFiles = new ArrayList<>(3);
-      for (StoreFile sf : region.getStore(family).getStorefiles()) {
+      for (HStoreFile sf : region.getStore(family).getStorefiles()) {
         storeFiles.add(sf.getPath().getName());
       }
 
@@ -4052,8 +4053,8 @@ public class TestHRegion {
       }
       // before compaction
       HStore store = (HStore) region.getStore(fam1);
-      Collection<StoreFile> storeFiles = store.getStorefiles();
-      for (StoreFile storefile : storeFiles) {
+      Collection<HStoreFile> storeFiles = store.getStorefiles();
+      for (HStoreFile storefile : storeFiles) {
         StoreFileReader reader = storefile.getReader();
         reader.loadFileInfo();
         reader.loadBloomfilter();
@@ -4065,7 +4066,7 @@ public class TestHRegion {
 
       // after compaction
       storeFiles = store.getStorefiles();
-      for (StoreFile storefile : storeFiles) {
+      for (HStoreFile storefile : storeFiles) {
         StoreFileReader reader = storefile.getReader();
         reader.loadFileInfo();
         reader.loadBloomfilter();
@@ -4814,7 +4815,7 @@ public class TestHRegion {
       secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF);
 
       // move the file of the primary region to the archive, simulating a compaction
-      Collection<StoreFile> storeFiles = primaryRegion.getStore(families[0]).getStorefiles();
+      Collection<HStoreFile> storeFiles = primaryRegion.getStore(families[0]).getStorefiles();
       primaryRegion.getRegionFileSystem().removeStoreFiles(Bytes.toString(families[0]), storeFiles);
       Collection<StoreFileInfo> storeFileInfos = primaryRegion.getRegionFileSystem()
           .getStoreFiles(families[0]);
@@ -5843,9 +5844,9 @@ public class TestHRegion {
     put.addColumn(fam1, qual1, Bytes.toBytes("c1-value"));
     region.put(put);
     region.flush(true);
-    Store store = region.getStore(fam1);
-    Collection<StoreFile> storefiles = store.getStorefiles();
-    for (StoreFile sf : storefiles) {
+    HStore store = region.getStore(fam1);
+    Collection<HStoreFile> storefiles = store.getStorefiles();
+    for (HStoreFile sf : storefiles) {
       assertFalse("Tags should not be present "
           ,sf.getReader().getHFileReader().getFileContext().isIncludesTags());
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
index 62ada8f..ab98ec9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java
@@ -238,9 +238,9 @@ public class TestHRegionReplayEvents {
     verifyData(secondaryRegion, 0, 1000, cq, families);
 
     // close the region, and inspect that it has not flushed
-    Map<byte[], List<StoreFile>> files = secondaryRegion.close(false);
+    Map<byte[], List<HStoreFile>> files = secondaryRegion.close(false);
     // assert that there are no files (due to flush)
-    for (List<StoreFile> f : files.values()) {
+    for (List<HStoreFile> f : files.values()) {
       assertTrue(f.isEmpty());
     }
   }
@@ -1524,8 +1524,8 @@ public class TestHRegionReplayEvents {
       storeFileName.addAll(storeDesc.getStoreFileList());
     }
     // assert that the bulk loaded files are picked
-    for (Store s : secondaryRegion.getStores()) {
-      for (StoreFile sf : s.getStorefiles()) {
+    for (HStore s : secondaryRegion.getStores()) {
+      for (HStoreFile sf : s.getStorefiles()) {
         storeFileName.remove(sf.getPath().getName());
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index 97f8ce3..af64be6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -17,11 +17,10 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertThat;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
@@ -68,17 +67,13 @@ import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -87,6 +82,11 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
+
 /**
  * Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of
  * the region server's bullkLoad functionality.
@@ -160,7 +160,7 @@ public class TestHRegionServerBulkLoad {
         KeyValue kv = new KeyValue(rowkey(i), family, qualifier, now, value);
         writer.append(kv);
       }
-      writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(now));
+      writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(now));
     } finally {
       writer.close();
     }


[04/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
new file mode 100644
index 0000000..b685115
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java
@@ -0,0 +1,1743 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+import java.lang.ref.SoftReference;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FilterFileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MemoryCompactionPolicy;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+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.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
+import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
+import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
+import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.util.Progressable;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+/**
+ * Test class for the Store
+ */
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestHStore {
+  private static final Log LOG = LogFactory.getLog(TestHStore.class);
+  @Rule
+  public TestName name = new TestName();
+
+  HStore store;
+  byte [] table = Bytes.toBytes("table");
+  byte [] family = Bytes.toBytes("family");
+
+  byte [] row = Bytes.toBytes("row");
+  byte [] row2 = Bytes.toBytes("row2");
+  byte [] qf1 = Bytes.toBytes("qf1");
+  byte [] qf2 = Bytes.toBytes("qf2");
+  byte [] qf3 = Bytes.toBytes("qf3");
+  byte [] qf4 = Bytes.toBytes("qf4");
+  byte [] qf5 = Bytes.toBytes("qf5");
+  byte [] qf6 = Bytes.toBytes("qf6");
+
+  NavigableSet<byte[]> qualifiers = new ConcurrentSkipListSet<>(Bytes.BYTES_COMPARATOR);
+
+  List<Cell> expected = new ArrayList<>();
+  List<Cell> result = new ArrayList<>();
+
+  long id = System.currentTimeMillis();
+  Get get = new Get(row);
+
+  private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString();
+
+
+  /**
+   * Setup
+   * @throws IOException
+   */
+  @Before
+  public void setUp() throws IOException {
+    qualifiers.add(qf1);
+    qualifiers.add(qf3);
+    qualifiers.add(qf5);
+
+    Iterator<byte[]> iter = qualifiers.iterator();
+    while(iter.hasNext()){
+      byte [] next = iter.next();
+      expected.add(new KeyValue(row, family, next, 1, (byte[])null));
+      get.addColumn(family, next);
+    }
+  }
+
+  private void init(String methodName) throws IOException {
+    init(methodName, TEST_UTIL.getConfiguration());
+  }
+
+  private Store init(String methodName, Configuration conf) throws IOException {
+    HColumnDescriptor hcd = new HColumnDescriptor(family);
+    // some of the tests write 4 versions and then flush
+    // (with HBASE-4241, lower versions are collected on flush)
+    hcd.setMaxVersions(4);
+    return init(methodName, conf, hcd);
+  }
+
+  private HStore init(String methodName, Configuration conf, HColumnDescriptor hcd)
+      throws IOException {
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
+    return init(methodName, conf, htd, hcd);
+  }
+
+  private HStore init(String methodName, Configuration conf, HTableDescriptor htd,
+      HColumnDescriptor hcd) throws IOException {
+    return init(methodName, conf, htd, hcd, null);
+  }
+
+  @SuppressWarnings("deprecation")
+  private HStore init(String methodName, Configuration conf, HTableDescriptor htd,
+      HColumnDescriptor hcd, MyStoreHook hook) throws IOException {
+    return init(methodName, conf, htd, hcd, hook, false);
+  }
+  @SuppressWarnings("deprecation")
+  private HStore init(String methodName, Configuration conf, HTableDescriptor htd,
+      HColumnDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {
+    //Setting up a Store
+    Path basedir = new Path(DIR+methodName);
+    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
+    final Path logdir = new Path(basedir, AbstractFSWALProvider.getWALDirectoryName(methodName));
+
+    FileSystem fs = FileSystem.get(conf);
+
+    fs.delete(logdir, true);
+
+    if (htd.hasFamily(hcd.getName())) {
+      htd.modifyFamily(hcd);
+    } else {
+      htd.addFamily(hcd);
+    }
+    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
+      MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null);
+    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
+    final Configuration walConf = new Configuration(conf);
+    FSUtils.setRootDir(walConf, basedir);
+    final WALFactory wals = new WALFactory(walConf, null, methodName);
+    HRegion region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes(),
+            info.getTable().getNamespace()), fs, conf, info, htd, null);
+    if (hook == null) {
+      store = new HStore(region, hcd, conf);
+    } else {
+      store = new MyStore(region, hcd, conf, hook, switchToPread);
+    }
+    return store;
+  }
+
+  /**
+   * Test we do not lose data if we fail a flush and then close.
+   * Part of HBase-10466
+   * @throws Exception
+   */
+  @Test
+  public void testFlushSizeAccounting() throws Exception {
+    LOG.info("Setting up a faulty file system that cannot write in " +
+      this.name.getMethodName());
+    final Configuration conf = HBaseConfiguration.create();
+    // Only retry once.
+    conf.setInt("hbase.hstore.flush.retries.number", 1);
+    User user = User.createUserForTesting(conf, this.name.getMethodName(),
+      new String[]{"foo"});
+    // Inject our faulty LocalFileSystem
+    conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);
+    user.runAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        // Make sure it worked (above is sensitive to caching details in hadoop core)
+        FileSystem fs = FileSystem.get(conf);
+        assertEquals(FaultyFileSystem.class, fs.getClass());
+        FaultyFileSystem ffs = (FaultyFileSystem)fs;
+
+        // Initialize region
+        init(name.getMethodName(), conf);
+
+        MemstoreSize size = store.memstore.getFlushableSize();
+        assertEquals(0, size.getDataSize());
+        LOG.info("Adding some data");
+        MemstoreSize kvSize = new MemstoreSize();
+        store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), kvSize);
+        // add the heap size of active (mutable) segment
+        kvSize.incMemstoreSize(0, MutableSegment.DEEP_OVERHEAD);
+        size = store.memstore.getFlushableSize();
+        assertEquals(kvSize, size);
+        // Flush.  Bug #1 from HBASE-10466.  Make sure size calculation on failed flush is right.
+        try {
+          LOG.info("Flushing");
+          flushStore(store, id++);
+          fail("Didn't bubble up IOE!");
+        } catch (IOException ioe) {
+          assertTrue(ioe.getMessage().contains("Fault injected"));
+        }
+        // due to snapshot, change mutable to immutable segment
+        kvSize.incMemstoreSize(0,
+            CSLMImmutableSegment.DEEP_OVERHEAD_CSLM-MutableSegment.DEEP_OVERHEAD);
+        size = store.memstore.getFlushableSize();
+        assertEquals(kvSize, size);
+        MemstoreSize kvSize2 = new MemstoreSize();
+        store.add(new KeyValue(row, family, qf2, 2, (byte[])null), kvSize2);
+        kvSize2.incMemstoreSize(0, MutableSegment.DEEP_OVERHEAD);
+        // Even though we add a new kv, we expect the flushable size to be 'same' since we have
+        // not yet cleared the snapshot -- the above flush failed.
+        assertEquals(kvSize, size);
+        ffs.fault.set(false);
+        flushStore(store, id++);
+        size = store.memstore.getFlushableSize();
+        // Size should be the foreground kv size.
+        assertEquals(kvSize2, size);
+        flushStore(store, id++);
+        size = store.memstore.getFlushableSize();
+        assertEquals(0, size.getDataSize());
+        assertEquals(MutableSegment.DEEP_OVERHEAD, size.getHeapSize());
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Verify that compression and data block encoding are respected by the
+   * Store.createWriterInTmp() method, used on store flush.
+   */
+  @Test
+  public void testCreateWriter() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    FileSystem fs = FileSystem.get(conf);
+
+    HColumnDescriptor hcd = new HColumnDescriptor(family);
+    hcd.setCompressionType(Compression.Algorithm.GZ);
+    hcd.setDataBlockEncoding(DataBlockEncoding.DIFF);
+    init(name.getMethodName(), conf, hcd);
+
+    // Test createWriterInTmp()
+    StoreFileWriter writer = store.createWriterInTmp(4, hcd.getCompressionType(), false, true, false);
+    Path path = writer.getPath();
+    writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1)));
+    writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2)));
+    writer.append(new KeyValue(row2, family, qf1, Bytes.toBytes(3)));
+    writer.append(new KeyValue(row2, family, qf2, Bytes.toBytes(4)));
+    writer.close();
+
+    // Verify that compression and encoding settings are respected
+    HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), true, conf);
+    assertEquals(hcd.getCompressionType(), reader.getCompressionAlgorithm());
+    assertEquals(hcd.getDataBlockEncoding(), reader.getDataBlockEncoding());
+    reader.close();
+  }
+
+  @Test
+  public void testDeleteExpiredStoreFiles() throws Exception {
+    testDeleteExpiredStoreFiles(0);
+    testDeleteExpiredStoreFiles(1);
+  }
+
+  /*
+   * @param minVersions the MIN_VERSIONS for the column family
+   */
+  public void testDeleteExpiredStoreFiles(int minVersions) throws Exception {
+    int storeFileNum = 4;
+    int ttl = 4;
+    IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge();
+    EnvironmentEdgeManagerTestHelper.injectEdge(edge);
+
+    Configuration conf = HBaseConfiguration.create();
+    // Enable the expired store file deletion
+    conf.setBoolean("hbase.store.delete.expired.storefile", true);
+    // Set the compaction threshold higher to avoid normal compactions.
+    conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 5);
+
+    HColumnDescriptor hcd = new HColumnDescriptor(family);
+    hcd.setMinVersions(minVersions);
+    hcd.setTimeToLive(ttl);
+    init(name.getMethodName() + "-" + minVersions, conf, hcd);
+
+    long storeTtl = this.store.getScanInfo().getTtl();
+    long sleepTime = storeTtl / storeFileNum;
+    long timeStamp;
+    // There are 4 store files and the max time stamp difference among these
+    // store files will be (this.store.ttl / storeFileNum)
+    for (int i = 1; i <= storeFileNum; i++) {
+      LOG.info("Adding some data for the store file #" + i);
+      timeStamp = EnvironmentEdgeManager.currentTime();
+      this.store.add(new KeyValue(row, family, qf1, timeStamp, (byte[]) null), null);
+      this.store.add(new KeyValue(row, family, qf2, timeStamp, (byte[]) null), null);
+      this.store.add(new KeyValue(row, family, qf3, timeStamp, (byte[]) null), null);
+      flush(i);
+      edge.incrementTime(sleepTime);
+    }
+
+    // Verify the total number of store files
+    assertEquals(storeFileNum, this.store.getStorefiles().size());
+
+     // Each call will find one expired store file and delete it before compaction happens.
+     // There will be no compaction due to threshold above. Last file will not be replaced.
+    for (int i = 1; i <= storeFileNum - 1; i++) {
+      // verify the expired store file.
+      assertFalse(this.store.requestCompaction().isPresent());
+      Collection<HStoreFile> sfs = this.store.getStorefiles();
+      // Ensure i files are gone.
+      if (minVersions == 0) {
+        assertEquals(storeFileNum - i, sfs.size());
+        // Ensure only non-expired files remain.
+        for (HStoreFile sf : sfs) {
+          assertTrue(sf.getReader().getMaxTimestamp() >= (edge.currentTime() - storeTtl));
+        }
+      } else {
+        assertEquals(storeFileNum, sfs.size());
+      }
+      // Let the next store file expired.
+      edge.incrementTime(sleepTime);
+    }
+    assertFalse(this.store.requestCompaction().isPresent());
+
+    Collection<HStoreFile> sfs = this.store.getStorefiles();
+    // Assert the last expired file is not removed.
+    if (minVersions == 0) {
+      assertEquals(1, sfs.size());
+    }
+    long ts = sfs.iterator().next().getReader().getMaxTimestamp();
+    assertTrue(ts < (edge.currentTime() - storeTtl));
+
+    for (HStoreFile sf : sfs) {
+      sf.closeStoreFile(true);
+    }
+  }
+
+  @Test
+  public void testLowestModificationTime() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    FileSystem fs = FileSystem.get(conf);
+    // Initialize region
+    init(name.getMethodName(), conf);
+
+    int storeFileNum = 4;
+    for (int i = 1; i <= storeFileNum; i++) {
+      LOG.info("Adding some data for the store file #"+i);
+      this.store.add(new KeyValue(row, family, qf1, i, (byte[])null), null);
+      this.store.add(new KeyValue(row, family, qf2, i, (byte[])null), null);
+      this.store.add(new KeyValue(row, family, qf3, i, (byte[])null), null);
+      flush(i);
+    }
+    // after flush; check the lowest time stamp
+    long lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles());
+    long lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles());
+    assertEquals(lowestTimeStampFromManager,lowestTimeStampFromFS);
+
+    // after compact; check the lowest time stamp
+    store.compact(store.requestCompaction().get(), NoLimitThroughputController.INSTANCE, null);
+    lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles());
+    lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles());
+    assertEquals(lowestTimeStampFromManager, lowestTimeStampFromFS);
+  }
+
+  private static long getLowestTimeStampFromFS(FileSystem fs,
+      final Collection<HStoreFile> candidates) throws IOException {
+    long minTs = Long.MAX_VALUE;
+    if (candidates.isEmpty()) {
+      return minTs;
+    }
+    Path[] p = new Path[candidates.size()];
+    int i = 0;
+    for (HStoreFile sf : candidates) {
+      p[i] = sf.getPath();
+      ++i;
+    }
+
+    FileStatus[] stats = fs.listStatus(p);
+    if (stats == null || stats.length == 0) {
+      return minTs;
+    }
+    for (FileStatus s : stats) {
+      minTs = Math.min(minTs, s.getModificationTime());
+    }
+    return minTs;
+  }
+
+  //////////////////////////////////////////////////////////////////////////////
+  // Get tests
+  //////////////////////////////////////////////////////////////////////////////
+
+  private static final int BLOCKSIZE_SMALL = 8192;
+  /**
+   * Test for hbase-1686.
+   * @throws IOException
+   */
+  @Test
+  public void testEmptyStoreFile() throws IOException {
+    init(this.name.getMethodName());
+    // Write a store file.
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
+    flush(1);
+    // Now put in place an empty store file.  Its a little tricky.  Have to
+    // do manually with hacked in sequence id.
+    HStoreFile f = this.store.getStorefiles().iterator().next();
+    Path storedir = f.getPath().getParent();
+    long seqid = f.getMaxSequenceId();
+    Configuration c = HBaseConfiguration.create();
+    FileSystem fs = FileSystem.get(c);
+    HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();
+    StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c),
+        fs)
+            .withOutputDir(storedir)
+            .withFileContext(meta)
+            .build();
+    w.appendMetadata(seqid + 1, false);
+    w.close();
+    this.store.close();
+    // Reopen it... should pick up two files
+    this.store = new HStore(this.store.getHRegion(), this.store.getColumnFamilyDescriptor(), c);
+    assertEquals(2, this.store.getStorefilesCount());
+
+    result = HBaseTestingUtility.getFromStoreFile(store,
+        get.getRow(),
+        qualifiers);
+    assertEquals(1, result.size());
+  }
+
+  /**
+   * Getting data from memstore only
+   * @throws IOException
+   */
+  @Test
+  public void testGet_FromMemStoreOnly() throws IOException {
+    init(this.name.getMethodName());
+
+    //Put data in memstore
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null), null);
+
+    //Get
+    result = HBaseTestingUtility.getFromStoreFile(store,
+        get.getRow(), qualifiers);
+
+    //Compare
+    assertCheck();
+  }
+
+  /**
+   * Getting data from files only
+   * @throws IOException
+   */
+  @Test
+  public void testGet_FromFilesOnly() throws IOException {
+    init(this.name.getMethodName());
+
+    //Put data in memstore
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
+    //flush
+    flush(1);
+
+    //Add more data
+    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null), null);
+    //flush
+    flush(2);
+
+    //Add more data
+    this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null), null);
+    //flush
+    flush(3);
+
+    //Get
+    result = HBaseTestingUtility.getFromStoreFile(store,
+        get.getRow(),
+        qualifiers);
+    //this.store.get(get, qualifiers, result);
+
+    //Need to sort the result since multiple files
+    Collections.sort(result, CellComparator.COMPARATOR);
+
+    //Compare
+    assertCheck();
+  }
+
+  /**
+   * Getting data from memstore and files
+   * @throws IOException
+   */
+  @Test
+  public void testGet_FromMemStoreAndFiles() throws IOException {
+    init(this.name.getMethodName());
+
+    //Put data in memstore
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
+    //flush
+    flush(1);
+
+    //Add more data
+    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null), null);
+    //flush
+    flush(2);
+
+    //Add more data
+    this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null), null);
+    this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null), null);
+
+    //Get
+    result = HBaseTestingUtility.getFromStoreFile(store,
+        get.getRow(), qualifiers);
+
+    //Need to sort the result since multiple files
+    Collections.sort(result, CellComparator.COMPARATOR);
+
+    //Compare
+    assertCheck();
+  }
+
+  private void flush(int storeFilessize) throws IOException{
+    this.store.snapshot();
+    flushStore(store, id++);
+    assertEquals(storeFilessize, this.store.getStorefiles().size());
+    assertEquals(0, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
+  }
+
+  private void assertCheck() {
+    assertEquals(expected.size(), result.size());
+    for(int i=0; i<expected.size(); i++) {
+      assertEquals(expected.get(i), result.get(i));
+    }
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvironmentEdgeManagerTestHelper.reset();
+  }
+
+  @Test
+  public void testHandleErrorsInFlush() throws Exception {
+    LOG.info("Setting up a faulty file system that cannot write");
+
+    final Configuration conf = HBaseConfiguration.create();
+    User user = User.createUserForTesting(conf,
+        "testhandleerrorsinflush", new String[]{"foo"});
+    // Inject our faulty LocalFileSystem
+    conf.setClass("fs.file.impl", FaultyFileSystem.class,
+        FileSystem.class);
+    user.runAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        // Make sure it worked (above is sensitive to caching details in hadoop core)
+        FileSystem fs = FileSystem.get(conf);
+        assertEquals(FaultyFileSystem.class, fs.getClass());
+
+        // Initialize region
+        init(name.getMethodName(), conf);
+
+        LOG.info("Adding some data");
+        store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
+        store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
+        store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
+
+        LOG.info("Before flush, we should have no files");
+
+        Collection<StoreFileInfo> files =
+          store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName());
+        assertEquals(0, files != null ? files.size() : 0);
+
+        //flush
+        try {
+          LOG.info("Flushing");
+          flush(1);
+          fail("Didn't bubble up IOE!");
+        } catch (IOException ioe) {
+          assertTrue(ioe.getMessage().contains("Fault injected"));
+        }
+
+        LOG.info("After failed flush, we should still have no files!");
+        files = store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName());
+        assertEquals(0, files != null ? files.size() : 0);
+        store.getHRegion().getWAL().close();
+        return null;
+      }
+    });
+    FileSystem.closeAllForUGI(user.getUGI());
+  }
+
+  /**
+   * Faulty file system that will fail if you write past its fault position the FIRST TIME
+   * only; thereafter it will succeed.  Used by {@link TestHRegion} too.
+   */
+  static class FaultyFileSystem extends FilterFileSystem {
+    List<SoftReference<FaultyOutputStream>> outStreams = new ArrayList<>();
+    private long faultPos = 200;
+    AtomicBoolean fault = new AtomicBoolean(true);
+
+    public FaultyFileSystem() {
+      super(new LocalFileSystem());
+      System.err.println("Creating faulty!");
+    }
+
+    @Override
+    public FSDataOutputStream create(Path p) throws IOException {
+      return new FaultyOutputStream(super.create(p), faultPos, fault);
+    }
+
+    @Override
+    public FSDataOutputStream create(Path f, FsPermission permission,
+        boolean overwrite, int bufferSize, short replication, long blockSize,
+        Progressable progress) throws IOException {
+      return new FaultyOutputStream(super.create(f, permission,
+          overwrite, bufferSize, replication, blockSize, progress), faultPos, fault);
+    }
+
+    @Override
+    public FSDataOutputStream createNonRecursive(Path f, boolean overwrite,
+        int bufferSize, short replication, long blockSize, Progressable progress)
+    throws IOException {
+      // Fake it.  Call create instead.  The default implementation throws an IOE
+      // that this is not supported.
+      return create(f, overwrite, bufferSize, replication, blockSize, progress);
+    }
+  }
+
+  static class FaultyOutputStream extends FSDataOutputStream {
+    volatile long faultPos = Long.MAX_VALUE;
+    private final AtomicBoolean fault;
+
+    public FaultyOutputStream(FSDataOutputStream out, long faultPos, final AtomicBoolean fault)
+    throws IOException {
+      super(out, null);
+      this.faultPos = faultPos;
+      this.fault = fault;
+    }
+
+    @Override
+    public void write(byte[] buf, int offset, int length) throws IOException {
+      System.err.println("faulty stream write at pos " + getPos());
+      injectFault();
+      super.write(buf, offset, length);
+    }
+
+    private void injectFault() throws IOException {
+      if (this.fault.get() && getPos() >= faultPos) {
+        throw new IOException("Fault injected");
+      }
+    }
+  }
+
+  private static void flushStore(HStore store, long id) throws IOException {
+    StoreFlushContext storeFlushCtx = store.createFlushContext(id);
+    storeFlushCtx.prepare();
+    storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
+    storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
+  }
+
+  /**
+   * Generate a list of KeyValues for testing based on given parameters
+   * @param timestamps
+   * @param numRows
+   * @param qualifier
+   * @param family
+   * @return
+   */
+  List<Cell> getKeyValueSet(long[] timestamps, int numRows,
+      byte[] qualifier, byte[] family) {
+    List<Cell> kvList = new ArrayList<>();
+    for (int i=1;i<=numRows;i++) {
+      byte[] b = Bytes.toBytes(i);
+      for (long timestamp: timestamps) {
+        kvList.add(new KeyValue(b, family, qualifier, timestamp, b));
+      }
+    }
+    return kvList;
+  }
+
+  /**
+   * Test to ensure correctness when using Stores with multiple timestamps
+   * @throws IOException
+   */
+  @Test
+  public void testMultipleTimestamps() throws IOException {
+    int numRows = 1;
+    long[] timestamps1 = new long[] {1,5,10,20};
+    long[] timestamps2 = new long[] {30,80};
+
+    init(this.name.getMethodName());
+
+    List<Cell> kvList1 = getKeyValueSet(timestamps1,numRows, qf1, family);
+    for (Cell kv : kvList1) {
+      this.store.add(kv, null);
+    }
+
+    this.store.snapshot();
+    flushStore(store, id++);
+
+    List<Cell> kvList2 = getKeyValueSet(timestamps2,numRows, qf1, family);
+    for(Cell kv : kvList2) {
+      this.store.add(kv, null);
+    }
+
+    List<Cell> result;
+    Get get = new Get(Bytes.toBytes(1));
+    get.addColumn(family,qf1);
+
+    get.setTimeRange(0,15);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
+    assertTrue(result.size()>0);
+
+    get.setTimeRange(40,90);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
+    assertTrue(result.size()>0);
+
+    get.setTimeRange(10,45);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
+    assertTrue(result.size()>0);
+
+    get.setTimeRange(80,145);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
+    assertTrue(result.size()>0);
+
+    get.setTimeRange(1,2);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
+    assertTrue(result.size()>0);
+
+    get.setTimeRange(90,200);
+    result = HBaseTestingUtility.getFromStoreFile(store, get);
+    assertTrue(result.size()==0);
+  }
+
+  /**
+   * Test for HBASE-3492 - Test split on empty colfam (no store files).
+   *
+   * @throws IOException When the IO operations fail.
+   */
+  @Test
+  public void testSplitWithEmptyColFam() throws IOException {
+    init(this.name.getMethodName());
+    assertFalse(store.getSplitPoint().isPresent());
+    store.getHRegion().forceSplit(null);
+    assertFalse(store.getSplitPoint().isPresent());
+    store.getHRegion().clearSplit();
+  }
+
+  @Test
+  public void testStoreUsesConfigurationFromHcdAndHtd() throws Exception {
+    final String CONFIG_KEY = "hbase.regionserver.thread.compaction.throttle";
+    long anyValue = 10;
+
+    // We'll check that it uses correct config and propagates it appropriately by going thru
+    // the simplest "real" path I can find - "throttleCompaction", which just checks whether
+    // a number we pass in is higher than some config value, inside compactionPolicy.
+    Configuration conf = HBaseConfiguration.create();
+    conf.setLong(CONFIG_KEY, anyValue);
+    init(name.getMethodName() + "-xml", conf);
+    assertTrue(store.throttleCompaction(anyValue + 1));
+    assertFalse(store.throttleCompaction(anyValue));
+
+    // HTD overrides XML.
+    --anyValue;
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
+    HColumnDescriptor hcd = new HColumnDescriptor(family);
+    htd.setConfiguration(CONFIG_KEY, Long.toString(anyValue));
+    init(name.getMethodName() + "-htd", conf, htd, hcd);
+    assertTrue(store.throttleCompaction(anyValue + 1));
+    assertFalse(store.throttleCompaction(anyValue));
+
+    // HCD overrides them both.
+    --anyValue;
+    hcd.setConfiguration(CONFIG_KEY, Long.toString(anyValue));
+    init(name.getMethodName() + "-hcd", conf, htd, hcd);
+    assertTrue(store.throttleCompaction(anyValue + 1));
+    assertFalse(store.throttleCompaction(anyValue));
+  }
+
+  public static class DummyStoreEngine extends DefaultStoreEngine {
+    public static DefaultCompactor lastCreatedCompactor = null;
+
+    @Override
+    protected void createComponents(Configuration conf, HStore store, CellComparator comparator)
+        throws IOException {
+      super.createComponents(conf, store, comparator);
+      lastCreatedCompactor = this.compactor;
+    }
+  }
+
+  @Test
+  public void testStoreUsesSearchEngineOverride() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(StoreEngine.STORE_ENGINE_CLASS_KEY, DummyStoreEngine.class.getName());
+    init(this.name.getMethodName(), conf);
+    assertEquals(DummyStoreEngine.lastCreatedCompactor,
+      this.store.storeEngine.getCompactor());
+  }
+
+  private void addStoreFile() throws IOException {
+    HStoreFile f = this.store.getStorefiles().iterator().next();
+    Path storedir = f.getPath().getParent();
+    long seqid = this.store.getMaxSequenceId();
+    Configuration c = TEST_UTIL.getConfiguration();
+    FileSystem fs = FileSystem.get(c);
+    HFileContext fileContext = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();
+    StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c),
+        fs)
+            .withOutputDir(storedir)
+            .withFileContext(fileContext)
+            .build();
+    w.appendMetadata(seqid + 1, false);
+    w.close();
+    LOG.info("Added store file:" + w.getPath());
+  }
+
+  private void archiveStoreFile(int index) throws IOException {
+    Collection<HStoreFile> files = this.store.getStorefiles();
+    HStoreFile sf = null;
+    Iterator<HStoreFile> it = files.iterator();
+    for (int i = 0; i <= index; i++) {
+      sf = it.next();
+    }
+    store.getRegionFileSystem().removeStoreFiles(store.getColumnFamilyName(), Lists.newArrayList(sf));
+  }
+
+  private void closeCompactedFile(int index) throws IOException {
+    Collection<HStoreFile> files =
+        this.store.getStoreEngine().getStoreFileManager().getCompactedfiles();
+    HStoreFile sf = null;
+    Iterator<HStoreFile> it = files.iterator();
+    for (int i = 0; i <= index; i++) {
+      sf = it.next();
+    }
+    sf.closeStoreFile(true);
+    store.getStoreEngine().getStoreFileManager().removeCompactedFiles(Lists.newArrayList(sf));
+  }
+
+  @Test
+  public void testRefreshStoreFiles() throws Exception {
+    init(name.getMethodName());
+
+    assertEquals(0, this.store.getStorefilesCount());
+
+    // Test refreshing store files when no store files are there
+    store.refreshStoreFiles();
+    assertEquals(0, this.store.getStorefilesCount());
+
+    // add some data, flush
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
+    flush(1);
+    assertEquals(1, this.store.getStorefilesCount());
+
+    // add one more file
+    addStoreFile();
+
+    assertEquals(1, this.store.getStorefilesCount());
+    store.refreshStoreFiles();
+    assertEquals(2, this.store.getStorefilesCount());
+
+    // add three more files
+    addStoreFile();
+    addStoreFile();
+    addStoreFile();
+
+    assertEquals(2, this.store.getStorefilesCount());
+    store.refreshStoreFiles();
+    assertEquals(5, this.store.getStorefilesCount());
+
+    closeCompactedFile(0);
+    archiveStoreFile(0);
+
+    assertEquals(5, this.store.getStorefilesCount());
+    store.refreshStoreFiles();
+    assertEquals(4, this.store.getStorefilesCount());
+
+    archiveStoreFile(0);
+    archiveStoreFile(1);
+    archiveStoreFile(2);
+
+    assertEquals(4, this.store.getStorefilesCount());
+    store.refreshStoreFiles();
+    assertEquals(1, this.store.getStorefilesCount());
+
+    archiveStoreFile(0);
+    store.refreshStoreFiles();
+    assertEquals(0, this.store.getStorefilesCount());
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testRefreshStoreFilesNotChanged() throws IOException {
+    init(name.getMethodName());
+
+    assertEquals(0, this.store.getStorefilesCount());
+
+    // add some data, flush
+    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
+    flush(1);
+    // add one more file
+    addStoreFile();
+
+    HStore spiedStore = spy(store);
+
+    // call first time after files changed
+    spiedStore.refreshStoreFiles();
+    assertEquals(2, this.store.getStorefilesCount());
+    verify(spiedStore, times(1)).replaceStoreFiles(any(Collection.class), any(Collection.class));
+
+    // call second time
+    spiedStore.refreshStoreFiles();
+
+    //ensure that replaceStoreFiles is not called if files are not refreshed
+    verify(spiedStore, times(0)).replaceStoreFiles(null, null);
+  }
+
+  private long countMemStoreScanner(StoreScanner scanner) {
+    if (scanner.currentScanners == null) {
+      return 0;
+    }
+    return scanner.currentScanners.stream()
+            .filter(s -> !s.isFileScanner())
+            .count();
+  }
+
+  @Test
+  public void testNumberOfMemStoreScannersAfterFlush() throws IOException {
+    long seqId = 100;
+    long timestamp = System.currentTimeMillis();
+    Cell cell0 = CellUtil.createCell(row, family, qf1, timestamp,
+            KeyValue.Type.Put.getCode(), qf1);
+    CellUtil.setSequenceId(cell0, seqId);
+    testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Collections.EMPTY_LIST);
+
+    Cell cell1 = CellUtil.createCell(row, family, qf2, timestamp,
+            KeyValue.Type.Put.getCode(), qf1);
+    CellUtil.setSequenceId(cell1, seqId);
+    testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1));
+
+    seqId = 101;
+    timestamp = System.currentTimeMillis();
+    Cell cell2 = CellUtil.createCell(row2, family, qf2, timestamp,
+            KeyValue.Type.Put.getCode(), qf1);
+     CellUtil.setSequenceId(cell2, seqId);
+    testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1, cell2));
+  }
+
+  private void testNumberOfMemStoreScannersAfterFlush(List<Cell> inputCellsBeforeSnapshot,
+      List<Cell> inputCellsAfterSnapshot) throws IOException {
+    init(this.name.getMethodName() + "-" + inputCellsBeforeSnapshot.size());
+    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+    long seqId = Long.MIN_VALUE;
+    for (Cell c : inputCellsBeforeSnapshot) {
+      quals.add(CellUtil.cloneQualifier(c));
+      seqId = Math.max(seqId, c.getSequenceId());
+    }
+    for (Cell c : inputCellsAfterSnapshot) {
+      quals.add(CellUtil.cloneQualifier(c));
+      seqId = Math.max(seqId, c.getSequenceId());
+    }
+    inputCellsBeforeSnapshot.forEach(c -> store.add(c, null));
+    StoreFlushContext storeFlushCtx = store.createFlushContext(id++);
+    storeFlushCtx.prepare();
+    inputCellsAfterSnapshot.forEach(c -> store.add(c, null));
+    int numberOfMemScannersWhenScaning = inputCellsAfterSnapshot.isEmpty() ? 1 : 2;
+    try (StoreScanner s = (StoreScanner) store.getScanner(new Scan(), quals, seqId)) {
+      // snaptshot + active (if it isn't empty)
+      assertEquals(numberOfMemScannersWhenScaning, countMemStoreScanner(s));
+      storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
+      storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
+      boolean more;
+      int cellCount = 0;
+      do {
+        List<Cell> cells = new ArrayList<>();
+        more = s.next(cells);
+        cellCount += cells.size();
+        assertEquals(more ? numberOfMemScannersWhenScaning : 0, countMemStoreScanner(s));
+      } while (more);
+      assertEquals("The number of cells added before snapshot is " + inputCellsBeforeSnapshot.size()
+          + ", The number of cells added after snapshot is " + inputCellsAfterSnapshot.size(),
+          inputCellsBeforeSnapshot.size() + inputCellsAfterSnapshot.size(), cellCount);
+      // the current scanners is cleared
+      assertEquals(0, countMemStoreScanner(s));
+    }
+  }
+
+  private Cell createCell(byte[] qualifier, long ts, long sequenceId, byte[] value) throws IOException {
+    Cell c = CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put.getCode(), value);
+    CellUtil.setSequenceId(c, sequenceId);
+    return c;
+  }
+
+  private Cell createCell(byte[] row, byte[] qualifier, long ts, long sequenceId, byte[] value)
+      throws IOException {
+    Cell c = CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put.getCode(), value);
+    CellUtil.setSequenceId(c, sequenceId);
+    return c;
+  }
+
+  @Test
+  public void testFlushBeforeCompletingScanWoFilter() throws IOException, InterruptedException {
+    final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);
+    final int expectedSize = 3;
+    testFlushBeforeCompletingScan(new MyListHook() {
+      @Override
+      public void hook(int currentSize) {
+        if (currentSize == expectedSize - 1) {
+          try {
+            flushStore(store, id++);
+            timeToGoNextRow.set(true);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      }
+    }, new FilterBase() {
+      @Override
+      public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
+        return ReturnCode.INCLUDE;
+      }
+    }, expectedSize);
+  }
+
+  @Test
+  public void testFlushBeforeCompletingScanWithFilter() throws IOException, InterruptedException {
+    final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);
+    final int expectedSize = 2;
+    testFlushBeforeCompletingScan(new MyListHook() {
+      @Override
+      public void hook(int currentSize) {
+        if (currentSize == expectedSize - 1) {
+          try {
+            flushStore(store, id++);
+            timeToGoNextRow.set(true);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      }
+    }, new FilterBase() {
+      @Override
+      public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
+        if (timeToGoNextRow.get()) {
+          timeToGoNextRow.set(false);
+          return ReturnCode.NEXT_ROW;
+        } else {
+          return ReturnCode.INCLUDE;
+        }
+      }
+    }, expectedSize);
+  }
+
+  @Test
+  public void testFlushBeforeCompletingScanWithFilterHint() throws IOException, InterruptedException {
+    final AtomicBoolean timeToGetHint = new AtomicBoolean(false);
+    final int expectedSize = 2;
+    testFlushBeforeCompletingScan(new MyListHook() {
+      @Override
+      public void hook(int currentSize) {
+        if (currentSize == expectedSize - 1) {
+          try {
+            flushStore(store, id++);
+            timeToGetHint.set(true);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      }
+    }, new FilterBase() {
+      @Override
+      public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
+        if (timeToGetHint.get()) {
+          timeToGetHint.set(false);
+          return Filter.ReturnCode.SEEK_NEXT_USING_HINT;
+        } else {
+          return Filter.ReturnCode.INCLUDE;
+        }
+      }
+      @Override
+      public Cell getNextCellHint(Cell currentCell) throws IOException {
+        return currentCell;
+      }
+    }, expectedSize);
+  }
+
+  private void testFlushBeforeCompletingScan(MyListHook hook, Filter filter, int expectedSize)
+          throws IOException, InterruptedException {
+    Configuration conf = HBaseConfiguration.create();
+    HColumnDescriptor hcd = new HColumnDescriptor(family);
+    hcd.setMaxVersions(1);
+    byte[] r0 = Bytes.toBytes("row0");
+    byte[] r1 = Bytes.toBytes("row1");
+    byte[] r2 = Bytes.toBytes("row2");
+    byte[] value0 = Bytes.toBytes("value0");
+    byte[] value1 = Bytes.toBytes("value1");
+    byte[] value2 = Bytes.toBytes("value2");
+    MemstoreSize memStoreSize = new MemstoreSize();
+    long ts = EnvironmentEdgeManager.currentTime();
+    long seqId = 100;
+    init(name.getMethodName(), conf, new HTableDescriptor(TableName.valueOf(table)), hcd, new MyStoreHook() {
+      @Override
+      public long getSmallestReadPoint(HStore store) {
+        return seqId + 3;
+      }
+    });
+    // The cells having the value0 won't be flushed to disk because the value of max version is 1
+    store.add(createCell(r0, qf1, ts, seqId, value0), memStoreSize);
+    store.add(createCell(r0, qf2, ts, seqId, value0), memStoreSize);
+    store.add(createCell(r0, qf3, ts, seqId, value0), memStoreSize);
+    store.add(createCell(r1, qf1, ts + 1, seqId + 1, value1), memStoreSize);
+    store.add(createCell(r1, qf2, ts + 1, seqId + 1, value1), memStoreSize);
+    store.add(createCell(r1, qf3, ts + 1, seqId + 1, value1), memStoreSize);
+    store.add(createCell(r2, qf1, ts + 2, seqId + 2, value2), memStoreSize);
+    store.add(createCell(r2, qf2, ts + 2, seqId + 2, value2), memStoreSize);
+    store.add(createCell(r2, qf3, ts + 2, seqId + 2, value2), memStoreSize);
+    store.add(createCell(r1, qf1, ts + 3, seqId + 3, value1), memStoreSize);
+    store.add(createCell(r1, qf2, ts + 3, seqId + 3, value1), memStoreSize);
+    store.add(createCell(r1, qf3, ts + 3, seqId + 3, value1), memStoreSize);
+    List<Cell> myList = new MyList<>(hook);
+    Scan scan = new Scan()
+            .withStartRow(r1)
+            .setFilter(filter);
+    try (InternalScanner scanner = (InternalScanner) store.getScanner(
+          scan, null, seqId + 3)){
+      // r1
+      scanner.next(myList);
+      assertEquals(expectedSize, myList.size());
+      for (Cell c : myList) {
+        byte[] actualValue = CellUtil.cloneValue(c);
+        assertTrue("expected:" + Bytes.toStringBinary(value1)
+          + ", actual:" + Bytes.toStringBinary(actualValue)
+          , Bytes.equals(actualValue, value1));
+      }
+      List<Cell> normalList = new ArrayList<>(3);
+      // r2
+      scanner.next(normalList);
+      assertEquals(3, normalList.size());
+      for (Cell c : normalList) {
+        byte[] actualValue = CellUtil.cloneValue(c);
+        assertTrue("expected:" + Bytes.toStringBinary(value2)
+          + ", actual:" + Bytes.toStringBinary(actualValue)
+          , Bytes.equals(actualValue, value2));
+      }
+    }
+  }
+
+  @Test
+  public void testCreateScannerAndSnapshotConcurrently() throws IOException, InterruptedException {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore.class.getName());
+    HColumnDescriptor hcd = new HColumnDescriptor(family);
+    hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
+    init(name.getMethodName(), conf, hcd);
+    byte[] value = Bytes.toBytes("value");
+    MemstoreSize memStoreSize = new MemstoreSize();
+    long ts = EnvironmentEdgeManager.currentTime();
+    long seqId = 100;
+    // older data whihc shouldn't be "seen" by client
+    store.add(createCell(qf1, ts, seqId, value), memStoreSize);
+    store.add(createCell(qf2, ts, seqId, value), memStoreSize);
+    store.add(createCell(qf3, ts, seqId, value), memStoreSize);
+    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+    quals.add(qf1);
+    quals.add(qf2);
+    quals.add(qf3);
+    StoreFlushContext storeFlushCtx = store.createFlushContext(id++);
+    MyCompactingMemStore.START_TEST.set(true);
+    Runnable flush = () -> {
+      // this is blocked until we create first scanner from pipeline and snapshot -- phase (1/5)
+      // recreate the active memstore -- phase (4/5)
+      storeFlushCtx.prepare();
+    };
+    ExecutorService service = Executors.newSingleThreadExecutor();
+    service.submit(flush);
+    // we get scanner from pipeline and snapshot but they are empty. -- phase (2/5)
+    // this is blocked until we recreate the active memstore -- phase (3/5)
+    // we get scanner from active memstore but it is empty -- phase (5/5)
+    InternalScanner scanner = (InternalScanner) store.getScanner(
+          new Scan(new Get(row)), quals, seqId + 1);
+    service.shutdown();
+    service.awaitTermination(20, TimeUnit.SECONDS);
+    try {
+      try {
+        List<Cell> results = new ArrayList<>();
+        scanner.next(results);
+        assertEquals(3, results.size());
+        for (Cell c : results) {
+          byte[] actualValue = CellUtil.cloneValue(c);
+          assertTrue("expected:" + Bytes.toStringBinary(value)
+            + ", actual:" + Bytes.toStringBinary(actualValue)
+            , Bytes.equals(actualValue, value));
+        }
+      } finally {
+        scanner.close();
+      }
+    } finally {
+      MyCompactingMemStore.START_TEST.set(false);
+      storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
+      storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
+    }
+  }
+
+  @Test
+  public void testScanWithDoubleFlush() throws IOException {
+    Configuration conf = HBaseConfiguration.create();
+    // Initialize region
+    MyStore myStore = initMyStore(name.getMethodName(), conf, new MyStoreHook(){
+      @Override
+      public void getScanners(MyStore store) throws IOException {
+        final long tmpId = id++;
+        ExecutorService s = Executors.newSingleThreadExecutor();
+        s.submit(() -> {
+          try {
+            // flush the store before storescanner updates the scanners from store.
+            // The current data will be flushed into files, and the memstore will
+            // be clear.
+            // -- phase (4/4)
+            flushStore(store, tmpId);
+          }catch (IOException ex) {
+            throw new RuntimeException(ex);
+          }
+        });
+        s.shutdown();
+        try {
+          // wait for the flush, the thread will be blocked in HStore#notifyChangedReadersObservers.
+          s.awaitTermination(3, TimeUnit.SECONDS);
+        } catch (InterruptedException ex) {
+        }
+      }
+    });
+    byte[] oldValue = Bytes.toBytes("oldValue");
+    byte[] currentValue = Bytes.toBytes("currentValue");
+    MemstoreSize memStoreSize = new MemstoreSize();
+    long ts = EnvironmentEdgeManager.currentTime();
+    long seqId = 100;
+    // older data whihc shouldn't be "seen" by client
+    myStore.add(createCell(qf1, ts, seqId, oldValue), memStoreSize);
+    myStore.add(createCell(qf2, ts, seqId, oldValue), memStoreSize);
+    myStore.add(createCell(qf3, ts, seqId, oldValue), memStoreSize);
+    long snapshotId = id++;
+    // push older data into snapshot -- phase (1/4)
+    StoreFlushContext storeFlushCtx = store.createFlushContext(snapshotId);
+    storeFlushCtx.prepare();
+
+    // insert current data into active -- phase (2/4)
+    myStore.add(createCell(qf1, ts + 1, seqId + 1, currentValue), memStoreSize);
+    myStore.add(createCell(qf2, ts + 1, seqId + 1, currentValue), memStoreSize);
+    myStore.add(createCell(qf3, ts + 1, seqId + 1, currentValue), memStoreSize);
+    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+    quals.add(qf1);
+    quals.add(qf2);
+    quals.add(qf3);
+    try (InternalScanner scanner = (InternalScanner) myStore.getScanner(
+        new Scan(new Get(row)), quals, seqId + 1)) {
+      // complete the flush -- phase (3/4)
+      storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
+      storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
+
+      List<Cell> results = new ArrayList<>();
+      scanner.next(results);
+      assertEquals(3, results.size());
+      for (Cell c : results) {
+        byte[] actualValue = CellUtil.cloneValue(c);
+        assertTrue("expected:" + Bytes.toStringBinary(currentValue)
+          + ", actual:" + Bytes.toStringBinary(actualValue)
+          , Bytes.equals(actualValue, currentValue));
+      }
+    }
+  }
+
+  @Test
+  public void testReclaimChunkWhenScaning() throws IOException {
+    init("testReclaimChunkWhenScaning");
+    long ts = EnvironmentEdgeManager.currentTime();
+    long seqId = 100;
+    byte[] value = Bytes.toBytes("value");
+    // older data whihc shouldn't be "seen" by client
+    store.add(createCell(qf1, ts, seqId, value), null);
+    store.add(createCell(qf2, ts, seqId, value), null);
+    store.add(createCell(qf3, ts, seqId, value), null);
+    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
+    quals.add(qf1);
+    quals.add(qf2);
+    quals.add(qf3);
+    try (InternalScanner scanner = (InternalScanner) store.getScanner(
+        new Scan(new Get(row)), quals, seqId)) {
+      List<Cell> results = new MyList<>(size -> {
+        switch (size) {
+          // 1) we get the first cell (qf1)
+          // 2) flush the data to have StoreScanner update inner scanners
+          // 3) the chunk will be reclaimed after updaing
+          case 1:
+            try {
+              flushStore(store, id++);
+            } catch (IOException e) {
+              throw new RuntimeException(e);
+            }
+            break;
+          // 1) we get the second cell (qf2)
+          // 2) add some cell to fill some byte into the chunk (we have only one chunk)
+          case 2:
+            try {
+              byte[] newValue = Bytes.toBytes("newValue");
+              // older data whihc shouldn't be "seen" by client
+              store.add(createCell(qf1, ts + 1, seqId + 1, newValue), null);
+              store.add(createCell(qf2, ts + 1, seqId + 1, newValue), null);
+              store.add(createCell(qf3, ts + 1, seqId + 1, newValue), null);
+            } catch (IOException e) {
+              throw new RuntimeException(e);
+            }
+            break;
+          default:
+            break;
+        }
+      });
+      scanner.next(results);
+      assertEquals(3, results.size());
+      for (Cell c : results) {
+        byte[] actualValue = CellUtil.cloneValue(c);
+        assertTrue("expected:" + Bytes.toStringBinary(value)
+          + ", actual:" + Bytes.toStringBinary(actualValue)
+          , Bytes.equals(actualValue, value));
+      }
+    }
+  }
+
+  /**
+   * If there are two running InMemoryFlushRunnable, the later InMemoryFlushRunnable
+   * may change the versionedList. And the first InMemoryFlushRunnable will use the chagned
+   * versionedList to remove the corresponding segments.
+   * In short, there will be some segements which isn't in merge are removed.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test (timeout=30000)
+  public void testRunDoubleMemStoreCompactors() throws IOException, InterruptedException {
+    int flushSize = 500;
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStoreWithCustomCompactor.class.getName());
+    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushSize));
+    // Set the lower threshold to invoke the "MERGE" policy
+    conf.set(MemStoreCompactor.COMPACTING_MEMSTORE_THRESHOLD_KEY, String.valueOf(0));
+    HColumnDescriptor hcd = new HColumnDescriptor(family);
+    hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
+    init(name.getMethodName(), conf, hcd);
+    byte[] value = Bytes.toBytes("thisisavarylargevalue");
+    MemstoreSize memStoreSize = new MemstoreSize();
+    long ts = EnvironmentEdgeManager.currentTime();
+    long seqId = 100;
+    // older data whihc shouldn't be "seen" by client
+    store.add(createCell(qf1, ts, seqId, value), memStoreSize);
+    store.add(createCell(qf2, ts, seqId, value), memStoreSize);
+    store.add(createCell(qf3, ts, seqId, value), memStoreSize);
+    assertEquals(1, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());
+    StoreFlushContext storeFlushCtx = store.createFlushContext(id++);
+    storeFlushCtx.prepare();
+    // This shouldn't invoke another in-memory flush because the first compactor thread
+    // hasn't accomplished the in-memory compaction.
+    store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSize);
+    store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSize);
+    store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSize);
+    assertEquals(1, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());
+    //okay. Let the compaction be completed
+    MyMemStoreCompactor.START_COMPACTOR_LATCH.countDown();
+    CompactingMemStore mem = (CompactingMemStore) ((HStore)store).memstore;
+    while (mem.isMemStoreFlushingInMemory()) {
+      TimeUnit.SECONDS.sleep(1);
+    }
+    // This should invoke another in-memory flush.
+    store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSize);
+    store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSize);
+    store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSize);
+    assertEquals(2, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());
+    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
+      String.valueOf(TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE));
+    storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
+    storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
+  }
+
+  private MyStore initMyStore(String methodName, Configuration conf, MyStoreHook hook)
+      throws IOException {
+    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
+    HColumnDescriptor hcd = new HColumnDescriptor(family);
+    hcd.setMaxVersions(5);
+    return (MyStore) init(methodName, conf, htd, hcd, hook);
+  }
+
+  class MyStore extends HStore {
+    private final MyStoreHook hook;
+
+    MyStore(final HRegion region, final HColumnDescriptor family, final Configuration confParam,
+        MyStoreHook hook, boolean switchToPread) throws IOException {
+      super(region, family, confParam);
+      this.hook = hook;
+    }
+
+    @Override
+    public List<KeyValueScanner> getScanners(List<HStoreFile> files, boolean cacheBlocks,
+        boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
+        boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
+        boolean includeMemstoreScanner) throws IOException {
+      hook.getScanners(this);
+      return super.getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true,
+        stopRow, false, readPt, includeMemstoreScanner);
+    }
+
+    @Override
+    public long getSmallestReadPoint() {
+      return hook.getSmallestReadPoint(this);
+    }
+  }
+
+  private abstract class MyStoreHook {
+    void getScanners(MyStore store) throws IOException {
+    }
+    long getSmallestReadPoint(HStore store) {
+      return store.getHRegion().getSmallestReadPoint();
+    }
+  }
+
+  @Test
+  public void testSwitchingPreadtoStreamParallelyWithCompactionDischarger() throws Exception {
+    int flushSize = 500;
+    Configuration conf = HBaseConfiguration.create();
+    conf.set("hbase.hstore.engine.class", DummyStoreEngine.class.getName());
+    conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 0);
+    // Set the lower threshold to invoke the "MERGE" policy
+    HColumnDescriptor hcd = new HColumnDescriptor(family);
+    hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
+    MyStore store = initMyStore(name.getMethodName(), conf, new MyStoreHook() {});
+    MemstoreSize memStoreSize = new MemstoreSize();
+    long ts = System.currentTimeMillis();
+    long seqID = 1l;
+    // Add some data to the region and do some flushes
+    for (int i = 1; i < 10; i++) {
+      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
+        memStoreSize);
+    }
+    // flush them
+    flushStore(store, seqID);
+    for (int i = 11; i < 20; i++) {
+      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
+        memStoreSize);
+    }
+    // flush them
+    flushStore(store, seqID);
+    for (int i = 21; i < 30; i++) {
+      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
+        memStoreSize);
+    }
+    // flush them
+    flushStore(store, seqID);
+
+    assertEquals(3, store.getStorefilesCount());
+    ScanInfo scanInfo = store.getScanInfo();
+    Scan scan = new Scan();
+    scan.addFamily(family);
+    Collection<HStoreFile> storefiles2 = store.getStorefiles();
+    ArrayList<HStoreFile> actualStorefiles = Lists.newArrayList(storefiles2);
+    StoreScanner storeScanner =
+        (StoreScanner) store.getScanner(scan, scan.getFamilyMap().get(family), Long.MAX_VALUE);
+    // get the current heap
+    KeyValueHeap heap = storeScanner.heap;
+    // create more store files
+    for (int i = 31; i < 40; i++) {
+      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
+        memStoreSize);
+    }
+    // flush them
+    flushStore(store, seqID);
+
+    for (int i = 41; i < 50; i++) {
+      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
+        memStoreSize);
+    }
+    // flush them
+    flushStore(store, seqID);
+    storefiles2 = store.getStorefiles();
+    ArrayList<HStoreFile> actualStorefiles1 = Lists.newArrayList(storefiles2);
+    actualStorefiles1.removeAll(actualStorefiles);
+    // Do compaction
+    List<Exception> exceptions = new ArrayList<Exception>();
+    MyThread thread = new MyThread(storeScanner);
+    thread.start();
+    store.replaceStoreFiles(actualStorefiles, actualStorefiles1);
+    thread.join();
+    KeyValueHeap heap2 = thread.getHeap();
+    assertFalse(heap.equals(heap2));
+  }
+
+  private static class MyThread extends Thread {
+    private StoreScanner scanner;
+    private KeyValueHeap heap;
+
+    public MyThread(StoreScanner scanner) {
+      this.scanner = scanner;
+    }
+
+    public KeyValueHeap getHeap() {
+      return this.heap;
+    }
+
+    public void run() {
+      scanner.trySwitchToStreamRead();
+      heap = scanner.heap;
+    }
+  }
+
+  private static class MyMemStoreCompactor extends MemStoreCompactor {
+    private static final AtomicInteger RUNNER_COUNT = new AtomicInteger(0);
+    private static final CountDownLatch START_COMPACTOR_LATCH = new CountDownLatch(1);
+    public MyMemStoreCompactor(CompactingMemStore compactingMemStore, MemoryCompactionPolicy compactionPolicy) {
+      super(compactingMemStore, compactionPolicy);
+    }
+
+    @Override
+    public boolean start() throws IOException {
+      boolean isFirst = RUNNER_COUNT.getAndIncrement() == 0;
+      boolean rval = super.start();
+      if (isFirst) {
+        try {
+          START_COMPACTOR_LATCH.await();
+        } catch (InterruptedException ex) {
+          throw new RuntimeException(ex);
+        }
+      }
+      return rval;
+    }
+  }
+
+  public static class MyCompactingMemStoreWithCustomCompactor extends CompactingMemStore {
+    private static final AtomicInteger RUNNER_COUNT = new AtomicInteger(0);
+    public MyCompactingMemStoreWithCustomCompactor(Configuration conf, CellComparator c,
+        HStore store, RegionServicesForStores regionServices,
+        MemoryCompactionPolicy compactionPolicy) throws IOException {
+      super(conf, c, store, regionServices, compactionPolicy);
+    }
+
+    @Override
+    protected MemStoreCompactor createMemStoreCompactor(MemoryCompactionPolicy compactionPolicy) {
+      return new MyMemStoreCompactor(this, compactionPolicy);
+    }
+
+    @Override
+    protected boolean shouldFlushInMemory() {
+      boolean rval = super.shouldFlushInMemory();
+      if (rval) {
+        RUNNER_COUNT.incrementAndGet();
+      }
+      return rval;
+    }
+  }
+
+  public static class MyCompactingMemStore extends CompactingMemStore {
+    private static final AtomicBoolean START_TEST = new AtomicBoolean(false);
+    private final CountDownLatch getScannerLatch = new CountDownLatch(1);
+    private final CountDownLatch snapshotLatch = new CountDownLatch(1);
+    public MyCompactingMemStore(Configuration conf, CellComparator c,
+        HStore store, RegionServicesForStores regionServices,
+        MemoryCompactionPolicy compactionPolicy) throws IOException {
+      super(conf, c, store, regionServices, compactionPolicy);
+    }
+
+    @Override
+    protected List<KeyValueScanner> createList(int capacity) {
+      if (START_TEST.get()) {
+        try {
+          getScannerLatch.countDown();
+          snapshotLatch.await();
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+      return new ArrayList<>(capacity);
+    }
+    @Override
+    protected void pushActiveToPipeline(MutableSegment active) {
+      if (START_TEST.get()) {
+        try {
+          getScannerLatch.await();
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      super.pushActiveToPipeline(active);
+      if (START_TEST.get()) {
+        snapshotLatch.countDown();
+      }
+    }
+  }
+
+  interface MyListHook {
+    void hook(int currentSize);
+  }
+
+  private static class MyList<T> implements List<T> {
+    private final List<T> delegatee = new ArrayList<>();
+    private final MyListHook hookAtAdd;
+    MyList(final MyListHook hookAtAdd) {
+      this.hookAtAdd = hookAtAdd;
+    }
+    @Override
+    public int size() {return delegatee.size();}
+
+    @Override
+    public boolean isEmpty() {return delegatee.isEmpty();}
+
+    @Override
+    public boolean contains(Object o) {return delegatee.contains(o);}
+
+    @Override
+    public Iterator<T> iterator() {return delegatee.iterator();}
+
+    @Override
+    public Object[] toArray() {return delegatee.toArray();}
+
+    @Override
+    public <T> T[] toArray(T[] a) {return delegatee.toArray(a);}
+
+    @Override
+    public boolean add(T e) {
+      hookAtAdd.hook(size());
+      return delegatee.add(e);
+    }
+
+    @Override
+    public boolean remove(Object o) {return delegatee.remove(o);}
+
+    @Override
+    public boolean containsAll(Collection<?> c) {return delegatee.containsAll(c);}
+
+    @Override
+    public boolean addAll(Collection<? extends T> c) {return delegatee.addAll(c);}
+
+    @Override
+    public boolean addAll(int index, Collection<? extends T> c) {return delegatee.addAll(index, c);}
+
+    @Override
+    public boolean removeAll(Collection<?> c) {return delegatee.removeAll(c);}
+
+    @Override
+    public boolean retainAll(Collection<?> c) {return delegatee.retainAll(c);}
+
+    @Override
+    public void clear() {delegatee.clear();}
+
+    @Override
+    public T get(int index) {return delegatee.get(index);}
+
+    @Override
+    public T set(int index, T element) {return delegatee.set(index, element);}
+
+    @Override
+    public void add(int index, T element) {delegatee.add(index, element);}
+
+    @Override
+    public T remove(int index) {return delegatee.remove(index);}
+
+    @Override
+    public int indexOf(Object o) {return delegatee.indexOf(o);}
+
+    @Override
+    public int lastIndexOf(Object o) {return delegatee.lastIndexOf(o);}
+
+    @Override
+    public ListIterator<T> listIterator() {return delegatee.listIterator();}
+
+    @Override
+    public ListIterator<T> listIterator(int index) {return delegatee.listIterator(index);}
+
+    @Override
+    public List<T> subList(int fromIndex, int toIndex) {return delegatee.subList(fromIndex, toIndex);}
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
index e74e939..b20cae8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
@@ -176,11 +176,9 @@ public class TestHStoreFile extends HBaseTestCase {
     // Split on a row, not in middle of row.  Midkey returned by reader
     // may be in middle of row.  Create new one with empty column and
     // timestamp.
-    Cell kv = reader.midkey();
-    byte [] midRow = CellUtil.cloneRow(kv);
-    kv = reader.getLastKey();
-    byte [] finalRow = CellUtil.cloneRow(kv);
-    hsf.closeReader(true);
+    byte [] midRow = CellUtil.cloneRow(reader.midKey().get());
+    byte [] finalRow = CellUtil.cloneRow(reader.getLastKey().get());
+    hsf.closeStoreFile(true);
 
     // Make a reference
     HRegionInfo splitHri = new HRegionInfo(hri.getTable(), null, midRow);
@@ -190,7 +188,8 @@ public class TestHStoreFile extends HBaseTestCase {
     // Now confirm that I can read from the reference and that it only gets
     // keys from top half of the file.
     HFileScanner s = refHsf.getReader().getScanner(false, false);
-    for(boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) {
+    Cell kv = null;
+    for (boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) {
       ByteBuffer bb = ByteBuffer.wrap(((KeyValue) s.getKey()).getKey());
       kv = KeyValueUtil.createKeyValueFromKey(bb);
       if (first) {
@@ -301,7 +300,7 @@ public class TestHStoreFile extends HBaseTestCase {
     f.initReader();
     Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top
     Path pathB = splitStoreFile(cloneRegionFs, splitHriB, TEST_FAMILY, f, SPLITKEY, false);// bottom
-    f.closeReader(true);
+    f.closeStoreFile(true);
     // OK test the thing
     FSUtils.logFileSystemState(fs, testDir, LOG);
 
@@ -342,7 +341,7 @@ public class TestHStoreFile extends HBaseTestCase {
   private void checkHalfHFile(final HRegionFileSystem regionFs, final HStoreFile f)
       throws IOException {
     f.initReader();
-    Cell midkey = f.getReader().midkey();
+    Cell midkey = f.getReader().midKey().get();
     KeyValue midKV = (KeyValue)midkey;
     byte [] midRow = CellUtil.cloneRow(midKV);
     // Create top split.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
index 707540a..0c33bdb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java
@@ -84,7 +84,7 @@ public class TestMajorCompaction {
   private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU();
   protected Configuration conf = UTIL.getConfiguration();
 
-  private Region r = null;
+  private HRegion r = null;
   private HTableDescriptor htd = null;
   private static final byte [] COLUMN_FAMILY = fam1;
   private final byte [] STARTROW = Bytes.toBytes(START_KEY);
@@ -328,7 +328,7 @@ public class TestMajorCompaction {
       // ensure that major compaction time is deterministic
       RatioBasedCompactionPolicy
           c = (RatioBasedCompactionPolicy)s.storeEngine.getCompactionPolicy();
-      Collection<StoreFile> storeFiles = s.getStorefiles();
+      Collection<HStoreFile> storeFiles = s.getStorefiles();
       long mcTime = c.getNextMajorCompactTime(storeFiles);
       for (int i = 0; i < 10; ++i) {
         assertEquals(mcTime, c.getNextMajorCompactTime(storeFiles));
@@ -358,7 +358,7 @@ public class TestMajorCompaction {
   private void verifyCounts(int countRow1, int countRow2) throws Exception {
     int count1 = 0;
     int count2 = 0;
-    for (StoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) {
+    for (HStoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) {
       HFileScanner scanner = f.getReader().getScanner(false, false);
       scanner.seekTo();
       do {
@@ -377,7 +377,7 @@ public class TestMajorCompaction {
 
   private int count() throws IOException {
     int count = 0;
-    for (StoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) {
+    for (HStoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) {
       HFileScanner scanner = f.getReader().getScanner(false, false);
       if (!scanner.seekTo()) {
         continue;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
----------------------------------------------------------------------
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 9ab1440..c08bd71 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
+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.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -318,10 +320,10 @@ public class TestMobStoreCompaction {
     if (fs.exists(mobDirPath)) {
       FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath);
       for (FileStatus file : files) {
-        StoreFile sf = new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true);
+        HStoreFile sf = new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true);
         sf.initReader();
         Map<byte[], byte[]> fileInfo = sf.getReader().loadFileInfo();
-        byte[] count = fileInfo.get(StoreFile.MOB_CELLS_COUNT);
+        byte[] count = fileInfo.get(MOB_CELLS_COUNT);
         assertTrue(count != null);
         mobCellsCount += Bytes.toLong(count);
       }
@@ -349,7 +351,7 @@ public class TestMobStoreCompaction {
           Bytes.toBytes("colX"), now, dummyData);
       writer.append(kv);
     } finally {
-      writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis()));
+      writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis()));
       writer.close();
     }
   }
@@ -428,20 +430,20 @@ public class TestMobStoreCompaction {
     copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
     CacheConfig cacheConfig = new CacheConfig(copyOfConf);
     Path mobDirPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(), hcd.getNameAsString());
-    List<StoreFile> sfs = new ArrayList<>();
+    List<HStoreFile> sfs = new ArrayList<>();
     int numDelfiles = 0;
     int size = 0;
     if (fs.exists(mobDirPath)) {
       for (FileStatus f : fs.listStatus(mobDirPath)) {
-        StoreFile sf = new HStoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true);
+        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);
+      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,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
index 72a968c..86fe5af 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
@@ -45,9 +45,6 @@ import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -55,9 +52,17 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.*;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+
 /**
  * Tests for region replicas. Sad that we cannot isolate these without bringing up a whole
  * cluster. See {@link TestRegionServerNoMaster}.
@@ -472,7 +477,7 @@ public class TestRegionReplicas {
       // should be able to deal with it giving us all the result we expect.
       int keys = 0;
       int sum = 0;
-      for (StoreFile sf: secondaryRegion.getStore(f).getStorefiles()) {
+      for (HStoreFile sf : ((HStore) secondaryRegion.getStore(f)).getStorefiles()) {
         // Our file does not exist anymore. was moved by the compaction above.
         LOG.debug(getRS().getFileSystem().exists(sf.getPath()));
         Assert.assertFalse(getRS().getFileSystem().exists(sf.getPath()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
index cad060e..06c0bfd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java
@@ -25,13 +25,14 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Optional;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -246,7 +247,7 @@ public class TestRegionSplitPolicy {
     HStore mockStore = Mockito.mock(HStore.class);
     Mockito.doReturn(2000L).when(mockStore).getSize();
     Mockito.doReturn(true).when(mockStore).canSplit();
-    Mockito.doReturn(Bytes.toBytes("abcd")).when(mockStore).getSplitPoint();
+    Mockito.doReturn(Optional.of(Bytes.toBytes("abcd"))).when(mockStore).getSplitPoint();
     stores.add(mockStore);
 
     KeyPrefixRegionSplitPolicy policy = (KeyPrefixRegionSplitPolicy) RegionSplitPolicy
@@ -322,8 +323,7 @@ public class TestRegionSplitPolicy {
     HStore mockStore = Mockito.mock(HStore.class);
     Mockito.doReturn(2000L).when(mockStore).getSize();
     Mockito.doReturn(true).when(mockStore).canSplit();
-    Mockito.doReturn(Bytes.toBytes("store 1 split"))
-      .when(mockStore).getSplitPoint();
+    Mockito.doReturn(Optional.of(Bytes.toBytes("store 1 split"))).when(mockStore).getSplitPoint();
     stores.add(mockStore);
 
     assertEquals("store 1 split",
@@ -333,8 +333,7 @@ public class TestRegionSplitPolicy {
     HStore mockStore2 = Mockito.mock(HStore.class);
     Mockito.doReturn(4000L).when(mockStore2).getSize();
     Mockito.doReturn(true).when(mockStore2).canSplit();
-    Mockito.doReturn(Bytes.toBytes("store 2 split"))
-      .when(mockStore2).getSplitPoint();
+    Mockito.doReturn(Optional.of(Bytes.toBytes("store 2 split"))).when(mockStore2).getSplitPoint();
     stores.add(mockStore2);
 
     assertEquals("store 2 split",
@@ -355,7 +354,7 @@ public class TestRegionSplitPolicy {
     HStore mockStore = Mockito.mock(HStore.class);
     Mockito.doReturn(2000L).when(mockStore).getSize();
     Mockito.doReturn(true).when(mockStore).canSplit();
-    Mockito.doReturn(Bytes.toBytes("ab,cd")).when(mockStore).getSplitPoint();
+    Mockito.doReturn(Optional.of(Bytes.toBytes("ab,cd"))).when(mockStore).getSplitPoint();
     stores.add(mockStore);
 
     DelimitedKeyPrefixRegionSplitPolicy policy = (DelimitedKeyPrefixRegionSplitPolicy) RegionSplitPolicy

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
index 8b34a2f..dbf3be0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java
@@ -113,7 +113,7 @@ public class TestReversibleScanners {
           .withFileContext(hFileContext).build();
       writeStoreFile(writer);
 
-      StoreFile sf = new HStoreFile(fs, writer.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+      HStoreFile sf = new HStoreFile(fs, writer.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
           BloomType.NONE, true);
 
       List<StoreFileScanner> scanners = StoreFileScanner
@@ -167,10 +167,10 @@ public class TestReversibleScanners {
     writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1,
         writer2 });
 
-    StoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+    HStoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
         BloomType.NONE, true);
 
-    StoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+    HStoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
         BloomType.NONE, true);
     /**
      * Test without MVCC
@@ -257,10 +257,10 @@ public class TestReversibleScanners {
     writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1,
         writer2 });
 
-    StoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+    HStoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
         BloomType.NONE, true);
 
-    StoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
+    HStoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
         BloomType.NONE, true);
 
     ScanInfo scanInfo =
@@ -418,19 +418,15 @@ public class TestReversibleScanners {
     verifyCountAndOrder(scanner, expectedRowNum * 2 * 2, expectedRowNum, false);
   }
 
-  private StoreScanner getReversibleStoreScanner(MemStore memstore,
-      StoreFile sf1, StoreFile sf2, Scan scan,
-      ScanInfo scanInfo, int readPoint) throws IOException {
-    List<KeyValueScanner> scanners = getScanners(memstore, sf1, sf2, null,
-        false, readPoint);
+  private StoreScanner getReversibleStoreScanner(MemStore memstore, HStoreFile sf1, HStoreFile sf2,
+      Scan scan, ScanInfo scanInfo, int readPoint) throws IOException {
+    List<KeyValueScanner> scanners = getScanners(memstore, sf1, sf2, null, false, readPoint);
     NavigableSet<byte[]> columns = null;
-    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap()
-        .entrySet()) {
+    for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
       // Should only one family
       columns = entry.getValue();
     }
-    StoreScanner storeScanner = new ReversedStoreScanner(scan, scanInfo,
-         columns, scanners);
+    StoreScanner storeScanner = new ReversedStoreScanner(scan, scanInfo, columns, scanners);
     return storeScanner;
   }
 
@@ -487,22 +483,17 @@ public class TestReversibleScanners {
     assertEquals(null, kvHeap.peek());
   }
 
-  private ReversedKeyValueHeap getReversibleKeyValueHeap(MemStore memstore,
-      StoreFile sf1, StoreFile sf2, byte[] startRow, int readPoint)
-      throws IOException {
-    List<KeyValueScanner> scanners = getScanners(memstore, sf1, sf2, startRow,
-        true, readPoint);
-    ReversedKeyValueHeap kvHeap = new ReversedKeyValueHeap(scanners,
-        CellComparator.COMPARATOR);
+  private ReversedKeyValueHeap getReversibleKeyValueHeap(MemStore memstore, HStoreFile sf1,
+      HStoreFile sf2, byte[] startRow, int readPoint) throws IOException {
+    List<KeyValueScanner> scanners = getScanners(memstore, sf1, sf2, startRow, true, readPoint);
+    ReversedKeyValueHeap kvHeap = new ReversedKeyValueHeap(scanners, CellComparator.COMPARATOR);
     return kvHeap;
   }
 
-  private List<KeyValueScanner> getScanners(MemStore memstore, StoreFile sf1,
-      StoreFile sf2, byte[] startRow, boolean doSeek, int readPoint)
-      throws IOException {
-    List<StoreFileScanner> fileScanners = StoreFileScanner
-        .getScannersForStoreFiles(Lists.newArrayList(sf1, sf2), false, true,
-            false, false, readPoint);
+  private List<KeyValueScanner> getScanners(MemStore memstore, HStoreFile sf1, HStoreFile sf2,
+      byte[] startRow, boolean doSeek, int readPoint) throws IOException {
+    List<StoreFileScanner> fileScanners = StoreFileScanner.getScannersForStoreFiles(
+      Lists.newArrayList(sf1, sf2), false, true, false, false, readPoint);
     List<KeyValueScanner> memScanners = memstore.getScanners(readPoint);
     List<KeyValueScanner> scanners = new ArrayList<>(fileScanners.size() + 1);
     scanners.addAll(fileScanners);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
index 5dce4ad..0c014fd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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,6 +17,9 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY;
+
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
@@ -171,10 +174,10 @@ public class TestScannerWithBulkload {
       // Set a big MAX_SEQ_ID_KEY. Scan should not look at this seq id in a bulk loaded file.
       // Scan should only look at the seq id appended at the bulk load time, and not skip its
       // kv.
-      writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(new Long(9999999)));
+      writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(new Long(9999999)));
     }
     else {
-    writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis()));
+    writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis()));
     }
     writer.close();
     return hfilePath;


[02/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java
index 76959c6..6e5aeed 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java
@@ -22,7 +22,6 @@ import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_K
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -87,10 +86,10 @@ public class TestStripeStoreFileManager {
   @Test
   public void testInsertFilesIntoL0() throws Exception {
     StripeStoreFileManager manager = createManager();
-    MockStoreFile sf = createFile();
+    MockHStoreFile sf = createFile();
     manager.insertNewFiles(al(sf));
     assertEquals(1, manager.getStorefileCount());
-    Collection<StoreFile> filesForGet = manager.getFilesForScan(KEY_A, true, KEY_A, true);
+    Collection<HStoreFile> filesForGet = manager.getFilesForScan(KEY_A, true, KEY_A, true);
     assertEquals(1, filesForGet.size());
     assertTrue(filesForGet.contains(sf));
 
@@ -109,14 +108,14 @@ public class TestStripeStoreFileManager {
     manager.addCompactionResults(al(), al(createFile(OPEN_KEY, KEY_B),
         createFile(KEY_B, OPEN_KEY)));
     assertEquals(4, manager.getStorefileCount());
-    Collection<StoreFile> allFiles = manager.clearFiles();
+    Collection<HStoreFile> allFiles = manager.clearFiles();
     assertEquals(4, allFiles.size());
     assertEquals(0, manager.getStorefileCount());
     assertEquals(0, manager.getStorefiles().size());
   }
 
-  private static ArrayList<StoreFile> dumpIterator(Iterator<StoreFile> iter) {
-    ArrayList<StoreFile> result = new ArrayList<>();
+  private static ArrayList<HStoreFile> dumpIterator(Iterator<HStoreFile> iter) {
+    ArrayList<HStoreFile> result = new ArrayList<>();
     for (; iter.hasNext(); result.add(iter.next()));
     return result;
   }
@@ -124,23 +123,23 @@ public class TestStripeStoreFileManager {
   @Test
   public void testRowKeyBefore() throws Exception {
     StripeStoreFileManager manager = createManager();
-    StoreFile l0File = createFile(), l0File2 = createFile();
+    HStoreFile l0File = createFile(), l0File2 = createFile();
     manager.insertNewFiles(al(l0File));
     manager.insertNewFiles(al(l0File2));
     // Get candidate files.
-    Iterator<StoreFile> sfs = manager.getCandidateFilesForRowKeyBefore(KV_B);
+    Iterator<HStoreFile> sfs = manager.getCandidateFilesForRowKeyBefore(KV_B);
     sfs.next();
     sfs.remove();
     // Suppose we found a candidate in this file... make sure L0 file remaining is not removed.
     sfs = manager.updateCandidateFilesForRowKeyBefore(sfs, KV_B, KV_A);
     assertTrue(sfs.hasNext());
     // Now add some stripes (remove L0 file too)
-    MockStoreFile stripe0a = createFile(0, 100, OPEN_KEY, KEY_B),
+    MockHStoreFile stripe0a = createFile(0, 100, OPEN_KEY, KEY_B),
         stripe1 = createFile(KEY_B, OPEN_KEY);
     manager.addCompactionResults(al(l0File), al(stripe0a, stripe1));
     manager.removeCompactedFiles(al(l0File));
     // If we want a key <= KEY_A, we should get everything except stripe1.
-    ArrayList<StoreFile> sfsDump = dumpIterator(manager.getCandidateFilesForRowKeyBefore(KV_A));
+    ArrayList<HStoreFile> sfsDump = dumpIterator(manager.getCandidateFilesForRowKeyBefore(KV_A));
     assertEquals(2, sfsDump.size());
     assertTrue(sfsDump.contains(stripe0a));
     assertFalse(sfsDump.contains(stripe1));
@@ -162,7 +161,7 @@ public class TestStripeStoreFileManager {
     // Add one more, later, file to stripe0, remove the last annoying L0 file.
     // This file should be returned in preference to older L0 file; also, after we get
     // a candidate from the first file, the old one should not be removed.
-    StoreFile stripe0b = createFile(0, 101, OPEN_KEY, KEY_B);
+    HStoreFile stripe0b = createFile(0, 101, OPEN_KEY, KEY_B);
     manager.addCompactionResults(al(l0File2), al(stripe0b));
     manager.removeCompactedFiles(al(l0File2));
     sfs = manager.getCandidateFilesForRowKeyBefore(KV_A);
@@ -176,24 +175,24 @@ public class TestStripeStoreFileManager {
   public void testGetSplitPointEdgeCases() throws Exception {
     StripeStoreFileManager manager = createManager();
     // No files => no split.
-    assertNull(manager.getSplitPoint());
+    assertFalse(manager.getSplitPoint().isPresent());
 
     // If there are no stripes, should pick midpoint from the biggest file in L0.
-    MockStoreFile sf5 = createFile(5, 0);
+    MockHStoreFile sf5 = createFile(5, 0);
     sf5.splitPoint = new byte[] { 1 };
     manager.insertNewFiles(al(sf5));
     manager.insertNewFiles(al(createFile(1, 0)));
-    assertArrayEquals(sf5.splitPoint, manager.getSplitPoint());
+    assertArrayEquals(sf5.splitPoint, manager.getSplitPoint().get());
 
     // Same if there's one stripe but the biggest file is still in L0.
     manager.addCompactionResults(al(), al(createFile(2, 0, OPEN_KEY, OPEN_KEY)));
-    assertArrayEquals(sf5.splitPoint, manager.getSplitPoint());
+    assertArrayEquals(sf5.splitPoint, manager.getSplitPoint().get());
 
     // If the biggest file is in the stripe, should get from it.
-    MockStoreFile sf6 = createFile(6, 0, OPEN_KEY, OPEN_KEY);
+    MockHStoreFile sf6 = createFile(6, 0, OPEN_KEY, OPEN_KEY);
     sf6.splitPoint = new byte[] { 2 };
     manager.addCompactionResults(al(), al(sf6));
-    assertArrayEquals(sf6.splitPoint, manager.getSplitPoint());
+    assertArrayEquals(sf6.splitPoint, manager.getSplitPoint().get());
   }
 
   @Test
@@ -234,11 +233,11 @@ public class TestStripeStoreFileManager {
   private void verifySplitPointScenario(int splitPointAfter, boolean shouldSplitStripe,
       float splitRatioToVerify, int... sizes) throws Exception {
     assertTrue(sizes.length > 1);
-    ArrayList<StoreFile> sfs = new ArrayList<>();
+    ArrayList<HStoreFile> sfs = new ArrayList<>();
     for (int sizeIx = 0; sizeIx < sizes.length; ++sizeIx) {
       byte[] startKey = (sizeIx == 0) ? OPEN_KEY : Bytes.toBytes(sizeIx - 1);
       byte[] endKey = (sizeIx == sizes.length - 1) ? OPEN_KEY : Bytes.toBytes(sizeIx);
-      MockStoreFile sf = createFile(sizes[sizeIx], 0, startKey, endKey);
+      MockHStoreFile sf = createFile(sizes[sizeIx], 0, startKey, endKey);
       sf.splitPoint = Bytes.toBytes(-sizeIx); // set split point to the negative index
       sfs.add(sf);
     }
@@ -249,7 +248,7 @@ public class TestStripeStoreFileManager {
     }
     StripeStoreFileManager manager = createManager(al(), conf);
     manager.addCompactionResults(al(), sfs);
-    int result = Bytes.toInt(manager.getSplitPoint());
+    int result = Bytes.toInt(manager.getSplitPoint().get());
     // Either end key and thus positive index, or "middle" of the file and thus negative index.
     assertEquals(splitPointAfter * (shouldSplitStripe ? -1 : 1), result);
   }
@@ -265,7 +264,7 @@ public class TestStripeStoreFileManager {
     verifyGetAndScanScenario(manager, KEY_B, KEY_C);
 
     // Populate one L0 file.
-    MockStoreFile sf0 = createFile();
+    MockHStoreFile sf0 = createFile();
     manager.insertNewFiles(al(sf0));
     verifyGetAndScanScenario(manager, null, null,   sf0);
     verifyGetAndScanScenario(manager, null, KEY_C,  sf0);
@@ -273,11 +272,11 @@ public class TestStripeStoreFileManager {
     verifyGetAndScanScenario(manager, KEY_B, KEY_C, sf0);
 
     // Populate a bunch of files for stripes, keep L0.
-    MockStoreFile sfA = createFile(OPEN_KEY, KEY_A);
-    MockStoreFile sfB = createFile(KEY_A, KEY_B);
-    MockStoreFile sfC = createFile(KEY_B, KEY_C);
-    MockStoreFile sfD = createFile(KEY_C, KEY_D);
-    MockStoreFile sfE = createFile(KEY_D, OPEN_KEY);
+    MockHStoreFile sfA = createFile(OPEN_KEY, KEY_A);
+    MockHStoreFile sfB = createFile(KEY_A, KEY_B);
+    MockHStoreFile sfC = createFile(KEY_B, KEY_C);
+    MockHStoreFile sfD = createFile(KEY_C, KEY_D);
+    MockHStoreFile sfE = createFile(KEY_D, OPEN_KEY);
     manager.addCompactionResults(al(), al(sfA, sfB, sfC, sfD, sfE));
 
     verifyGetAndScanScenario(manager, null, null,              sf0, sfA, sfB, sfC, sfD, sfE);
@@ -292,7 +291,7 @@ public class TestStripeStoreFileManager {
   }
 
   private void verifyGetAndScanScenario(StripeStoreFileManager manager, byte[] start, byte[] end,
-      StoreFile... results) throws Exception {
+      HStoreFile... results) throws Exception {
     verifyGetOrScanScenario(manager, start, end, results);
   }
 
@@ -302,18 +301,18 @@ public class TestStripeStoreFileManager {
     // In L0, there will be file w/o metadata (real L0, 3 files with invalid metadata, and 3
     // files that overlap valid stripes in various ways). Note that the 4th way to overlap the
     // stripes will cause the structure to be mostly scraped, and is tested separately.
-    ArrayList<StoreFile> validStripeFiles = al(createFile(OPEN_KEY, KEY_B),
+    ArrayList<HStoreFile> validStripeFiles = al(createFile(OPEN_KEY, KEY_B),
         createFile(KEY_B, KEY_C), createFile(KEY_C, OPEN_KEY),
         createFile(KEY_C, OPEN_KEY));
-    ArrayList<StoreFile> filesToGoToL0 = al(createFile(), createFile(null, KEY_A),
+    ArrayList<HStoreFile> filesToGoToL0 = al(createFile(), createFile(null, KEY_A),
         createFile(KEY_D, null), createFile(KEY_D, KEY_A), createFile(keyAfter(KEY_A), KEY_C),
         createFile(OPEN_KEY, KEY_D), createFile(KEY_D, keyAfter(KEY_D)));
-    ArrayList<StoreFile> allFilesToGo = flattenLists(validStripeFiles, filesToGoToL0);
+    ArrayList<HStoreFile> allFilesToGo = flattenLists(validStripeFiles, filesToGoToL0);
     Collections.shuffle(allFilesToGo);
     StripeStoreFileManager manager = createManager(allFilesToGo);
-    List<StoreFile> l0Files = manager.getLevel0Files();
+    List<HStoreFile> l0Files = manager.getLevel0Files();
     assertEquals(filesToGoToL0.size(), l0Files.size());
-    for (StoreFile sf : filesToGoToL0) {
+    for (HStoreFile sf : filesToGoToL0) {
       assertTrue(l0Files.contains(sf));
     }
     verifyAllFiles(manager, allFilesToGo);
@@ -323,7 +322,7 @@ public class TestStripeStoreFileManager {
   public void testLoadFilesWithBadStripe() throws Exception {
     // Current "algorithm" will see the after-B key before C key, add it as valid stripe,
     // and then fail all other stripes. So everything would end up in L0.
-    ArrayList<StoreFile> allFilesToGo = al(createFile(OPEN_KEY, KEY_B),
+    ArrayList<HStoreFile> allFilesToGo = al(createFile(OPEN_KEY, KEY_B),
         createFile(KEY_B, KEY_C), createFile(KEY_C, OPEN_KEY),
         createFile(KEY_B, keyAfter(KEY_B)));
     Collections.shuffle(allFilesToGo);
@@ -346,7 +345,7 @@ public class TestStripeStoreFileManager {
   @Test
   public void testLoadFilesAfterSplit() throws Exception {
     // If stripes are good but have non-open ends, they must be treated as open ends.
-    MockStoreFile sf = createFile(KEY_B, KEY_C);
+    MockHStoreFile sf = createFile(KEY_B, KEY_C);
     StripeStoreFileManager manager = createManager(al(createFile(OPEN_KEY, KEY_B), sf));
     assertEquals(0, manager.getLevel0Files().size());
     // Here, [B, C] is logically [B, inf), so we should be able to compact it to that only.
@@ -367,7 +366,7 @@ public class TestStripeStoreFileManager {
   public void testAddingCompactionResults() throws Exception {
     StripeStoreFileManager manager = createManager();
     // First, add some L0 files and "compact" one with new stripe creation.
-    StoreFile sf_L0_0a = createFile(), sf_L0_0b = createFile();
+    HStoreFile sf_L0_0a = createFile(), sf_L0_0b = createFile();
     manager.insertNewFiles(al(sf_L0_0a, sf_L0_0b));
 
     // Try compacting with invalid new branches (gaps, overlaps) - no effect.
@@ -379,24 +378,24 @@ public class TestStripeStoreFileManager {
     verifyInvalidCompactionScenario(manager, al(sf_L0_0a), al(createFile(OPEN_KEY, KEY_B),
         createFile(KEY_A, KEY_B), createFile(KEY_B, OPEN_KEY)));
 
-    StoreFile sf_i2B_0 = createFile(OPEN_KEY, KEY_B);
-    StoreFile sf_B2C_0 = createFile(KEY_B, KEY_C);
-    StoreFile sf_C2i_0 = createFile(KEY_C, OPEN_KEY);
+    HStoreFile sf_i2B_0 = createFile(OPEN_KEY, KEY_B);
+    HStoreFile sf_B2C_0 = createFile(KEY_B, KEY_C);
+    HStoreFile sf_C2i_0 = createFile(KEY_C, OPEN_KEY);
     manager.addCompactionResults(al(sf_L0_0a), al(sf_i2B_0, sf_B2C_0, sf_C2i_0));
     manager.removeCompactedFiles(al(sf_L0_0a));
     verifyAllFiles(manager, al(sf_L0_0b, sf_i2B_0, sf_B2C_0, sf_C2i_0));
 
     // Add another l0 file, "compact" both L0 into two stripes
-    StoreFile sf_L0_1 = createFile();
-    StoreFile sf_i2B_1 = createFile(OPEN_KEY, KEY_B);
-    StoreFile sf_B2C_1 = createFile(KEY_B, KEY_C);
+    HStoreFile sf_L0_1 = createFile();
+    HStoreFile sf_i2B_1 = createFile(OPEN_KEY, KEY_B);
+    HStoreFile sf_B2C_1 = createFile(KEY_B, KEY_C);
     manager.insertNewFiles(al(sf_L0_1));
     manager.addCompactionResults(al(sf_L0_0b, sf_L0_1), al(sf_i2B_1, sf_B2C_1));
     manager.removeCompactedFiles(al(sf_L0_0b, sf_L0_1));
     verifyAllFiles(manager, al(sf_i2B_0, sf_B2C_0, sf_C2i_0, sf_i2B_1, sf_B2C_1));
 
     // Try compacting with invalid file (no metadata) - should add files to L0.
-    StoreFile sf_L0_2 = createFile(null, null);
+    HStoreFile sf_L0_2 = createFile(null, null);
     manager.addCompactionResults(al(), al(sf_L0_2));
     manager.removeCompactedFiles(al());
     verifyAllFiles(manager, al(sf_i2B_0, sf_B2C_0, sf_C2i_0, sf_i2B_1, sf_B2C_1, sf_L0_2));
@@ -405,46 +404,46 @@ public class TestStripeStoreFileManager {
     manager.removeCompactedFiles(al(sf_L0_2));
 
     // Do regular compaction in the first stripe.
-    StoreFile sf_i2B_3 = createFile(OPEN_KEY, KEY_B);
+    HStoreFile sf_i2B_3 = createFile(OPEN_KEY, KEY_B);
     manager.addCompactionResults(al(sf_i2B_0, sf_i2B_1), al(sf_i2B_3));
     manager.removeCompactedFiles(al(sf_i2B_0, sf_i2B_1));
     verifyAllFiles(manager, al(sf_B2C_0, sf_C2i_0, sf_B2C_1, sf_i2B_3));
 
     // Rebalance two stripes.
-    StoreFile sf_B2D_4 = createFile(KEY_B, KEY_D);
-    StoreFile sf_D2i_4 = createFile(KEY_D, OPEN_KEY);
+    HStoreFile sf_B2D_4 = createFile(KEY_B, KEY_D);
+    HStoreFile sf_D2i_4 = createFile(KEY_D, OPEN_KEY);
     manager.addCompactionResults(al(sf_B2C_0, sf_C2i_0, sf_B2C_1), al(sf_B2D_4, sf_D2i_4));
     manager.removeCompactedFiles(al(sf_B2C_0, sf_C2i_0, sf_B2C_1));
     verifyAllFiles(manager, al(sf_i2B_3, sf_B2D_4, sf_D2i_4));
 
     // Split the first stripe.
-    StoreFile sf_i2A_5 = createFile(OPEN_KEY, KEY_A);
-    StoreFile sf_A2B_5 = createFile(KEY_A, KEY_B);
+    HStoreFile sf_i2A_5 = createFile(OPEN_KEY, KEY_A);
+    HStoreFile sf_A2B_5 = createFile(KEY_A, KEY_B);
     manager.addCompactionResults(al(sf_i2B_3), al(sf_i2A_5, sf_A2B_5));
     manager.removeCompactedFiles(al(sf_i2B_3));
     verifyAllFiles(manager, al(sf_B2D_4, sf_D2i_4, sf_i2A_5, sf_A2B_5));
 
     // Split the middle stripe.
-    StoreFile sf_B2C_6 = createFile(KEY_B, KEY_C);
-    StoreFile sf_C2D_6 = createFile(KEY_C, KEY_D);
+    HStoreFile sf_B2C_6 = createFile(KEY_B, KEY_C);
+    HStoreFile sf_C2D_6 = createFile(KEY_C, KEY_D);
     manager.addCompactionResults(al(sf_B2D_4), al(sf_B2C_6, sf_C2D_6));
     manager.removeCompactedFiles(al(sf_B2D_4));
     verifyAllFiles(manager, al(sf_D2i_4, sf_i2A_5, sf_A2B_5, sf_B2C_6, sf_C2D_6));
 
     // Merge two different middle stripes.
-    StoreFile sf_A2C_7 = createFile(KEY_A, KEY_C);
+    HStoreFile sf_A2C_7 = createFile(KEY_A, KEY_C);
     manager.addCompactionResults(al(sf_A2B_5, sf_B2C_6), al(sf_A2C_7));
     manager.removeCompactedFiles(al(sf_A2B_5, sf_B2C_6));
     verifyAllFiles(manager, al(sf_D2i_4, sf_i2A_5, sf_C2D_6, sf_A2C_7));
 
     // Merge lower half.
-    StoreFile sf_i2C_8 = createFile(OPEN_KEY, KEY_C);
+    HStoreFile sf_i2C_8 = createFile(OPEN_KEY, KEY_C);
     manager.addCompactionResults(al(sf_i2A_5, sf_A2C_7), al(sf_i2C_8));
     manager.removeCompactedFiles(al(sf_i2A_5, sf_A2C_7));
     verifyAllFiles(manager, al(sf_D2i_4, sf_C2D_6, sf_i2C_8));
 
     // Merge all.
-    StoreFile sf_i2i_9 = createFile(OPEN_KEY, OPEN_KEY);
+    HStoreFile sf_i2i_9 = createFile(OPEN_KEY, OPEN_KEY);
     manager.addCompactionResults(al(sf_D2i_4, sf_C2D_6, sf_i2C_8), al(sf_i2i_9));
     manager.removeCompactedFiles(al(sf_D2i_4, sf_C2D_6, sf_i2C_8));
     verifyAllFiles(manager, al(sf_i2i_9));
@@ -455,11 +454,11 @@ public class TestStripeStoreFileManager {
     // Add file flush into stripes
     StripeStoreFileManager sfm = createManager();
     assertEquals(0, sfm.getStripeCount());
-    StoreFile sf_i2c = createFile(OPEN_KEY, KEY_C), sf_c2i = createFile(KEY_C, OPEN_KEY);
+    HStoreFile sf_i2c = createFile(OPEN_KEY, KEY_C), sf_c2i = createFile(KEY_C, OPEN_KEY);
     sfm.insertNewFiles(al(sf_i2c, sf_c2i));
     assertEquals(2, sfm.getStripeCount());
     // Now try to add conflicting flush - should throw.
-    StoreFile sf_i2d = createFile(OPEN_KEY, KEY_D), sf_d2i = createFile(KEY_D, OPEN_KEY);
+    HStoreFile sf_i2d = createFile(OPEN_KEY, KEY_D), sf_d2i = createFile(KEY_D, OPEN_KEY);
     sfm.insertNewFiles(al(sf_i2d, sf_d2i));
     assertEquals(2, sfm.getStripeCount());
     assertEquals(2, sfm.getLevel0Files().size());
@@ -470,7 +469,7 @@ public class TestStripeStoreFileManager {
     assertEquals(0, sfm.getLevel0Files().size());
     // Add another file to stripe; then "rebalance" stripes w/o it - the file, which was
     // presumably flushed during compaction, should go to L0.
-    StoreFile sf_i2c_2 = createFile(OPEN_KEY, KEY_C);
+    HStoreFile sf_i2c_2 = createFile(OPEN_KEY, KEY_C);
     sfm.insertNewFiles(al(sf_i2c_2));
     sfm.addCompactionResults(al(sf_i2c, sf_c2i), al(sf_i2d, sf_d2i));
     sfm.removeCompactedFiles(al(sf_i2c, sf_c2i));
@@ -482,16 +481,16 @@ public class TestStripeStoreFileManager {
   public void testEmptyResultsForStripes() throws Exception {
     // Test that we can compact L0 into a subset of stripes.
     StripeStoreFileManager manager = createManager();
-    StoreFile sf0a = createFile();
-    StoreFile sf0b = createFile();
+    HStoreFile sf0a = createFile();
+    HStoreFile sf0b = createFile();
     manager.insertNewFiles(al(sf0a));
     manager.insertNewFiles(al(sf0b));
-    ArrayList<StoreFile> compacted = al(createFile(OPEN_KEY, KEY_B),
+    ArrayList<HStoreFile> compacted = al(createFile(OPEN_KEY, KEY_B),
         createFile(KEY_B, KEY_C), createFile(KEY_C, OPEN_KEY));
     manager.addCompactionResults(al(sf0a), compacted);
     manager.removeCompactedFiles(al(sf0a));
     // Next L0 compaction only produces file for the first and last stripe.
-    ArrayList<StoreFile> compacted2 = al(createFile(OPEN_KEY, KEY_B), createFile(KEY_C, OPEN_KEY));
+    ArrayList<HStoreFile> compacted2 = al(createFile(OPEN_KEY, KEY_B), createFile(KEY_C, OPEN_KEY));
     manager.addCompactionResults(al(sf0b), compacted2);
     manager.removeCompactedFiles(al(sf0b));
     compacted.addAll(compacted2);
@@ -526,7 +525,7 @@ public class TestStripeStoreFileManager {
       sfm.insertNewFiles(al(createFile()));
     }
     for (int i = 0; i < filesInStripe; ++i) {
-      ArrayList<StoreFile> stripe = new ArrayList<>();
+      ArrayList<HStoreFile> stripe = new ArrayList<>();
       for (int j = 0; j < stripes; ++j) {
         stripe.add(createFile(
             (j == 0) ? OPEN_KEY : keys[j - 1], (j == stripes - 1) ? OPEN_KEY : keys[j]));
@@ -537,8 +536,8 @@ public class TestStripeStoreFileManager {
   }
 
   private void verifyInvalidCompactionScenario(StripeStoreFileManager manager,
-      ArrayList<StoreFile> filesToCompact, ArrayList<StoreFile> filesToInsert) throws Exception {
-    Collection<StoreFile> allFiles = manager.getStorefiles();
+      ArrayList<HStoreFile> filesToCompact, ArrayList<HStoreFile> filesToInsert) throws Exception {
+    Collection<HStoreFile> allFiles = manager.getStorefiles();
     try {
        manager.addCompactionResults(filesToCompact, filesToInsert);
        fail("Should have thrown");
@@ -549,33 +548,33 @@ public class TestStripeStoreFileManager {
   }
 
   private void verifyGetOrScanScenario(StripeStoreFileManager manager, byte[] start, byte[] end,
-      StoreFile... results) throws Exception {
+      HStoreFile... results) throws Exception {
     verifyGetOrScanScenario(manager, start, end, Arrays.asList(results));
   }
 
   private void verifyGetOrScanScenario(StripeStoreFileManager manager, byte[] start, byte[] end,
-      Collection<StoreFile> results) throws Exception {
+      Collection<HStoreFile> results) throws Exception {
     start = start != null ? start : HConstants.EMPTY_START_ROW;
     end = end != null ? end : HConstants.EMPTY_END_ROW;
-    Collection<StoreFile> sfs = manager.getFilesForScan(start, true, end, false);
+    Collection<HStoreFile> sfs = manager.getFilesForScan(start, true, end, false);
     assertEquals(results.size(), sfs.size());
-    for (StoreFile result : results) {
+    for (HStoreFile result : results) {
       assertTrue(sfs.contains(result));
     }
   }
 
   private void verifyAllFiles(
-      StripeStoreFileManager manager, Collection<StoreFile> results) throws Exception {
+      StripeStoreFileManager manager, Collection<HStoreFile> results) throws Exception {
     verifyGetOrScanScenario(manager, null, null, results);
   }
 
   // TODO: replace with Mockito?
-  private static MockStoreFile createFile(
+  private static MockHStoreFile createFile(
       long size, long seqNum, byte[] startKey, byte[] endKey) throws Exception {
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     Path testFilePath = StoreFileWriter.getUniqueFile(fs, CFDIR);
     fs.create(testFilePath).close();
-    MockStoreFile sf = new MockStoreFile(TEST_UTIL, testFilePath, size, 0, false, seqNum);
+    MockHStoreFile sf = new MockHStoreFile(TEST_UTIL, testFilePath, size, 0, false, seqNum);
     if (startKey != null) {
       sf.setMetadataValue(StripeStoreFileManager.STRIPE_START_KEY, startKey);
     }
@@ -585,15 +584,15 @@ public class TestStripeStoreFileManager {
     return sf;
   }
 
-  private static MockStoreFile createFile(long size, long seqNum) throws Exception {
+  private static MockHStoreFile createFile(long size, long seqNum) throws Exception {
     return createFile(size, seqNum, null, null);
   }
 
-  private static MockStoreFile createFile(byte[] startKey, byte[] endKey) throws Exception {
+  private static MockHStoreFile createFile(byte[] startKey, byte[] endKey) throws Exception {
     return createFile(0, 0, startKey, endKey);
   }
 
-  private static MockStoreFile createFile() throws Exception {
+  private static MockHStoreFile createFile() throws Exception {
     return createFile(null, null);
   }
 
@@ -601,12 +600,12 @@ public class TestStripeStoreFileManager {
     return createManager(new ArrayList<>());
   }
 
-  private static StripeStoreFileManager createManager(ArrayList<StoreFile> sfs) throws Exception {
+  private static StripeStoreFileManager createManager(ArrayList<HStoreFile> sfs) throws Exception {
     return createManager(sfs, TEST_UTIL.getConfiguration());
   }
 
   private static StripeStoreFileManager createManager(
-      ArrayList<StoreFile> sfs, Configuration conf) throws Exception {
+      ArrayList<HStoreFile> sfs, Configuration conf) throws Exception {
     StripeStoreConfig config = new StripeStoreConfig(
         conf, Mockito.mock(StoreConfigInformation.class));
     StripeStoreFileManager result = new StripeStoreFileManager(CellComparator.COMPARATOR, conf,
@@ -615,13 +614,13 @@ public class TestStripeStoreFileManager {
     return result;
   }
 
-  private static ArrayList<StoreFile> al(StoreFile... sfs) {
+  private static ArrayList<HStoreFile> al(HStoreFile... sfs) {
     return new ArrayList<>(Arrays.asList(sfs));
   }
 
-  private static ArrayList<StoreFile> flattenLists(ArrayList<StoreFile>... sfls) {
-    ArrayList<StoreFile> result = new ArrayList<>();
-    for (ArrayList<StoreFile> sfl : sfls) {
+  private static ArrayList<HStoreFile> flattenLists(ArrayList<HStoreFile>... sfls) {
+    ArrayList<HStoreFile> result = new ArrayList<>();
+    for (ArrayList<HStoreFile> sfl : sfls) {
       result.addAll(sfl);
     }
     return result;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java
index 767ad2e..730696c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java
@@ -28,12 +28,12 @@ import java.util.concurrent.ThreadLocalRandom;
 
 import org.apache.hadoop.hbase.Cell;
 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.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -67,7 +67,10 @@ public class TestSwitchToStreamRead {
     }
     VALUE_PREFIX = sb.append("-").toString();
     REGION = UTIL.createLocalHRegion(
-      new HTableDescriptor(TABLE_NAME).addFamily(new HColumnDescriptor(FAMILY).setBlocksize(1024)),
+      TableDescriptorBuilder.newBuilder(TABLE_NAME)
+          .addColumnFamily(
+            ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setBlocksize(1024).build())
+          .build(),
       null, null);
     for (int i = 0; i < 900; i++) {
       REGION
@@ -122,7 +125,7 @@ public class TestSwitchToStreamRead {
       }
     }
     // make sure all scanners are closed.
-    for (StoreFile sf : REGION.getStore(FAMILY).getStorefiles()) {
+    for (HStoreFile sf : REGION.getStore(FAMILY).getStorefiles()) {
       assertFalse(sf.isReferencedInReads());
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java
index 5014b41..2e47566 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java
@@ -15,14 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 
 /**
  * Class to generate several lists of StoreFiles that are all the same size.
@@ -37,8 +36,8 @@ class ConstantSizeFileListGenerator extends StoreFileListGenerator {
   }
 
   @Override
-  public final Iterator<List<StoreFile>> iterator() {
-    return new Iterator<List<StoreFile>>() {
+  public final Iterator<List<HStoreFile>> iterator() {
+    return new Iterator<List<HStoreFile>>() {
       private int count = 0;
 
       @Override
@@ -47,9 +46,9 @@ class ConstantSizeFileListGenerator extends StoreFileListGenerator {
       }
 
       @Override
-      public List<StoreFile> next() {
+      public List<HStoreFile> next() {
         count += 1;
-        ArrayList<StoreFile> files = new ArrayList<>(NUM_FILES_GEN);
+        ArrayList<HStoreFile> files = new ArrayList<>(NUM_FILES_GEN);
         for (int i = 0; i < NUM_FILES_GEN; i++) {
           files.add(createMockStoreFile(FILESIZE));
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/EverythingPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/EverythingPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/EverythingPolicy.java
index 46bb639..ca65bf1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/EverythingPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/EverythingPolicy.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 
@@ -23,8 +22,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 
 /**
  * Test Policy to compact everything every time.
@@ -36,19 +35,16 @@ public class EverythingPolicy extends RatioBasedCompactionPolicy {
    * @param conf            The Conf.
    * @param storeConfigInfo Info about the store.
    */
-  public EverythingPolicy(final Configuration conf,
-                          final StoreConfigInformation storeConfigInfo) {
+  public EverythingPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) {
     super(conf, storeConfigInfo);
   }
 
   @Override
-  protected final ArrayList<StoreFile> applyCompactionPolicy(final ArrayList<StoreFile> candidates,
-    final boolean mayUseOffPeak, final boolean mayBeStuck) throws IOException {
-
+  protected final ArrayList<HStoreFile> applyCompactionPolicy(ArrayList<HStoreFile> candidates,
+      boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
     if (candidates.size() < comConf.getMinFilesToCompact()) {
       return new ArrayList<>(0);
     }
-
     return new ArrayList<>(candidates);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java
index 24302b8..e14f696 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java
@@ -15,13 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 
 /**
  * Class to create list of mock storefiles of specified length.
@@ -53,8 +52,8 @@ class ExplicitFileListGenerator extends StoreFileListGenerator {
   }
 
   @Override
-  public final Iterator<List<StoreFile>> iterator() {
-    return new Iterator<List<StoreFile>>() {
+  public final Iterator<List<HStoreFile>> iterator() {
+    return new Iterator<List<HStoreFile>>() {
       private int nextIndex = 0;
       @Override
       public boolean hasNext() {
@@ -62,8 +61,8 @@ class ExplicitFileListGenerator extends StoreFileListGenerator {
       }
 
       @Override
-      public List<StoreFile> next() {
-        List<StoreFile> files =  createStoreFileList(fileSizes[nextIndex]);
+      public List<HStoreFile> next() {
+        List<HStoreFile> files =  createStoreFileList(fileSizes[nextIndex]);
         nextIndex += 1;
         return files;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java
index a2964ff..fe5f904 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import java.util.ArrayList;
@@ -24,7 +23,7 @@ import java.util.List;
 
 import org.apache.commons.math3.random.GaussianRandomGenerator;
 import org.apache.commons.math3.random.MersenneTwister;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 
 class GaussianFileListGenerator extends StoreFileListGenerator {
 
@@ -33,8 +32,8 @@ class GaussianFileListGenerator extends StoreFileListGenerator {
   }
 
   @Override
-  public Iterator<List<StoreFile>> iterator() {
-    return new Iterator<List<StoreFile>>() {
+  public Iterator<List<HStoreFile>> iterator() {
+    return new Iterator<List<HStoreFile>>() {
       private GaussianRandomGenerator gen =
           new GaussianRandomGenerator(new MersenneTwister(random.nextInt()));
       private int count = 0;
@@ -45,9 +44,9 @@ class GaussianFileListGenerator extends StoreFileListGenerator {
       }
 
       @Override
-      public List<StoreFile> next() {
+      public List<HStoreFile> next() {
         count += 1;
-        ArrayList<StoreFile> files = new ArrayList<>(NUM_FILES_GEN);
+        ArrayList<HStoreFile> files = new ArrayList<>(NUM_FILES_GEN);
         for (int i = 0; i < NUM_FILES_GEN; i++) {
           files.add(createMockStoreFile(
               (int) Math.ceil(Math.max(0, gen.nextNormalizedDouble() * 32 + 32)))

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java
index 85350ca..21cee13 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java
@@ -21,18 +21,18 @@ package org.apache.hadoop.hbase.regionserver.compactions;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.base.MoreObjects;
-
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
 
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.util.StringUtils;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.base.MoreObjects;
+
 /**
  * Base class of objects that can create mock store files with a given size.
  */
@@ -42,31 +42,31 @@ class MockStoreFileGenerator {
   /** The random number generator. */
   protected Random random;
 
-  MockStoreFileGenerator(Class klass) {
+  MockStoreFileGenerator(Class<?> klass) {
     random = new Random(klass.getSimpleName().hashCode());
   }
 
-  protected List<StoreFile> createStoreFileList(final int[] fs) {
-    List<StoreFile> storeFiles = new LinkedList<>();
+  protected List<HStoreFile> createStoreFileList(final int[] fs) {
+    List<HStoreFile> storeFiles = new LinkedList<>();
     for (int fileSize : fs) {
       storeFiles.add(createMockStoreFile(fileSize));
     }
     return storeFiles;
   }
 
-  protected StoreFile createMockStoreFile(final long size) {
+  protected HStoreFile createMockStoreFile(final long size) {
     return createMockStoreFile(size * 1024 * 1024, -1L);
   }
 
-  protected StoreFile createMockStoreFileBytes(final long size) {
+  protected HStoreFile createMockStoreFileBytes(final long size) {
     return createMockStoreFile(size, -1L);
   }
 
-  protected StoreFile createMockStoreFile(final long sizeInBytes, final long seqId) {
-    StoreFile mockSf = mock(StoreFile.class);
+  protected HStoreFile createMockStoreFile(final long sizeInBytes, final long seqId) {
+    HStoreFile mockSf = mock(HStoreFile.class);
     StoreFileReader reader = mock(StoreFileReader.class);
-    String stringPath = "/hbase/testTable/regionA/"
-        + RandomStringUtils.random(FILENAME_LENGTH, 0, 0, true, true, null, random);
+    String stringPath = "/hbase/testTable/regionA/" +
+        RandomStringUtils.random(FILENAME_LENGTH, 0, 0, true, true, null, random);
     Path path = new Path(stringPath);
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java
index 2dbc26f..5554683 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java
@@ -26,13 +26,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
@@ -45,9 +43,6 @@ import org.junit.runners.Parameterized;
 @RunWith(Parameterized.class)
 public class PerfTestCompactionPolicies extends MockStoreFileGenerator {
 
-
-  private static final Log LOG = LogFactory.getLog(PerfTestCompactionPolicies.class);
-
   private final RatioBasedCompactionPolicy cp;
   private final StoreFileListGenerator generator;
   private final HStore store;
@@ -62,13 +57,13 @@ public class PerfTestCompactionPolicies extends MockStoreFileGenerator {
 
 
 
-    Class[] policyClasses = new Class[]{
+    Class<?>[] policyClasses = new Class[]{
         EverythingPolicy.class,
         RatioBasedCompactionPolicy.class,
         ExploringCompactionPolicy.class,
     };
 
-    Class[] fileListGenClasses = new Class[]{
+    Class<?>[] fileListGenClasses = new Class[]{
         ExplicitFileListGenerator.class,
         ConstantSizeFileListGenerator.class,
         SemiConstantSizeFileListGenerator.class,
@@ -88,12 +83,12 @@ public class PerfTestCompactionPolicies extends MockStoreFileGenerator {
         * policyClasses.length);
 
 
-    for (Class policyClass :  policyClasses) {
-      for (Class genClass: fileListGenClasses) {
-        for (int maxFile:maxFileValues) {
-          for (int minFile:minFilesValues) {
-            for (float ratio:ratioValues) {
-              params.add(new Object[] {policyClass, genClass, maxFile, minFile, ratio});
+    for (Class<?> policyClass : policyClasses) {
+      for (Class<?> genClass : fileListGenClasses) {
+        for (int maxFile : maxFileValues) {
+          for (int minFile : minFilesValues) {
+            for (float ratio : ratioValues) {
+              params.add(new Object[] { policyClass, genClass, maxFile, minFile, ratio });
             }
           }
         }
@@ -150,9 +145,9 @@ public class PerfTestCompactionPolicies extends MockStoreFileGenerator {
   @Test
   public final void testSelection() throws Exception {
     long fileDiff = 0;
-    for (List<StoreFile> storeFileList : generator) {
-      List<StoreFile> currentFiles = new ArrayList<>(18);
-      for (StoreFile file : storeFileList) {
+    for (List<HStoreFile> storeFileList : generator) {
+      List<HStoreFile> currentFiles = new ArrayList<>(18);
+      for (HStoreFile file : storeFileList) {
         currentFiles.add(file);
         currentFiles = runIteration(currentFiles);
       }
@@ -172,21 +167,20 @@ public class PerfTestCompactionPolicies extends MockStoreFileGenerator {
   }
 
 
-  private List<StoreFile> runIteration(List<StoreFile> startingStoreFiles) throws IOException {
-
-    List<StoreFile> storeFiles = new ArrayList<>(startingStoreFiles);
+  private List<HStoreFile> runIteration(List<HStoreFile> startingStoreFiles) throws IOException {
+    List<HStoreFile> storeFiles = new ArrayList<>(startingStoreFiles);
     CompactionRequest req = cp.selectCompaction(
         storeFiles, new ArrayList<>(), false, false, false);
     long newFileSize = 0;
 
-    Collection<StoreFile> filesToCompact = req.getFiles();
+    Collection<HStoreFile> filesToCompact = req.getFiles();
 
     if (!filesToCompact.isEmpty()) {
 
       storeFiles = new ArrayList<>(storeFiles);
       storeFiles.removeAll(filesToCompact);
 
-      for (StoreFile storeFile : filesToCompact) {
+      for (HStoreFile storeFile : filesToCompact) {
         newFileSize += storeFile.getReader().length();
       }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java
index 5fe47f3..c89f136 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java
@@ -15,14 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 
 class SemiConstantSizeFileListGenerator extends StoreFileListGenerator {
   SemiConstantSizeFileListGenerator() {
@@ -30,8 +29,8 @@ class SemiConstantSizeFileListGenerator extends StoreFileListGenerator {
   }
 
   @Override
-  public Iterator<List<StoreFile>> iterator() {
-    return new Iterator<List<StoreFile>>() {
+  public Iterator<List<HStoreFile>> iterator() {
+    return new Iterator<List<HStoreFile>>() {
       private int count = 0;
 
       @Override
@@ -40,9 +39,9 @@ class SemiConstantSizeFileListGenerator extends StoreFileListGenerator {
       }
 
       @Override
-      public List<StoreFile> next() {
+      public List<HStoreFile> next() {
         count += 1;
-        ArrayList<StoreFile> files = new ArrayList<>(NUM_FILES_GEN);
+        ArrayList<HStoreFile> files = new ArrayList<>(NUM_FILES_GEN);
         for (int i = 0; i < NUM_FILES_GEN; i++) {
           files.add(createMockStoreFile(random.nextInt(5) + 30));
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java
index f5f36ac..d270da7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java
@@ -15,14 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 
 class SinusoidalFileListGenerator extends StoreFileListGenerator {
   SinusoidalFileListGenerator() {
@@ -30,10 +29,8 @@ class SinusoidalFileListGenerator extends StoreFileListGenerator {
   }
 
   @Override
-  public Iterator<List<StoreFile>> iterator() {
-
-
-    return new Iterator<List<StoreFile>>() {
+  public Iterator<List<HStoreFile>> iterator() {
+    return new Iterator<List<HStoreFile>>() {
       private int count = 0;
       @Override
       public boolean hasNext() {
@@ -41,9 +38,9 @@ class SinusoidalFileListGenerator extends StoreFileListGenerator {
       }
 
       @Override
-      public List<StoreFile> next() {
+      public List<HStoreFile> next() {
         count += 1;
-        ArrayList<StoreFile> files = new ArrayList<>(NUM_FILES_GEN);
+        ArrayList<HStoreFile> files = new ArrayList<>(NUM_FILES_GEN);
         for (int x = 0; x < NUM_FILES_GEN; x++) {
           int fileSize = (int) Math.abs(64 * Math.sin((Math.PI * x) / 50.0)) + 1;
           files.add(createMockStoreFile(fileSize));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java
index 5201eb7..bed1342 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java
@@ -15,14 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 
 class SpikyFileListGenerator extends StoreFileListGenerator {
 
@@ -31,8 +30,8 @@ class SpikyFileListGenerator extends StoreFileListGenerator {
   }
 
   @Override
-  public Iterator<List<StoreFile>> iterator() {
-    return new Iterator<List<StoreFile>>() {
+  public Iterator<List<HStoreFile>> iterator() {
+    return new Iterator<List<HStoreFile>>() {
       private int count = 0;
 
       @Override
@@ -41,9 +40,9 @@ class SpikyFileListGenerator extends StoreFileListGenerator {
       }
 
       @Override
-      public List<StoreFile> next() {
+      public List<HStoreFile> next() {
         count += 1;
-        ArrayList<StoreFile> files = new ArrayList<>(NUM_FILES_GEN);
+        ArrayList<HStoreFile> files = new ArrayList<>(NUM_FILES_GEN);
         for (int x = 0; x < NUM_FILES_GEN; x++) {
           int fileSize = random.nextInt(5) + 1;
           if ( x % 10 == 0) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java
index 643f771..8854676 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java
@@ -15,20 +15,19 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver.compactions;
 
 import java.util.List;
 
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 
-public abstract class StoreFileListGenerator
-    extends MockStoreFileGenerator implements Iterable<List<StoreFile>> {
+public abstract class StoreFileListGenerator extends MockStoreFileGenerator
+    implements Iterable<List<HStoreFile>> {
 
   public static final int MAX_FILE_GEN_ITERS = 10;
   public static final int NUM_FILES_GEN = 1000;
 
-  StoreFileListGenerator(final Class klass) {
+  StoreFileListGenerator(Class<?> klass) {
     super(klass);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
index a71e766..e30383e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java
@@ -47,8 +47,6 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -60,7 +58,7 @@ import org.junit.experimental.categories.Category;
 @Category({ MediumTests.class, RegionServerTests.class })
 public class TestCompactedHFilesDischarger {
   private final HBaseTestingUtility testUtil = new HBaseTestingUtility();
-  private Region region;
+  private HRegion region;
   private final static byte[] fam = Bytes.toBytes("cf_1");
   private final static byte[] qual1 = Bytes.toBytes("qf_1");
   private final static byte[] val = Bytes.toBytes("val");
@@ -120,21 +118,21 @@ public class TestCompactedHFilesDischarger {
     // flush them
     region.flush(true);
 
-    Store store = region.getStore(fam);
+    HStore store = region.getStore(fam);
     assertEquals(3, store.getStorefilesCount());
 
-    Collection<StoreFile> storefiles = store.getStorefiles();
-    Collection<StoreFile> compactedfiles =
-        ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
+    Collection<HStoreFile> storefiles = store.getStorefiles();
+    Collection<HStoreFile> compactedfiles =
+        store.getStoreEngine().getStoreFileManager().getCompactedfiles();
     // None of the files should be in compacted state.
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       assertFalse(file.isCompactedAway());
     }
     // Try to run the cleaner without compaction. there should not be any change
     cleaner.chore();
     storefiles = store.getStorefiles();
     // None of the files should be in compacted state.
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       assertFalse(file.isCompactedAway());
     }
     // now do some compaction
@@ -149,7 +147,7 @@ public class TestCompactedHFilesDischarger {
     cleaner.chore();
     assertEquals(1, store.getStorefilesCount());
     storefiles = store.getStorefiles();
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       // Should not be in compacted state
       assertFalse(file.isCompactedAway());
     }
@@ -186,14 +184,14 @@ public class TestCompactedHFilesDischarger {
     // flush them
     region.flush(true);
 
-    Store store = region.getStore(fam);
+    HStore store = region.getStore(fam);
     assertEquals(3, store.getStorefilesCount());
 
-    Collection<StoreFile> storefiles = store.getStorefiles();
-    Collection<StoreFile> compactedfiles =
-        ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
+    Collection<HStoreFile> storefiles = store.getStorefiles();
+    Collection<HStoreFile> compactedfiles =
+        store.getStoreEngine().getStoreFileManager().getCompactedfiles();
     // None of the files should be in compacted state.
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       assertFalse(file.isCompactedAway());
     }
     // Do compaction
@@ -203,13 +201,13 @@ public class TestCompactedHFilesDischarger {
     storefiles = store.getStorefiles();
     int usedReaderCount = 0;
     int unusedReaderCount = 0;
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       if (((HStoreFile) file).getRefCount() == 3) {
         usedReaderCount++;
       }
     }
     compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
-    for(StoreFile file : compactedfiles) {
+    for(HStoreFile file : compactedfiles) {
       assertEquals("Refcount should be 3", 0, ((HStoreFile) file).getRefCount());
       unusedReaderCount++;
     }
@@ -221,7 +219,7 @@ public class TestCompactedHFilesDischarger {
     countDown();
     assertEquals(1, store.getStorefilesCount());
     storefiles = store.getStorefiles();
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       // Should not be in compacted state
       assertFalse(file.isCompactedAway());
     }
@@ -257,14 +255,14 @@ public class TestCompactedHFilesDischarger {
     // flush them
     region.flush(true);
 
-    Store store = region.getStore(fam);
+    HStore store = region.getStore(fam);
     assertEquals(3, store.getStorefilesCount());
 
-    Collection<StoreFile> storefiles = store.getStorefiles();
-    Collection<StoreFile> compactedfiles =
-        ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
+    Collection<HStoreFile> storefiles = store.getStorefiles();
+    Collection<HStoreFile> compactedfiles =
+        store.getStoreEngine().getStoreFileManager().getCompactedfiles();
     // None of the files should be in compacted state.
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       assertFalse(file.isCompactedAway());
     }
     startScannerThreads();
@@ -274,14 +272,13 @@ public class TestCompactedHFilesDischarger {
     storefiles = store.getStorefiles();
     int usedReaderCount = 0;
     int unusedReaderCount = 0;
-    for (StoreFile file : storefiles) {
-      if (((HStoreFile) file).getRefCount() == 0) {
+    for (HStoreFile file : storefiles) {
+      if (file.getRefCount() == 0) {
         unusedReaderCount++;
       }
     }
-    compactedfiles =
-        ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
-    for(StoreFile file : compactedfiles) {
+    compactedfiles = store.getStoreEngine().getStoreFileManager().getCompactedfiles();
+    for(HStoreFile file : compactedfiles) {
       assertEquals("Refcount should be 3", 3, ((HStoreFile) file).getRefCount());
       usedReaderCount++;
     }
@@ -307,14 +304,14 @@ public class TestCompactedHFilesDischarger {
     storefiles = store.getStorefiles();
     usedReaderCount = 0;
     unusedReaderCount = 0;
-    for (StoreFile file : storefiles) {
-      if (((HStoreFile) file).getRefCount() == 3) {
+    for (HStoreFile file : storefiles) {
+      if (file.getRefCount() == 3) {
         usedReaderCount++;
       }
     }
     compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
-    for(StoreFile file : compactedfiles) {
-      assertEquals("Refcount should be 0", 0, ((HStoreFile) file).getRefCount());
+    for (HStoreFile file : compactedfiles) {
+      assertEquals("Refcount should be 0", 0, file.getRefCount());
       unusedReaderCount++;
     }
     // Though there are files we are not using them for reads
@@ -329,7 +326,7 @@ public class TestCompactedHFilesDischarger {
     // Now the cleaner should be able to clear it up because there are no active readers
     assertEquals(1, store.getStorefilesCount());
     storefiles = store.getStorefiles();
-    for (StoreFile file : storefiles) {
+    for (HStoreFile file : storefiles) {
       // Should not be in compacted state
       assertFalse(file.isCompactedAway());
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
index 170fba2..eb4801a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java
@@ -43,9 +43,9 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
@@ -56,10 +56,10 @@ import org.mockito.stubbing.Answer;
 
 public class TestCompactor {
 
-  public static StoreFile createDummyStoreFile(long maxSequenceId) throws Exception {
+  public static HStoreFile createDummyStoreFile(long maxSequenceId) throws Exception {
     // "Files" are totally unused, it's Scanner class below that gives compactor fake KVs.
     // But compaction depends on everything under the sun, so stub everything with dummies.
-    StoreFile sf = mock(StoreFile.class);
+    HStoreFile sf = mock(HStoreFile.class);
     StoreFileReader r = mock(StoreFileReader.class);
     when(r.length()).thenReturn(1L);
     when(r.getBloomFilterType()).thenReturn(BloomType.NONE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
index ca7c0ae..ef0c931 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java
@@ -42,11 +42,11 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.ScanType;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreUtils;
 import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner;
@@ -87,14 +87,14 @@ public class TestDateTieredCompactor {
   public boolean usePrivateReaders;
 
   private DateTieredCompactor createCompactor(StoreFileWritersCapture writers,
-      final KeyValue[] input, List<StoreFile> storefiles) throws Exception {
+      final KeyValue[] input, List<HStoreFile> storefiles) throws Exception {
     Configuration conf = HBaseConfiguration.create();
     conf.setBoolean("hbase.regionserver.compaction.private.readers", usePrivateReaders);
     final Scanner scanner = new Scanner(input);
     // Create store mock that is satisfactory for compactor.
     HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS);
     ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparator.COMPARATOR);
-    final Store store = mock(Store.class);
+    HStore store = mock(HStore.class);
     when(store.getStorefiles()).thenReturn(storefiles);
     when(store.getColumnFamilyDescriptor()).thenReturn(col);
     when(store.getScanInfo()).thenReturn(si);
@@ -109,14 +109,14 @@ public class TestDateTieredCompactor {
 
     return new DateTieredCompactor(conf, store) {
       @Override
-      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+      protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
           long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
           byte[] dropDeletesToRow) throws IOException {
         return scanner;
       }
 
       @Override
-      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+      protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
           ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
         return scanner;
       }
@@ -126,8 +126,8 @@ public class TestDateTieredCompactor {
   private void verify(KeyValue[] input, List<Long> boundaries, KeyValue[][] output,
       boolean allFiles) throws Exception {
     StoreFileWritersCapture writers = new StoreFileWritersCapture();
-    StoreFile sf1 = createDummyStoreFile(1L);
-    StoreFile sf2 = createDummyStoreFile(2L);
+    HStoreFile sf1 = createDummyStoreFile(1L);
+    HStoreFile sf2 = createDummyStoreFile(2L);
     DateTieredCompactor dtc = createCompactor(writers, input, Arrays.asList(sf1, sf2));
     List<Path> paths = dtc.compact(new CompactionRequest(Arrays.asList(sf1)),
       boundaries.subList(0, boundaries.size() - 1), NoLimitThroughputController.INSTANCE, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
index 49c3cba..1249fee 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
@@ -36,9 +36,6 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -58,12 +55,12 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 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.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
-import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileReader;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
@@ -88,6 +85,9 @@ import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
 import org.mockito.ArgumentMatcher;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
 @RunWith(Parameterized.class)
 @Category({RegionServerTests.class, SmallTests.class})
 public class TestStripeCompactionPolicy {
@@ -163,14 +163,14 @@ public class TestStripeCompactionPolicy {
     StripeCompactionPolicy policy = new StripeCompactionPolicy(conf, sci, ssc) {
       @Override
       public StripeCompactionRequest selectCompaction(StripeInformationProvider si,
-          List<StoreFile> filesCompacting, boolean isOffpeak) throws IOException {
+          List<HStoreFile> filesCompacting, boolean isOffpeak) throws IOException {
         if (!filesCompacting.isEmpty()) return null;
         return selectSingleStripeCompaction(si, false, false, isOffpeak);
       }
 
       @Override
       public boolean needsCompactions(
-          StripeInformationProvider si, List<StoreFile> filesCompacting) {
+          StripeInformationProvider si, List<HStoreFile> filesCompacting) {
         if (!filesCompacting.isEmpty()) return false;
         return needsSingleStripeCompaction(si);
       }
@@ -202,7 +202,7 @@ public class TestStripeCompactionPolicy {
     verifySingleStripeCompaction(policy, si, 1, null);
     // Verify max count is respected.
     si = createStripesWithSizes(0, 0, new Long[] { 5L }, new Long[] { 5L, 4L, 4L, 4L, 4L });
-    List<StoreFile> sfs = si.getStripes().get(1).subList(1, 5);
+    List<HStoreFile> sfs = si.getStripes().get(1).subList(1, 5);
     verifyCompaction(policy, si, sfs, null, 1, null, si.getStartRow(1), si.getEndRow(1), true);
     // Verify ratio is applied.
     si = createStripesWithSizes(0, 0, new Long[] { 5L }, new Long[] { 50L, 4L, 4L, 4L, 4L });
@@ -222,10 +222,10 @@ public class TestStripeCompactionPolicy {
   public void testWithReferences() throws Exception {
     StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create());
     StripeCompactor sc = mock(StripeCompactor.class);
-    StoreFile ref = createFile();
+    HStoreFile ref = createFile();
     when(ref.isReference()).thenReturn(true);
     StripeInformationProvider si = mock(StripeInformationProvider.class);
-    Collection<StoreFile> sfs = al(ref, createFile());
+    Collection<HStoreFile> sfs = al(ref, createFile());
     when(si.getStorefiles()).thenReturn(sfs);
 
     assertTrue(policy.needsCompactions(si, al()));
@@ -349,12 +349,12 @@ public class TestStripeCompactionPolicy {
     edge.setValue(now);
     EnvironmentEdgeManager.injectEdge(edge);
     try {
-      StoreFile expiredFile = createFile(), notExpiredFile = createFile();
+      HStoreFile expiredFile = createFile(), notExpiredFile = createFile();
       when(expiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl - 1);
       when(notExpiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl + 1);
-      List<StoreFile> expired = Lists.newArrayList(expiredFile, expiredFile);
-      List<StoreFile> notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile);
-      List<StoreFile> mixed = Lists.newArrayList(expiredFile, notExpiredFile);
+      List<HStoreFile> expired = Lists.newArrayList(expiredFile, expiredFile);
+      List<HStoreFile> notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile);
+      List<HStoreFile> mixed = Lists.newArrayList(expiredFile, notExpiredFile);
 
       StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create(),
           defaultSplitSize, defaultSplitCount, defaultInitialCount, true);
@@ -389,11 +389,11 @@ public class TestStripeCompactionPolicy {
     edge.setValue(now);
     EnvironmentEdgeManager.injectEdge(edge);
     try {
-      StoreFile expiredFile = createFile(), notExpiredFile = createFile();
+      HStoreFile expiredFile = createFile(), notExpiredFile = createFile();
       when(expiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl - 1);
       when(notExpiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl + 1);
-      List<StoreFile> expired = Lists.newArrayList(expiredFile, expiredFile);
-      List<StoreFile> notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile);
+      List<HStoreFile> expired = Lists.newArrayList(expiredFile, expiredFile);
+      List<HStoreFile> notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile);
 
       StripeCompactionPolicy policy =
           createPolicy(HBaseConfiguration.create(), defaultSplitSize, defaultSplitCount,
@@ -414,7 +414,7 @@ public class TestStripeCompactionPolicy {
 
   @SuppressWarnings("unchecked")
   private static StripeCompactionPolicy.StripeInformationProvider createStripesWithFiles(
-      List<StoreFile>... stripeFiles) throws Exception {
+      List<HStoreFile>... stripeFiles) throws Exception {
     return createStripesWithFiles(createBoundaries(stripeFiles.length),
         Lists.newArrayList(stripeFiles), new ArrayList<>());
   }
@@ -434,7 +434,7 @@ public class TestStripeCompactionPolicy {
     verifySingleStripeCompaction(policy, si, 0, false);
     // Unless there are enough to cause L0 compaction.
     si = createStripesWithSizes(6, 2, stripes);
-    ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
+    ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>();
     sfs.addSublist(si.getLevel0Files());
     sfs.addSublist(si.getStripes().get(0));
     verifyCompaction(
@@ -447,12 +447,13 @@ public class TestStripeCompactionPolicy {
     // if all files of stripe aren't selected, delete must not be dropped.
     stripes = new Long[][] { new Long[] { 100L, 3L, 2L, 2L, 2L }, new Long[] { 6L } };
     si = createStripesWithSizes(0, 0, stripes);
-    List<StoreFile> compact_file = new ArrayList<>();
-    Iterator<StoreFile> iter = si.getStripes().get(0).listIterator(1);
+    List<HStoreFile> compactFile = new ArrayList<>();
+    Iterator<HStoreFile> iter = si.getStripes().get(0).listIterator(1);
     while (iter.hasNext()) {
-        compact_file.add(iter.next());
+        compactFile.add(iter.next());
     }
-    verifyCompaction(policy, si, compact_file, false, 1, null, si.getStartRow(0), si.getEndRow(0), true);
+    verifyCompaction(policy, si, compactFile, false, 1, null, si.getStartRow(0), si.getEndRow(0),
+      true);
   }
 
   /********* HELPER METHODS ************/
@@ -472,14 +473,14 @@ public class TestStripeCompactionPolicy {
     return new StripeCompactionPolicy(conf, sci, ssc);
   }
 
-  private static ArrayList<StoreFile> al(StoreFile... sfs) {
+  private static ArrayList<HStoreFile> al(HStoreFile... sfs) {
     return new ArrayList<>(Arrays.asList(sfs));
   }
 
   private void verifyMergeCompatcion(StripeCompactionPolicy policy, StripeInformationProvider si,
       int from, int to) throws Exception {
     StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false);
-    Collection<StoreFile> sfs = getAllFiles(si, from, to);
+    Collection<HStoreFile> sfs = getAllFiles(si, from, to);
     verifyCollectionsEqual(sfs, scr.getRequest().getFiles());
 
     // All the Stripes are expired, so the Compactor will not create any Writers. We need to create
@@ -538,7 +539,7 @@ public class TestStripeCompactionPolicy {
    * @param boundaries Expected target stripe boundaries.
    */
   private void verifyCompaction(StripeCompactionPolicy policy, StripeInformationProvider si,
-      Collection<StoreFile> sfs, byte[] dropDeletesFrom, byte[] dropDeletesTo,
+      Collection<HStoreFile> sfs, byte[] dropDeletesFrom, byte[] dropDeletesTo,
       final List<byte[]> boundaries) throws Exception {
     StripeCompactor sc = mock(StripeCompactor.class);
     assertTrue(policy.needsCompactions(si, al()));
@@ -573,7 +574,7 @@ public class TestStripeCompactionPolicy {
    * @param righr Right boundary of the compaction.
    */
   private void verifyCompaction(StripeCompactionPolicy policy, StripeInformationProvider si,
-      Collection<StoreFile> sfs, Boolean dropDeletes, Integer count, Long size,
+      Collection<HStoreFile> sfs, Boolean dropDeletes, Integer count, Long size,
       byte[] start, byte[] end, boolean needsCompaction) throws IOException {
     StripeCompactor sc = mock(StripeCompactor.class);
     assertTrue(!needsCompaction || policy.needsCompactions(si, al()));
@@ -612,15 +613,15 @@ public class TestStripeCompactionPolicy {
             : (dropDeletes.booleanValue() ? aryEq(value) : isNull(byte[].class));
   }
 
-  private void verifyCollectionsEqual(Collection<StoreFile> sfs, Collection<StoreFile> scr) {
+  private void verifyCollectionsEqual(Collection<HStoreFile> sfs, Collection<HStoreFile> scr) {
     // Dumb.
     assertEquals(sfs.size(), scr.size());
     assertTrue(scr.containsAll(sfs));
   }
 
-  private static List<StoreFile> getAllFiles(
+  private static List<HStoreFile> getAllFiles(
       StripeInformationProvider si, int fromStripe, int toStripe) {
-    ArrayList<StoreFile> expected = new ArrayList<>();
+    ArrayList<HStoreFile> expected = new ArrayList<>();
     for (int i = fromStripe; i <= toStripe; ++i) {
       expected.addAll(si.getStripes().get(i));
     }
@@ -694,15 +695,15 @@ public class TestStripeCompactionPolicy {
 
   private static StripeInformationProvider createStripes(List<byte[]> boundaries,
       List<List<Long>> stripeSizes, List<Long> l0Sizes) throws Exception {
-    List<List<StoreFile>> stripeFiles = new ArrayList<>(stripeSizes.size());
+    List<List<HStoreFile>> stripeFiles = new ArrayList<>(stripeSizes.size());
     for (List<Long> sizes : stripeSizes) {
-      List<StoreFile> sfs = new ArrayList<>(sizes.size());
+      List<HStoreFile> sfs = new ArrayList<>(sizes.size());
       for (Long size : sizes) {
         sfs.add(createFile(size));
       }
       stripeFiles.add(sfs);
     }
-    List<StoreFile> l0Files = new ArrayList<>();
+    List<HStoreFile> l0Files = new ArrayList<>();
     for (Long size : l0Sizes) {
       l0Files.add(createFile(size));
     }
@@ -713,8 +714,8 @@ public class TestStripeCompactionPolicy {
    * This method actually does all the work.
    */
   private static StripeInformationProvider createStripesWithFiles(List<byte[]> boundaries,
-      List<List<StoreFile>> stripeFiles, List<StoreFile> l0Files) throws Exception {
-    ArrayList<ImmutableList<StoreFile>> stripes = new ArrayList<>();
+      List<List<HStoreFile>> stripeFiles, List<HStoreFile> l0Files) throws Exception {
+    ArrayList<ImmutableList<HStoreFile>> stripes = new ArrayList<>();
     ArrayList<byte[]> boundariesList = new ArrayList<>();
     StripeInformationProvider si = mock(StripeInformationProvider.class);
     if (!stripeFiles.isEmpty()) {
@@ -724,7 +725,7 @@ public class TestStripeCompactionPolicy {
         byte[] startKey = ((i == 0) ? OPEN_KEY : boundaries.get(i - 1));
         byte[] endKey = ((i == boundaries.size()) ? OPEN_KEY : boundaries.get(i));
         boundariesList.add(endKey);
-        for (StoreFile sf : stripeFiles.get(i)) {
+        for (HStoreFile sf : stripeFiles.get(i)) {
           setFileStripe(sf, startKey, endKey);
         }
         stripes.add(ImmutableList.copyOf(stripeFiles.get(i)));
@@ -732,7 +733,7 @@ public class TestStripeCompactionPolicy {
         when(si.getEndRow(eq(i))).thenReturn(endKey);
       }
     }
-    ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
+    ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>();
     sfs.addAllSublists(stripes);
     sfs.addSublist(l0Files);
     when(si.getStorefiles()).thenReturn(sfs);
@@ -743,8 +744,8 @@ public class TestStripeCompactionPolicy {
     return si;
   }
 
-  private static StoreFile createFile(long size) throws Exception {
-    StoreFile sf = mock(StoreFile.class);
+  private static HStoreFile createFile(long size) throws Exception {
+    HStoreFile sf = mock(HStoreFile.class);
     when(sf.getPath()).thenReturn(new Path("moo"));
     StoreFileReader r = mock(StoreFileReader.class);
     when(r.getEntries()).thenReturn(size);
@@ -758,11 +759,11 @@ public class TestStripeCompactionPolicy {
     return sf;
   }
 
-  private static StoreFile createFile() throws Exception {
+  private static HStoreFile createFile() throws Exception {
     return createFile(0);
   }
 
-  private static void setFileStripe(StoreFile sf, byte[] startKey, byte[] endKey) {
+  private static void setFileStripe(HStoreFile sf, byte[] startKey, byte[] endKey) {
     when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_START_KEY)).thenReturn(startKey);
     when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_END_KEY)).thenReturn(endKey);
   }
@@ -770,7 +771,7 @@ public class TestStripeCompactionPolicy {
   private StripeCompactor createCompactor() throws Exception {
     HColumnDescriptor col = new HColumnDescriptor(Bytes.toBytes("foo"));
     StoreFileWritersCapture writers = new StoreFileWritersCapture();
-    Store store = mock(Store.class);
+    HStore store = mock(HStore.class);
     HRegionInfo info = mock(HRegionInfo.class);
     when(info.getRegionNameAsString()).thenReturn("testRegion");
     when(store.getColumnFamilyDescriptor()).thenReturn(col);
@@ -784,14 +785,14 @@ public class TestStripeCompactionPolicy {
     final Scanner scanner = new Scanner();
     return new StripeCompactor(conf, store) {
       @Override
-      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+      protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
           long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
           byte[] dropDeletesToRow) throws IOException {
         return scanner;
       }
 
       @Override
-      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+      protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
           ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
         return scanner;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
index e51a675..bd3a803 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java
@@ -42,10 +42,10 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.ScanInfo;
 import org.apache.hadoop.hbase.regionserver.ScanType;
-import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner;
 import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture;
@@ -195,7 +195,7 @@ public class TestStripeCompactor {
     // Create store mock that is satisfactory for compactor.
     HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS);
     ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparator.COMPARATOR);
-    Store store = mock(Store.class);
+    HStore store = mock(HStore.class);
     when(store.getColumnFamilyDescriptor()).thenReturn(col);
     when(store.getScanInfo()).thenReturn(si);
     when(store.areWritesEnabled()).thenReturn(true);
@@ -207,14 +207,14 @@ public class TestStripeCompactor {
 
     return new StripeCompactor(conf, store) {
       @Override
-      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+      protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
           long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
           byte[] dropDeletesToRow) throws IOException {
         return scanner;
       }
 
       @Override
-      protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
+      protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
           ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
         return scanner;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index d25829d..391155e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -51,7 +51,17 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -60,7 +70,6 @@ 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.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher;
 import org.apache.hadoop.hbase.regionserver.FlushRequestListener;
@@ -101,6 +110,8 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
+
 /**
  * Test replay of edits out of a WAL split.
  */
@@ -636,7 +647,7 @@ public abstract class AbstractTestWALReplay {
     // Switch between throw and not throw exception in flush
     static final AtomicBoolean throwExceptionWhenFlushing = new AtomicBoolean(false);
 
-    public CustomStoreFlusher(Configuration conf, Store store) {
+    public CustomStoreFlusher(Configuration conf, HStore store) {
       super(conf, store);
     }
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
index 2b5c78c..9dc13d9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java
@@ -62,9 +62,9 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.Visibil
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
@@ -843,10 +843,10 @@ public abstract class TestVisibilityLabels {
     }
     TEST_UTIL.getAdmin().flush(tableName);
     List<HRegion> regions = TEST_UTIL.getHBaseCluster().getRegions(tableName);
-    Store store = regions.get(0).getStore(fam);
-    Collection<StoreFile> storefiles = store.getStorefiles();
+    HStore store = regions.get(0).getStore(fam);
+    Collection<HStoreFile> storefiles = store.getStorefiles();
     assertTrue(storefiles.size() > 0);
-    for (StoreFile storeFile : storefiles) {
+    for (HStoreFile storeFile : storefiles) {
       assertTrue(storeFile.getReader().getHFileReader().getFileContext().isIncludesTags());
     }
   }


[09/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
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 daad241..de41087 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
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
 import java.util.Optional;
+import java.util.OptionalDouble;
 import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CompletionService;
@@ -43,6 +44,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Predicate;
+import java.util.function.ToLongFunction;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -171,7 +175,7 @@ public class HStore implements Store {
   private ScanInfo scanInfo;
 
   // TODO: ideally, this should be part of storeFileManager, as we keep passing this to it.
-  final List<StoreFile> filesCompacting = Lists.newArrayList();
+  final List<HStoreFile> filesCompacting = Lists.newArrayList();
 
   // All access must be synchronized.
   private final Set<ChangedReadersObserver> changedReaderObservers =
@@ -335,7 +339,7 @@ public class HStore implements Store {
    * @param kvComparator KVComparator for storeFileManager.
    * @return StoreEngine to use.
    */
-  protected StoreEngine<?, ?, ?, ?> createStoreEngine(Store store, Configuration conf,
+  protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf,
       CellComparator kvComparator) throws IOException {
     return StoreEngine.create(store, conf, comparator);
   }
@@ -517,12 +521,12 @@ public class HStore implements Store {
    * from the given directory.
    * @throws IOException
    */
-  private List<StoreFile> loadStoreFiles() throws IOException {
+  private List<HStoreFile> loadStoreFiles() throws IOException {
     Collection<StoreFileInfo> files = fs.getStoreFiles(getColumnFamilyName());
     return openStoreFiles(files);
   }
 
-  private List<StoreFile> openStoreFiles(Collection<StoreFileInfo> files) throws IOException {
+  private List<HStoreFile> openStoreFiles(Collection<StoreFileInfo> files) throws IOException {
     if (files == null || files.isEmpty()) {
       return new ArrayList<>();
     }
@@ -530,28 +534,21 @@ public class HStore implements Store {
     ThreadPoolExecutor storeFileOpenerThreadPool =
       this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" +
           this.getColumnFamilyName());
-    CompletionService<StoreFile> completionService = new ExecutorCompletionService<>(storeFileOpenerThreadPool);
+    CompletionService<HStoreFile> completionService = new ExecutorCompletionService<>(storeFileOpenerThreadPool);
 
     int totalValidStoreFile = 0;
-    for (final StoreFileInfo storeFileInfo: files) {
+    for (StoreFileInfo storeFileInfo : files) {
       // open each store file in parallel
-      completionService.submit(new Callable<StoreFile>() {
-        @Override
-        public StoreFile call() throws IOException {
-          StoreFile storeFile = createStoreFileAndReader(storeFileInfo);
-          return storeFile;
-        }
-      });
+      completionService.submit(() -> this.createStoreFileAndReader(storeFileInfo));
       totalValidStoreFile++;
     }
 
-    ArrayList<StoreFile> results = new ArrayList<>(files.size());
+    ArrayList<HStoreFile> results = new ArrayList<>(files.size());
     IOException ioe = null;
     try {
       for (int i = 0; i < totalValidStoreFile; i++) {
         try {
-          Future<StoreFile> future = completionService.take();
-          StoreFile storeFile = future.get();
+          HStoreFile storeFile = completionService.take().get();
           if (storeFile != null) {
             long length = storeFile.getReader().length();
             this.storeSize += length;
@@ -574,9 +571,9 @@ public class HStore implements Store {
       // close StoreFile readers
       boolean evictOnClose =
           cacheConf != null? cacheConf.shouldEvictOnClose(): true;
-      for (StoreFile file : results) {
+      for (HStoreFile file : results) {
         try {
-          if (file != null) file.closeReader(evictOnClose);
+          if (file != null) file.closeStoreFile(evictOnClose);
         } catch (IOException e) {
           LOG.warn(e.getMessage());
         }
@@ -618,19 +615,18 @@ public class HStore implements Store {
    */
   private void refreshStoreFilesInternal(Collection<StoreFileInfo> newFiles) throws IOException {
     StoreFileManager sfm = storeEngine.getStoreFileManager();
-    Collection<StoreFile> currentFiles = sfm.getStorefiles();
-    Collection<StoreFile> compactedFiles = sfm.getCompactedfiles();
+    Collection<HStoreFile> currentFiles = sfm.getStorefiles();
+    Collection<HStoreFile> compactedFiles = sfm.getCompactedfiles();
     if (currentFiles == null) currentFiles = Collections.emptySet();
     if (newFiles == null) newFiles = Collections.emptySet();
     if (compactedFiles == null) compactedFiles = Collections.emptySet();
 
-    HashMap<StoreFileInfo, StoreFile> currentFilesSet = new HashMap<>(currentFiles.size());
-    for (StoreFile sf : currentFiles) {
+    HashMap<StoreFileInfo, HStoreFile> currentFilesSet = new HashMap<>(currentFiles.size());
+    for (HStoreFile sf : currentFiles) {
       currentFilesSet.put(sf.getFileInfo(), sf);
     }
-    HashMap<StoreFileInfo, StoreFile> compactedFilesSet =
-        new HashMap<StoreFileInfo, StoreFile>(compactedFiles.size());
-    for (StoreFile sf : compactedFiles) {
+    HashMap<StoreFileInfo, HStoreFile> compactedFilesSet = new HashMap<>(compactedFiles.size());
+    for (HStoreFile sf : compactedFiles) {
       compactedFilesSet.put(sf.getFileInfo(), sf);
     }
 
@@ -647,13 +643,13 @@ public class HStore implements Store {
     LOG.info("Refreshing store files for region " + this.getRegionInfo().getRegionNameAsString()
       + " files to add: " + toBeAddedFiles + " files to remove: " + toBeRemovedFiles);
 
-    Set<StoreFile> toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size());
+    Set<HStoreFile> toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size());
     for (StoreFileInfo sfi : toBeRemovedFiles) {
       toBeRemovedStoreFiles.add(currentFilesSet.get(sfi));
     }
 
     // try to open the files
-    List<StoreFile> openedFiles = openStoreFiles(toBeAddedFiles);
+    List<HStoreFile> openedFiles = openStoreFiles(toBeAddedFiles);
 
     // propogate the file changes to the underlying store file manager
     replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); //won't throw an exception
@@ -668,14 +664,14 @@ public class HStore implements Store {
     completeCompaction(toBeRemovedStoreFiles);
   }
 
-  private StoreFile createStoreFileAndReader(final Path p) throws IOException {
+  private HStoreFile createStoreFileAndReader(final Path p) throws IOException {
     StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(), p);
     return createStoreFileAndReader(info);
   }
 
-  private StoreFile createStoreFileAndReader(final StoreFileInfo info) throws IOException {
+  private HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException {
     info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
-    StoreFile storeFile = new HStoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
+    HStoreFile storeFile = new HStoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
         this.family.getBloomFilterType(), isPrimaryReplicaStore());
     storeFile.initReader();
     return storeFile;
@@ -734,12 +730,12 @@ public class HStore implements Store {
    * @return All store files.
    */
   @Override
-  public Collection<StoreFile> getStorefiles() {
+  public Collection<HStoreFile> getStorefiles() {
     return this.storeEngine.getStoreFileManager().getStorefiles();
   }
 
   @Override
-  public Collection<StoreFile> getCompactedFiles() {
+  public Collection<HStoreFile> getCompactedFiles() {
     return this.storeEngine.getStoreFileManager().getCompactedfiles();
   }
 
@@ -756,19 +752,19 @@ public class HStore implements Store {
         isPrimaryReplicaStore(), conf);
       reader.loadFileInfo();
 
-      byte[] firstKey = reader.getFirstRowKey();
-      Preconditions.checkState(firstKey != null, "First key can not be null");
-      Cell lk = reader.getLastKey();
-      Preconditions.checkState(lk != null, "Last key can not be null");
-      byte[] lastKey =  CellUtil.cloneRow(lk);
+      Optional<byte[]> firstKey = reader.getFirstRowKey();
+      Preconditions.checkState(firstKey.isPresent(), "First key can not be null");
+      Optional<Cell> lk = reader.getLastKey();
+      Preconditions.checkState(lk.isPresent(), "Last key can not be null");
+      byte[] lastKey =  CellUtil.cloneRow(lk.get());
 
-      LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) +
+      LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey.get()) +
           " last=" + Bytes.toStringBinary(lastKey));
       LOG.debug("Region bounds: first=" +
           Bytes.toStringBinary(getRegionInfo().getStartKey()) +
           " last=" + Bytes.toStringBinary(getRegionInfo().getEndKey()));
 
-      if (!this.getRegionInfo().containsRange(firstKey, lastKey)) {
+      if (!this.getRegionInfo().containsRange(firstKey.get(), lastKey)) {
         throw new WrongRegionException(
             "Bulk load file " + srcPath.toString() + " does not fit inside region "
             + this.getRegionInfo().getRegionNameAsString());
@@ -842,7 +838,7 @@ public class HStore implements Store {
     LOG.info("Loaded HFile " + srcPath + " into store '" + getColumnFamilyName() + "' as "
         + dstPath + " - updating store file list.");
 
-    StoreFile sf = createStoreFileAndReader(dstPath);
+    HStoreFile sf = createStoreFileAndReader(dstPath);
     bulkLoadHFile(sf);
 
     LOG.info("Successfully loaded store file " + srcPath + " into store " + this
@@ -852,11 +848,11 @@ public class HStore implements Store {
   }
 
   public void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException {
-    StoreFile sf = createStoreFileAndReader(fileInfo);
+    HStoreFile sf = createStoreFileAndReader(fileInfo);
     bulkLoadHFile(sf);
   }
 
-  private void bulkLoadHFile(StoreFile sf) throws IOException {
+  private void bulkLoadHFile(HStoreFile sf) throws IOException {
     StoreFileReader r = sf.getReader();
     this.storeSize += r.length();
     this.totalUncompressedBytes += r.getTotalUncompressedBytes();
@@ -883,13 +879,13 @@ public class HStore implements Store {
   }
 
   @Override
-  public ImmutableCollection<StoreFile> close() throws IOException {
+  public ImmutableCollection<HStoreFile> close() throws IOException {
     this.archiveLock.lock();
     this.lock.writeLock().lock();
     try {
       // Clear so metrics doesn't find them.
-      ImmutableCollection<StoreFile> result = storeEngine.getStoreFileManager().clearFiles();
-      Collection<StoreFile> compactedfiles =
+      ImmutableCollection<HStoreFile> result = storeEngine.getStoreFileManager().clearFiles();
+      Collection<HStoreFile> compactedfiles =
           storeEngine.getStoreFileManager().clearCompactedFiles();
       // clear the compacted files
       if (compactedfiles != null && !compactedfiles.isEmpty()) {
@@ -904,13 +900,13 @@ public class HStore implements Store {
         // close each store file in parallel
         CompletionService<Void> completionService =
           new ExecutorCompletionService<>(storeFileCloserThreadPool);
-        for (final StoreFile f : result) {
+        for (HStoreFile f : result) {
           completionService.submit(new Callable<Void>() {
             @Override
             public Void call() throws IOException {
               boolean evictOnClose =
                   cacheConf != null? cacheConf.shouldEvictOnClose(): true;
-              f.closeReader(evictOnClose);
+              f.closeStoreFile(evictOnClose);
               return null;
             }
           });
@@ -1012,20 +1008,20 @@ public class HStore implements Store {
     throw lastException;
   }
 
-  /*
+  /**
    * @param path The pathname of the tmp file into which the store was flushed
    * @param logCacheFlushId
    * @param status
-   * @return StoreFile created.
+   * @return store file created.
    * @throws IOException
    */
-  private StoreFile commitFile(final Path path, final long logCacheFlushId, MonitoredTask status)
+  private HStoreFile commitFile(Path path, long logCacheFlushId, MonitoredTask status)
       throws IOException {
     // Write-out finished successfully, move into the right spot
     Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path);
 
     status.setStatus("Flushing " + this + ": reopening flushed file");
-    StoreFile sf = createStoreFileAndReader(dstPath);
+    HStoreFile sf = createStoreFileAndReader(dstPath);
 
     StoreFileReader r = sf.getReader();
     this.storeSize += r.length();
@@ -1041,35 +1037,32 @@ public class HStore implements Store {
 
   @Override
   public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
-                                            boolean isCompaction, boolean includeMVCCReadpoint,
-                                            boolean includesTag)
-      throws IOException {
+      boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag) throws IOException {
     return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
-        includesTag, false);
+      includesTag, false);
   }
 
-  /*
+  /**
    * @param maxKeyCount
    * @param compression Compression algorithm to use
    * @param isCompaction whether we are creating a new file in a compaction
-   * @param includesMVCCReadPoint - whether to include MVCC or not
+   * @param includeMVCCReadpoint - whether to include MVCC or not
    * @param includesTag - includesTag or not
    * @return Writer for a new StoreFile in the tmp dir.
    */
   @Override
   public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
       boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
-      boolean shouldDropBehind)
-  throws IOException {
+      boolean shouldDropBehind) throws IOException {
     return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
-        includesTag, shouldDropBehind, null);
+      includesTag, shouldDropBehind, null);
   }
 
-  /*
+  /**
    * @param maxKeyCount
    * @param compression Compression algorithm to use
    * @param isCompaction whether we are creating a new file in a compaction
-   * @param includesMVCCReadPoint - whether to include MVCC or not
+   * @param includeMVCCReadpoint - whether to include MVCC or not
    * @param includesTag - includesTag or not
    * @return Writer for a new StoreFile in the tmp dir.
    */
@@ -1078,8 +1071,7 @@ public class HStore implements Store {
   @Override
   public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
       boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
-      boolean shouldDropBehind, final TimeRangeTracker trt)
-  throws IOException {
+      boolean shouldDropBehind, final TimeRangeTracker trt) throws IOException {
     final CacheConfig writerCacheConf;
     if (isCompaction) {
       // Don't cache data on write on compactions.
@@ -1133,15 +1125,18 @@ public class HStore implements Store {
   }
 
 
-  /*
+  private long getTotalSize(Collection<HStoreFile> sfs) {
+    return sfs.stream().mapToLong(sf -> sf.getReader().length()).sum();
+  }
+
+  /**
    * Change storeFiles adding into place the Reader produced by this new flush.
    * @param sfs Store files
    * @param snapshotId
    * @throws IOException
    * @return Whether compaction is required.
    */
-  private boolean updateStorefiles(final List<StoreFile> sfs, final long snapshotId)
-      throws IOException {
+  private boolean updateStorefiles(List<HStoreFile> sfs, long snapshotId) throws IOException {
     this.lock.writeLock().lock();
     try {
       this.storeEngine.getStoreFileManager().insertNewFiles(sfs);
@@ -1159,10 +1154,7 @@ public class HStore implements Store {
     // notify to be called here - only in case of flushes
     notifyChangedReadersObservers(sfs);
     if (LOG.isTraceEnabled()) {
-      long totalSize = 0;
-      for (StoreFile sf : sfs) {
-        totalSize += sf.getReader().length();
-      }
+      long totalSize = getTotalSize(sfs);
       String traceMessage = "FLUSH time,count,size,store size,store files ["
           + EnvironmentEdgeManager.currentTime() + "," + sfs.size() + "," + totalSize
           + "," + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]";
@@ -1171,11 +1163,11 @@ public class HStore implements Store {
     return needsCompaction();
   }
 
-  /*
+  /**
    * Notify all observers that set of Readers has changed.
    * @throws IOException
    */
-  private void notifyChangedReadersObservers(List<StoreFile> sfs) throws IOException {
+  private void notifyChangedReadersObservers(List<HStoreFile> sfs) throws IOException {
     for (ChangedReadersObserver o : this.changedReaderObservers) {
       List<KeyValueScanner> memStoreScanners;
       this.lock.readLock().lock();
@@ -1190,13 +1182,39 @@ public class HStore implements Store {
 
   /**
    * Get all scanners with no filtering based on TTL (that happens further down the line).
+   * @param cacheBlocks cache the blocks or not
+   * @param usePread true to use pread, false if not
+   * @param isCompaction true if the scanner is created for compaction
+   * @param matcher the scan query matcher
+   * @param startRow the start row
+   * @param stopRow the stop row
+   * @param readPt the read point of the current scan
+   * @return all scanners for this store
+   */
+  public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet, boolean usePread,
+      boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt)
+      throws IOException {
+    return getScanners(cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, false,
+      readPt);
+  }
+
+  /**
+   * Get all scanners with no filtering based on TTL (that happens further down the line).
+   * @param cacheBlocks cache the blocks or not
+   * @param usePread true to use pread, false if not
+   * @param isCompaction true if the scanner is created for compaction
+   * @param matcher the scan query matcher
+   * @param startRow the start row
+   * @param includeStartRow true to include start row, false if not
+   * @param stopRow the stop row
+   * @param includeStopRow true to include stop row, false if not
+   * @param readPt the read point of the current scan
    * @return all scanners for this store
    */
-  @Override
   public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean usePread,
       boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow,
       byte[] stopRow, boolean includeStopRow, long readPt) throws IOException {
-    Collection<StoreFile> storeFilesToScan;
+    Collection<HStoreFile> storeFilesToScan;
     List<KeyValueScanner> memStoreScanners;
     this.lock.readLock().lock();
     try {
@@ -1221,8 +1239,45 @@ public class HStore implements Store {
     return scanners;
   }
 
-  @Override
-  public List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks,
+  /**
+   * Create scanners on the given files and if needed on the memstore with no filtering based on TTL
+   * (that happens further down the line).
+   * @param files the list of files on which the scanners has to be created
+   * @param cacheBlocks cache the blocks or not
+   * @param usePread true to use pread, false if not
+   * @param isCompaction true if the scanner is created for compaction
+   * @param matcher the scan query matcher
+   * @param startRow the start row
+   * @param stopRow the stop row
+   * @param readPt the read point of the current scan
+   * @param includeMemstoreScanner true if memstore has to be included
+   * @return scanners on the given files and on the memstore if specified
+   */
+  public List<KeyValueScanner> getScanners(List<HStoreFile> files, boolean cacheBlocks,
+      boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
+      byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner)
+      throws IOException {
+    return getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow,
+      false, readPt, includeMemstoreScanner);
+  }
+
+  /**
+   * Create scanners on the given files and if needed on the memstore with no filtering based on TTL
+   * (that happens further down the line).
+   * @param files the list of files on which the scanners has to be created
+   * @param cacheBlocks ache the blocks or not
+   * @param usePread true to use pread, false if not
+   * @param isCompaction true if the scanner is created for compaction
+   * @param matcher the scan query matcher
+   * @param startRow the start row
+   * @param includeStartRow true to include start row, false if not
+   * @param stopRow the stop row
+   * @param includeStopRow true to include stop row, false if not
+   * @param readPt the read point of the current scan
+   * @param includeMemstoreScanner true if memstore has to be included
+   * @return scanners on the given files and on the memstore if specified
+   */
+  public List<KeyValueScanner> getScanners(List<HStoreFile> files, boolean cacheBlocks,
       boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
       boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
       boolean includeMemstoreScanner) throws IOException {
@@ -1305,16 +1360,16 @@ public class HStore implements Store {
    * @return Storefile we compacted into or null if we failed or opted out early.
    */
   @Override
-  public List<StoreFile> compact(CompactionContext compaction,
+  public List<HStoreFile> compact(CompactionContext compaction,
       ThroughputController throughputController) throws IOException {
     return compact(compaction, throughputController, null);
   }
 
   @Override
-  public List<StoreFile> compact(CompactionContext compaction,
+  public List<HStoreFile> compact(CompactionContext compaction,
     ThroughputController throughputController, User user) throws IOException {
     assert compaction != null;
-    List<StoreFile> sfs = null;
+    List<HStoreFile> sfs = null;
     CompactionRequest cr = compaction.getRequest();
     try {
       // Do all sanity checking in here if we have a valid CompactionRequest
@@ -1322,7 +1377,7 @@ public class HStore implements Store {
       // block below
       long compactionStartTime = EnvironmentEdgeManager.currentTime();
       assert compaction.hasSelection();
-      Collection<StoreFile> filesToCompact = cr.getFiles();
+      Collection<HStoreFile> filesToCompact = cr.getFiles();
       assert !filesToCompact.isEmpty();
       synchronized (filesCompacting) {
         // sanity check: we're compacting files that this store knows about
@@ -1338,7 +1393,6 @@ public class HStore implements Store {
       // Commence the compaction.
       List<Path> newFiles = compaction.compact(throughputController, user);
 
-      long outputBytes = 0L;
       // TODO: get rid of this!
       if (!this.conf.getBoolean("hbase.hstore.compaction.complete", true)) {
         LOG.warn("hbase.hstore.compaction.complete is set to false");
@@ -1347,8 +1401,8 @@ public class HStore implements Store {
             cacheConf != null? cacheConf.shouldEvictOnClose(): true;
         for (Path newFile : newFiles) {
           // Create storefile around what we wrote with a reader on it.
-          StoreFile sf = createStoreFileAndReader(newFile);
-          sf.closeReader(evictOnClose);
+          HStoreFile sf = createStoreFileAndReader(newFile);
+          sf.closeStoreFile(evictOnClose);
           sfs.add(sf);
         }
         return sfs;
@@ -1364,10 +1418,7 @@ public class HStore implements Store {
         compactedCellsCount += getCompactionProgress().totalCompactingKVs;
         compactedCellsSize += getCompactionProgress().totalCompactedSize;
       }
-
-      for (StoreFile sf : sfs) {
-        outputBytes += sf.getReader().length();
-      }
+      long outputBytes = getTotalSize(sfs);
 
       // At this point the store will use new files for all new scanners.
       completeCompaction(filesToCompact); // update store size.
@@ -1387,12 +1438,12 @@ public class HStore implements Store {
     }
   }
 
-  private List<StoreFile> moveCompatedFilesIntoPlace(CompactionRequest cr, List<Path> newFiles,
+  private List<HStoreFile> moveCompatedFilesIntoPlace(CompactionRequest cr, List<Path> newFiles,
       User user) throws IOException {
-    List<StoreFile> sfs = new ArrayList<>(newFiles.size());
+    List<HStoreFile> sfs = new ArrayList<>(newFiles.size());
     for (Path newFile : newFiles) {
       assert newFile != null;
-      StoreFile sf = moveFileIntoPlace(newFile);
+      HStoreFile sf = moveFileIntoPlace(newFile);
       if (this.getCoprocessorHost() != null) {
         getCoprocessorHost().postCompact(this, sf, cr.getTracker(), user);
       }
@@ -1403,7 +1454,7 @@ public class HStore implements Store {
   }
 
   // Package-visible for tests
-  StoreFile moveFileIntoPlace(final Path newFile) throws IOException {
+  HStoreFile moveFileIntoPlace(Path newFile) throws IOException {
     validateStoreFile(newFile);
     // Move the file into the right spot
     Path destPath = fs.commitStoreFile(getColumnFamilyName(), newFile);
@@ -1415,17 +1466,15 @@ public class HStore implements Store {
    * @param filesCompacted Files compacted (input).
    * @param newFiles Files from compaction.
    */
-  private void writeCompactionWalRecord(Collection<StoreFile> filesCompacted,
-      Collection<StoreFile> newFiles) throws IOException {
-    if (region.getWAL() == null) return;
-    List<Path> inputPaths = new ArrayList<>(filesCompacted.size());
-    for (StoreFile f : filesCompacted) {
-      inputPaths.add(f.getPath());
-    }
-    List<Path> outputPaths = new ArrayList<>(newFiles.size());
-    for (StoreFile f : newFiles) {
-      outputPaths.add(f.getPath());
+  private void writeCompactionWalRecord(Collection<HStoreFile> filesCompacted,
+      Collection<HStoreFile> newFiles) throws IOException {
+    if (region.getWAL() == null) {
+      return;
     }
+    List<Path> inputPaths =
+        filesCompacted.stream().map(HStoreFile::getPath).collect(Collectors.toList());
+    List<Path> outputPaths =
+        newFiles.stream().map(HStoreFile::getPath).collect(Collectors.toList());
     HRegionInfo info = this.region.getRegionInfo();
     CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info,
         family.getName(), inputPaths, outputPaths, fs.getStoreDir(getColumnFamilyDescriptor().getNameAsString()));
@@ -1437,8 +1486,8 @@ public class HStore implements Store {
   }
 
   @VisibleForTesting
-  void replaceStoreFiles(final Collection<StoreFile> compactedFiles,
-      final Collection<StoreFile> result) throws IOException {
+  void replaceStoreFiles(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> result)
+      throws IOException {
     this.lock.writeLock().lock();
     try {
       this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
@@ -1455,7 +1504,7 @@ public class HStore implements Store {
    * @param compactionStartTime Start time.
    */
   private void logCompactionEndMessage(
-      CompactionRequest cr, List<StoreFile> sfs, long now, long compactionStartTime) {
+      CompactionRequest cr, List<HStoreFile> sfs, long now, long compactionStartTime) {
     StringBuilder message = new StringBuilder(
       "Completed" + (cr.isMajor() ? " major" : "") + " compaction of "
       + cr.getFiles().size() + (cr.isAllFiles() ? " (all)" : "") + " file(s) in "
@@ -1463,7 +1512,7 @@ public class HStore implements Store {
     if (sfs.isEmpty()) {
       message.append("none, ");
     } else {
-      for (StoreFile sf: sfs) {
+      for (HStoreFile sf: sfs) {
         message.append(sf.getPath().getName());
         message.append("(size=");
         message.append(TraditionalBinaryPrefix.long2String(sf.getReader().length(), "", 1));
@@ -1479,10 +1528,7 @@ public class HStore implements Store {
     LOG.info(message.toString());
     if (LOG.isTraceEnabled()) {
       int fileCount = storeEngine.getStoreFileManager().getStorefileCount();
-      long resultSize = 0;
-      for (StoreFile sf : sfs) {
-        resultSize += sf.getReader().length();
-      }
+      long resultSize = getTotalSize(sfs);
       String traceMessage = "COMPACTION start,end,size out,files in,files out,store size,"
         + "store files [" + compactionStartTime + "," + now + "," + resultSize + ","
           + cr.getFiles().size() + "," + sfs.size() + "," +  storeSize + "," + fileCount + "]";
@@ -1496,9 +1542,8 @@ public class HStore implements Store {
    * See HBASE-2231.
    * @param compaction
    */
-  public void replayCompactionMarker(CompactionDescriptor compaction,
-      boolean pickCompactionFiles, boolean removeFiles)
-      throws IOException {
+  public void replayCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
+      boolean removeFiles) throws IOException {
     LOG.debug("Completing compaction from the WAL marker");
     List<String> compactionInputs = compaction.getCompactionInputList();
     List<String> compactionOutputs = Lists.newArrayList(compaction.getCompactionOutputList());
@@ -1525,23 +1570,23 @@ public class HStore implements Store {
     }
 
     //some of the input files might already be deleted
-    List<StoreFile> inputStoreFiles = new ArrayList<>(compactionInputs.size());
-    for (StoreFile sf : this.getStorefiles()) {
+    List<HStoreFile> inputStoreFiles = new ArrayList<>(compactionInputs.size());
+    for (HStoreFile sf : this.getStorefiles()) {
       if (inputFiles.contains(sf.getPath().getName())) {
         inputStoreFiles.add(sf);
       }
     }
 
     // check whether we need to pick up the new files
-    List<StoreFile> outputStoreFiles = new ArrayList<>(compactionOutputs.size());
+    List<HStoreFile> outputStoreFiles = new ArrayList<>(compactionOutputs.size());
 
     if (pickCompactionFiles) {
-      for (StoreFile sf : this.getStorefiles()) {
+      for (HStoreFile sf : this.getStorefiles()) {
         compactionOutputs.remove(sf.getPath().getName());
       }
       for (String compactionOutput : compactionOutputs) {
         StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), compactionOutput);
-        StoreFile storeFile = createStoreFileAndReader(storeFileInfo);
+        HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
         outputStoreFiles.add(storeFile);
       }
     }
@@ -1561,8 +1606,9 @@ public class HStore implements Store {
    * but instead makes a compaction candidate list by itself.
    * @param N Number of files.
    */
+  @VisibleForTesting
   public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
-    List<StoreFile> filesToCompact;
+    List<HStoreFile> filesToCompact;
     boolean isMajor;
 
     this.lock.readLock().lock();
@@ -1572,7 +1618,7 @@ public class HStore implements Store {
         if (!filesCompacting.isEmpty()) {
           // exclude all files older than the newest file we're currently
           // compacting. this allows us to preserve contiguity (HBASE-2856)
-          StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
+          HStoreFile last = filesCompacting.get(filesCompacting.size() - 1);
           int idx = filesToCompact.indexOf(last);
           Preconditions.checkArgument(idx != -1);
           filesToCompact.subList(0, idx + 1).clear();
@@ -1598,11 +1644,11 @@ public class HStore implements Store {
           .compactForTesting(filesToCompact, isMajor);
       for (Path newFile: newFiles) {
         // Move the compaction into place.
-        StoreFile sf = moveFileIntoPlace(newFile);
+        HStoreFile sf = moveFileIntoPlace(newFile);
         if (this.getCoprocessorHost() != null) {
           this.getCoprocessorHost().postCompact(this, sf, null, null);
         }
-        replaceStoreFiles(filesToCompact, Lists.newArrayList(sf));
+        replaceStoreFiles(filesToCompact, Collections.singletonList(sf));
         completeCompaction(filesToCompact);
       }
     } finally {
@@ -1624,7 +1670,7 @@ public class HStore implements Store {
 
   @Override
   public boolean isMajorCompaction() throws IOException {
-    for (StoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) {
+    for (HStoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) {
       // TODO: what are these reader checks all over the place?
       if (sf.getReader() == null) {
         LOG.debug("StoreFile " + sf + " has null Reader");
@@ -1652,7 +1698,7 @@ public class HStore implements Store {
       synchronized (filesCompacting) {
         // First, see if coprocessor would want to override selection.
         if (this.getCoprocessorHost() != null) {
-          final List<StoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
+          final List<HStoreFile> candidatesForCoproc = compaction.preSelect(this.filesCompacting);
           boolean override = false;
           override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc,
             tracker, user);
@@ -1688,7 +1734,7 @@ public class HStore implements Store {
         }
         // Finally, we have the resulting files list. Check if we have any files at all.
         request = compaction.getRequest();
-        Collection<StoreFile> selectedFiles = request.getFiles();
+        Collection<HStoreFile> selectedFiles = request.getFiles();
         if (selectedFiles.isEmpty()) {
           return Optional.empty();
         }
@@ -1716,7 +1762,7 @@ public class HStore implements Store {
   }
 
   /** Adds the files to compacting files. filesCompacting must be locked. */
-  private void addToCompactingFiles(final Collection<StoreFile> filesToAdd) {
+  private void addToCompactingFiles(Collection<HStoreFile> filesToAdd) {
     if (filesToAdd == null) return;
     // Check that we do not try to compact the same StoreFile twice.
     if (!Collections.disjoint(filesCompacting, filesToAdd)) {
@@ -1734,7 +1780,7 @@ public class HStore implements Store {
       return;
     }
     this.lock.readLock().lock();
-    Collection<StoreFile> delSfs = null;
+    Collection<HStoreFile> delSfs = null;
     try {
       synchronized (filesCompacting) {
         long cfTtl = getStoreFileTtl();
@@ -1749,7 +1795,7 @@ public class HStore implements Store {
     }
     if (delSfs == null || delSfs.isEmpty()) return;
 
-    Collection<StoreFile> newFiles = new ArrayList<>(); // No new files.
+    Collection<HStoreFile> newFiles = new ArrayList<>(); // No new files.
     writeCompactionWalRecord(delSfs, newFiles);
     replaceStoreFiles(delSfs, newFiles);
     completeCompaction(delSfs);
@@ -1775,23 +1821,20 @@ public class HStore implements Store {
   }
 
   /**
-   * Validates a store file by opening and closing it. In HFileV2 this should
-   * not be an expensive operation.
-   *
+   * Validates a store file by opening and closing it. In HFileV2 this should not be an expensive
+   * operation.
    * @param path the path to the store file
    */
-  private void validateStoreFile(Path path)
-      throws IOException {
-    StoreFile storeFile = null;
+  private void validateStoreFile(Path path) throws IOException {
+    HStoreFile storeFile = null;
     try {
       storeFile = createStoreFileAndReader(path);
     } catch (IOException e) {
-      LOG.error("Failed to open store file : " + path
-          + ", keeping it in tmp location", e);
+      LOG.error("Failed to open store file : " + path + ", keeping it in tmp location", e);
       throw e;
     } finally {
       if (storeFile != null) {
-        storeFile.closeReader(false);
+        storeFile.closeStoreFile(false);
       }
     }
   }
@@ -1811,11 +1854,11 @@ public class HStore implements Store {
    * @param compactedFiles list of files that were compacted
    */
   @VisibleForTesting
-  protected void completeCompaction(final Collection<StoreFile> compactedFiles)
+  protected void completeCompaction(Collection<HStoreFile> compactedFiles)
     throws IOException {
     this.storeSize = 0L;
     this.totalUncompressedBytes = 0L;
-    for (StoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
+    for (HStoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
       StoreFileReader r = hsf.getReader();
       if (r == null) {
         LOG.warn("StoreFile " + hsf + " has a null Reader");
@@ -1857,7 +1900,7 @@ public class HStore implements Store {
   }
 
   @Override
-  public byte[] getSplitPoint() {
+  public Optional<byte[]> getSplitPoint() {
     this.lock.readLock().lock();
     try {
       // Should already be enforced by the split policy!
@@ -1867,7 +1910,7 @@ public class HStore implements Store {
         if (LOG.isTraceEnabled()) {
           LOG.trace("Not splittable; has references: " + this);
         }
-        return null;
+        return Optional.empty();
       }
       return this.storeEngine.getStoreFileManager().getSplitPoint();
     } catch(IOException e) {
@@ -1875,7 +1918,7 @@ public class HStore implements Store {
     } finally {
       this.lock.readLock().unlock();
     }
-    return null;
+    return Optional.empty();
   }
 
   @Override
@@ -1924,24 +1967,39 @@ public class HStore implements Store {
     return scanner;
   }
 
-  @Override
+  /**
+   * Recreates the scanners on the current list of active store file scanners
+   * @param currentFileScanners the current set of active store file scanners
+   * @param cacheBlocks cache the blocks or not
+   * @param usePread use pread or not
+   * @param isCompaction is the scanner for compaction
+   * @param matcher the scan query matcher
+   * @param startRow the scan's start row
+   * @param includeStartRow should the scan include the start row
+   * @param stopRow the scan's stop row
+   * @param includeStopRow should the scan include the stop row
+   * @param readPt the read point of the current scane
+   * @param includeMemstoreScanner whether the current scanner should include memstorescanner
+   * @return list of scanners recreated on the current Scanners
+   * @throws IOException
+   */
   public List<KeyValueScanner> recreateScanners(List<KeyValueScanner> currentFileScanners,
       boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
       byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
       boolean includeMemstoreScanner) throws IOException {
     this.lock.readLock().lock();
     try {
-      Map<String, StoreFile> name2File =
+      Map<String, HStoreFile> name2File =
           new HashMap<>(getStorefilesCount() + getCompactedFilesCount());
-      for (StoreFile file : getStorefiles()) {
+      for (HStoreFile file : getStorefiles()) {
         name2File.put(file.getFileInfo().getActiveFileName(), file);
       }
       if (getCompactedFiles() != null) {
-        for (StoreFile file : getCompactedFiles()) {
+        for (HStoreFile file : getCompactedFiles()) {
           name2File.put(file.getFileInfo().getActiveFileName(), file);
         }
       }
-      List<StoreFile> filesToReopen = new ArrayList<>();
+      List<HStoreFile> filesToReopen = new ArrayList<>();
       for (KeyValueScanner kvs : currentFileScanners) {
         assert kvs.isFileScanner();
         if (kvs.peek() == null) {
@@ -1974,87 +2032,45 @@ public class HStore implements Store {
     return this.storeEngine.getStoreFileManager().getCompactedFilesCount();
   }
 
+  private LongStream getStoreFileCreatedTimestampStream() {
+    return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> {
+      if (sf.getReader() == null) {
+        LOG.warn("StoreFile " + sf + " has a null Reader");
+        return false;
+      } else {
+        return true;
+      }
+    }).filter(HStoreFile::isHFile).mapToLong(sf -> sf.getFileInfo().getCreatedTimestamp());
+  }
+
   @Override
   public long getMaxStoreFileAge() {
-    long earliestTS = Long.MAX_VALUE;
-    for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
-      StoreFileReader r = s.getReader();
-      if (r == null) {
-        LOG.warn("StoreFile " + s + " has a null Reader");
-        continue;
-      }
-      if (!s.isHFile()) {
-        continue;
-      }
-      long createdTS = s.getFileInfo().getCreatedTimestamp();
-      earliestTS = (createdTS < earliestTS) ? createdTS : earliestTS;
-    }
-    long now = EnvironmentEdgeManager.currentTime();
-    return now - earliestTS;
+    return EnvironmentEdgeManager.currentTime() -
+        getStoreFileCreatedTimestampStream().min().orElse(Long.MAX_VALUE);
   }
 
   @Override
   public long getMinStoreFileAge() {
-    long latestTS = 0;
-    for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
-      StoreFileReader r = s.getReader();
-      if (r == null) {
-        LOG.warn("StoreFile " + s + " has a null Reader");
-        continue;
-      }
-      if (!s.isHFile()) {
-        continue;
-      }
-      long createdTS = s.getFileInfo().getCreatedTimestamp();
-      latestTS = (createdTS > latestTS) ? createdTS : latestTS;
-    }
-    long now = EnvironmentEdgeManager.currentTime();
-    return now - latestTS;
+    return EnvironmentEdgeManager.currentTime() -
+        getStoreFileCreatedTimestampStream().max().orElse(0L);
   }
 
   @Override
   public long getAvgStoreFileAge() {
-    long sum = 0, count = 0;
-    for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
-      StoreFileReader r = s.getReader();
-      if (r == null) {
-        LOG.warn("StoreFile " + s + " has a null Reader");
-        continue;
-      }
-      if (!s.isHFile()) {
-        continue;
-      }
-      sum += s.getFileInfo().getCreatedTimestamp();
-      count++;
-    }
-    if (count == 0) {
-      return 0;
-    }
-    long avgTS = sum / count;
-    long now = EnvironmentEdgeManager.currentTime();
-    return now - avgTS;
+    OptionalDouble avg = getStoreFileCreatedTimestampStream().average();
+    return avg.isPresent() ? EnvironmentEdgeManager.currentTime() - (long) avg.getAsDouble() : 0L;
   }
 
   @Override
   public long getNumReferenceFiles() {
-    long numRefFiles = 0;
-    for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
-      if (s.isReference()) {
-        numRefFiles++;
-      }
-    }
-    return numRefFiles;
+    return this.storeEngine.getStoreFileManager().getStorefiles().stream()
+        .filter(HStoreFile::isReference).count();
   }
 
   @Override
   public long getNumHFiles() {
-    long numHFiles = 0;
-    for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
-      if (s.isHFile()) {
-        numHFiles++;
-      }
-    }
-    return numHFiles;
+    return this.storeEngine.getStoreFileManager().getStorefiles().stream()
+        .filter(HStoreFile::isHFile).count();
   }
 
   @Override
@@ -2074,59 +2090,41 @@ public class HStore implements Store {
     return getStorefilesSize(storeFile -> storeFile.isHFile());
   }
 
-  private long getStorefilesSize(Predicate<StoreFile> predicate) {
-    long size = 0;
-    for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
-      StoreFileReader r = s.getReader();
-      if (r == null) {
-        LOG.warn("StoreFile " + s + " has a null Reader");
-        continue;
+  private long getStorefilesSize(Predicate<HStoreFile> predicate) {
+    return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> {
+      if (sf.getReader() == null) {
+        LOG.warn("StoreFile " + sf + " has a null Reader");
+        return false;
+      } else {
+        return true;
       }
-      if (predicate.test(s)) {
-        size += r.length();
+    }).filter(predicate).mapToLong(sf -> sf.getReader().length()).sum();
+  }
+
+  private long getStoreFileFieldSize(ToLongFunction<StoreFileReader> f) {
+    return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> {
+      if (sf.getReader() == null) {
+        LOG.warn("StoreFile " + sf + " has a null Reader");
+        return false;
+      } else {
+        return true;
       }
-    }
-    return size;
+    }).map(HStoreFile::getReader).mapToLong(f).sum();
   }
 
   @Override
   public long getStorefilesIndexSize() {
-    long size = 0;
-    for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) {
-      StoreFileReader r = s.getReader();
-      if (r == null) {
-        LOG.warn("StoreFile " + s + " has a null Reader");
-        continue;
-      }
-      size += r.indexSize();
-    }
-    return size;
+    return getStoreFileFieldSize(StoreFileReader::indexSize);
   }
 
   @Override
   public long getTotalStaticIndexSize() {
-    long size = 0;
-    for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
-      StoreFileReader r = s.getReader();
-      if (r == null) {
-        continue;
-      }
-      size += r.getUncompressedDataIndexSize();
-    }
-    return size;
+    return getStoreFileFieldSize(StoreFileReader::getUncompressedDataIndexSize);
   }
 
   @Override
   public long getTotalStaticBloomSize() {
-    long size = 0;
-    for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) {
-      StoreFileReader r = s.getReader();
-      if (r == null) {
-        continue;
-      }
-      size += r.getTotalBloomSize();
-    }
-    return size;
+    return getStoreFileFieldSize(StoreFileReader::getTotalBloomSize);
   }
 
   @Override
@@ -2247,19 +2245,19 @@ public class HStore implements Store {
       if (this.tempFiles == null || this.tempFiles.isEmpty()) {
         return false;
       }
-      List<StoreFile> storeFiles = new ArrayList<>(this.tempFiles.size());
+      List<HStoreFile> storeFiles = new ArrayList<>(this.tempFiles.size());
       for (Path storeFilePath : tempFiles) {
         try {
-          StoreFile sf = HStore.this.commitFile(storeFilePath, cacheFlushSeqNum, status);
+          HStoreFile sf = HStore.this.commitFile(storeFilePath, cacheFlushSeqNum, status);
           outputFileSize += sf.getReader().length();
           storeFiles.add(sf);
         } catch (IOException ex) {
           LOG.error("Failed to commit store file " + storeFilePath, ex);
           // Try to delete the files we have committed before.
-          for (StoreFile sf : storeFiles) {
+          for (HStoreFile sf : storeFiles) {
             Path pathToDelete = sf.getPath();
             try {
-              sf.deleteReader();
+              sf.deleteStoreFile();
             } catch (IOException deleteEx) {
               LOG.fatal("Failed to delete store file we committed, halting " + pathToDelete, ex);
               Runtime.getRuntime().halt(1);
@@ -2269,7 +2267,7 @@ public class HStore implements Store {
         }
       }
 
-      for (StoreFile sf : storeFiles) {
+      for (HStoreFile sf : storeFiles) {
         if (HStore.this.getCoprocessorHost() != null) {
           HStore.this.getCoprocessorHost().postFlush(HStore.this, sf);
         }
@@ -2305,11 +2303,11 @@ public class HStore implements Store {
     @Override
     public void replayFlush(List<String> fileNames, boolean dropMemstoreSnapshot)
         throws IOException {
-      List<StoreFile> storeFiles = new ArrayList<>(fileNames.size());
+      List<HStoreFile> storeFiles = new ArrayList<>(fileNames.size());
       for (String file : fileNames) {
         // open the file as a store file (hfile link, etc)
         StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), file);
-        StoreFile storeFile = createStoreFileAndReader(storeFileInfo);
+        HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
         storeFiles.add(storeFile);
         HStore.this.storeSize += storeFile.getReader().length();
         HStore.this.totalUncompressedBytes += storeFile.getReader().getTotalUncompressedBytes();
@@ -2498,9 +2496,9 @@ public class HStore implements Store {
     archiveLock.lock();
     try {
       lock.readLock().lock();
-      Collection<StoreFile> copyCompactedfiles = null;
+      Collection<HStoreFile> copyCompactedfiles = null;
       try {
-        Collection<StoreFile> compactedfiles =
+        Collection<HStoreFile> compactedfiles =
             this.getStoreEngine().getStoreFileManager().getCompactedfiles();
         if (compactedfiles != null && compactedfiles.size() != 0) {
           // Do a copy under read lock
@@ -2527,10 +2525,10 @@ public class HStore implements Store {
    * @param compactedfiles The compacted files in this store that are not active in reads
    * @throws IOException
    */
-  private void removeCompactedfiles(Collection<StoreFile> compactedfiles)
+  private void removeCompactedfiles(Collection<HStoreFile> compactedfiles)
       throws IOException {
-    final List<StoreFile> filesToRemove = new ArrayList<>(compactedfiles.size());
-    for (final StoreFile file : compactedfiles) {
+    final List<HStoreFile> filesToRemove = new ArrayList<>(compactedfiles.size());
+    for (final HStoreFile file : compactedfiles) {
       synchronized (file) {
         try {
           StoreFileReader r = file.getReader();
@@ -2573,7 +2571,7 @@ public class HStore implements Store {
           // files which were successfully archived.  Otherwise we will receive a
           // FileNotFoundException when we attempt to re-archive them in the next go around.
           Collection<Path> failedFiles = fae.getFailedFiles();
-          Iterator<StoreFile> iter = filesToRemove.iterator();
+          Iterator<HStoreFile> iter = filesToRemove.iterator();
           while (iter.hasNext()) {
             if (failedFiles.contains(iter.next().getPath())) {
               iter.remove();
@@ -2601,7 +2599,7 @@ public class HStore implements Store {
     return this.memstore.isSloppy();
   }
 
-  private void clearCompactedfiles(final List<StoreFile> filesToRemove) throws IOException {
+  private void clearCompactedfiles(List<HStoreFile> filesToRemove) throws IOException {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Clearing the compacted file " + filesToRemove + " from this store");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
----------------------------------------------------------------------
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 c43b788..a79af13 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
@@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.Map;
+import java.util.Optional;
 import java.util.OptionalLong;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -32,14 +32,15 @@ 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.CellComparator;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.io.hfile.BlockType;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
@@ -61,8 +62,50 @@ public class HStoreFile implements StoreFile {
 
   private static final Log LOG = LogFactory.getLog(HStoreFile.class.getName());
 
+  public static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
+
   private static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false;
 
+  // Keys for fileinfo values in HFile
+
+  /** Max Sequence ID in FileInfo */
+  public static final byte[] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
+
+  /** Major compaction flag in FileInfo */
+  public static final byte[] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
+
+  /** Minor compaction flag in FileInfo */
+  public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
+      Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
+
+  /** Bloom filter Type in FileInfo */
+  public static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
+
+  /** Delete Family Count in FileInfo */
+  public static final byte[] DELETE_FAMILY_COUNT = Bytes.toBytes("DELETE_FAMILY_COUNT");
+
+  /** Last Bloom filter key in FileInfo */
+  public static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
+
+  /** Key for Timerange information in metadata */
+  public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
+
+  /** Key for timestamp of earliest-put in metadata */
+  public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
+
+  /** Key for the number of mob cells in metadata */
+  public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
+
+  /** 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");
+
+  /**
+   * Key for skipping resetting sequence id in metadata. For bulk loaded hfiles, the scanner resets
+   * the cell seqId with the latest one, if this metadata is set as true, the reset is skipped.
+   */
+  public static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
+
   private final StoreFileInfo fileInfo;
   private final FileSystem fs;
 
@@ -90,29 +133,28 @@ public class HStoreFile implements StoreFile {
   private long maxMemstoreTS = -1;
 
   // firstKey, lastkey and cellComparator will be set when openReader.
-  private Cell firstKey;
+  private Optional<Cell> firstKey;
 
-  private Cell lastKey;
+  private Optional<Cell> lastKey;
 
-  private Comparator<Cell> comparator;
+  private CellComparator comparator;
 
-  @Override
   public CacheConfig getCacheConf() {
     return cacheConf;
   }
 
   @Override
-  public Cell getFirstKey() {
+  public Optional<Cell> getFirstKey() {
     return firstKey;
   }
 
   @Override
-  public Cell getLastKey() {
+  public Optional<Cell> getLastKey() {
     return lastKey;
   }
 
   @Override
-  public Comparator<Cell> getComparator() {
+  public CellComparator getComparator() {
     return comparator;
   }
 
@@ -155,27 +197,6 @@ public class HStoreFile implements StoreFile {
    *          configuration. This may or may not be the same as the Bloom filter type actually
    *          present in the HFile, because column family configuration might change. If this is
    *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
-   * @deprecated Now we will specific whether the StoreFile is for primary replica when
-   *             constructing, so please use {@link #HStoreFile(FileSystem, Path, Configuration,
-   *             CacheConfig, BloomType, boolean)} directly.
-   */
-  @Deprecated
-  public HStoreFile(final FileSystem fs, final Path p, final Configuration conf,
-      final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
-    this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType);
-  }
-
-  /**
-   * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
-   * depending on the underlying files (10-20MB?).
-   * @param fs The current file system to use.
-   * @param p The path of the file.
-   * @param conf The current configuration.
-   * @param cacheConf The cache configuration and block cache reference.
-   * @param cfBloomType The bloom type to use for this store file as specified by column family
-   *          configuration. This may or may not be the same as the Bloom filter type actually
-   *          present in the HFile, because column family configuration might change. If this is
-   *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
    * @param primaryReplica true if this is a store file for primary replica, otherwise false.
    * @throws IOException
    */
@@ -187,27 +208,6 @@ public class HStoreFile implements StoreFile {
   /**
    * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
    * depending on the underlying files (10-20MB?).
-   * @param fs The current file system to use.
-   * @param fileInfo The store file information.
-   * @param conf The current configuration.
-   * @param cacheConf The cache configuration and block cache reference.
-   * @param cfBloomType The bloom type to use for this store file as specified by column family
-   *          configuration. This may or may not be the same as the Bloom filter type actually
-   *          present in the HFile, because column family configuration might change. If this is
-   *          {@link BloomType#NONE}, the existing Bloom filter is ignored.
-   * @deprecated Now we will specific whether the StoreFile is for primary replica when
-   *             constructing, so please use {@link #HStoreFile(FileSystem, StoreFileInfo,
-   *             Configuration, CacheConfig, BloomType, boolean)} directly.
-   */
-  @Deprecated
-  public HStoreFile(final FileSystem fs, final StoreFileInfo fileInfo, final Configuration conf,
-      final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
-    this(fs, fileInfo, conf, cacheConf, cfBloomType, true);
-  }
-
-  /**
-   * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
-   * depending on the underlying files (10-20MB?).
    * @param fs fs The current file system to use.
    * @param fileInfo The store file information.
    * @param conf The current configuration.
@@ -235,7 +235,10 @@ public class HStoreFile implements StoreFile {
     this.primaryReplica = primaryReplica;
   }
 
-  @Override
+  /**
+   * @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a
+   *         reference.
+   */
   public StoreFileInfo getFileInfo() {
     return this.fileInfo;
   }
@@ -283,7 +286,11 @@ public class HStoreFile implements StoreFile {
     return fileInfo.getModificationTime();
   }
 
-  @Override
+  /**
+   * Only used by the Striped Compaction Policy
+   * @param key
+   * @return value associated with the metadata key
+   */
   public byte[] getMetadataValue(byte[] key) {
     return metadataMap.get(key);
   }
@@ -299,7 +306,6 @@ public class HStoreFile implements StoreFile {
     return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY));
   }
 
-  @Override
   public boolean isCompactedAway() {
     return compactedAway;
   }
@@ -309,7 +315,9 @@ public class HStoreFile implements StoreFile {
     return refCount.get();
   }
 
-  @Override
+  /**
+   * @return true if the file is still used in reads
+   */
   public boolean isReferencedInReads() {
     int rc = refCount.get();
     assert rc >= 0; // we should not go negative.
@@ -331,7 +339,7 @@ public class HStoreFile implements StoreFile {
   /**
    * Opens reader on this store file. Called by Constructor.
    * @throws IOException
-   * @see #closeReader(boolean)
+   * @see #closeStoreFile(boolean)
    */
   private void open() throws IOException {
     if (this.reader != null) {
@@ -440,7 +448,9 @@ public class HStoreFile implements StoreFile {
     comparator = reader.getComparator();
   }
 
-  @Override
+  /**
+   * Initialize the reader used for pread.
+   */
   public void initReader() throws IOException {
     if (reader == null) {
       try {
@@ -448,7 +458,7 @@ public class HStoreFile implements StoreFile {
       } catch (Exception e) {
         try {
           boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true;
-          this.closeReader(evictOnClose);
+          this.closeStoreFile(evictOnClose);
         } catch (IOException ee) {
           LOG.warn("failed to close reader", ee);
         }
@@ -465,14 +475,22 @@ public class HStoreFile implements StoreFile {
     return reader;
   }
 
-  @Override
+  /**
+   * Get a scanner which uses pread.
+   * <p>
+   * Must be called after initReader.
+   */
   public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
       boolean canOptimizeForNonNullColumn) {
     return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder,
       canOptimizeForNonNullColumn);
   }
 
-  @Override
+  /**
+   * Get a scanner which uses streaming read.
+   * <p>
+   * Must be called after initReader.
+   */
   public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
       boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
       throws IOException {
@@ -480,31 +498,37 @@ public class HStoreFile implements StoreFile {
       isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn);
   }
 
-  @Override
+  /**
+   * @return Current reader. Must call initReader first else returns null.
+   * @see #initReader()
+   */
   public StoreFileReader getReader() {
     return this.reader;
   }
 
-  @Override
-  public synchronized void closeReader(boolean evictOnClose)
-      throws IOException {
+  /**
+   * @param evictOnClose whether to evict blocks belonging to this file
+   * @throws IOException
+   */
+  public synchronized void closeStoreFile(boolean evictOnClose) throws IOException {
     if (this.reader != null) {
       this.reader.close(evictOnClose);
       this.reader = null;
     }
   }
 
-  @Override
-  public void markCompactedAway() {
-    this.compactedAway = true;
+  /**
+   * Delete this file
+   * @throws IOException
+   */
+  public void deleteStoreFile() throws IOException {
+    boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true;
+    closeStoreFile(evictOnClose);
+    this.fs.delete(getPath(), true);
   }
 
-  @Override
-  public void deleteReader() throws IOException {
-    boolean evictOnClose =
-        cacheConf != null? cacheConf.shouldEvictOnClose(): true;
-    closeReader(evictOnClose);
-    this.fs.delete(getPath(), true);
+  public void markCompactedAway() {
+    this.compactedAway = true;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
index 4528517..8af33b6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
@@ -246,30 +246,27 @@ public class MemStoreCompactor {
     MemStoreSegmentsIterator iterator = null;
 
     switch (action) {
-    case COMPACT:
-      iterator =
-          new MemStoreCompactorSegmentsIterator(versionedList.getStoreSegments(),
-              compactingMemStore.getComparator(),
-              compactionKVMax, compactingMemStore.getStore());
+      case COMPACT:
+        iterator = new MemStoreCompactorSegmentsIterator(versionedList.getStoreSegments(),
+            compactingMemStore.getComparator(), compactionKVMax, compactingMemStore.getStore());
 
-      result = SegmentFactory.instance().createImmutableSegmentByCompaction(
+        result = SegmentFactory.instance().createImmutableSegmentByCompaction(
           compactingMemStore.getConfiguration(), compactingMemStore.getComparator(), iterator,
           versionedList.getNumOfCells(), compactingMemStore.getIndexType());
-      iterator.close();
-      break;
-    case MERGE:
-      iterator =
-          new MemStoreMergerSegmentsIterator(versionedList.getStoreSegments(),
-              compactingMemStore.getComparator(),
-              compactionKVMax);
+        iterator.close();
+        break;
+      case MERGE:
+        iterator = new MemStoreMergerSegmentsIterator(versionedList.getStoreSegments(),
+            compactingMemStore.getComparator(), compactionKVMax);
 
-      result = SegmentFactory.instance().createImmutableSegmentByMerge(
+        result = SegmentFactory.instance().createImmutableSegmentByMerge(
           compactingMemStore.getConfiguration(), compactingMemStore.getComparator(), iterator,
           versionedList.getNumOfCells(), versionedList.getStoreSegments(),
           compactingMemStore.getIndexType());
-      iterator.close();
-      break;
-    default: throw new RuntimeException("Unknown action " + action); // sanity check
+        iterator.close();
+        break;
+      default:
+        throw new RuntimeException("Unknown action " + action); // sanity check
     }
 
     return result;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactorSegmentsIterator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactorSegmentsIterator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactorSegmentsIterator.java
index 3d88955..b3ba998 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactorSegmentsIterator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactorSegmentsIterator.java
@@ -45,10 +45,8 @@ public class MemStoreCompactorSegmentsIterator extends MemStoreSegmentsIterator
   private StoreScanner compactingScanner;
 
   // C-tor
-  public MemStoreCompactorSegmentsIterator(
-      List<ImmutableSegment> segments,
-      CellComparator comparator, int compactionKVMax, Store store
-  ) throws IOException {
+  public MemStoreCompactorSegmentsIterator(List<ImmutableSegment> segments,
+      CellComparator comparator, int compactionKVMax, HStore store) throws IOException {
     super(compactionKVMax);
 
     List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
@@ -108,7 +106,7 @@ public class MemStoreCompactorSegmentsIterator extends MemStoreSegmentsIterator
    * Creates the scanner for compacting the pipeline.
    * @return the scanner
    */
-  private StoreScanner createScanner(Store store, List<KeyValueScanner> scanners)
+  private StoreScanner createScanner(HStore store, List<KeyValueScanner> scanners)
       throws IOException {
     // Get all available versions
     return new StoreScanner(store, store.getScanInfo(), OptionalInt.of(Integer.MAX_VALUE), scanners,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java
index 9bdeedc..b9f9af8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java
@@ -23,9 +23,9 @@ import java.util.List;
 import java.util.NavigableSet;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Scanner scans both the memstore and the MOB Store. Coalesce KeyValue stream into
@@ -39,7 +39,7 @@ public class MobStoreScanner extends StoreScanner {
   private boolean readEmptyValueOnMobCellMiss = false;
   private final HMobStore mobStore;
 
-  public MobStoreScanner(Store store, ScanInfo scanInfo, Scan scan,
+  public MobStoreScanner(HStore store, ScanInfo scanInfo, Scan scan,
       final NavigableSet<byte[]> columns, long readPt) throws IOException {
     super(store, scanInfo, scan, columns, readPt);
     cacheMobBlocks = MobUtils.isCacheMobBlocks(scan);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
index 044c4dc..fe0f30e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java
@@ -499,7 +499,7 @@ public class RegionCoprocessorHost
    * {@link RegionObserver#preCompactScannerOpen(ObserverContext, Store, List, ScanType, long,
    *   InternalScanner, CompactionLifeCycleTracker, long)}
    */
-  public InternalScanner preCompactScannerOpen(Store store, List<StoreFileScanner> scanners,
+  public InternalScanner preCompactScannerOpen(HStore store, List<StoreFileScanner> scanners,
       ScanType scanType, long earliestPutTs, CompactionLifeCycleTracker tracker, User user,
       long readPoint) throws IOException {
     return execOperationWithResult(null,
@@ -514,7 +514,7 @@ public class RegionCoprocessorHost
   }
 
   /**
-   * Called prior to selecting the {@link StoreFile}s for compaction from the list of currently
+   * Called prior to selecting the {@link HStoreFile}s for compaction from the list of currently
    * available candidates.
    * @param store The store where compaction is being requested
    * @param candidates The currently available store files
@@ -522,7 +522,7 @@ public class RegionCoprocessorHost
    * @return If {@code true}, skip the normal selection process and use the current list
    * @throws IOException
    */
-  public boolean preCompactSelection(Store store, List<StoreFile> candidates,
+  public boolean preCompactSelection(HStore store, List<HStoreFile> candidates,
       CompactionLifeCycleTracker tracker, User user) throws IOException {
     return execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
       @Override
@@ -534,13 +534,13 @@ public class RegionCoprocessorHost
   }
 
   /**
-   * Called after the {@link StoreFile}s to be compacted have been selected from the available
+   * Called after the {@link HStoreFile}s to be compacted have been selected from the available
    * candidates.
    * @param store The store where compaction is being requested
    * @param selected The store files selected to compact
    * @param tracker used to track the life cycle of a compaction
    */
-  public void postCompactSelection(Store store, ImmutableList<StoreFile> selected,
+  public void postCompactSelection(HStore store, ImmutableList<HStoreFile> selected,
       CompactionLifeCycleTracker tracker, User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
       @Override
@@ -559,7 +559,7 @@ public class RegionCoprocessorHost
    * @param tracker used to track the life cycle of a compaction
    * @throws IOException
    */
-  public InternalScanner preCompact(Store store, InternalScanner scanner, ScanType scanType,
+  public InternalScanner preCompact(HStore store, InternalScanner scanner, ScanType scanType,
       CompactionLifeCycleTracker tracker, User user) throws IOException {
     return execOperationWithResult(false, scanner,
         coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>(user) {
@@ -578,7 +578,7 @@ public class RegionCoprocessorHost
    * @param tracker used to track the life cycle of a compaction
    * @throws IOException
    */
-  public void postCompact(Store store, StoreFile resultFile, CompactionLifeCycleTracker tracker,
+  public void postCompact(HStore store, HStoreFile resultFile, CompactionLifeCycleTracker tracker,
       User user) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
       @Override
@@ -593,7 +593,7 @@ public class RegionCoprocessorHost
    * Invoked before a memstore flush
    * @throws IOException
    */
-  public InternalScanner preFlush(final Store store, final InternalScanner scanner)
+  public InternalScanner preFlush(HStore store, final InternalScanner scanner)
       throws IOException {
     return execOperationWithResult(false, scanner,
         coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>() {
@@ -623,16 +623,16 @@ public class RegionCoprocessorHost
    * See
    * {@link RegionObserver#preFlushScannerOpen(ObserverContext, Store, List, InternalScanner, long)}
    */
-  public InternalScanner preFlushScannerOpen(final Store store,
-      final List<KeyValueScanner> scanners, final long readPoint) throws IOException {
+  public InternalScanner preFlushScannerOpen(HStore store, List<KeyValueScanner> scanners,
+      long readPoint) throws IOException {
     return execOperationWithResult(null,
-        coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>() {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        setResult(oserver.preFlushScannerOpen(ctx, store, scanners, getResult(), readPoint));
-      }
-    });
+      coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>() {
+        @Override
+        public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
+            throws IOException {
+          setResult(oserver.preFlushScannerOpen(ctx, store, scanners, getResult(), readPoint));
+        }
+      });
   }
 
   /**
@@ -653,7 +653,7 @@ public class RegionCoprocessorHost
    * Invoked after a memstore flush
    * @throws IOException
    */
-  public void postFlush(final Store store, final StoreFile storeFile) throws IOException {
+  public void postFlush(HStore store, HStoreFile storeFile) throws IOException {
     execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
       @Override
       public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
@@ -1136,16 +1136,16 @@ public class RegionCoprocessorHost
    * See
    * {@link RegionObserver#preStoreScannerOpen(ObserverContext, Store, Scan, NavigableSet, KeyValueScanner, long)}
    */
-  public KeyValueScanner preStoreScannerOpen(final Store store, final Scan scan,
-      final NavigableSet<byte[]> targetCols, final long readPt) throws IOException {
+  public KeyValueScanner preStoreScannerOpen(HStore store, Scan scan,
+      NavigableSet<byte[]> targetCols, long readPt) throws IOException {
     return execOperationWithResult(null,
-        coprocessors.isEmpty() ? null : new RegionOperationWithResult<KeyValueScanner>() {
-      @Override
-      public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
-          throws IOException {
-        setResult(oserver.preStoreScannerOpen(ctx, store, scan, targetCols, getResult(), readPt));
-      }
-    });
+      coprocessors.isEmpty() ? null : new RegionOperationWithResult<KeyValueScanner>() {
+        @Override
+        public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
+            throws IOException {
+          setResult(oserver.preStoreScannerOpen(ctx, store, scan, targetCols, getResult(), readPt));
+        }
+      });
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
index 5ccd6e3..71b7b9b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
@@ -19,14 +19,15 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Optional;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 
@@ -80,12 +81,12 @@ public abstract class RegionSplitPolicy extends Configured {
 
     byte[] splitPointFromLargestStore = null;
     long largestStoreSize = 0;
-    for (Store s : stores) {
-      byte[] splitPoint = s.getSplitPoint();
+    for (HStore s : stores) {
+      Optional<byte[]> splitPoint = s.getSplitPoint();
       // Store also returns null if it has references as way of indicating it is not splittable
       long storeSize = s.getSize();
-      if (splitPoint != null && largestStoreSize < storeSize) {
-        splitPointFromLargestStore = splitPoint;
+      if (splitPoint.isPresent() && largestStoreSize < storeSize) {
+        splitPointFromLargestStore = splitPoint.get();
         largestStoreSize = storeSize;
       }
     }
@@ -131,7 +132,7 @@ public abstract class RegionSplitPolicy extends Configured {
 
   /**
    * In {@link HRegionFileSystem#splitStoreFile(org.apache.hadoop.hbase.HRegionInfo, String,
-   * StoreFile, byte[], boolean, RegionSplitPolicy)} we are not creating the split reference
+   * HStoreFile, byte[], boolean, RegionSplitPolicy)} we are not creating the split reference
    * if split row not lies in the StoreFile range. But in some use cases we may need to create
    * the split reference even when the split row not lies in the range. This method can be used
    * to decide, whether to skip the the StoreFile range check or not.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedMobStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedMobStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedMobStoreScanner.java
index bfe20ba..d64c372 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedMobStoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedMobStoreScanner.java
@@ -23,9 +23,9 @@ import java.util.List;
 import java.util.NavigableSet;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * ReversedMobStoreScanner extends from ReversedStoreScanner, and is used to support
@@ -40,7 +40,7 @@ public class ReversedMobStoreScanner extends ReversedStoreScanner {
   private boolean readEmptyValueOnMobCellMiss = false;
   protected final HMobStore mobStore;
 
-  ReversedMobStoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
+  ReversedMobStoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
       long readPt) throws IOException {
     super(store, scanInfo, scan, columns, readPt);
     cacheMobBlocks = MobUtils.isCacheMobBlocks(scan);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
index 04e77e9..0089d3f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java
@@ -22,12 +22,12 @@ import java.io.IOException;
 import java.util.List;
 import java.util.NavigableSet;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * ReversedStoreScanner extends from StoreScanner, and is used to support
@@ -46,7 +46,7 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
    * @param columns which columns we are scanning
    * @throws IOException
    */
-  ReversedStoreScanner(Store store, ScanInfo scanInfo, Scan scan,
+  ReversedStoreScanner(HStore store, ScanInfo scanInfo, Scan scan,
       NavigableSet<byte[]> columns, long readPt)
       throws IOException {
     super(store, scanInfo, scan, columns, readPt);


[03/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
index 3ba2299..c2c3171 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java
@@ -82,12 +82,6 @@ import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -109,6 +103,13 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
 
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
+
 /**
  * The below tests are testing split region against a running cluster
  */
@@ -750,8 +751,8 @@ public class TestSplitTransactionOnCluster {
         region.put(p);
       }
       region.flush(true);
-      Store store = region.getStore(Bytes.toBytes("f"));
-      Collection<StoreFile> storefiles = store.getStorefiles();
+      HStore store = region.getStore(Bytes.toBytes("f"));
+      Collection<HStoreFile> storefiles = store.getStorefiles();
       assertEquals(storefiles.size(), 1);
       assertFalse(region.hasReferences());
       Path referencePath =

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
deleted file mode 100644
index 2095dcd..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java
+++ /dev/null
@@ -1,1740 +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.regionserver;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
-import java.io.IOException;
-import java.lang.ref.SoftReference;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.NavigableSet;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentSkipListSet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FilterFileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellComparator;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.MemoryCompactionPolicy;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
-import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterBase;
-import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-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.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
-import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
-import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
-import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.util.Progressable;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-import org.mockito.Mockito;
-
-/**
- * Test class for the Store
- */
-@Category({RegionServerTests.class, MediumTests.class})
-public class TestStore {
-  private static final Log LOG = LogFactory.getLog(TestStore.class);
-  @Rule public TestName name = new TestName();
-
-  HStore store;
-  byte [] table = Bytes.toBytes("table");
-  byte [] family = Bytes.toBytes("family");
-
-  byte [] row = Bytes.toBytes("row");
-  byte [] row2 = Bytes.toBytes("row2");
-  byte [] qf1 = Bytes.toBytes("qf1");
-  byte [] qf2 = Bytes.toBytes("qf2");
-  byte [] qf3 = Bytes.toBytes("qf3");
-  byte [] qf4 = Bytes.toBytes("qf4");
-  byte [] qf5 = Bytes.toBytes("qf5");
-  byte [] qf6 = Bytes.toBytes("qf6");
-
-  NavigableSet<byte[]> qualifiers = new ConcurrentSkipListSet<>(Bytes.BYTES_COMPARATOR);
-
-  List<Cell> expected = new ArrayList<>();
-  List<Cell> result = new ArrayList<>();
-
-  long id = System.currentTimeMillis();
-  Get get = new Get(row);
-
-  private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString();
-
-
-  /**
-   * Setup
-   * @throws IOException
-   */
-  @Before
-  public void setUp() throws IOException {
-    qualifiers.add(qf1);
-    qualifiers.add(qf3);
-    qualifiers.add(qf5);
-
-    Iterator<byte[]> iter = qualifiers.iterator();
-    while(iter.hasNext()){
-      byte [] next = iter.next();
-      expected.add(new KeyValue(row, family, next, 1, (byte[])null));
-      get.addColumn(family, next);
-    }
-  }
-
-  private void init(String methodName) throws IOException {
-    init(methodName, TEST_UTIL.getConfiguration());
-  }
-
-  private Store init(String methodName, Configuration conf)
-  throws IOException {
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    // some of the tests write 4 versions and then flush
-    // (with HBASE-4241, lower versions are collected on flush)
-    hcd.setMaxVersions(4);
-    return init(methodName, conf, hcd);
-  }
-
-  private Store init(String methodName, Configuration conf,
-      HColumnDescriptor hcd) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
-    return init(methodName, conf, htd, hcd);
-  }
-
-  private Store init(String methodName, Configuration conf, HTableDescriptor htd,
-      HColumnDescriptor hcd) throws IOException {
-    return init(methodName, conf, htd, hcd, null);
-  }
-
-  @SuppressWarnings("deprecation")
-  private Store init(String methodName, Configuration conf, HTableDescriptor htd,
-      HColumnDescriptor hcd, MyStoreHook hook) throws IOException {
-    return init(methodName, conf, htd, hcd, hook, false);
-  }
-  @SuppressWarnings("deprecation")
-  private Store init(String methodName, Configuration conf, HTableDescriptor htd,
-      HColumnDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {
-    //Setting up a Store
-    Path basedir = new Path(DIR+methodName);
-    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
-    final Path logdir = new Path(basedir, AbstractFSWALProvider.getWALDirectoryName(methodName));
-
-    FileSystem fs = FileSystem.get(conf);
-
-    fs.delete(logdir, true);
-
-    if (htd.hasFamily(hcd.getName())) {
-      htd.modifyFamily(hcd);
-    } else {
-      htd.addFamily(hcd);
-    }
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
-      MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null);
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
-    final Configuration walConf = new Configuration(conf);
-    FSUtils.setRootDir(walConf, basedir);
-    final WALFactory wals = new WALFactory(walConf, null, methodName);
-    HRegion region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes(),
-            info.getTable().getNamespace()), fs, conf, info, htd, null);
-    if (hook == null) {
-      store = new HStore(region, hcd, conf);
-    } else {
-      store = new MyStore(region, hcd, conf, hook, switchToPread);
-    }
-    return store;
-  }
-
-  /**
-   * Test we do not lose data if we fail a flush and then close.
-   * Part of HBase-10466
-   * @throws Exception
-   */
-  @Test
-  public void testFlushSizeAccounting() throws Exception {
-    LOG.info("Setting up a faulty file system that cannot write in " +
-      this.name.getMethodName());
-    final Configuration conf = HBaseConfiguration.create();
-    // Only retry once.
-    conf.setInt("hbase.hstore.flush.retries.number", 1);
-    User user = User.createUserForTesting(conf, this.name.getMethodName(),
-      new String[]{"foo"});
-    // Inject our faulty LocalFileSystem
-    conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);
-    user.runAs(new PrivilegedExceptionAction<Object>() {
-      @Override
-      public Object run() throws Exception {
-        // Make sure it worked (above is sensitive to caching details in hadoop core)
-        FileSystem fs = FileSystem.get(conf);
-        Assert.assertEquals(FaultyFileSystem.class, fs.getClass());
-        FaultyFileSystem ffs = (FaultyFileSystem)fs;
-
-        // Initialize region
-        init(name.getMethodName(), conf);
-
-        MemstoreSize size = store.memstore.getFlushableSize();
-        Assert.assertEquals(0, size.getDataSize());
-        LOG.info("Adding some data");
-        MemstoreSize kvSize = new MemstoreSize();
-        store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), kvSize);
-        // add the heap size of active (mutable) segment
-        kvSize.incMemstoreSize(0, MutableSegment.DEEP_OVERHEAD);
-        size = store.memstore.getFlushableSize();
-        Assert.assertEquals(kvSize, size);
-        // Flush.  Bug #1 from HBASE-10466.  Make sure size calculation on failed flush is right.
-        try {
-          LOG.info("Flushing");
-          flushStore(store, id++);
-          Assert.fail("Didn't bubble up IOE!");
-        } catch (IOException ioe) {
-          Assert.assertTrue(ioe.getMessage().contains("Fault injected"));
-        }
-        // due to snapshot, change mutable to immutable segment
-        kvSize.incMemstoreSize(0,
-            CSLMImmutableSegment.DEEP_OVERHEAD_CSLM-MutableSegment.DEEP_OVERHEAD);
-        size = store.memstore.getFlushableSize();
-        Assert.assertEquals(kvSize, size);
-        MemstoreSize kvSize2 = new MemstoreSize();
-        store.add(new KeyValue(row, family, qf2, 2, (byte[])null), kvSize2);
-        kvSize2.incMemstoreSize(0, MutableSegment.DEEP_OVERHEAD);
-        // Even though we add a new kv, we expect the flushable size to be 'same' since we have
-        // not yet cleared the snapshot -- the above flush failed.
-        Assert.assertEquals(kvSize, size);
-        ffs.fault.set(false);
-        flushStore(store, id++);
-        size = store.memstore.getFlushableSize();
-        // Size should be the foreground kv size.
-        Assert.assertEquals(kvSize2, size);
-        flushStore(store, id++);
-        size = store.memstore.getFlushableSize();
-        assertEquals(0, size.getDataSize());
-        assertEquals(MutableSegment.DEEP_OVERHEAD, size.getHeapSize());
-        return null;
-      }
-    });
-  }
-
-  /**
-   * Verify that compression and data block encoding are respected by the
-   * Store.createWriterInTmp() method, used on store flush.
-   */
-  @Test
-  public void testCreateWriter() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    FileSystem fs = FileSystem.get(conf);
-
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setCompressionType(Compression.Algorithm.GZ);
-    hcd.setDataBlockEncoding(DataBlockEncoding.DIFF);
-    init(name.getMethodName(), conf, hcd);
-
-    // Test createWriterInTmp()
-    StoreFileWriter writer = store.createWriterInTmp(4, hcd.getCompressionType(), false, true, false);
-    Path path = writer.getPath();
-    writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1)));
-    writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2)));
-    writer.append(new KeyValue(row2, family, qf1, Bytes.toBytes(3)));
-    writer.append(new KeyValue(row2, family, qf2, Bytes.toBytes(4)));
-    writer.close();
-
-    // Verify that compression and encoding settings are respected
-    HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), true, conf);
-    Assert.assertEquals(hcd.getCompressionType(), reader.getCompressionAlgorithm());
-    Assert.assertEquals(hcd.getDataBlockEncoding(), reader.getDataBlockEncoding());
-    reader.close();
-  }
-
-  @Test
-  public void testDeleteExpiredStoreFiles() throws Exception {
-    testDeleteExpiredStoreFiles(0);
-    testDeleteExpiredStoreFiles(1);
-  }
-
-  /*
-   * @param minVersions the MIN_VERSIONS for the column family
-   */
-  public void testDeleteExpiredStoreFiles(int minVersions) throws Exception {
-    int storeFileNum = 4;
-    int ttl = 4;
-    IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge();
-    EnvironmentEdgeManagerTestHelper.injectEdge(edge);
-
-    Configuration conf = HBaseConfiguration.create();
-    // Enable the expired store file deletion
-    conf.setBoolean("hbase.store.delete.expired.storefile", true);
-    // Set the compaction threshold higher to avoid normal compactions.
-    conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 5);
-
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setMinVersions(minVersions);
-    hcd.setTimeToLive(ttl);
-    init(name.getMethodName() + "-" + minVersions, conf, hcd);
-
-    long storeTtl = this.store.getScanInfo().getTtl();
-    long sleepTime = storeTtl / storeFileNum;
-    long timeStamp;
-    // There are 4 store files and the max time stamp difference among these
-    // store files will be (this.store.ttl / storeFileNum)
-    for (int i = 1; i <= storeFileNum; i++) {
-      LOG.info("Adding some data for the store file #" + i);
-      timeStamp = EnvironmentEdgeManager.currentTime();
-      this.store.add(new KeyValue(row, family, qf1, timeStamp, (byte[]) null), null);
-      this.store.add(new KeyValue(row, family, qf2, timeStamp, (byte[]) null), null);
-      this.store.add(new KeyValue(row, family, qf3, timeStamp, (byte[]) null), null);
-      flush(i);
-      edge.incrementTime(sleepTime);
-    }
-
-    // Verify the total number of store files
-    Assert.assertEquals(storeFileNum, this.store.getStorefiles().size());
-
-     // Each call will find one expired store file and delete it before compaction happens.
-     // There will be no compaction due to threshold above. Last file will not be replaced.
-    for (int i = 1; i <= storeFileNum - 1; i++) {
-      // verify the expired store file.
-      assertFalse(this.store.requestCompaction().isPresent());
-      Collection<StoreFile> sfs = this.store.getStorefiles();
-      // Ensure i files are gone.
-      if (minVersions == 0) {
-        assertEquals(storeFileNum - i, sfs.size());
-        // Ensure only non-expired files remain.
-        for (StoreFile sf : sfs) {
-          assertTrue(sf.getReader().getMaxTimestamp() >= (edge.currentTime() - storeTtl));
-        }
-      } else {
-        assertEquals(storeFileNum, sfs.size());
-      }
-      // Let the next store file expired.
-      edge.incrementTime(sleepTime);
-    }
-    assertFalse(this.store.requestCompaction().isPresent());
-
-    Collection<StoreFile> sfs = this.store.getStorefiles();
-    // Assert the last expired file is not removed.
-    if (minVersions == 0) {
-      assertEquals(1, sfs.size());
-    }
-    long ts = sfs.iterator().next().getReader().getMaxTimestamp();
-    assertTrue(ts < (edge.currentTime() - storeTtl));
-
-    for (StoreFile sf : sfs) {
-      sf.closeReader(true);
-    }
-  }
-
-  @Test
-  public void testLowestModificationTime() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    FileSystem fs = FileSystem.get(conf);
-    // Initialize region
-    init(name.getMethodName(), conf);
-
-    int storeFileNum = 4;
-    for (int i = 1; i <= storeFileNum; i++) {
-      LOG.info("Adding some data for the store file #"+i);
-      this.store.add(new KeyValue(row, family, qf1, i, (byte[])null), null);
-      this.store.add(new KeyValue(row, family, qf2, i, (byte[])null), null);
-      this.store.add(new KeyValue(row, family, qf3, i, (byte[])null), null);
-      flush(i);
-    }
-    // after flush; check the lowest time stamp
-    long lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles());
-    long lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles());
-    Assert.assertEquals(lowestTimeStampFromManager,lowestTimeStampFromFS);
-
-    // after compact; check the lowest time stamp
-    store.compact(store.requestCompaction().get(), NoLimitThroughputController.INSTANCE, null);
-    lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles());
-    lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles());
-    Assert.assertEquals(lowestTimeStampFromManager, lowestTimeStampFromFS);
-  }
-
-  private static long getLowestTimeStampFromFS(FileSystem fs,
-      final Collection<StoreFile> candidates) throws IOException {
-    long minTs = Long.MAX_VALUE;
-    if (candidates.isEmpty()) {
-      return minTs;
-    }
-    Path[] p = new Path[candidates.size()];
-    int i = 0;
-    for (StoreFile sf : candidates) {
-      p[i] = sf.getPath();
-      ++i;
-    }
-
-    FileStatus[] stats = fs.listStatus(p);
-    if (stats == null || stats.length == 0) {
-      return minTs;
-    }
-    for (FileStatus s : stats) {
-      minTs = Math.min(minTs, s.getModificationTime());
-    }
-    return minTs;
-  }
-
-  //////////////////////////////////////////////////////////////////////////////
-  // Get tests
-  //////////////////////////////////////////////////////////////////////////////
-
-  private static final int BLOCKSIZE_SMALL = 8192;
-  /**
-   * Test for hbase-1686.
-   * @throws IOException
-   */
-  @Test
-  public void testEmptyStoreFile() throws IOException {
-    init(this.name.getMethodName());
-    // Write a store file.
-    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
-    flush(1);
-    // Now put in place an empty store file.  Its a little tricky.  Have to
-    // do manually with hacked in sequence id.
-    StoreFile f = this.store.getStorefiles().iterator().next();
-    Path storedir = f.getPath().getParent();
-    long seqid = f.getMaxSequenceId();
-    Configuration c = HBaseConfiguration.create();
-    FileSystem fs = FileSystem.get(c);
-    HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();
-    StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c),
-        fs)
-            .withOutputDir(storedir)
-            .withFileContext(meta)
-            .build();
-    w.appendMetadata(seqid + 1, false);
-    w.close();
-    this.store.close();
-    // Reopen it... should pick up two files
-    this.store = new HStore(this.store.getHRegion(), this.store.getColumnFamilyDescriptor(), c);
-    Assert.assertEquals(2, this.store.getStorefilesCount());
-
-    result = HBaseTestingUtility.getFromStoreFile(store,
-        get.getRow(),
-        qualifiers);
-    Assert.assertEquals(1, result.size());
-  }
-
-  /**
-   * Getting data from memstore only
-   * @throws IOException
-   */
-  @Test
-  public void testGet_FromMemStoreOnly() throws IOException {
-    init(this.name.getMethodName());
-
-    //Put data in memstore
-    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null), null);
-
-    //Get
-    result = HBaseTestingUtility.getFromStoreFile(store,
-        get.getRow(), qualifiers);
-
-    //Compare
-    assertCheck();
-  }
-
-  /**
-   * Getting data from files only
-   * @throws IOException
-   */
-  @Test
-  public void testGet_FromFilesOnly() throws IOException {
-    init(this.name.getMethodName());
-
-    //Put data in memstore
-    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
-    //flush
-    flush(1);
-
-    //Add more data
-    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null), null);
-    //flush
-    flush(2);
-
-    //Add more data
-    this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null), null);
-    //flush
-    flush(3);
-
-    //Get
-    result = HBaseTestingUtility.getFromStoreFile(store,
-        get.getRow(),
-        qualifiers);
-    //this.store.get(get, qualifiers, result);
-
-    //Need to sort the result since multiple files
-    Collections.sort(result, CellComparator.COMPARATOR);
-
-    //Compare
-    assertCheck();
-  }
-
-  /**
-   * Getting data from memstore and files
-   * @throws IOException
-   */
-  @Test
-  public void testGet_FromMemStoreAndFiles() throws IOException {
-    init(this.name.getMethodName());
-
-    //Put data in memstore
-    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
-    //flush
-    flush(1);
-
-    //Add more data
-    this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null), null);
-    //flush
-    flush(2);
-
-    //Add more data
-    this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null), null);
-    this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null), null);
-
-    //Get
-    result = HBaseTestingUtility.getFromStoreFile(store,
-        get.getRow(), qualifiers);
-
-    //Need to sort the result since multiple files
-    Collections.sort(result, CellComparator.COMPARATOR);
-
-    //Compare
-    assertCheck();
-  }
-
-  private void flush(int storeFilessize) throws IOException{
-    this.store.snapshot();
-    flushStore(store, id++);
-    Assert.assertEquals(storeFilessize, this.store.getStorefiles().size());
-    Assert.assertEquals(0, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
-  }
-
-  private void assertCheck() {
-    Assert.assertEquals(expected.size(), result.size());
-    for(int i=0; i<expected.size(); i++) {
-      Assert.assertEquals(expected.get(i), result.get(i));
-    }
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    EnvironmentEdgeManagerTestHelper.reset();
-  }
-
-  @Test
-  public void testHandleErrorsInFlush() throws Exception {
-    LOG.info("Setting up a faulty file system that cannot write");
-
-    final Configuration conf = HBaseConfiguration.create();
-    User user = User.createUserForTesting(conf,
-        "testhandleerrorsinflush", new String[]{"foo"});
-    // Inject our faulty LocalFileSystem
-    conf.setClass("fs.file.impl", FaultyFileSystem.class,
-        FileSystem.class);
-    user.runAs(new PrivilegedExceptionAction<Object>() {
-      @Override
-      public Object run() throws Exception {
-        // Make sure it worked (above is sensitive to caching details in hadoop core)
-        FileSystem fs = FileSystem.get(conf);
-        Assert.assertEquals(FaultyFileSystem.class, fs.getClass());
-
-        // Initialize region
-        init(name.getMethodName(), conf);
-
-        LOG.info("Adding some data");
-        store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
-        store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
-        store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
-
-        LOG.info("Before flush, we should have no files");
-
-        Collection<StoreFileInfo> files =
-          store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName());
-        Assert.assertEquals(0, files != null ? files.size() : 0);
-
-        //flush
-        try {
-          LOG.info("Flushing");
-          flush(1);
-          Assert.fail("Didn't bubble up IOE!");
-        } catch (IOException ioe) {
-          Assert.assertTrue(ioe.getMessage().contains("Fault injected"));
-        }
-
-        LOG.info("After failed flush, we should still have no files!");
-        files = store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName());
-        Assert.assertEquals(0, files != null ? files.size() : 0);
-        store.getHRegion().getWAL().close();
-        return null;
-      }
-    });
-    FileSystem.closeAllForUGI(user.getUGI());
-  }
-
-  /**
-   * Faulty file system that will fail if you write past its fault position the FIRST TIME
-   * only; thereafter it will succeed.  Used by {@link TestHRegion} too.
-   */
-  static class FaultyFileSystem extends FilterFileSystem {
-    List<SoftReference<FaultyOutputStream>> outStreams = new ArrayList<>();
-    private long faultPos = 200;
-    AtomicBoolean fault = new AtomicBoolean(true);
-
-    public FaultyFileSystem() {
-      super(new LocalFileSystem());
-      System.err.println("Creating faulty!");
-    }
-
-    @Override
-    public FSDataOutputStream create(Path p) throws IOException {
-      return new FaultyOutputStream(super.create(p), faultPos, fault);
-    }
-
-    @Override
-    public FSDataOutputStream create(Path f, FsPermission permission,
-        boolean overwrite, int bufferSize, short replication, long blockSize,
-        Progressable progress) throws IOException {
-      return new FaultyOutputStream(super.create(f, permission,
-          overwrite, bufferSize, replication, blockSize, progress), faultPos, fault);
-    }
-
-    @Override
-    public FSDataOutputStream createNonRecursive(Path f, boolean overwrite,
-        int bufferSize, short replication, long blockSize, Progressable progress)
-    throws IOException {
-      // Fake it.  Call create instead.  The default implementation throws an IOE
-      // that this is not supported.
-      return create(f, overwrite, bufferSize, replication, blockSize, progress);
-    }
-  }
-
-  static class FaultyOutputStream extends FSDataOutputStream {
-    volatile long faultPos = Long.MAX_VALUE;
-    private final AtomicBoolean fault;
-
-    public FaultyOutputStream(FSDataOutputStream out, long faultPos, final AtomicBoolean fault)
-    throws IOException {
-      super(out, null);
-      this.faultPos = faultPos;
-      this.fault = fault;
-    }
-
-    @Override
-    public void write(byte[] buf, int offset, int length) throws IOException {
-      System.err.println("faulty stream write at pos " + getPos());
-      injectFault();
-      super.write(buf, offset, length);
-    }
-
-    private void injectFault() throws IOException {
-      if (this.fault.get() && getPos() >= faultPos) {
-        throw new IOException("Fault injected");
-      }
-    }
-  }
-
-  private static void flushStore(HStore store, long id) throws IOException {
-    StoreFlushContext storeFlushCtx = store.createFlushContext(id);
-    storeFlushCtx.prepare();
-    storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
-    storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
-  }
-
-  /**
-   * Generate a list of KeyValues for testing based on given parameters
-   * @param timestamps
-   * @param numRows
-   * @param qualifier
-   * @param family
-   * @return
-   */
-  List<Cell> getKeyValueSet(long[] timestamps, int numRows,
-      byte[] qualifier, byte[] family) {
-    List<Cell> kvList = new ArrayList<>();
-    for (int i=1;i<=numRows;i++) {
-      byte[] b = Bytes.toBytes(i);
-      for (long timestamp: timestamps) {
-        kvList.add(new KeyValue(b, family, qualifier, timestamp, b));
-      }
-    }
-    return kvList;
-  }
-
-  /**
-   * Test to ensure correctness when using Stores with multiple timestamps
-   * @throws IOException
-   */
-  @Test
-  public void testMultipleTimestamps() throws IOException {
-    int numRows = 1;
-    long[] timestamps1 = new long[] {1,5,10,20};
-    long[] timestamps2 = new long[] {30,80};
-
-    init(this.name.getMethodName());
-
-    List<Cell> kvList1 = getKeyValueSet(timestamps1,numRows, qf1, family);
-    for (Cell kv : kvList1) {
-      this.store.add(kv, null);
-    }
-
-    this.store.snapshot();
-    flushStore(store, id++);
-
-    List<Cell> kvList2 = getKeyValueSet(timestamps2,numRows, qf1, family);
-    for(Cell kv : kvList2) {
-      this.store.add(kv, null);
-    }
-
-    List<Cell> result;
-    Get get = new Get(Bytes.toBytes(1));
-    get.addColumn(family,qf1);
-
-    get.setTimeRange(0,15);
-    result = HBaseTestingUtility.getFromStoreFile(store, get);
-    Assert.assertTrue(result.size()>0);
-
-    get.setTimeRange(40,90);
-    result = HBaseTestingUtility.getFromStoreFile(store, get);
-    Assert.assertTrue(result.size()>0);
-
-    get.setTimeRange(10,45);
-    result = HBaseTestingUtility.getFromStoreFile(store, get);
-    Assert.assertTrue(result.size()>0);
-
-    get.setTimeRange(80,145);
-    result = HBaseTestingUtility.getFromStoreFile(store, get);
-    Assert.assertTrue(result.size()>0);
-
-    get.setTimeRange(1,2);
-    result = HBaseTestingUtility.getFromStoreFile(store, get);
-    Assert.assertTrue(result.size()>0);
-
-    get.setTimeRange(90,200);
-    result = HBaseTestingUtility.getFromStoreFile(store, get);
-    Assert.assertTrue(result.size()==0);
-  }
-
-  /**
-   * Test for HBASE-3492 - Test split on empty colfam (no store files).
-   *
-   * @throws IOException When the IO operations fail.
-   */
-  @Test
-  public void testSplitWithEmptyColFam() throws IOException {
-    init(this.name.getMethodName());
-    Assert.assertNull(store.getSplitPoint());
-    store.getHRegion().forceSplit(null);
-    Assert.assertNull(store.getSplitPoint());
-    store.getHRegion().clearSplit();
-  }
-
-  @Test
-  public void testStoreUsesConfigurationFromHcdAndHtd() throws Exception {
-    final String CONFIG_KEY = "hbase.regionserver.thread.compaction.throttle";
-    long anyValue = 10;
-
-    // We'll check that it uses correct config and propagates it appropriately by going thru
-    // the simplest "real" path I can find - "throttleCompaction", which just checks whether
-    // a number we pass in is higher than some config value, inside compactionPolicy.
-    Configuration conf = HBaseConfiguration.create();
-    conf.setLong(CONFIG_KEY, anyValue);
-    init(name.getMethodName() + "-xml", conf);
-    Assert.assertTrue(store.throttleCompaction(anyValue + 1));
-    Assert.assertFalse(store.throttleCompaction(anyValue));
-
-    // HTD overrides XML.
-    --anyValue;
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    htd.setConfiguration(CONFIG_KEY, Long.toString(anyValue));
-    init(name.getMethodName() + "-htd", conf, htd, hcd);
-    Assert.assertTrue(store.throttleCompaction(anyValue + 1));
-    Assert.assertFalse(store.throttleCompaction(anyValue));
-
-    // HCD overrides them both.
-    --anyValue;
-    hcd.setConfiguration(CONFIG_KEY, Long.toString(anyValue));
-    init(name.getMethodName() + "-hcd", conf, htd, hcd);
-    Assert.assertTrue(store.throttleCompaction(anyValue + 1));
-    Assert.assertFalse(store.throttleCompaction(anyValue));
-  }
-
-  public static class DummyStoreEngine extends DefaultStoreEngine {
-    public static DefaultCompactor lastCreatedCompactor = null;
-
-    @Override
-    protected void createComponents(Configuration conf, Store store, CellComparator comparator)
-        throws IOException {
-      super.createComponents(conf, store, comparator);
-      lastCreatedCompactor = this.compactor;
-    }
-  }
-
-  @Test
-  public void testStoreUsesSearchEngineOverride() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(StoreEngine.STORE_ENGINE_CLASS_KEY, DummyStoreEngine.class.getName());
-    init(this.name.getMethodName(), conf);
-    Assert.assertEquals(DummyStoreEngine.lastCreatedCompactor,
-      this.store.storeEngine.getCompactor());
-  }
-
-  private void addStoreFile() throws IOException {
-    StoreFile f = this.store.getStorefiles().iterator().next();
-    Path storedir = f.getPath().getParent();
-    long seqid = this.store.getMaxSequenceId();
-    Configuration c = TEST_UTIL.getConfiguration();
-    FileSystem fs = FileSystem.get(c);
-    HFileContext fileContext = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();
-    StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c),
-        fs)
-            .withOutputDir(storedir)
-            .withFileContext(fileContext)
-            .build();
-    w.appendMetadata(seqid + 1, false);
-    w.close();
-    LOG.info("Added store file:" + w.getPath());
-  }
-
-  private void archiveStoreFile(int index) throws IOException {
-    Collection<StoreFile> files = this.store.getStorefiles();
-    StoreFile sf = null;
-    Iterator<StoreFile> it = files.iterator();
-    for (int i = 0; i <= index; i++) {
-      sf = it.next();
-    }
-    store.getRegionFileSystem().removeStoreFiles(store.getColumnFamilyName(), Lists.newArrayList(sf));
-  }
-
-  private void closeCompactedFile(int index) throws IOException {
-    Collection<StoreFile> files =
-        this.store.getStoreEngine().getStoreFileManager().getCompactedfiles();
-    StoreFile sf = null;
-    Iterator<StoreFile> it = files.iterator();
-    for (int i = 0; i <= index; i++) {
-      sf = it.next();
-    }
-    sf.closeReader(true);
-    store.getStoreEngine().getStoreFileManager().removeCompactedFiles(Lists.newArrayList(sf));
-  }
-
-  @Test
-  public void testRefreshStoreFiles() throws Exception {
-    init(name.getMethodName());
-
-    assertEquals(0, this.store.getStorefilesCount());
-
-    // Test refreshing store files when no store files are there
-    store.refreshStoreFiles();
-    assertEquals(0, this.store.getStorefilesCount());
-
-    // add some data, flush
-    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
-    flush(1);
-    assertEquals(1, this.store.getStorefilesCount());
-
-    // add one more file
-    addStoreFile();
-
-    assertEquals(1, this.store.getStorefilesCount());
-    store.refreshStoreFiles();
-    assertEquals(2, this.store.getStorefilesCount());
-
-    // add three more files
-    addStoreFile();
-    addStoreFile();
-    addStoreFile();
-
-    assertEquals(2, this.store.getStorefilesCount());
-    store.refreshStoreFiles();
-    assertEquals(5, this.store.getStorefilesCount());
-
-    closeCompactedFile(0);
-    archiveStoreFile(0);
-
-    assertEquals(5, this.store.getStorefilesCount());
-    store.refreshStoreFiles();
-    assertEquals(4, this.store.getStorefilesCount());
-
-    archiveStoreFile(0);
-    archiveStoreFile(1);
-    archiveStoreFile(2);
-
-    assertEquals(4, this.store.getStorefilesCount());
-    store.refreshStoreFiles();
-    assertEquals(1, this.store.getStorefilesCount());
-
-    archiveStoreFile(0);
-    store.refreshStoreFiles();
-    assertEquals(0, this.store.getStorefilesCount());
-  }
-
-  @SuppressWarnings("unchecked")
-  @Test
-  public void testRefreshStoreFilesNotChanged() throws IOException {
-    init(name.getMethodName());
-
-    assertEquals(0, this.store.getStorefilesCount());
-
-    // add some data, flush
-    this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
-    flush(1);
-    // add one more file
-    addStoreFile();
-
-    HStore spiedStore = spy(store);
-
-    // call first time after files changed
-    spiedStore.refreshStoreFiles();
-    assertEquals(2, this.store.getStorefilesCount());
-    verify(spiedStore, times(1)).replaceStoreFiles(any(Collection.class), any(Collection.class));
-
-    // call second time
-    spiedStore.refreshStoreFiles();
-
-    //ensure that replaceStoreFiles is not called if files are not refreshed
-    verify(spiedStore, times(0)).replaceStoreFiles(null, null);
-  }
-
-  private long countMemStoreScanner(StoreScanner scanner) {
-    if (scanner.currentScanners == null) {
-      return 0;
-    }
-    return scanner.currentScanners.stream()
-            .filter(s -> !s.isFileScanner())
-            .count();
-  }
-
-  @Test
-  public void testNumberOfMemStoreScannersAfterFlush() throws IOException {
-    long seqId = 100;
-    long timestamp = System.currentTimeMillis();
-    Cell cell0 = CellUtil.createCell(row, family, qf1, timestamp,
-            KeyValue.Type.Put.getCode(), qf1);
-    CellUtil.setSequenceId(cell0, seqId);
-    testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Collections.EMPTY_LIST);
-
-    Cell cell1 = CellUtil.createCell(row, family, qf2, timestamp,
-            KeyValue.Type.Put.getCode(), qf1);
-    CellUtil.setSequenceId(cell1, seqId);
-    testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1));
-
-    seqId = 101;
-    timestamp = System.currentTimeMillis();
-    Cell cell2 = CellUtil.createCell(row2, family, qf2, timestamp,
-            KeyValue.Type.Put.getCode(), qf1);
-     CellUtil.setSequenceId(cell2, seqId);
-    testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1, cell2));
-  }
-
-  private void testNumberOfMemStoreScannersAfterFlush(List<Cell> inputCellsBeforeSnapshot,
-      List<Cell> inputCellsAfterSnapshot) throws IOException {
-    init(this.name.getMethodName() + "-" + inputCellsBeforeSnapshot.size());
-    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
-    long seqId = Long.MIN_VALUE;
-    for (Cell c : inputCellsBeforeSnapshot) {
-      quals.add(CellUtil.cloneQualifier(c));
-      seqId = Math.max(seqId, c.getSequenceId());
-    }
-    for (Cell c : inputCellsAfterSnapshot) {
-      quals.add(CellUtil.cloneQualifier(c));
-      seqId = Math.max(seqId, c.getSequenceId());
-    }
-    inputCellsBeforeSnapshot.forEach(c -> store.add(c, null));
-    StoreFlushContext storeFlushCtx = store.createFlushContext(id++);
-    storeFlushCtx.prepare();
-    inputCellsAfterSnapshot.forEach(c -> store.add(c, null));
-    int numberOfMemScannersWhenScaning = inputCellsAfterSnapshot.isEmpty() ? 1 : 2;
-    try (StoreScanner s = (StoreScanner) store.getScanner(new Scan(), quals, seqId)) {
-      // snaptshot + active (if it isn't empty)
-      assertEquals(numberOfMemScannersWhenScaning, countMemStoreScanner(s));
-      storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
-      storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
-      boolean more;
-      int cellCount = 0;
-      do {
-        List<Cell> cells = new ArrayList<>();
-        more = s.next(cells);
-        cellCount += cells.size();
-        assertEquals(more ? numberOfMemScannersWhenScaning : 0, countMemStoreScanner(s));
-      } while (more);
-      assertEquals("The number of cells added before snapshot is " + inputCellsBeforeSnapshot.size()
-          + ", The number of cells added after snapshot is " + inputCellsAfterSnapshot.size(),
-          inputCellsBeforeSnapshot.size() + inputCellsAfterSnapshot.size(), cellCount);
-      // the current scanners is cleared
-      assertEquals(0, countMemStoreScanner(s));
-    }
-  }
-
-  private Cell createCell(byte[] qualifier, long ts, long sequenceId, byte[] value) throws IOException {
-    Cell c = CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put.getCode(), value);
-    CellUtil.setSequenceId(c, sequenceId);
-    return c;
-  }
-
-  private Cell createCell(byte[] row, byte[] qualifier, long ts, long sequenceId, byte[] value)
-      throws IOException {
-    Cell c = CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put.getCode(), value);
-    CellUtil.setSequenceId(c, sequenceId);
-    return c;
-  }
-
-  @Test
-  public void testFlushBeforeCompletingScanWoFilter() throws IOException, InterruptedException {
-    final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);
-    final int expectedSize = 3;
-    testFlushBeforeCompletingScan(new MyListHook() {
-      @Override
-      public void hook(int currentSize) {
-        if (currentSize == expectedSize - 1) {
-          try {
-            flushStore(store, id++);
-            timeToGoNextRow.set(true);
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
-      }
-    }, new FilterBase() {
-      @Override
-      public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
-        return ReturnCode.INCLUDE;
-      }
-    }, expectedSize);
-  }
-
-  @Test
-  public void testFlushBeforeCompletingScanWithFilter() throws IOException, InterruptedException {
-    final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);
-    final int expectedSize = 2;
-    testFlushBeforeCompletingScan(new MyListHook() {
-      @Override
-      public void hook(int currentSize) {
-        if (currentSize == expectedSize - 1) {
-          try {
-            flushStore(store, id++);
-            timeToGoNextRow.set(true);
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
-      }
-    }, new FilterBase() {
-      @Override
-      public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
-        if (timeToGoNextRow.get()) {
-          timeToGoNextRow.set(false);
-          return ReturnCode.NEXT_ROW;
-        } else {
-          return ReturnCode.INCLUDE;
-        }
-      }
-    }, expectedSize);
-  }
-
-  @Test
-  public void testFlushBeforeCompletingScanWithFilterHint() throws IOException, InterruptedException {
-    final AtomicBoolean timeToGetHint = new AtomicBoolean(false);
-    final int expectedSize = 2;
-    testFlushBeforeCompletingScan(new MyListHook() {
-      @Override
-      public void hook(int currentSize) {
-        if (currentSize == expectedSize - 1) {
-          try {
-            flushStore(store, id++);
-            timeToGetHint.set(true);
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
-      }
-    }, new FilterBase() {
-      @Override
-      public Filter.ReturnCode filterKeyValue(Cell v) throws IOException {
-        if (timeToGetHint.get()) {
-          timeToGetHint.set(false);
-          return Filter.ReturnCode.SEEK_NEXT_USING_HINT;
-        } else {
-          return Filter.ReturnCode.INCLUDE;
-        }
-      }
-      @Override
-      public Cell getNextCellHint(Cell currentCell) throws IOException {
-        return currentCell;
-      }
-    }, expectedSize);
-  }
-
-  private void testFlushBeforeCompletingScan(MyListHook hook, Filter filter, int expectedSize)
-          throws IOException, InterruptedException {
-    Configuration conf = HBaseConfiguration.create();
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setMaxVersions(1);
-    byte[] r0 = Bytes.toBytes("row0");
-    byte[] r1 = Bytes.toBytes("row1");
-    byte[] r2 = Bytes.toBytes("row2");
-    byte[] value0 = Bytes.toBytes("value0");
-    byte[] value1 = Bytes.toBytes("value1");
-    byte[] value2 = Bytes.toBytes("value2");
-    MemstoreSize memStoreSize = new MemstoreSize();
-    long ts = EnvironmentEdgeManager.currentTime();
-    long seqId = 100;
-    init(name.getMethodName(), conf, new HTableDescriptor(TableName.valueOf(table)), hcd, new MyStoreHook() {
-      @Override
-      public long getSmallestReadPoint(HStore store) {
-        return seqId + 3;
-      }
-    });
-    // The cells having the value0 won't be flushed to disk because the value of max version is 1
-    store.add(createCell(r0, qf1, ts, seqId, value0), memStoreSize);
-    store.add(createCell(r0, qf2, ts, seqId, value0), memStoreSize);
-    store.add(createCell(r0, qf3, ts, seqId, value0), memStoreSize);
-    store.add(createCell(r1, qf1, ts + 1, seqId + 1, value1), memStoreSize);
-    store.add(createCell(r1, qf2, ts + 1, seqId + 1, value1), memStoreSize);
-    store.add(createCell(r1, qf3, ts + 1, seqId + 1, value1), memStoreSize);
-    store.add(createCell(r2, qf1, ts + 2, seqId + 2, value2), memStoreSize);
-    store.add(createCell(r2, qf2, ts + 2, seqId + 2, value2), memStoreSize);
-    store.add(createCell(r2, qf3, ts + 2, seqId + 2, value2), memStoreSize);
-    store.add(createCell(r1, qf1, ts + 3, seqId + 3, value1), memStoreSize);
-    store.add(createCell(r1, qf2, ts + 3, seqId + 3, value1), memStoreSize);
-    store.add(createCell(r1, qf3, ts + 3, seqId + 3, value1), memStoreSize);
-    List<Cell> myList = new MyList<>(hook);
-    Scan scan = new Scan()
-            .withStartRow(r1)
-            .setFilter(filter);
-    try (InternalScanner scanner = (InternalScanner) store.getScanner(
-          scan, null, seqId + 3)){
-      // r1
-      scanner.next(myList);
-      assertEquals(expectedSize, myList.size());
-      for (Cell c : myList) {
-        byte[] actualValue = CellUtil.cloneValue(c);
-        assertTrue("expected:" + Bytes.toStringBinary(value1)
-          + ", actual:" + Bytes.toStringBinary(actualValue)
-          , Bytes.equals(actualValue, value1));
-      }
-      List<Cell> normalList = new ArrayList<>(3);
-      // r2
-      scanner.next(normalList);
-      assertEquals(3, normalList.size());
-      for (Cell c : normalList) {
-        byte[] actualValue = CellUtil.cloneValue(c);
-        assertTrue("expected:" + Bytes.toStringBinary(value2)
-          + ", actual:" + Bytes.toStringBinary(actualValue)
-          , Bytes.equals(actualValue, value2));
-      }
-    }
-  }
-
-  @Test
-  public void testCreateScannerAndSnapshotConcurrently() throws IOException, InterruptedException {
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore.class.getName());
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
-    init(name.getMethodName(), conf, hcd);
-    byte[] value = Bytes.toBytes("value");
-    MemstoreSize memStoreSize = new MemstoreSize();
-    long ts = EnvironmentEdgeManager.currentTime();
-    long seqId = 100;
-    // older data whihc shouldn't be "seen" by client
-    store.add(createCell(qf1, ts, seqId, value), memStoreSize);
-    store.add(createCell(qf2, ts, seqId, value), memStoreSize);
-    store.add(createCell(qf3, ts, seqId, value), memStoreSize);
-    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
-    quals.add(qf1);
-    quals.add(qf2);
-    quals.add(qf3);
-    StoreFlushContext storeFlushCtx = store.createFlushContext(id++);
-    MyCompactingMemStore.START_TEST.set(true);
-    Runnable flush = () -> {
-      // this is blocked until we create first scanner from pipeline and snapshot -- phase (1/5)
-      // recreate the active memstore -- phase (4/5)
-      storeFlushCtx.prepare();
-    };
-    ExecutorService service = Executors.newSingleThreadExecutor();
-    service.submit(flush);
-    // we get scanner from pipeline and snapshot but they are empty. -- phase (2/5)
-    // this is blocked until we recreate the active memstore -- phase (3/5)
-    // we get scanner from active memstore but it is empty -- phase (5/5)
-    InternalScanner scanner = (InternalScanner) store.getScanner(
-          new Scan(new Get(row)), quals, seqId + 1);
-    service.shutdown();
-    service.awaitTermination(20, TimeUnit.SECONDS);
-    try {
-      try {
-        List<Cell> results = new ArrayList<>();
-        scanner.next(results);
-        assertEquals(3, results.size());
-        for (Cell c : results) {
-          byte[] actualValue = CellUtil.cloneValue(c);
-          assertTrue("expected:" + Bytes.toStringBinary(value)
-            + ", actual:" + Bytes.toStringBinary(actualValue)
-            , Bytes.equals(actualValue, value));
-        }
-      } finally {
-        scanner.close();
-      }
-    } finally {
-      MyCompactingMemStore.START_TEST.set(false);
-      storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
-      storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
-    }
-  }
-
-  @Test
-  public void testScanWithDoubleFlush() throws IOException {
-    Configuration conf = HBaseConfiguration.create();
-    // Initialize region
-    MyStore myStore = initMyStore(name.getMethodName(), conf, new MyStoreHook(){
-      @Override
-      public void getScanners(MyStore store) throws IOException {
-        final long tmpId = id++;
-        ExecutorService s = Executors.newSingleThreadExecutor();
-        s.submit(() -> {
-          try {
-            // flush the store before storescanner updates the scanners from store.
-            // The current data will be flushed into files, and the memstore will
-            // be clear.
-            // -- phase (4/4)
-            flushStore(store, tmpId);
-          }catch (IOException ex) {
-            throw new RuntimeException(ex);
-          }
-        });
-        s.shutdown();
-        try {
-          // wait for the flush, the thread will be blocked in HStore#notifyChangedReadersObservers.
-          s.awaitTermination(3, TimeUnit.SECONDS);
-        } catch (InterruptedException ex) {
-        }
-      }
-    });
-    byte[] oldValue = Bytes.toBytes("oldValue");
-    byte[] currentValue = Bytes.toBytes("currentValue");
-    MemstoreSize memStoreSize = new MemstoreSize();
-    long ts = EnvironmentEdgeManager.currentTime();
-    long seqId = 100;
-    // older data whihc shouldn't be "seen" by client
-    myStore.add(createCell(qf1, ts, seqId, oldValue), memStoreSize);
-    myStore.add(createCell(qf2, ts, seqId, oldValue), memStoreSize);
-    myStore.add(createCell(qf3, ts, seqId, oldValue), memStoreSize);
-    long snapshotId = id++;
-    // push older data into snapshot -- phase (1/4)
-    StoreFlushContext storeFlushCtx = store.createFlushContext(snapshotId);
-    storeFlushCtx.prepare();
-
-    // insert current data into active -- phase (2/4)
-    myStore.add(createCell(qf1, ts + 1, seqId + 1, currentValue), memStoreSize);
-    myStore.add(createCell(qf2, ts + 1, seqId + 1, currentValue), memStoreSize);
-    myStore.add(createCell(qf3, ts + 1, seqId + 1, currentValue), memStoreSize);
-    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
-    quals.add(qf1);
-    quals.add(qf2);
-    quals.add(qf3);
-    try (InternalScanner scanner = (InternalScanner) myStore.getScanner(
-        new Scan(new Get(row)), quals, seqId + 1)) {
-      // complete the flush -- phase (3/4)
-      storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
-      storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
-
-      List<Cell> results = new ArrayList<>();
-      scanner.next(results);
-      assertEquals(3, results.size());
-      for (Cell c : results) {
-        byte[] actualValue = CellUtil.cloneValue(c);
-        assertTrue("expected:" + Bytes.toStringBinary(currentValue)
-          + ", actual:" + Bytes.toStringBinary(actualValue)
-          , Bytes.equals(actualValue, currentValue));
-      }
-    }
-  }
-
-  @Test
-  public void testReclaimChunkWhenScaning() throws IOException {
-    init("testReclaimChunkWhenScaning");
-    long ts = EnvironmentEdgeManager.currentTime();
-    long seqId = 100;
-    byte[] value = Bytes.toBytes("value");
-    // older data whihc shouldn't be "seen" by client
-    store.add(createCell(qf1, ts, seqId, value), null);
-    store.add(createCell(qf2, ts, seqId, value), null);
-    store.add(createCell(qf3, ts, seqId, value), null);
-    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
-    quals.add(qf1);
-    quals.add(qf2);
-    quals.add(qf3);
-    try (InternalScanner scanner = (InternalScanner) store.getScanner(
-        new Scan(new Get(row)), quals, seqId)) {
-      List<Cell> results = new MyList<>(size -> {
-        switch (size) {
-          // 1) we get the first cell (qf1)
-          // 2) flush the data to have StoreScanner update inner scanners
-          // 3) the chunk will be reclaimed after updaing
-          case 1:
-            try {
-              flushStore(store, id++);
-            } catch (IOException e) {
-              throw new RuntimeException(e);
-            }
-            break;
-          // 1) we get the second cell (qf2)
-          // 2) add some cell to fill some byte into the chunk (we have only one chunk)
-          case 2:
-            try {
-              byte[] newValue = Bytes.toBytes("newValue");
-              // older data whihc shouldn't be "seen" by client
-              store.add(createCell(qf1, ts + 1, seqId + 1, newValue), null);
-              store.add(createCell(qf2, ts + 1, seqId + 1, newValue), null);
-              store.add(createCell(qf3, ts + 1, seqId + 1, newValue), null);
-            } catch (IOException e) {
-              throw new RuntimeException(e);
-            }
-            break;
-          default:
-            break;
-        }
-      });
-      scanner.next(results);
-      assertEquals(3, results.size());
-      for (Cell c : results) {
-        byte[] actualValue = CellUtil.cloneValue(c);
-        assertTrue("expected:" + Bytes.toStringBinary(value)
-          + ", actual:" + Bytes.toStringBinary(actualValue)
-          , Bytes.equals(actualValue, value));
-      }
-    }
-  }
-
-  /**
-   * If there are two running InMemoryFlushRunnable, the later InMemoryFlushRunnable
-   * may change the versionedList. And the first InMemoryFlushRunnable will use the chagned
-   * versionedList to remove the corresponding segments.
-   * In short, there will be some segements which isn't in merge are removed.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  @Test (timeout=30000)
-  public void testRunDoubleMemStoreCompactors() throws IOException, InterruptedException {
-    int flushSize = 500;
-    Configuration conf = HBaseConfiguration.create();
-    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStoreWithCustomCompactor.class.getName());
-    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushSize));
-    // Set the lower threshold to invoke the "MERGE" policy
-    conf.set(MemStoreCompactor.COMPACTING_MEMSTORE_THRESHOLD_KEY, String.valueOf(0));
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
-    init(name.getMethodName(), conf, hcd);
-    byte[] value = Bytes.toBytes("thisisavarylargevalue");
-    MemstoreSize memStoreSize = new MemstoreSize();
-    long ts = EnvironmentEdgeManager.currentTime();
-    long seqId = 100;
-    // older data whihc shouldn't be "seen" by client
-    store.add(createCell(qf1, ts, seqId, value), memStoreSize);
-    store.add(createCell(qf2, ts, seqId, value), memStoreSize);
-    store.add(createCell(qf3, ts, seqId, value), memStoreSize);
-    assertEquals(1, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());
-    StoreFlushContext storeFlushCtx = store.createFlushContext(id++);
-    storeFlushCtx.prepare();
-    // This shouldn't invoke another in-memory flush because the first compactor thread
-    // hasn't accomplished the in-memory compaction.
-    store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSize);
-    store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSize);
-    store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSize);
-    assertEquals(1, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());
-    //okay. Let the compaction be completed
-    MyMemStoreCompactor.START_COMPACTOR_LATCH.countDown();
-    CompactingMemStore mem = (CompactingMemStore) ((HStore)store).memstore;
-    while (mem.isMemStoreFlushingInMemory()) {
-      TimeUnit.SECONDS.sleep(1);
-    }
-    // This should invoke another in-memory flush.
-    store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSize);
-    store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSize);
-    store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSize);
-    assertEquals(2, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());
-    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
-      String.valueOf(TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE));
-    storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
-    storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
-  }
-
-  private MyStore initMyStore(String methodName, Configuration conf, MyStoreHook hook)
-      throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setMaxVersions(5);
-    return (MyStore) init(methodName, conf, htd, hcd, hook);
-  }
-
-  class MyStore extends HStore {
-    private final MyStoreHook hook;
-
-    MyStore(final HRegion region, final HColumnDescriptor family, final Configuration confParam,
-        MyStoreHook hook, boolean switchToPread) throws IOException {
-      super(region, family, confParam);
-      this.hook = hook;
-    }
-
-    @Override
-    public List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks,
-        boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
-        boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
-        boolean includeMemstoreScanner) throws IOException {
-      hook.getScanners(this);
-      return super.getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true,
-        stopRow, false, readPt, includeMemstoreScanner);
-    }
-    @Override
-    public long getSmallestReadPoint() {
-      return hook.getSmallestReadPoint(this);
-    }
-  }
-
-  private abstract class MyStoreHook {
-    void getScanners(MyStore store) throws IOException {
-    }
-    long getSmallestReadPoint(HStore store) {
-      return store.getHRegion().getSmallestReadPoint();
-    }
-  }
-
-  @Test
-  public void testSwitchingPreadtoStreamParallelyWithCompactionDischarger() throws Exception {
-    int flushSize = 500;
-    Configuration conf = HBaseConfiguration.create();
-    conf.set("hbase.hstore.engine.class", DummyStoreEngine.class.getName());
-    conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 0);
-    // Set the lower threshold to invoke the "MERGE" policy
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
-    MyStore store = initMyStore(name.getMethodName(), conf, new MyStoreHook() {});
-    MemstoreSize memStoreSize = new MemstoreSize();
-    long ts = System.currentTimeMillis();
-    long seqID = 1l;
-    // Add some data to the region and do some flushes
-    for (int i = 1; i < 10; i++) {
-      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
-        memStoreSize);
-    }
-    // flush them
-    flushStore(store, seqID);
-    for (int i = 11; i < 20; i++) {
-      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
-        memStoreSize);
-    }
-    // flush them
-    flushStore(store, seqID);
-    for (int i = 21; i < 30; i++) {
-      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
-        memStoreSize);
-    }
-    // flush them
-    flushStore(store, seqID);
-
-    assertEquals(3, store.getStorefilesCount());
-    ScanInfo scanInfo = store.getScanInfo();
-    Scan scan = new Scan();
-    scan.addFamily(family);
-    Collection<StoreFile> storefiles2 = store.getStorefiles();
-    ArrayList<StoreFile> actualStorefiles = Lists.newArrayList(storefiles2);
-    StoreScanner storeScanner =
-        (StoreScanner) store.getScanner(scan, scan.getFamilyMap().get(family), Long.MAX_VALUE);
-    // get the current heap
-    KeyValueHeap heap = storeScanner.heap;
-    // create more store files
-    for (int i = 31; i < 40; i++) {
-      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
-        memStoreSize);
-    }
-    // flush them
-    flushStore(store, seqID);
-
-    for (int i = 41; i < 50; i++) {
-      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
-        memStoreSize);
-    }
-    // flush them
-    flushStore(store, seqID);
-    storefiles2 = store.getStorefiles();
-    ArrayList<StoreFile> actualStorefiles1 = Lists.newArrayList(storefiles2);
-    actualStorefiles1.removeAll(actualStorefiles);
-    // Do compaction
-    List<Exception> exceptions = new ArrayList<Exception>();
-    MyThread thread = new MyThread(storeScanner);
-    thread.start();
-    store.replaceStoreFiles(actualStorefiles, actualStorefiles1);
-    thread.join();
-    KeyValueHeap heap2 = thread.getHeap();
-    assertFalse(heap.equals(heap2));
-  }
-
-  private static class MyThread extends Thread {
-    private StoreScanner scanner;
-    private KeyValueHeap heap;
-
-    public MyThread(StoreScanner scanner) {
-      this.scanner = scanner;
-    }
-
-    public KeyValueHeap getHeap() {
-      return this.heap;
-    }
-
-    public void run() {
-      scanner.trySwitchToStreamRead();
-      heap = scanner.heap;
-    }
-  }
-
-  private static class MyMemStoreCompactor extends MemStoreCompactor {
-    private static final AtomicInteger RUNNER_COUNT = new AtomicInteger(0);
-    private static final CountDownLatch START_COMPACTOR_LATCH = new CountDownLatch(1);
-    public MyMemStoreCompactor(CompactingMemStore compactingMemStore, MemoryCompactionPolicy compactionPolicy) {
-      super(compactingMemStore, compactionPolicy);
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      boolean isFirst = RUNNER_COUNT.getAndIncrement() == 0;
-      boolean rval = super.start();
-      if (isFirst) {
-        try {
-          START_COMPACTOR_LATCH.await();
-        } catch (InterruptedException ex) {
-          throw new RuntimeException(ex);
-        }
-      }
-      return rval;
-    }
-  }
-
-  public static class MyCompactingMemStoreWithCustomCompactor extends CompactingMemStore {
-    private static final AtomicInteger RUNNER_COUNT = new AtomicInteger(0);
-    public MyCompactingMemStoreWithCustomCompactor(Configuration conf, CellComparator c,
-        HStore store, RegionServicesForStores regionServices,
-        MemoryCompactionPolicy compactionPolicy) throws IOException {
-      super(conf, c, store, regionServices, compactionPolicy);
-    }
-
-    @Override
-    protected MemStoreCompactor createMemStoreCompactor(MemoryCompactionPolicy compactionPolicy) {
-      return new MyMemStoreCompactor(this, compactionPolicy);
-    }
-
-    @Override
-    protected boolean shouldFlushInMemory() {
-      boolean rval = super.shouldFlushInMemory();
-      if (rval) {
-        RUNNER_COUNT.incrementAndGet();
-      }
-      return rval;
-    }
-  }
-
-  public static class MyCompactingMemStore extends CompactingMemStore {
-    private static final AtomicBoolean START_TEST = new AtomicBoolean(false);
-    private final CountDownLatch getScannerLatch = new CountDownLatch(1);
-    private final CountDownLatch snapshotLatch = new CountDownLatch(1);
-    public MyCompactingMemStore(Configuration conf, CellComparator c,
-        HStore store, RegionServicesForStores regionServices,
-        MemoryCompactionPolicy compactionPolicy) throws IOException {
-      super(conf, c, store, regionServices, compactionPolicy);
-    }
-
-    @Override
-    protected List<KeyValueScanner> createList(int capacity) {
-      if (START_TEST.get()) {
-        try {
-          getScannerLatch.countDown();
-          snapshotLatch.await();
-        } catch (InterruptedException e) {
-          throw new RuntimeException(e);
-        }
-      }
-      return new ArrayList<>(capacity);
-    }
-    @Override
-    protected void pushActiveToPipeline(MutableSegment active) {
-      if (START_TEST.get()) {
-        try {
-          getScannerLatch.await();
-        } catch (InterruptedException e) {
-          throw new RuntimeException(e);
-        }
-      }
-
-      super.pushActiveToPipeline(active);
-      if (START_TEST.get()) {
-        snapshotLatch.countDown();
-      }
-    }
-  }
-
-  interface MyListHook {
-    void hook(int currentSize);
-  }
-
-  private static class MyList<T> implements List<T> {
-    private final List<T> delegatee = new ArrayList<>();
-    private final MyListHook hookAtAdd;
-    MyList(final MyListHook hookAtAdd) {
-      this.hookAtAdd = hookAtAdd;
-    }
-    @Override
-    public int size() {return delegatee.size();}
-
-    @Override
-    public boolean isEmpty() {return delegatee.isEmpty();}
-
-    @Override
-    public boolean contains(Object o) {return delegatee.contains(o);}
-
-    @Override
-    public Iterator<T> iterator() {return delegatee.iterator();}
-
-    @Override
-    public Object[] toArray() {return delegatee.toArray();}
-
-    @Override
-    public <T> T[] toArray(T[] a) {return delegatee.toArray(a);}
-
-    @Override
-    public boolean add(T e) {
-      hookAtAdd.hook(size());
-      return delegatee.add(e);
-    }
-
-    @Override
-    public boolean remove(Object o) {return delegatee.remove(o);}
-
-    @Override
-    public boolean containsAll(Collection<?> c) {return delegatee.containsAll(c);}
-
-    @Override
-    public boolean addAll(Collection<? extends T> c) {return delegatee.addAll(c);}
-
-    @Override
-    public boolean addAll(int index, Collection<? extends T> c) {return delegatee.addAll(index, c);}
-
-    @Override
-    public boolean removeAll(Collection<?> c) {return delegatee.removeAll(c);}
-
-    @Override
-    public boolean retainAll(Collection<?> c) {return delegatee.retainAll(c);}
-
-    @Override
-    public void clear() {delegatee.clear();}
-
-    @Override
-    public T get(int index) {return delegatee.get(index);}
-
-    @Override
-    public T set(int index, T element) {return delegatee.set(index, element);}
-
-    @Override
-    public void add(int index, T element) {delegatee.add(index, element);}
-
-    @Override
-    public T remove(int index) {return delegatee.remove(index);}
-
-    @Override
-    public int indexOf(Object o) {return delegatee.indexOf(o);}
-
-    @Override
-    public int lastIndexOf(Object o) {return delegatee.lastIndexOf(o);}
-
-    @Override
-    public ListIterator<T> listIterator() {return delegatee.listIterator();}
-
-    @Override
-    public ListIterator<T> listIterator(int index) {return delegatee.listIterator(index);}
-
-    @Override
-    public List<T> subList(int fromIndex, int toIndex) {return delegatee.subList(fromIndex, toIndex);}
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
index 8c460a6..a562af8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
@@ -82,8 +82,8 @@ public class TestStripeStoreEngine {
         .thenReturn(new ArrayList<>());
 
     // Produce 3 L0 files.
-    StoreFile sf = createFile();
-    ArrayList<StoreFile> compactUs = al(sf, createFile(), createFile());
+    HStoreFile sf = createFile();
+    ArrayList<HStoreFile> compactUs = al(sf, createFile(), createFile());
     se.getStoreFileManager().loadFiles(compactUs);
     // Create a compaction that would want to split the stripe.
     CompactionContext compaction = se.createCompaction();
@@ -103,8 +103,8 @@ public class TestStripeStoreEngine {
       NoLimitThroughputController.INSTANCE, null);
   }
 
-  private static StoreFile createFile() throws Exception {
-    StoreFile sf = mock(StoreFile.class);
+  private static HStoreFile createFile() throws Exception {
+    HStoreFile sf = mock(HStoreFile.class);
     when(sf.getMetadataValue(any(byte[].class)))
       .thenReturn(StripeStoreFileManager.INVALID_KEY);
     when(sf.getReader()).thenReturn(mock(StoreFileReader.class));
@@ -114,12 +114,12 @@ public class TestStripeStoreEngine {
   }
 
   private static TestStoreEngine createEngine(Configuration conf) throws Exception {
-    Store store = mock(Store.class);
+    HStore store = mock(HStore.class);
     CellComparator kvComparator = mock(CellComparator.class);
     return (TestStoreEngine)StoreEngine.create(store, conf, kvComparator);
   }
 
-  private static ArrayList<StoreFile> al(StoreFile... sfs) {
+  private static ArrayList<HStoreFile> al(HStoreFile... sfs) {
     return new ArrayList<>(Arrays.asList(sfs));
   }
 }


[11/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a5f84430
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a5f84430
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a5f84430

Branch: refs/heads/master
Commit: a5f84430a305db6a1eddd45f82d19babf43a8d01
Parents: 1540483
Author: zhangduo <zh...@apache.org>
Authored: Sun Sep 24 19:22:16 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Sep 25 09:35:39 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/backup/util/RestoreTool.java   |    4 +-
 .../example/ZooKeeperScanPolicyObserver.java    |   68 +-
 .../hbase/mapreduce/HFileOutputFormat2.java     |   30 +-
 .../hbase/regionserver/CompactionTool.java      |    6 +-
 .../hbase/mapreduce/TestHFileOutputFormat2.java |    4 +-
 .../hadoop/hbase/backup/HFileArchiver.java      |   30 +-
 .../hbase/coprocessor/RegionObserver.java       |    4 +-
 .../hadoop/hbase/io/HalfStoreFileReader.java    |   23 +-
 .../org/apache/hadoop/hbase/io/hfile/HFile.java |   11 +-
 .../hbase/io/hfile/HFilePrettyPrinter.java      |    2 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  |   36 +-
 .../assignment/SplitTableRegionProcedure.java   |   31 +-
 .../master/balancer/StochasticLoadBalancer.java |    5 +-
 .../apache/hadoop/hbase/mob/CachedMobFile.java  |    7 +-
 .../hbase/mob/DefaultMobStoreCompactor.java     |    2 +-
 .../hbase/mob/DefaultMobStoreFlusher.java       |    9 +-
 .../org/apache/hadoop/hbase/mob/MobFile.java    |   15 +-
 .../apache/hadoop/hbase/mob/MobStoreEngine.java |    8 +-
 .../org/apache/hadoop/hbase/mob/MobUtils.java   |   11 +-
 .../PartitionedMobCompactionRequest.java        |   13 +-
 .../compactions/PartitionedMobCompactor.java    |   58 +-
 .../regionserver/ChangedReadersObserver.java    |    4 +-
 .../hbase/regionserver/CompactingMemStore.java  |    4 +-
 .../regionserver/DateTieredStoreEngine.java     |    8 +-
 .../hbase/regionserver/DefaultStoreEngine.java  |   20 +-
 .../regionserver/DefaultStoreFileManager.java   |  155 +-
 .../hbase/regionserver/DefaultStoreFlusher.java |    2 +-
 .../hadoop/hbase/regionserver/HMobStore.java    |    8 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   90 +-
 .../hbase/regionserver/HRegionFileSystem.java   |   44 +-
 .../hbase/regionserver/HRegionServer.java       |   78 +-
 .../hadoop/hbase/regionserver/HStore.java       |  524 +++---
 .../hadoop/hbase/regionserver/HStoreFile.java   |  170 +-
 .../hbase/regionserver/MemStoreCompactor.java   |   31 +-
 .../MemStoreCompactorSegmentsIterator.java      |    8 +-
 .../hbase/regionserver/MobStoreScanner.java     |    4 +-
 .../regionserver/RegionCoprocessorHost.java     |   54 +-
 .../hbase/regionserver/RegionSplitPolicy.java   |   15 +-
 .../regionserver/ReversedMobStoreScanner.java   |    4 +-
 .../regionserver/ReversedStoreScanner.java      |    4 +-
 .../apache/hadoop/hbase/regionserver/Store.java |  121 +-
 .../hadoop/hbase/regionserver/StoreEngine.java  |   16 +-
 .../hadoop/hbase/regionserver/StoreFile.java    |  126 +-
 .../regionserver/StoreFileComparators.java      |   28 +-
 .../hbase/regionserver/StoreFileManager.java    |   42 +-
 .../hbase/regionserver/StoreFileReader.java     |   46 +-
 .../hbase/regionserver/StoreFileScanner.java    |   24 +-
 .../hbase/regionserver/StoreFileWriter.java     |   42 +-
 .../hadoop/hbase/regionserver/StoreFlusher.java |    6 +-
 .../hadoop/hbase/regionserver/StoreScanner.java |   29 +-
 .../hadoop/hbase/regionserver/StoreUtils.java   |   86 +-
 .../hbase/regionserver/StripeStoreEngine.java   |    8 +-
 .../regionserver/StripeStoreFileManager.java    |  225 +--
 .../hbase/regionserver/StripeStoreFlusher.java  |    6 +-
 .../AbstractMultiOutputCompactor.java           |    6 +-
 .../compactions/CompactionContext.java          |   11 +-
 .../compactions/CompactionPolicy.java           |    8 +-
 .../compactions/CompactionRequest.java          |   12 +-
 .../regionserver/compactions/Compactor.java     |   26 +-
 .../compactions/DateTieredCompactionPolicy.java |   38 +-
 .../DateTieredCompactionRequest.java            |    4 +-
 .../compactions/DateTieredCompactor.java        |    6 +-
 .../compactions/DefaultCompactor.java           |   10 +-
 .../compactions/ExploringCompactionPolicy.java  |   40 +-
 .../compactions/FIFOCompactionPolicy.java       |   29 +-
 .../compactions/RatioBasedCompactionPolicy.java |   18 +-
 .../compactions/SortedCompactionPolicy.java     |   60 +-
 .../compactions/StripeCompactionPolicy.java     |   92 +-
 .../compactions/StripeCompactor.java            |    6 +-
 .../hadoop/hbase/snapshot/SnapshotManifest.java |   21 +-
 .../hbase/tool/LoadIncrementalHFiles.java       |   25 +-
 .../hadoop/hbase/util/BloomFilterFactory.java   |   14 +-
 .../org/apache/hadoop/hbase/util/HBaseFsck.java |   17 +-
 .../hadoop/hbase/util/RowBloomContext.java      |    7 +-
 .../hadoop/hbase/util/RowColBloomContext.java   |    7 +-
 .../hbase-webapps/regionserver/region.jsp       |    2 +-
 .../org/apache/hadoop/hbase/TestIOFencing.java  |   12 +-
 .../apache/hadoop/hbase/client/TestAdmin1.java  |   17 +-
 ...estAvoidCellReferencesIntoShippedBlocks.java |   13 +-
 .../hadoop/hbase/client/TestFromClientSide.java |    7 +-
 .../hbase/coprocessor/SimpleRegionObserver.java |   15 +-
 .../TestRegionObserverScannerOpenHook.java      |    3 +-
 .../hbase/io/TestHalfStoreFileReader.java       |    4 +-
 .../apache/hadoop/hbase/io/hfile/TestHFile.java |    4 +-
 .../hbase/io/hfile/TestHFileBlockIndex.java     |    8 +-
 .../hadoop/hbase/io/hfile/TestHFileSeek.java    |    4 +-
 .../hbase/mob/compactions/TestMobCompactor.java |    6 +-
 .../TestPartitionedMobCompactor.java            |   74 +-
 .../hbase/quotas/SpaceQuotaHelperForTests.java  |   12 +-
 .../AbstractTestDateTieredCompactionPolicy.java |   24 +-
 .../regionserver/DataBlockEncodingTool.java     |    2 +-
 .../EncodedSeekPerformanceTest.java             |    8 +-
 .../hbase/regionserver/MockHStoreFile.java      |  229 +++
 .../hbase/regionserver/MockStoreFile.java       |  220 ---
 .../regionserver/NoOpScanPolicyObserver.java    |   15 +-
 .../regionserver/TestCacheOnWriteInSchema.java  |    2 +-
 .../hbase/regionserver/TestCompaction.java      |   41 +-
 .../TestCompactionArchiveConcurrentClose.java   |   15 +-
 .../TestCompactionArchiveIOException.java       |   18 +-
 .../regionserver/TestCompactionPolicy.java      |   26 +-
 .../regionserver/TestCompoundBloomFilter.java   |    2 +-
 .../TestDefaultCompactSelection.java            |   10 +-
 .../regionserver/TestDefaultStoreEngine.java    |   10 +-
 .../regionserver/TestEncryptionKeyRotation.java |   24 +-
 .../TestEncryptionRandomKeying.java             |    4 +-
 .../hbase/regionserver/TestFSErrorsExposed.java |    6 +-
 .../hbase/regionserver/TestHMobStore.java       |    6 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  135 +-
 .../regionserver/TestHRegionReplayEvents.java   |    8 +-
 .../regionserver/TestHRegionServerBulkLoad.java |   16 +-
 .../hadoop/hbase/regionserver/TestHStore.java   | 1743 ++++++++++++++++++
 .../hbase/regionserver/TestHStoreFile.java      |   15 +-
 .../hbase/regionserver/TestMajorCompaction.java |    8 +-
 .../regionserver/TestMobStoreCompaction.java    |   16 +-
 .../hbase/regionserver/TestRegionReplicas.java  |   15 +-
 .../regionserver/TestRegionSplitPolicy.java     |   13 +-
 .../regionserver/TestReversibleScanners.java    |   45 +-
 .../regionserver/TestScannerWithBulkload.java   |    9 +-
 .../TestSplitTransactionOnCluster.java          |   17 +-
 .../hadoop/hbase/regionserver/TestStore.java    | 1740 -----------------
 .../regionserver/TestStripeStoreEngine.java     |   12 +-
 .../TestStripeStoreFileManager.java             |  155 +-
 .../regionserver/TestSwitchToStreamRead.java    |   11 +-
 .../ConstantSizeFileListGenerator.java          |   11 +-
 .../compactions/EverythingPolicy.java           |   12 +-
 .../compactions/ExplicitFileListGenerator.java  |   11 +-
 .../compactions/GaussianFileListGenerator.java  |   11 +-
 .../compactions/MockStoreFileGenerator.java     |   24 +-
 .../compactions/PerfTestCompactionPolicies.java |   38 +-
 .../SemiConstantSizeFileListGenerator.java      |   11 +-
 .../SinusoidalFileListGenerator.java            |   13 +-
 .../compactions/SpikyFileListGenerator.java     |   11 +-
 .../compactions/StoreFileListGenerator.java     |    9 +-
 .../TestCompactedHFilesDischarger.java          |   63 +-
 .../regionserver/compactions/TestCompactor.java |    6 +-
 .../compactions/TestDateTieredCompactor.java    |   16 +-
 .../compactions/TestStripeCompactionPolicy.java |   89 +-
 .../compactions/TestStripeCompactor.java        |    8 +-
 .../regionserver/wal/AbstractTestWALReplay.java |   17 +-
 .../visibility/TestVisibilityLabels.java        |   10 +-
 .../apache/hadoop/hbase/util/HFileTestUtil.java |    4 +-
 .../hbase/util/TestCoprocessorScanPolicy.java   |   13 +-
 .../hbase/util/TestHBaseFsckEncryption.java     |   13 +-
 .../hadoop/hbase/spark/HBaseContext.scala       |   10 +-
 144 files changed, 3920 insertions(+), 4147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
index ca0d026..b00351b 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java
@@ -440,8 +440,8 @@ public class RestoreTool {
           final byte[] first, last;
           try {
             reader.loadFileInfo();
-            first = reader.getFirstRowKey();
-            last = reader.getLastRowKey();
+            first = reader.getFirstRowKey().get();
+            last = reader.getLastRowKey().get();
             LOG.debug("Trying to figure out region boundaries hfile=" + hfile + " first="
                 + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
index 6b31664..a730403 100644
--- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
-import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.zookeeper.KeeperException;
@@ -130,32 +129,32 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
 
     @Override
     public void process(WatchedEvent event) {
-      switch(event.getType()) {
-      case NodeDataChanged:
-      case NodeCreated:
-      try {
-        // get data and re-watch
-        data = zk.getData(node, this, null);
-        LOG.debug("Read asynchronously: "+(data == null ? "null" : Bytes.toLong(data)));
-      } catch (InterruptedException ix) {
-      } catch (KeeperException kx) {
-        needSetup = true;
-      }
-      break;
+      switch (event.getType()) {
+        case NodeDataChanged:
+        case NodeCreated:
+          try {
+            // get data and re-watch
+            data = zk.getData(node, this, null);
+            LOG.debug("Read asynchronously: " + (data == null ? "null" : Bytes.toLong(data)));
+          } catch (InterruptedException ix) {
+          } catch (KeeperException kx) {
+            needSetup = true;
+          }
+          break;
 
-      case NodeDeleted:
-      try {
-        // just re-watch
-        zk.exists(node, this);
-        data = null;
-      } catch (InterruptedException ix) {
-      } catch (KeeperException kx) {
-        needSetup = true;
-      }
-      break;
+        case NodeDeleted:
+          try {
+            // just re-watch
+            zk.exists(node, this);
+            data = null;
+          } catch (InterruptedException ix) {
+          } catch (KeeperException kx) {
+            needSetup = true;
+          }
+          break;
 
-      default:
-        // ignore
+        default:
+          // ignore
       }
     }
   }
@@ -166,15 +165,13 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
     if (!re.getSharedData().containsKey(zkkey)) {
       // there is a short race here
       // in the worst case we create a watcher that will be notified once
-      re.getSharedData().putIfAbsent(
-          zkkey,
-          new ZKWatcher(re.getRegionServerServices().getZooKeeper()
-              .getRecoverableZooKeeper().getZooKeeper()));
+      re.getSharedData().putIfAbsent(zkkey, new ZKWatcher(
+          re.getRegionServerServices().getZooKeeper().getRecoverableZooKeeper().getZooKeeper()));
     }
   }
 
   protected ScanInfo getScanInfo(Store store, RegionCoprocessorEnvironment e) {
-    byte[] data = ((ZKWatcher)e.getSharedData().get(zkkey)).getData();
+    byte[] data = ((ZKWatcher) e.getSharedData().get(zkkey)).getData();
     if (data == null) {
       return null;
     }
@@ -182,8 +179,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
     if (oldSI.getTtl() == Long.MAX_VALUE) {
       return null;
     }
-    long ttl = Math.max(EnvironmentEdgeManager.currentTime() -
-        Bytes.toLong(data), oldSI.getTtl());
+    long ttl = Math.max(EnvironmentEdgeManager.currentTime() - Bytes.toLong(data), oldSI.getTtl());
     return new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), ttl,
         oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
   }
@@ -197,7 +193,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
       // take default action
       return null;
     }
-    return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners,
+    return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners,
         ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP);
   }
 
@@ -210,7 +206,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
       // take default action
       return null;
     }
-    return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, scanType,
+    return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, scanType,
         store.getSmallestReadPoint(), earliestPutTs);
   }
 
@@ -223,7 +219,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
       // take default action
       return null;
     }
-    return new StoreScanner(store, scanInfo, scan, targetCols,
-      ((HStore)store).getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED));
+    return new StoreScanner((HStore) store, scanInfo, scan, targetCols,
+        ((HStore) store).getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
index 41a9839..e8b7d11 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TASK_KEY;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
+
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.net.InetSocketAddress;
@@ -43,8 +48,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+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.Connection;
@@ -52,10 +61,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@@ -65,13 +72,9 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStore;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -85,6 +88,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
@@ -413,13 +417,13 @@ public class HFileOutputFormat2
 
       private void close(final StoreFileWriter w) throws IOException {
         if (w != null) {
-          w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY,
+          w.appendFileInfo(BULKLOAD_TIME_KEY,
               Bytes.toBytes(System.currentTimeMillis()));
-          w.appendFileInfo(StoreFile.BULKLOAD_TASK_KEY,
+          w.appendFileInfo(BULKLOAD_TASK_KEY,
               Bytes.toBytes(context.getTaskAttemptID().toString()));
-          w.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY,
+          w.appendFileInfo(MAJOR_COMPACTION_KEY,
               Bytes.toBytes(true));
-          w.appendFileInfo(StoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY,
+          w.appendFileInfo(EXCLUDE_FROM_MINOR_COMPACTION_KEY,
               Bytes.toBytes(compactionExclude));
           w.appendTrackedTimestampsToMetadata();
           w.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
index bb01459..7912340 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.mapreduce.JobUtil;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
@@ -62,6 +61,7 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /*
  * The CompactionTool allows to execute a compaction specifying a:
@@ -165,11 +165,11 @@ public class CompactionTool extends Configured implements Tool {
         if (!compaction.isPresent()) {
           break;
         }
-        List<StoreFile> storeFiles =
+        List<HStoreFile> storeFiles =
             store.compact(compaction.get(), NoLimitThroughputController.INSTANCE);
         if (storeFiles != null && !storeFiles.isEmpty()) {
           if (keepCompactedFiles && deleteCompacted) {
-            for (StoreFile storeFile: storeFiles) {
+            for (HStoreFile storeFile: storeFiles) {
               fs.delete(storeFile.getPath(), false);
             }
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index cbff2de..0b5a929 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.mapreduce;
 
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -85,7 +86,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -1162,7 +1162,7 @@ public class TestHFileOutputFormat2  {
         Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf), true, conf);
         Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
 
-        byte[] bloomFilter = fileInfo.get(StoreFile.BLOOM_FILTER_TYPE_KEY);
+        byte[] bloomFilter = fileInfo.get(BLOOM_FILTER_TYPE_KEY);
         if (bloomFilter == null) bloomFilter = Bytes.toBytes("NONE");
         assertEquals("Incorrect bloom filter used for column family " + familyStr +
           "(reader: " + reader + ")",

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
index d17546e..4321dc8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java
@@ -33,14 +33,14 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.io.MultipleIOException;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Function;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
@@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Collections2;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
 /**
- * Utility class to handle the removal of HFiles (or the respective {@link StoreFile StoreFiles})
+ * Utility class to handle the removal of HFiles (or the respective {@link HStoreFile StoreFiles})
  * for a HRegion from the {@link FileSystem}. The hfiles will be archived or deleted, depending on
  * the state of the system.
  */
@@ -226,7 +226,7 @@ public class HFileArchiver {
    * @throws IOException if the files could not be correctly disposed.
    */
   public static void archiveStoreFiles(Configuration conf, FileSystem fs, HRegionInfo regionInfo,
-      Path tableDir, byte[] family, Collection<StoreFile> compactedFiles)
+      Path tableDir, byte[] family, Collection<HStoreFile> compactedFiles)
       throws IOException, FailedArchiveException {
 
     // sometimes in testing, we don't have rss, so we need to check for that
@@ -479,13 +479,13 @@ public class HFileArchiver {
    * @throws IOException if a file cannot be deleted. All files will be attempted to deleted before
    *           throwing the exception, rather than failing at the first file.
    */
-  private static void deleteStoreFilesWithoutArchiving(Collection<StoreFile> compactedFiles)
+  private static void deleteStoreFilesWithoutArchiving(Collection<HStoreFile> compactedFiles)
       throws IOException {
     LOG.debug("Deleting store files without archiving.");
     List<IOException> errors = new ArrayList<>(0);
-    for (StoreFile hsf : compactedFiles) {
+    for (HStoreFile hsf : compactedFiles) {
       try {
-        hsf.deleteReader();
+        hsf.deleteStoreFile();
       } catch (IOException e) {
         LOG.error("Failed to delete store file:" + hsf.getPath());
         errors.add(e);
@@ -524,16 +524,16 @@ public class HFileArchiver {
   }
 
   /**
-   * Convert the {@link StoreFile} into something we can manage in the archive
+   * Convert the {@link HStoreFile} into something we can manage in the archive
    * methods
    */
-  private static class StoreToFile extends FileConverter<StoreFile> {
+  private static class StoreToFile extends FileConverter<HStoreFile> {
     public StoreToFile(FileSystem fs) {
       super(fs);
     }
 
     @Override
-    public File apply(StoreFile input) {
+    public File apply(HStoreFile input) {
       return new FileableStoreFile(fs, input);
     }
   }
@@ -656,20 +656,20 @@ public class HFileArchiver {
   }
 
   /**
-   * {@link File} adapter for a {@link StoreFile} living on a {@link FileSystem}
+   * {@link File} adapter for a {@link HStoreFile} living on a {@link FileSystem}
    * .
    */
   private static class FileableStoreFile extends File {
-    StoreFile file;
+    HStoreFile file;
 
-    public FileableStoreFile(FileSystem fs, StoreFile store) {
+    public FileableStoreFile(FileSystem fs, HStoreFile store) {
       super(fs);
       this.file = store;
     }
 
     @Override
     public void delete() throws IOException {
-      file.deleteReader();
+      file.deleteStoreFile();
     }
 
     @Override
@@ -690,7 +690,7 @@ public class HFileArchiver {
 
     @Override
     public void close() throws IOException {
-      file.closeReader(true);
+      file.closeStoreFile(true);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index 9318b9a..60e5f40 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
@@ -189,7 +189,7 @@ public interface RegionObserver extends Coprocessor {
    * @param tracker tracker used to track the life cycle of a compaction
    */
   default void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      List<StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {}
+      List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {}
 
   /**
    * Called after the {@link StoreFile}s to compact have been selected from the available
@@ -200,7 +200,7 @@ public interface RegionObserver extends Coprocessor {
    * @param tracker tracker used to track the life cycle of a compaction
    */
   default void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
-      ImmutableList<StoreFile> selected, CompactionLifeCycleTracker tracker) {}
+      ImmutableList<? extends StoreFile> selected, CompactionLifeCycleTracker tracker) {}
 
   /**
    * Called prior to writing the {@link StoreFile}s selected for compaction into a new

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
index 0c915d1..18ddb6a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.io;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
@@ -48,7 +49,7 @@ import org.apache.hadoop.hbase.util.Bytes;
  * <p>This type works in tandem with the {@link Reference} type.  This class
  * is used reading while Reference is used writing.
  *
- * <p>This file is not splitable.  Calls to {@link #midkey()} return null.
+ * <p>This file is not splitable.  Calls to {@link #midKey()} return null.
  */
 @InterfaceAudience.Private
 public class HalfStoreFileReader extends StoreFileReader {
@@ -60,7 +61,7 @@ public class HalfStoreFileReader extends StoreFileReader {
 
   protected final Cell splitCell;
 
-  private Cell firstKey = null;
+  private Optional<Cell> firstKey = null;
 
   private boolean firstKeySeeked = false;
 
@@ -258,8 +259,8 @@ public class HalfStoreFileReader extends StoreFileReader {
       @Override
       public boolean seekBefore(Cell key) throws IOException {
         if (top) {
-          Cell fk = getFirstKey();
-          if (getComparator().compareKeyIgnoresMvcc(key, fk) <= 0) {
+          Optional<Cell> fk = getFirstKey();
+          if (getComparator().compareKeyIgnoresMvcc(key, fk.get()) <= 0) {
             return false;
           }
         } else {
@@ -303,7 +304,7 @@ public class HalfStoreFileReader extends StoreFileReader {
   }
   
   @Override
-  public Cell getLastKey() {
+  public Optional<Cell> getLastKey() {
     if (top) {
       return super.getLastKey();
     }
@@ -311,7 +312,7 @@ public class HalfStoreFileReader extends StoreFileReader {
     HFileScanner scanner = getScanner(true, true);
     try {
       if (scanner.seekBefore(this.splitCell)) {
-        return scanner.getKey();
+        return Optional.ofNullable(scanner.getKey());
       }
     } catch (IOException e) {
       LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
@@ -320,22 +321,22 @@ public class HalfStoreFileReader extends StoreFileReader {
         scanner.close();
       }
     }
-    return null;
+    return Optional.empty();
   }
 
   @Override
-  public Cell midkey() throws IOException {
+  public Optional<Cell> midKey() throws IOException {
     // Returns null to indicate file is not splitable.
-    return null;
+    return Optional.empty();
   }
 
   @Override
-  public Cell getFirstKey() {
+  public Optional<Cell> getFirstKey() {
     if (!firstKeySeeked) {
       HFileScanner scanner = getScanner(true, true, false);
       try {
         if (scanner.seekTo()) {
-          this.firstKey = scanner.getKey();
+          this.firstKey = Optional.ofNullable(scanner.getKey());
         }
         firstKeySeeked = true;
       } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index 3d6cdaf..9cf00b3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -31,6 +31,7 @@ import java.util.Collection;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -434,21 +435,21 @@ public class HFile {
 
     Map<byte[], byte[]> loadFileInfo() throws IOException;
 
-    Cell getLastKey();
+    Optional<Cell> getLastKey();
 
-    Cell midkey() throws IOException;
+    Optional<Cell> midKey() throws IOException;
 
     long length();
 
     long getEntries();
 
-    Cell getFirstKey();
+    Optional<Cell> getFirstKey();
 
     long indexSize();
 
-    byte[] getFirstRowKey();
+    Optional<byte[]> getFirstRowKey();
 
-    byte[] getLastRowKey();
+    Optional<byte[]> getLastRowKey();
 
     FixedFileTrailer getTrailer();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
index dcfffb8..4524350 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
@@ -544,7 +544,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
     }
 
     try {
-      out.println("Mid-key: " + (CellUtil.getCellKeyAsString(reader.midkey())));
+      out.println("Mid-key: " + reader.midKey().map(CellUtil::getCellKeyAsString));
     } catch (Exception e) {
       out.println ("Unable to retrieve the midkey");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index bf722de..381279a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import java.security.Key;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Optional;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
@@ -338,14 +339,12 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
     }
   }
 
-  private String toStringFirstKey() {
-    if(getFirstKey() == null)
-      return null;
-    return CellUtil.getCellKeyAsString(getFirstKey());
+  private Optional<String> toStringFirstKey() {
+    return getFirstKey().map(CellUtil::getCellKeyAsString);
   }
 
-  private String toStringLastKey() {
-    return CellUtil.toString(getLastKey(), false);
+  private Optional<String> toStringLastKey() {
+    return getLastKey().map(CellUtil::getCellKeyAsString);
   }
 
   @Override
@@ -382,12 +381,12 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    *         first KeyValue.
    */
   @Override
-  public Cell getFirstKey() {
+  public Optional<Cell> getFirstKey() {
     if (dataBlockIndexReader == null) {
       throw new BlockIndexNotLoadedException();
     }
-    return dataBlockIndexReader.isEmpty() ? null
-        : dataBlockIndexReader.getRootBlockKey(0);
+    return dataBlockIndexReader.isEmpty() ? Optional.empty()
+        : Optional.of(dataBlockIndexReader.getRootBlockKey(0));
   }
 
   /**
@@ -397,10 +396,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    * @return the first row key, or null if the file is empty.
    */
   @Override
-  public byte[] getFirstRowKey() {
-    Cell firstKey = getFirstKey();
+  public Optional<byte[]> getFirstRowKey() {
     // We have to copy the row part to form the row key alone
-    return firstKey == null? null: CellUtil.cloneRow(firstKey);
+    return getFirstKey().map(CellUtil::cloneRow);
   }
 
   /**
@@ -410,9 +408,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    * @return the last row key, or null if the file is empty.
    */
   @Override
-  public byte[] getLastRowKey() {
-    Cell lastKey = getLastKey();
-    return lastKey == null? null: CellUtil.cloneRow(lastKey);
+  public Optional<byte[]> getLastRowKey() {
+    // We have to copy the row part to form the row key alone
+    return getLastKey().map(CellUtil::cloneRow);
   }
 
   /** @return number of KV entries in this HFile */
@@ -1550,8 +1548,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    *         key
    */
   @Override
-  public Cell getLastKey() {
-    return dataBlockIndexReader.isEmpty() ? null : lastKeyCell;
+  public Optional<Cell> getLastKey() {
+    return dataBlockIndexReader.isEmpty() ? Optional.empty() : Optional.of(lastKeyCell);
   }
 
   /**
@@ -1560,8 +1558,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    * @throws IOException
    */
   @Override
-  public Cell midkey() throws IOException {
-    return dataBlockIndexReader.midkey();
+  public Optional<Cell> midKey() throws IOException {
+    return Optional.ofNullable(dataBlockIndexReader.midkey());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 98cd16d..f67aa5b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.hbase.master.assignment;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
@@ -41,7 +43,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MasterSwitchType;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@@ -58,19 +60,21 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
 import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.quotas.QuotaExceededException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
 import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
 
 /**
  * The procedure to split a region in a table.
@@ -636,8 +640,8 @@ public class SplitTableRegionProcedure
     }
   }
 
-  private Pair<Path, Path> splitStoreFile(final HRegionFileSystem regionFs,
-      final byte[] family, final StoreFile sf) throws IOException {
+  private Pair<Path, Path> splitStoreFile(HRegionFileSystem regionFs, byte[] family, HStoreFile sf)
+      throws IOException {
     if (LOG.isDebugEnabled()) {
       LOG.debug("pid=" + getProcId() + " splitting started for store file: " +
           sf.getPath() + " for region: " + getParentRegion().getShortNameToLog());
@@ -663,7 +667,7 @@ public class SplitTableRegionProcedure
   private class StoreFileSplitter implements Callable<Pair<Path,Path>> {
     private final HRegionFileSystem regionFs;
     private final byte[] family;
-    private final StoreFile sf;
+    private final HStoreFile sf;
 
     /**
      * Constructor that takes what it needs to split
@@ -671,8 +675,7 @@ public class SplitTableRegionProcedure
      * @param family Family that contains the store file
      * @param sf which file
      */
-    public StoreFileSplitter(final HRegionFileSystem regionFs, final byte[] family,
-        final StoreFile sf) {
+    public StoreFileSplitter(HRegionFileSystem regionFs, byte[] family, HStoreFile sf) {
       this.regionFs = regionFs;
       this.sf = sf;
       this.family = family;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index c1fef92..957c182 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.master.balancer;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -42,7 +43,6 @@ import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.MoveRegi
 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.SwapRegionsAction;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Optional;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
@@ -1241,7 +1242,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
 
   /**
    * Compute a cost of a potential cluster configuration based upon where
-   * {@link org.apache.hadoop.hbase.regionserver.StoreFile}s are located.
+   * {@link org.apache.hadoop.hbase.regionserver.HStoreFile}s are located.
    */
   static abstract class LocalityBasedCostFunction extends CostFunction {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java
index 9dc32be..397570c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java
@@ -24,11 +24,10 @@ 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.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Cached mob file.
@@ -39,7 +38,7 @@ public class CachedMobFile extends MobFile implements Comparable<CachedMobFile>
   private long accessCount;
   private AtomicLong referenceCount = new AtomicLong(0);
 
-  public CachedMobFile(StoreFile sf) {
+  public CachedMobFile(HStoreFile sf) {
     super(sf);
   }
 
@@ -47,7 +46,7 @@ public class CachedMobFile extends MobFile implements Comparable<CachedMobFile>
       CacheConfig cacheConf) throws IOException {
     // XXX: primaryReplica is only used for constructing the key of block cache so it is not a
     // critical problem if we pass the wrong value, so here we always pass true. Need to fix later.
-    StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
+    HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
     return new CachedMobFile(sf);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java
----------------------------------------------------------------------
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 1badeb2..e8ada97 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
@@ -91,7 +91,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
         }
       };
 
-  public DefaultMobStoreCompactor(Configuration conf, Store store) {
+  public DefaultMobStoreCompactor(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

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java
----------------------------------------------------------------------
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 bb0301e..bef73f2 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
@@ -32,19 +32,19 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher;
 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.MemStoreSnapshot;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
-import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * An implementation of the StoreFlusher. It extends the DefaultStoreFlusher.
@@ -70,8 +70,11 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
   private Path targetPath;
   private HMobStore mobStore;
 
-  public DefaultMobStoreFlusher(Configuration conf, Store store) throws IOException {
+  public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException {
     super(conf, store);
+    if (!(store instanceof HMobStore)) {
+      throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
+    }
     mobCellValueSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold();
     this.targetPath = MobUtils.getMobFamilyPath(conf, store.getTableName(),
         store.getColumnFamilyName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java
index 929bfd2..91702c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java
@@ -26,12 +26,11 @@ 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.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HStoreFile;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * The mob file.
@@ -39,13 +38,13 @@ import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 @InterfaceAudience.Private
 public class MobFile {
 
-  private StoreFile sf;
+  private HStoreFile sf;
 
   // internal use only for sub classes
   protected MobFile() {
   }
 
-  protected MobFile(StoreFile sf) {
+  protected MobFile(HStoreFile sf) {
     this.sf = sf;
   }
 
@@ -56,7 +55,7 @@ public class MobFile {
    * @throws IOException
    */
   public StoreFileScanner getScanner() throws IOException {
-    List<StoreFile> sfs = new ArrayList<>();
+    List<HStoreFile> sfs = new ArrayList<>();
     sfs.add(sf);
     List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true,
         false, false, sf.getMaxMemstoreTS());
@@ -86,7 +85,7 @@ public class MobFile {
   public Cell readCell(Cell search, boolean cacheMobBlocks, long readPt) throws IOException {
     Cell result = null;
     StoreFileScanner scanner = null;
-    List<StoreFile> sfs = new ArrayList<>();
+    List<HStoreFile> sfs = new ArrayList<>();
     sfs.add(sf);
     try {
       List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(sfs,
@@ -129,7 +128,7 @@ public class MobFile {
    */
   public void close() throws IOException {
     if (sf != null) {
-      sf.closeReader(false);
+      sf.closeStoreFile(false);
       sf = null;
     }
   }
@@ -147,7 +146,7 @@ public class MobFile {
       throws IOException {
     // XXX: primaryReplica is only used for constructing the key of block cache so it is not a
     // critical problem if we pass the wrong value, so here we always pass true. Need to fix later.
-    StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
+    HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
     return new MobFile(sf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java
----------------------------------------------------------------------
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 bdd336f..ee1fe7d 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
@@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.mob;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
-import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * MobStoreEngine creates the mob specific compactor, and store flusher.
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.regionserver.Store;
 public class MobStoreEngine extends DefaultStoreEngine {
 
   @Override
-  protected void createStoreFlusher(Configuration conf, Store store) throws IOException {
+  protected void createStoreFlusher(Configuration conf, HStore store) throws IOException {
     // When using MOB, we use DefaultMobStoreFlusher always
     // Just use the compactor and compaction policy as that in DefaultStoreEngine. We can have MOB
     // specific compactor and policy when that is implemented.
@@ -42,7 +42,7 @@ public class MobStoreEngine extends DefaultStoreEngine {
    * Creates the DefaultMobCompactor.
    */
   @Override
-  protected void createCompactor(Configuration conf, Store store) throws IOException {
+  protected void createCompactor(Configuration conf, HStore store) throws IOException {
     compactor = new DefaultMobStoreCompactor(conf, store);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
----------------------------------------------------------------------
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 2592b72..54f1373 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
@@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
 import org.apache.hadoop.hbase.client.Scan;
@@ -70,7 +69,6 @@ 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;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
@@ -78,6 +76,7 @@ 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;
 
 /**
  * The mob utilities
@@ -315,7 +314,7 @@ public final class MobUtils {
       // no file found
       return;
     }
-    List<StoreFile> filesToClean = new ArrayList<>();
+    List<HStoreFile> filesToClean = new ArrayList<>();
     int deletedFileCount = 0;
     for (FileStatus file : stats) {
       String fileName = file.getPath().getName();
@@ -467,7 +466,7 @@ public final class MobUtils {
    * @throws IOException
    */
   public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName,
-      Path tableDir, byte[] family, Collection<StoreFile> storeFiles) throws IOException {
+      Path tableDir, byte[] family, Collection<HStoreFile> storeFiles) throws IOException {
     HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family,
         storeFiles);
   }
@@ -721,7 +720,7 @@ public final class MobUtils {
    */
   private static void validateMobFile(Configuration conf, FileSystem fs, Path path,
       CacheConfig cacheConfig, boolean primaryReplica) throws IOException {
-    StoreFile storeFile = null;
+    HStoreFile storeFile = null;
     try {
       storeFile = new HStoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica);
       storeFile.initReader();
@@ -730,7 +729,7 @@ public final class MobUtils {
       throw e;
     } finally {
       if (storeFile != null) {
-        storeFile.closeReader(false);
+        storeFile.closeStoreFile(false);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java
----------------------------------------------------------------------
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
index 7916779..aaf545b 100644
--- 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
@@ -25,10 +25,10 @@ import java.util.List;
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.regionserver.StoreFile;
+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
@@ -106,8 +106,7 @@ public class PartitionedMobCompactionRequest extends MobCompactionRequest {
      * Set start key of this partition, only if the input startKey is less than
      * the current start key.
      */
-    public void setStartKey(final byte[] startKey)
-    {
+    public void setStartKey(final byte[] startKey) {
       if ((this.startKey == null) || (Bytes.compareTo(startKey, this.startKey) < 0)) {
         this.startKey = startKey;
       }
@@ -227,7 +226,7 @@ public class PartitionedMobCompactionRequest extends MobCompactionRequest {
    */
   protected static class CompactionDelPartition {
     private List<Path> delFiles = new ArrayList<Path>();
-    private List<StoreFile> storeFiles = new ArrayList<>();
+    private List<HStoreFile> storeFiles = new ArrayList<>();
     private CompactionDelPartitionId id;
 
     public CompactionDelPartition(CompactionDelPartitionId id) {
@@ -241,11 +240,11 @@ public class PartitionedMobCompactionRequest extends MobCompactionRequest {
     void addDelFile(FileStatus file) {
       delFiles.add(file.getPath());
     }
-    public void addStoreFile(final StoreFile file) {
+    public void addStoreFile(HStoreFile file) {
       storeFiles.add(file);
     }
 
-    public List<StoreFile> getStoreFiles() {
+    public List<HStoreFile> getStoreFiles() {
       return storeFiles;
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
----------------------------------------------------------------------
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
index e40f3a7..1fc2902 100644
--- 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
@@ -18,6 +18,10 @@
  */
 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;
@@ -50,7 +54,6 @@ 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.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -75,7 +78,6 @@ 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.StoreFile;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
@@ -85,6 +87,7 @@ import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 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.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
@@ -225,8 +228,8 @@ public class PartitionedMobCompactor extends MobCompactor {
 
         // Get delId from the file
         try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) {
-          delId.setStartKey(reader.getFirstRowKey());
-          delId.setEndKey(reader.getLastRowKey());
+          delId.setStartKey(reader.getFirstRowKey().get());
+          delId.setEndKey(reader.getLastRowKey().get());
         }
         CompactionDelPartition delPartition = delFilesToCompact.get(delId);
         if (delPartition == null) {
@@ -266,8 +269,8 @@ public class PartitionedMobCompactor extends MobCompactor {
             // 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());
-              compactionPartition.setEndKey(reader.getLastRowKey());
+              compactionPartition.setStartKey(reader.getFirstRowKey().get());
+              compactionPartition.setEndKey(reader.getLastRowKey().get());
             }
           }
 
@@ -335,7 +338,7 @@ public class PartitionedMobCompactor extends MobCompactor {
     try {
       for (CompactionDelPartition delPartition : request.getDelPartitions()) {
         for (Path newDelPath : delPartition.listDelFiles()) {
-          StoreFile sf =
+          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.
@@ -361,7 +364,8 @@ public class PartitionedMobCompactor extends MobCompactor {
       for (CompactionDelPartition delPartition : request.getDelPartitions()) {
         LOG.info(delPartition.listDelFiles());
         try {
-          MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delPartition.getStoreFiles());
+          MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(),
+            delPartition.getStoreFiles());
         } catch (IOException e) {
           LOG.error("Failed to archive the del files " + delPartition.getStoreFiles(), e);
         }
@@ -397,11 +401,11 @@ public class PartitionedMobCompactor extends MobCompactor {
   }
 
   @VisibleForTesting
-  List<StoreFile> getListOfDelFilesForPartition(final CompactionPartition partition,
+  List<HStoreFile> getListOfDelFilesForPartition(final CompactionPartition partition,
       final List<CompactionDelPartition> delPartitions) {
     // Binary search for startKey and endKey
 
-    List<StoreFile> result = new ArrayList<>();
+    List<HStoreFile> result = new ArrayList<>();
 
     DelPartitionComparator comparator = new DelPartitionComparator(false);
     CompactionDelPartitionId id = new CompactionDelPartitionId(null, partition.getStartKey());
@@ -473,7 +477,7 @@ public class PartitionedMobCompactor extends MobCompactor {
         // 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<StoreFile> delFiles = getListOfDelFilesForPartition(partition,
+        List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition,
             request.getDelPartitions());
 
         results.put(partition.getPartitionId(), pool.submit(new Callable<List<Path>>() {
@@ -521,7 +525,7 @@ public class PartitionedMobCompactor extends MobCompactor {
    */
   private List<Path> compactMobFilePartition(PartitionedMobCompactionRequest request,
                                              CompactionPartition partition,
-                                             List<StoreFile> delFiles,
+                                             List<HStoreFile> delFiles,
                                              Connection connection,
                                              Table table) throws IOException {
     if (MobUtils.isMobFileExpired(column, EnvironmentEdgeManager.currentTime(),
@@ -550,9 +554,9 @@ public class PartitionedMobCompactor extends MobCompactor {
       // clean the bulkload directory to avoid loading old files.
       fs.delete(bulkloadPathOfPartition, true);
       // add the selected mob files and del files into filesToCompact
-      List<StoreFile> filesToCompact = new ArrayList<>();
+      List<HStoreFile> filesToCompact = new ArrayList<>();
       for (int i = offset; i < batch + offset; i++) {
-        StoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
+        HStoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
             BloomType.NONE, true);
         filesToCompact.add(sf);
       }
@@ -572,10 +576,10 @@ public class PartitionedMobCompactor extends MobCompactor {
    * Closes the readers of store files.
    * @param storeFiles The store files to be closed.
    */
-  private void closeStoreFileReaders(List<StoreFile> storeFiles) {
-    for (StoreFile storeFile : storeFiles) {
+  private void closeStoreFileReaders(List<HStoreFile> storeFiles) {
+    for (HStoreFile storeFile : storeFiles) {
       try {
-        storeFile.closeReader(true);
+        storeFile.closeStoreFile(true);
       } catch (IOException e) {
         LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
       }
@@ -600,14 +604,14 @@ public class PartitionedMobCompactor extends MobCompactor {
   private void compactMobFilesInBatch(PartitionedMobCompactionRequest request,
                                       CompactionPartition partition,
                                       Connection connection, Table table,
-                                      List<StoreFile> filesToCompact, int batch,
+                                      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<StoreFile> mobFilesToCompact = filesToCompact.subList(0, batch);
+    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.
@@ -726,7 +730,7 @@ public class PartitionedMobCompactor extends MobCompactor {
       if (delFilePaths.size() - offset < compactionBatchSize) {
         batch = delFilePaths.size() - offset;
       }
-      List<StoreFile> batchedDelFiles = new ArrayList<>();
+      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));
@@ -753,7 +757,7 @@ public class PartitionedMobCompactor extends MobCompactor {
    * @throws IOException if IO failure is encountered
    */
   private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request,
-    List<StoreFile> delFiles) throws IOException {
+    List<HStoreFile> delFiles) throws IOException {
     // create a scanner for the del files.
     StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES);
     StoreFileWriter writer = null;
@@ -803,7 +807,7 @@ public class PartitionedMobCompactor extends MobCompactor {
    * @return The store scanner.
    * @throws IOException if IO failure is encountered
    */
-  private StoreScanner createScanner(List<StoreFile> filesToCompact, ScanType scanType)
+  private StoreScanner createScanner(List<HStoreFile> filesToCompact, ScanType scanType)
       throws IOException {
     List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact,
       false, true, false, false, HConstants.LATEST_TIMESTAMP);
@@ -864,8 +868,8 @@ public class PartitionedMobCompactor extends MobCompactor {
     throws IOException {
     if (writer != null) {
       writer.appendMetadata(maxSeqId, false);
-      writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime));
-      writer.appendFileInfo(StoreFile.SKIP_RESET_SEQ_ID, Bytes.toBytes(true));
+      writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime));
+      writer.appendFileInfo(SKIP_RESET_SEQ_ID, Bytes.toBytes(true));
       try {
         writer.close();
       } catch (IOException e) {
@@ -880,14 +884,14 @@ public class PartitionedMobCompactor extends MobCompactor {
    * @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<StoreFile> storeFiles) throws IOException {
+  private Pair<Long, Long> getFileInfo(List<HStoreFile> storeFiles) throws IOException {
     long maxSeqId = 0;
     long maxKeyCount = 0;
-    for (StoreFile sf : storeFiles) {
+    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(StoreFile.MOB_CELLS_COUNT);
+      byte[] count = sf.getReader().loadFileInfo().get(MOB_CELLS_COUNT);
       if (count != null) {
         maxKeyCount += Bytes.toLong(count);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
index 07b72e2..065fd37 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java
@@ -16,11 +16,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.util.List;
+
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -40,5 +40,5 @@ public interface ChangedReadersObserver {
    * @param memStoreScanners scanner of current memstore
    * @throws IOException e
    */
-  void updateReaders(List<StoreFile> sfs, List<KeyValueScanner> memStoreScanners) throws IOException;
+  void updateReaders(List<HStoreFile> sfs, List<KeyValueScanner> memStoreScanners) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
index c31a1cf..da502c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
@@ -69,7 +69,7 @@ public class CompactingMemStore extends AbstractMemStore {
   private static final double IN_MEMORY_FLUSH_THRESHOLD_FACTOR_DEFAULT = 0.25;
 
   private static final Log LOG = LogFactory.getLog(CompactingMemStore.class);
-  private Store store;
+  private HStore store;
   private RegionServicesForStores regionServices;
   private CompactionPipeline pipeline;
   private MemStoreCompactor compactor;
@@ -341,7 +341,7 @@ public class CompactingMemStore extends AbstractMemStore {
     return store.getSmallestReadPoint();
   }
 
-  public Store getStore() {
+  public HStore getStore() {
     return store;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java
index 25e1609..f7c18f9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.security.User;
 public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
   DateTieredCompactionPolicy, DateTieredCompactor, DefaultStoreFileManager> {
   @Override
-  public boolean needsCompaction(List<StoreFile> filesCompacting) {
+  public boolean needsCompaction(List<HStoreFile> filesCompacting) {
     return compactionPolicy.needsCompaction(storeFileManager.getStorefiles(),
       filesCompacting);
   }
@@ -54,7 +54,7 @@ public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
   }
 
   @Override
-  protected void createComponents(Configuration conf, Store store, CellComparator kvComparator)
+  protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator)
       throws IOException {
     this.compactionPolicy = new DateTieredCompactionPolicy(conf, store);
     this.storeFileManager =
@@ -67,13 +67,13 @@ public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
   private final class DateTieredCompactionContext extends CompactionContext {
 
     @Override
-    public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
+    public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
       return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStorefiles(),
         filesCompacting);
     }
 
     @Override
-    public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
+    public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
         boolean mayUseOffPeak, boolean forceMajor) throws IOException {
       request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting,
         isUserCompaction, mayUseOffPeak, forceMajor);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
index 70421e1..58f8bbb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
 import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
@@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPoli
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Default StoreEngine creates the default compactor, policy, and store file manager, or
@@ -57,14 +57,14 @@ public class DefaultStoreEngine extends StoreEngine<
     DEFAULT_COMPACTION_POLICY_CLASS = ExploringCompactionPolicy.class;
 
   @Override
-  public boolean needsCompaction(List<StoreFile> filesCompacting) {
+  public boolean needsCompaction(List<HStoreFile> filesCompacting) {
     return compactionPolicy.needsCompaction(
         this.storeFileManager.getStorefiles(), filesCompacting);
   }
 
   @Override
   protected void createComponents(
-      Configuration conf, Store store, CellComparator kvComparator) throws IOException {
+      Configuration conf, HStore store, CellComparator kvComparator) throws IOException {
     createCompactor(conf, store);
     createCompactionPolicy(conf, store);
     createStoreFlusher(conf, store);
@@ -73,17 +73,17 @@ public class DefaultStoreEngine extends StoreEngine<
             compactionPolicy.getConf());
   }
 
-  protected void createCompactor(Configuration conf, Store store) throws IOException {
+  protected void createCompactor(Configuration conf, HStore store) throws IOException {
     String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
     try {
       compactor = ReflectionUtils.instantiateWithCustomCtor(className,
-          new Class[] { Configuration.class, Store.class }, new Object[] { conf, store });
+          new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
     } catch (Exception e) {
       throw new IOException("Unable to load configured compactor '" + className + "'", e);
     }
   }
 
-  protected void createCompactionPolicy(Configuration conf, Store store) throws IOException {
+  protected void createCompactionPolicy(Configuration conf, HStore store) throws IOException {
     String className = conf.get(
         DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
     try {
@@ -95,12 +95,12 @@ public class DefaultStoreEngine extends StoreEngine<
     }
   }
 
-  protected void createStoreFlusher(Configuration conf, Store store) throws IOException {
+  protected void createStoreFlusher(Configuration conf, HStore store) throws IOException {
     String className = conf.get(
         DEFAULT_STORE_FLUSHER_CLASS_KEY, DEFAULT_STORE_FLUSHER_CLASS.getName());
     try {
       storeFlusher = ReflectionUtils.instantiateWithCustomCtor(className,
-          new Class[] { Configuration.class, Store.class }, new Object[] { conf, store });
+          new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
     } catch (Exception e) {
       throw new IOException("Unable to load configured store flusher '" + className + "'", e);
     }
@@ -113,7 +113,7 @@ public class DefaultStoreEngine extends StoreEngine<
 
   private class DefaultCompactionContext extends CompactionContext {
     @Override
-    public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
+    public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
         boolean mayUseOffPeak, boolean forceMajor) throws IOException {
       request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(),
           filesCompacting, isUserCompaction, mayUseOffPeak, forceMajor);
@@ -127,7 +127,7 @@ public class DefaultStoreEngine extends StoreEngine<
     }
 
     @Override
-    public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
+    public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
       return compactionPolicy.preSelectCompactionForCoprocessor(
           storeFileManager.getStorefiles(), filesCompacting);
     }


[10/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
index 95945c6..e1f31bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java
@@ -19,13 +19,12 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Optional;
+import java.util.stream.Collectors;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -33,12 +32,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
 
 /**
  * Default implementation of StoreFileManager. Not thread-safe.
@@ -47,27 +46,27 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 class DefaultStoreFileManager implements StoreFileManager {
   private static final Log LOG = LogFactory.getLog(DefaultStoreFileManager.class);
 
-  private final CellComparator kvComparator;
+  private final CellComparator cellComparator;
   private final CompactionConfiguration comConf;
   private final int blockingFileCount;
-  private final Comparator<StoreFile> storeFileComparator;
+  private final Comparator<HStoreFile> storeFileComparator;
   /**
    * List of store files inside this store. This is an immutable list that
    * is atomically replaced when its contents change.
    */
-  private volatile ImmutableList<StoreFile> storefiles = null;
+  private volatile ImmutableList<HStoreFile> storefiles = ImmutableList.of();
   /**
    * List of compacted files inside this store that needs to be excluded in reads
    * because further new reads will be using only the newly created files out of compaction.
    * These compacted files will be deleted/cleared once all the existing readers on these
    * compacted files are done.
    */
-  private volatile List<StoreFile> compactedfiles = null;
+  private volatile ImmutableList<HStoreFile> compactedfiles = ImmutableList.of();
 
-  public DefaultStoreFileManager(CellComparator kvComparator,
-      Comparator<StoreFile> storeFileComparator, Configuration conf,
+  public DefaultStoreFileManager(CellComparator cellComparator,
+      Comparator<HStoreFile> storeFileComparator, Configuration conf,
       CompactionConfiguration comConf) {
-    this.kvComparator = kvComparator;
+    this.cellComparator = cellComparator;
     this.storeFileComparator = storeFileComparator;
     this.comConf = comConf;
     this.blockingFileCount =
@@ -75,39 +74,37 @@ class DefaultStoreFileManager implements StoreFileManager {
   }
 
   @Override
-  public void loadFiles(List<StoreFile> storeFiles) {
-    sortAndSetStoreFiles(storeFiles);
+  public void loadFiles(List<HStoreFile> storeFiles) {
+    this.storefiles = ImmutableList.sortedCopyOf(storeFileComparator, storeFiles);
   }
 
   @Override
-  public final Collection<StoreFile> getStorefiles() {
-    // TODO: I can return a null list of StoreFiles? That'll mess up clients. St.Ack 20151111
+  public final Collection<HStoreFile> getStorefiles() {
     return storefiles;
   }
 
   @Override
-  public Collection<StoreFile> getCompactedfiles() {
+  public Collection<HStoreFile> getCompactedfiles() {
     return compactedfiles;
   }
 
   @Override
-  public void insertNewFiles(Collection<StoreFile> sfs) throws IOException {
-    ArrayList<StoreFile> newFiles = new ArrayList<>(storefiles);
-    newFiles.addAll(sfs);
-    sortAndSetStoreFiles(newFiles);
+  public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
+    this.storefiles =
+        ImmutableList.sortedCopyOf(storeFileComparator, Iterables.concat(this.storefiles, sfs));
   }
 
   @Override
-  public ImmutableCollection<StoreFile> clearFiles() {
-    ImmutableList<StoreFile> result = storefiles;
+  public ImmutableCollection<HStoreFile> clearFiles() {
+    ImmutableList<HStoreFile> result = storefiles;
     storefiles = ImmutableList.of();
     return result;
   }
 
   @Override
-  public Collection<StoreFile> clearCompactedFiles() {
-    List<StoreFile> result = compactedfiles;
-    compactedfiles = new ArrayList<>();
+  public Collection<HStoreFile> clearCompactedFiles() {
+    List<HStoreFile> result = compactedfiles;
+    compactedfiles = ImmutableList.of();
     return result;
   }
 
@@ -118,60 +115,39 @@ class DefaultStoreFileManager implements StoreFileManager {
 
   @Override
   public final int getCompactedFilesCount() {
-    if (compactedfiles == null) {
-      return 0;
-    }
     return compactedfiles.size();
   }
 
   @Override
-  public void addCompactionResults(
-    Collection<StoreFile> newCompactedfiles, Collection<StoreFile> results) {
-    ArrayList<StoreFile> newStoreFiles = Lists.newArrayList(storefiles);
-    newStoreFiles.removeAll(newCompactedfiles);
-    if (!results.isEmpty()) {
-      newStoreFiles.addAll(results);
-    }
-    sortAndSetStoreFiles(newStoreFiles);
-    ArrayList<StoreFile> updatedCompactedfiles = null;
-    if (this.compactedfiles != null) {
-      updatedCompactedfiles = new ArrayList<>(this.compactedfiles);
-      updatedCompactedfiles.addAll(newCompactedfiles);
-    } else {
-      updatedCompactedfiles = new ArrayList<>(newCompactedfiles);
-    }
-    markCompactedAway(newCompactedfiles);
-    this.compactedfiles = sortCompactedfiles(updatedCompactedfiles);
-  }
-
-  // Mark the files as compactedAway once the storefiles and compactedfiles list is finalized
-  // Let a background thread close the actual reader on these compacted files and also
-  // ensure to evict the blocks from block cache so that they are no longer in
-  // cache
-  private void markCompactedAway(Collection<StoreFile> compactedFiles) {
-    for (StoreFile file : compactedFiles) {
-      file.markCompactedAway();
-    }
+  public void addCompactionResults(Collection<HStoreFile> newCompactedfiles,
+      Collection<HStoreFile> results) {
+    this.storefiles = ImmutableList.sortedCopyOf(storeFileComparator, Iterables
+        .concat(Iterables.filter(storefiles, sf -> !newCompactedfiles.contains(sf)), results));
+    // Mark the files as compactedAway once the storefiles and compactedfiles list is finalized
+    // Let a background thread close the actual reader on these compacted files and also
+    // ensure to evict the blocks from block cache so that they are no longer in
+    // cache
+    newCompactedfiles.forEach(HStoreFile::markCompactedAway);
+    this.compactedfiles = ImmutableList.sortedCopyOf(storeFileComparator,
+      Iterables.concat(this.compactedfiles, newCompactedfiles));
   }
 
   @Override
-  public void removeCompactedFiles(Collection<StoreFile> removedCompactedfiles) throws IOException {
-    ArrayList<StoreFile> updatedCompactedfiles = null;
-    if (this.compactedfiles != null) {
-      updatedCompactedfiles = new ArrayList<>(this.compactedfiles);
-      updatedCompactedfiles.removeAll(removedCompactedfiles);
-      this.compactedfiles = sortCompactedfiles(updatedCompactedfiles);
-    }
+  public void removeCompactedFiles(Collection<HStoreFile> removedCompactedfiles)
+      throws IOException {
+    this.compactedfiles =
+        this.compactedfiles.stream().filter(sf -> !removedCompactedfiles.contains(sf))
+            .sorted(storeFileComparator).collect(ImmutableList.toImmutableList());
   }
 
   @Override
-  public final Iterator<StoreFile> getCandidateFilesForRowKeyBefore(final KeyValue targetKey) {
-    return new ArrayList<>(Lists.reverse(this.storefiles)).iterator();
+  public final Iterator<HStoreFile> getCandidateFilesForRowKeyBefore(KeyValue targetKey) {
+    return this.storefiles.reverse().iterator();
   }
 
   @Override
-  public Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
-      Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
+  public Iterator<HStoreFile> updateCandidateFilesForRowKeyBefore(
+      Iterator<HStoreFile> candidateFiles, KeyValue targetKey, Cell candidate) {
     // Default store has nothing useful to do here.
     // TODO: move this comment when implementing Level:
     // Level store can trim the list by range, removing all the files which cannot have
@@ -180,18 +156,12 @@ class DefaultStoreFileManager implements StoreFileManager {
   }
 
   @Override
-  public final byte[] getSplitPoint() throws IOException {
-    List<StoreFile> storefiles = this.storefiles;
-    if (storefiles.isEmpty()) {
-      return null;
-    }
-    Optional<StoreFile> largestFile = StoreUtils.getLargestFile(storefiles);
-    return largestFile.isPresent()
-        ? StoreUtils.getFileSplitPoint(largestFile.get(), kvComparator).orElse(null) : null;
+  public final Optional<byte[]> getSplitPoint() throws IOException {
+    return StoreUtils.getSplitPoint(storefiles, cellComparator);
   }
 
   @Override
-  public final Collection<StoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
+  public final Collection<HStoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
       byte[] stopRow, boolean includeStopRow) {
     // We cannot provide any useful input and already have the files sorted by seqNum.
     return getStorefiles();
@@ -204,35 +174,20 @@ class DefaultStoreFileManager implements StoreFileManager {
   }
 
   @Override
-  public Collection<StoreFile> getUnneededFiles(long maxTs, List<StoreFile> filesCompacting) {
-    Collection<StoreFile> expiredStoreFiles = null;
-    ImmutableList<StoreFile> files = storefiles;
+  public Collection<HStoreFile> getUnneededFiles(long maxTs, List<HStoreFile> filesCompacting) {
+    ImmutableList<HStoreFile> files = storefiles;
     // 1) We can never get rid of the last file which has the maximum seqid.
     // 2) Files that are not the latest can't become one due to (1), so the rest are fair game.
-    for (int i = 0; i < files.size() - 1; ++i) {
-      StoreFile sf = files.get(i);
+    return files.stream().limit(Math.max(0, files.size() - 1)).filter(sf -> {
       long fileTs = sf.getReader().getMaxTimestamp();
       if (fileTs < maxTs && !filesCompacting.contains(sf)) {
-        LOG.info("Found an expired store file: " + sf.getPath()
-            + " whose maxTimeStamp is " + fileTs + ", which is below " + maxTs);
-        if (expiredStoreFiles == null) {
-          expiredStoreFiles = new ArrayList<>();
-        }
-        expiredStoreFiles.add(sf);
+        LOG.info("Found an expired store file: " + sf.getPath() + " whose maxTimeStamp is " +
+            fileTs + ", which is below " + maxTs);
+        return true;
+      } else {
+        return false;
       }
-    }
-    return expiredStoreFiles;
-  }
-
-  private void sortAndSetStoreFiles(List<StoreFile> storeFiles) {
-    Collections.sort(storeFiles, storeFileComparator);
-    storefiles = ImmutableList.copyOf(storeFiles);
-  }
-
-  private List<StoreFile> sortCompactedfiles(List<StoreFile> storefiles) {
-    // Sorting may not be really needed here for the compacted files?
-    Collections.sort(storefiles, storeFileComparator);
-    return new ArrayList<>(storefiles);
+    }).collect(Collectors.toList());
   }
 
   @Override
@@ -246,7 +201,7 @@ class DefaultStoreFileManager implements StoreFileManager {
   }
 
   @Override
-  public Comparator<StoreFile> getStoreFileComparator() {
+  public Comparator<HStoreFile> getStoreFileComparator() {
     return storeFileComparator;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
index 14c3f92..a5dd9f7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java
@@ -39,7 +39,7 @@ public class DefaultStoreFlusher extends StoreFlusher {
   private static final Log LOG = LogFactory.getLog(DefaultStoreFlusher.class);
   private final Object flushLock = new Object();
 
-  public DefaultStoreFlusher(Configuration conf, Store store) {
+  public DefaultStoreFlusher(Configuration conf, HStore store) {
     super(conf, store);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
----------------------------------------------------------------------
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 43e4a17..95bbf74 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
@@ -43,7 +43,6 @@ 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.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -59,6 +58,7 @@ import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.IdLock;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * The store implementation to save MOBs (medium objects), it extends the HStore.
@@ -166,7 +166,7 @@ public class HMobStore extends HStore {
    * Creates the mob store engine.
    */
   @Override
-  protected StoreEngine<?, ?, ?, ?> createStoreEngine(Store store, Configuration conf,
+  protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf,
       CellComparator cellComparator) throws IOException {
     MobStoreEngine engine = new MobStoreEngine();
     engine.createComponents(conf, store, cellComparator);
@@ -291,7 +291,7 @@ public class HMobStore extends HStore {
    * @param path the path to the mob file
    */
   private void validateMobFile(Path path) throws IOException {
-    StoreFile storeFile = null;
+    HStoreFile storeFile = null;
     try {
       storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig,
           BloomType.NONE, isPrimaryReplicaStore());
@@ -301,7 +301,7 @@ public class HMobStore extends HStore {
       throw e;
     } finally {
       if (storeFile != null) {
-        storeFile.closeReader(false);
+        storeFile.closeStoreFile(false);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 9554d7f..4fa2c70 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
 
 import java.io.EOFException;
@@ -100,7 +101,6 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.UnknownScannerException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Delete;
@@ -147,31 +147,8 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack
 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.shaded.com.google.common.io.Closeables;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -187,6 +164,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALSplitter;
@@ -195,6 +173,30 @@ import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.shaded.com.google.common.io.Closeables;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
 
 @SuppressWarnings("deprecation")
 @InterfaceAudience.Private
@@ -1066,12 +1068,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private NavigableMap<byte[], List<Path>> getStoreFiles() {
     NavigableMap<byte[], List<Path>> allStoreFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
     for (HStore store : stores.values()) {
-      Collection<StoreFile> storeFiles = store.getStorefiles();
+      Collection<HStoreFile> storeFiles = store.getStorefiles();
       if (storeFiles == null) {
         continue;
       }
       List<Path> storeFileNames = new ArrayList<>();
-      for (StoreFile storeFile : storeFiles) {
+      for (HStoreFile storeFile : storeFiles) {
         storeFileNames.add(storeFile.getPath());
       }
       allStoreFiles.put(store.getColumnFamilyDescriptor().getName(), storeFileNames);
@@ -1124,7 +1126,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public HDFSBlocksDistribution getHDFSBlocksDistribution() {
     HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
     stores.values().stream().filter(s -> s.getStorefiles() != null)
-        .flatMap(s -> s.getStorefiles().stream()).map(StoreFile::getHDFSBlockDistribution)
+        .flatMap(s -> s.getStorefiles().stream()).map(HStoreFile::getHDFSBlockDistribution)
         .forEachOrdered(hdfsBlocksDistribution::add);
     return hdfsBlocksDistribution;
   }
@@ -1384,7 +1386,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     LOG.info("DEBUG LIST ALL FILES");
     for (HStore store : this.stores.values()) {
       LOG.info("store " + store.getColumnFamilyName());
-      for (StoreFile sf : store.getStorefiles()) {
+      for (HStoreFile sf : store.getStorefiles()) {
         LOG.info(sf.toStringDetailed());
       }
     }
@@ -1458,7 +1460,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * because a Snapshot was not properly persisted. The region is put in closing mode, and the
    * caller MUST abort after this.
    */
-  public Map<byte[], List<StoreFile>> close() throws IOException {
+  public Map<byte[], List<HStoreFile>> close() throws IOException {
     return close(false);
   }
 
@@ -1499,7 +1501,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * because a Snapshot was not properly persisted. The region is put in closing mode, and the
    * caller MUST abort after this.
    */
-  public Map<byte[], List<StoreFile>> close(final boolean abort) throws IOException {
+  public Map<byte[], List<HStoreFile>> close(boolean abort) throws IOException {
     // Only allow one thread to close at a time. Serialize them so dual
     // threads attempting to close will run up against each other.
     MonitoredTask status = TaskMonitor.get().createStatus(
@@ -1537,7 +1539,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH",
       justification="I think FindBugs is confused")
-  private Map<byte[], List<StoreFile>> doClose(final boolean abort, MonitoredTask status)
+  private Map<byte[], List<HStoreFile>> doClose(boolean abort, MonitoredTask status)
       throws IOException {
     if (isClosed()) {
       LOG.warn("Region " + this + " already closed");
@@ -1632,13 +1634,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
       }
 
-      Map<byte[], List<StoreFile>> result = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+      Map<byte[], List<HStoreFile>> result = new TreeMap<>(Bytes.BYTES_COMPARATOR);
       if (!stores.isEmpty()) {
         // initialize the thread pool for closing stores in parallel.
         ThreadPoolExecutor storeCloserThreadPool =
           getStoreOpenAndCloseThreadPool("StoreCloserThread-" +
             getRegionInfo().getRegionNameAsString());
-        CompletionService<Pair<byte[], Collection<StoreFile>>> completionService =
+        CompletionService<Pair<byte[], Collection<HStoreFile>>> completionService =
           new ExecutorCompletionService<>(storeCloserThreadPool);
 
         // close each store in parallel
@@ -1654,18 +1656,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
             }
           }
           completionService
-              .submit(new Callable<Pair<byte[], Collection<StoreFile>>>() {
+              .submit(new Callable<Pair<byte[], Collection<HStoreFile>>>() {
                 @Override
-                public Pair<byte[], Collection<StoreFile>> call() throws IOException {
+                public Pair<byte[], Collection<HStoreFile>> call() throws IOException {
                   return new Pair<>(store.getColumnFamilyDescriptor().getName(), store.close());
                 }
               });
         }
         try {
           for (int i = 0; i < stores.size(); i++) {
-            Future<Pair<byte[], Collection<StoreFile>>> future = completionService.take();
-            Pair<byte[], Collection<StoreFile>> storeFiles = future.get();
-            List<StoreFile> familyFiles = result.get(storeFiles.getFirst());
+            Future<Pair<byte[], Collection<HStoreFile>>> future = completionService.take();
+            Pair<byte[], Collection<HStoreFile>> storeFiles = future.get();
+            List<HStoreFile> familyFiles = result.get(storeFiles.getFirst());
             if (familyFiles == null) {
               familyFiles = new ArrayList<>();
               result.put(storeFiles.getFirst(), familyFiles);
@@ -1874,11 +1876,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException {
     long result = Long.MAX_VALUE;
     for (HStore store : stores.values()) {
-      Collection<StoreFile> storeFiles = store.getStorefiles();
+      Collection<HStoreFile> storeFiles = store.getStorefiles();
       if (storeFiles == null) {
         continue;
       }
-      for (StoreFile file : storeFiles) {
+      for (HStoreFile file : storeFiles) {
         StoreFileReader sfReader = file.getReader();
         if (sfReader == null) {
           continue;
@@ -1888,7 +1890,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           continue;
         }
         if (majorCompactionOnly) {
-          byte[] val = reader.loadFileInfo().get(StoreFile.MAJOR_COMPACTION_KEY);
+          byte[] val = reader.loadFileInfo().get(MAJOR_COMPACTION_KEY);
           if (val == null || !Bytes.toBoolean(val)) {
             continue;
           }
@@ -4182,7 +4184,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       // If this flag is set, make use of the hfile archiving by making recovered.edits a fake
       // column family. Have to fake out file type too by casting our recovered.edits as storefiles
       String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regiondir).getName();
-      Set<StoreFile> fakeStoreFiles = new HashSet<>(files.size());
+      Set<HStoreFile> fakeStoreFiles = new HashSet<>(files.size());
       for (Path file: files) {
         fakeStoreFiles.add(
           new HStoreFile(getRegionFileSystem().getFileSystem(), file, this.conf, null, null, true));
@@ -5296,11 +5298,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           throw new IllegalArgumentException(
               "No column family : " + new String(column) + " available");
         }
-        Collection<StoreFile> storeFiles = store.getStorefiles();
+        Collection<HStoreFile> storeFiles = store.getStorefiles();
         if (storeFiles == null) {
           continue;
         }
-        for (StoreFile storeFile : storeFiles) {
+        for (HStoreFile storeFile : storeFiles) {
           storeFileNames.add(storeFile.getPath().toString());
         }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 3cb5bdb..3f42466 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -25,7 +25,7 @@ import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.UUID;
 
 import org.apache.commons.logging.Log;
@@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.fs.HFileSystem;
@@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.util.FSHDFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
@@ -482,20 +482,6 @@ public class HRegionFileSystem {
   }
 
   /**
-   * Moves multiple store files to the relative region's family store directory.
-   * @param storeFiles list of store files divided by family
-   * @throws IOException
-   */
-  void commitStoreFiles(final Map<byte[], List<StoreFile>> storeFiles) throws IOException {
-    for (Map.Entry<byte[], List<StoreFile>> es: storeFiles.entrySet()) {
-      String familyName = Bytes.toString(es.getKey());
-      for (StoreFile sf: es.getValue()) {
-        commitStoreFile(familyName, sf.getPath());
-      }
-    }
-  }
-
-  /**
    * Archives the specified store file from the specified family.
    * @param familyName Family that contains the store files
    * @param filePath {@link Path} to the store file to remove
@@ -513,7 +499,7 @@ public class HRegionFileSystem {
    * @param storeFiles set of store files to remove
    * @throws IOException if the archiving fails
    */
-  public void removeStoreFiles(final String familyName, final Collection<StoreFile> storeFiles)
+  public void removeStoreFiles(String familyName, Collection<HStoreFile> storeFiles)
       throws IOException {
     HFileArchiver.archiveStoreFiles(this.conf, this.fs, this.regionInfoForFs,
         this.tableDir, Bytes.toBytes(familyName), storeFiles);
@@ -671,9 +657,8 @@ public class HRegionFileSystem {
    * @return Path to created reference.
    * @throws IOException
    */
-  public Path splitStoreFile(final HRegionInfo hri, final String familyName, final StoreFile f,
-      final byte[] splitRow, final boolean top, RegionSplitPolicy splitPolicy)
-          throws IOException {
+  public Path splitStoreFile(HRegionInfo hri, String familyName, HStoreFile f, byte[] splitRow,
+      boolean top, RegionSplitPolicy splitPolicy) throws IOException {
     if (splitPolicy == null || !splitPolicy.skipStoreFileRangeCheck(familyName)) {
       // Check whether the split row lies in the range of the store file
       // If it is outside the range, return directly.
@@ -682,28 +667,28 @@ public class HRegionFileSystem {
         if (top) {
           //check if larger than last key.
           Cell splitKey = CellUtil.createFirstOnRow(splitRow);
-          Cell lastKey = f.getLastKey();
+          Optional<Cell> lastKey = f.getLastKey();
           // If lastKey is null means storefile is empty.
-          if (lastKey == null) {
+          if (!lastKey.isPresent()) {
             return null;
           }
-          if (f.getComparator().compare(splitKey, lastKey) > 0) {
+          if (f.getComparator().compare(splitKey, lastKey.get()) > 0) {
             return null;
           }
         } else {
           //check if smaller than first key
           Cell splitKey = CellUtil.createLastOnRow(splitRow);
-          Cell firstKey = f.getFirstKey();
+          Optional<Cell> firstKey = f.getFirstKey();
           // If firstKey is null means storefile is empty.
-          if (firstKey == null) {
+          if (!firstKey.isPresent()) {
             return null;
           }
-          if (f.getComparator().compare(splitKey, firstKey) < 0) {
+          if (f.getComparator().compare(splitKey, firstKey.get()) < 0) {
             return null;
           }
         }
       } finally {
-        f.closeReader(f.getCacheConf() != null ? f.getCacheConf().shouldEvictOnClose() : true);
+        f.closeStoreFile(f.getCacheConf() != null ? f.getCacheConf().shouldEvictOnClose() : true);
       }
     }
 
@@ -791,9 +776,8 @@ public class HRegionFileSystem {
    * @return Path to created reference.
    * @throws IOException
    */
-  public Path mergeStoreFile(final HRegionInfo mergedRegion, final String familyName,
-      final StoreFile f, final Path mergedDir)
-      throws IOException {
+  public Path mergeStoreFile(HRegionInfo mergedRegion, String familyName, HStoreFile f,
+      Path mergedDir) throws IOException {
     Path referenceDir = new Path(new Path(mergedDir,
         mergedRegion.getEncodedName()), familyName);
     // A whole reference to the store file.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a5f84430/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index f648c2f..394826c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.Thread.UncaughtExceptionHandler;
@@ -84,7 +83,6 @@ import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.YouAreDeadException;
 import org.apache.hadoop.hbase.ZNodeClearer;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionUtils;
@@ -141,42 +139,6 @@ import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor.Builder;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -210,10 +172,48 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.data.Stat;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor.Builder;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
+
 import sun.misc.Signal;
 import sun.misc.SignalHandler;
 
@@ -3172,7 +3172,7 @@ public class HRegionServer extends HasThread implements
    for (int i = 0; i < regionEncodedName.size(); ++i) {
      Region regionToClose = this.getFromOnlineRegions(regionEncodedName.get(i));
      if (regionToClose != null) {
-       Map<byte[], List<StoreFile>> hstoreFiles = null;
+       Map<byte[], List<HStoreFile>> hstoreFiles = null;
        Exception exceptionToThrow = null;
        try{
          hstoreFiles = ((HRegion)regionToClose).close(false);