hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From zhang...@apache.org
Subject [07/11] hbase git commit: HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface
Date Mon, 25 Sep 2017 01:58:49 GMT
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();
     }


Mime
View raw message