hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject [3/4] hbase git commit: HBASE-15296 Break out writer and reader from StoreFile. Done using Intellij15 Refactor > Move. (Apekshit)
Date Thu, 14 Apr 2016 05:43:14 GMT
http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/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 21ae417..d66d8bd 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
@@ -20,22 +20,16 @@ package org.apache.hadoop.hbase.regionserver;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Ordering;
 
-import java.io.DataInput;
 import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Map;
-import java.util.SortedSet;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -45,33 +39,19 @@ 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.HConstants;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
-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.io.hfile.HFileBlock;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
-import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.nio.ByteBuff;
-import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
-import org.apache.hadoop.hbase.util.BloomFilter;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
-import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
-import org.apache.hadoop.io.WritableUtils;
 
 /**
  * A Store data file.  Stores usually have one or more of these files.  They
  * are produced by flushing the memstore to disk.  To
- * create, instantiate a writer using {@link StoreFile.WriterBuilder}
+ * create, instantiate a writer using {@link StoreFileWriter.Builder}
  * and append data. Be sure to add any metadata before calling close on the
  * Writer (Use the appendMetadata convenience methods). On close, a StoreFile
  * is sitting in the Filesystem.  To refer to it, create a StoreFile instance
@@ -107,7 +87,7 @@ public class StoreFile {
       Bytes.toBytes("DELETE_FAMILY_COUNT");
 
   /** Last Bloom filter key in FileInfo */
-  private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
+  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");
@@ -183,7 +163,7 @@ public class StoreFile {
   private Map<byte[], byte[]> metadataMap;
 
   // StoreFile.Reader
-  private volatile Reader reader;
+  private volatile StoreFileReader reader;
 
   /**
    * Bloom filter type specified in column family configuration. Does not
@@ -404,7 +384,7 @@ public class StoreFile {
 
   @VisibleForTesting
   public int getRefCount() {
-    return this.reader.refCount.get();
+    return this.reader.getRefCount().get();
   }
 
   /**
@@ -429,7 +409,7 @@ public class StoreFile {
    * @throws IOException
    * @see #closeReader(boolean)
    */
-  private Reader open(boolean canUseDropBehind) throws IOException {
+  private StoreFileReader open(boolean canUseDropBehind) throws IOException {
     if (this.reader != null) {
       throw new IllegalAccessError("Already open");
     }
@@ -535,7 +515,7 @@ public class StoreFile {
     return this.reader;
   }
 
-  public Reader createReader() throws IOException {
+  public StoreFileReader createReader() throws IOException {
     return createReader(false);
   }
 
@@ -543,7 +523,7 @@ public class StoreFile {
    * @return Reader for StoreFile. creates if necessary
    * @throws IOException
    */
-  public Reader createReader(boolean canUseDropBehind) throws IOException {
+  public StoreFileReader createReader(boolean canUseDropBehind) throws IOException {
     if (this.reader == null) {
       try {
         this.reader = open(canUseDropBehind);
@@ -565,7 +545,7 @@ public class StoreFile {
    * @return Current reader.  Must call createReader first else returns null.
    * @see #createReader()
    */
-  public Reader getReader() {
+  public StoreFileReader getReader() {
     return this.reader;
   }
 
@@ -636,125 +616,6 @@ public class StoreFile {
     return false;
   }
 
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ICAST_INTEGER_MULTIPLY_CAST_TO_LONG",
-      justification="Will not overflow")
-  public static class WriterBuilder {
-    private final Configuration conf;
-    private final CacheConfig cacheConf;
-    private final FileSystem fs;
-
-    private CellComparator comparator = CellComparator.COMPARATOR;
-    private BloomType bloomType = BloomType.NONE;
-    private long maxKeyCount = 0;
-    private Path dir;
-    private Path filePath;
-    private InetSocketAddress[] favoredNodes;
-    private HFileContext fileContext;
-
-    public WriterBuilder(Configuration conf, CacheConfig cacheConf,
-        FileSystem fs) {
-      this.conf = conf;
-      this.cacheConf = cacheConf;
-      this.fs = fs;
-    }
-
-    /**
-     * Use either this method or {@link #withFilePath}, but not both.
-     * @param dir Path to column family directory. The directory is created if
-     *          does not exist. The file is given a unique name within this
-     *          directory.
-     * @return this (for chained invocation)
-     */
-    public WriterBuilder withOutputDir(Path dir) {
-      Preconditions.checkNotNull(dir);
-      this.dir = dir;
-      return this;
-    }
-
-    /**
-     * Use either this method or {@link #withOutputDir}, but not both.
-     * @param filePath the StoreFile path to write
-     * @return this (for chained invocation)
-     */
-    public WriterBuilder withFilePath(Path filePath) {
-      Preconditions.checkNotNull(filePath);
-      this.filePath = filePath;
-      return this;
-    }
-
-    /**
-     * @param favoredNodes an array of favored nodes or possibly null
-     * @return this (for chained invocation)
-     */
-    public WriterBuilder withFavoredNodes(InetSocketAddress[] favoredNodes) {
-      this.favoredNodes = favoredNodes;
-      return this;
-    }
-
-    public WriterBuilder withComparator(CellComparator comparator) {
-      Preconditions.checkNotNull(comparator);
-      this.comparator = comparator;
-      return this;
-    }
-
-    public WriterBuilder withBloomType(BloomType bloomType) {
-      Preconditions.checkNotNull(bloomType);
-      this.bloomType = bloomType;
-      return this;
-    }
-
-    /**
-     * @param maxKeyCount estimated maximum number of keys we expect to add
-     * @return this (for chained invocation)
-     */
-    public WriterBuilder withMaxKeyCount(long maxKeyCount) {
-      this.maxKeyCount = maxKeyCount;
-      return this;
-    }
-
-    public WriterBuilder withFileContext(HFileContext fileContext) {
-      this.fileContext = fileContext;
-      return this;
-    }
-
-    public WriterBuilder withShouldDropCacheBehind(boolean shouldDropCacheBehind/*NOT USED!!*/) {
-      // TODO: HAS NO EFFECT!!! FIX!!
-      return this;
-    }
-    /**
-     * Create a store file writer. Client is responsible for closing file when
-     * done. If metadata, add BEFORE closing using
-     * {@link Writer#appendMetadata}.
-     */
-    public Writer build() throws IOException {
-      if ((dir == null ? 0 : 1) + (filePath == null ? 0 : 1) != 1) {
-        throw new IllegalArgumentException("Either specify parent directory " +
-            "or file path");
-      }
-
-      if (dir == null) {
-        dir = filePath.getParent();
-      }
-
-      if (!fs.exists(dir)) {
-        fs.mkdirs(dir);
-      }
-
-      if (filePath == null) {
-        filePath = getUniqueFile(fs, dir);
-        if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
-          bloomType = BloomType.NONE;
-        }
-      }
-
-      if (comparator == null) {
-        comparator = CellComparator.COMPARATOR;
-      }
-      return new Writer(fs, filePath,
-          conf, cacheConf, comparator, bloomType, maxKeyCount, favoredNodes, fileContext);
-    }
-  }
-
   /**
    * @param fs
    * @param dir Directory to create file in.
@@ -814,950 +675,6 @@ public class StoreFile {
   }
 
   /**
-   * A StoreFile writer.  Use this to read/write HBase Store Files. It is package
-   * local because it is an implementation detail of the HBase regionserver.
-   */
-  public static class Writer implements Compactor.CellSink {
-    private final BloomFilterWriter generalBloomFilterWriter;
-    private final BloomFilterWriter deleteFamilyBloomFilterWriter;
-    private final BloomType bloomType;
-    private byte[] lastBloomKey;
-    private int lastBloomKeyOffset, lastBloomKeyLen;
-    private Cell lastCell = null;
-    private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
-    private Cell lastDeleteFamilyCell = null;
-    private long deleteFamilyCnt = 0;
-
-    TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
-    /* isTimeRangeTrackerSet keeps track if the timeRange has already been set
-     * When flushing a memstore, we set TimeRange and use this variable to
-     * indicate that it doesn't need to be calculated again while
-     * appending KeyValues.
-     * It is not set in cases of compactions when it is recalculated using only
-     * the appended KeyValues*/
-    boolean isTimeRangeTrackerSet = false;
-
-    protected HFile.Writer writer;
-    private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
-
-    /**
-     * Creates an HFile.Writer that also write helpful meta data.
-     * @param fs file system to write to
-     * @param path file name to create
-     * @param conf user configuration
-     * @param comparator key comparator
-     * @param bloomType bloom filter setting
-     * @param maxKeys the expected maximum number of keys to be added. Was used
-     *        for Bloom filter size in {@link HFile} format version 1.
-     * @param favoredNodes
-     * @param fileContext - The HFile context
-     * @throws IOException problem writing to FS
-     */
-    private Writer(FileSystem fs, Path path,
-        final Configuration conf,
-        CacheConfig cacheConf,
-        final CellComparator comparator, BloomType bloomType, long maxKeys,
-        InetSocketAddress[] favoredNodes, HFileContext fileContext)
-            throws IOException {
-      writer = HFile.getWriterFactory(conf, cacheConf)
-          .withPath(fs, path)
-          .withComparator(comparator)
-          .withFavoredNodes(favoredNodes)
-          .withFileContext(fileContext)
-          .create();
-
-      generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(
-          conf, cacheConf, bloomType,
-          (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
-
-      if (generalBloomFilterWriter != null) {
-        this.bloomType = bloomType;
-        if(this.bloomType ==  BloomType.ROWCOL) {
-          lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue();
-        }
-        if (LOG.isTraceEnabled()) LOG.trace("Bloom filter type for " + path + ": " +
-          this.bloomType + ", " + generalBloomFilterWriter.getClass().getSimpleName());
-      } else {
-        // Not using Bloom filters.
-        this.bloomType = BloomType.NONE;
-      }
-
-      // initialize delete family Bloom filter when there is NO RowCol Bloom
-      // filter
-      if (this.bloomType != BloomType.ROWCOL) {
-        this.deleteFamilyBloomFilterWriter = BloomFilterFactory
-            .createDeleteBloomAtWrite(conf, cacheConf,
-                (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
-      } else {
-        deleteFamilyBloomFilterWriter = null;
-      }
-      if (deleteFamilyBloomFilterWriter != null) {
-        if (LOG.isTraceEnabled()) LOG.trace("Delete Family Bloom filter type for " + path + ": "
-            + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
-      }
-    }
-
-    /**
-     * Writes meta data.
-     * Call before {@link #close()} since its written as meta data to this file.
-     * @param maxSequenceId Maximum sequence id.
-     * @param majorCompaction True if this file is product of a major compaction
-     * @throws IOException problem writing to FS
-     */
-    public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
-    throws IOException {
-      writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
-      writer.appendFileInfo(MAJOR_COMPACTION_KEY,
-          Bytes.toBytes(majorCompaction));
-      appendTrackedTimestampsToMetadata();
-    }
-
-    /**
-     * Writes meta data.
-     * Call before {@link #close()} since its written as meta data to this file.
-     * @param maxSequenceId Maximum sequence id.
-     * @param majorCompaction True if this file is product of a major compaction
-     * @param mobCellsCount The number of mob cells.
-     * @throws IOException problem writing to FS
-     */
-    public void appendMetadata(final long maxSequenceId, final boolean majorCompaction,
-        final long mobCellsCount) throws IOException {
-      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();
-    }
-
-    /**
-     * Add TimestampRange and earliest put timestamp to Metadata
-     */
-    public void appendTrackedTimestampsToMetadata() throws IOException {
-      appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
-      appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
-    }
-
-    /**
-     * Set TimeRangeTracker
-     * @param trt
-     */
-    public void setTimeRangeTracker(final TimeRangeTracker trt) {
-      this.timeRangeTracker = trt;
-      isTimeRangeTrackerSet = true;
-    }
-
-    /**
-     * Record the earlest Put timestamp.
-     *
-     * If the timeRangeTracker is not set,
-     * update TimeRangeTracker to include the timestamp of this key
-     * @param cell
-     */
-    public void trackTimestamps(final Cell cell) {
-      if (KeyValue.Type.Put.getCode() == cell.getTypeByte()) {
-        earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp());
-      }
-      if (!isTimeRangeTrackerSet) {
-        timeRangeTracker.includeTimestamp(cell);
-      }
-    }
-
-    private void appendGeneralBloomfilter(final Cell cell) throws IOException {
-      if (this.generalBloomFilterWriter != null) {
-        // only add to the bloom filter on a new, unique key
-        boolean newKey = true;
-        if (this.lastCell != null) {
-          switch(bloomType) {
-          case ROW:
-            newKey = ! CellUtil.matchingRows(cell, lastCell);
-            break;
-          case ROWCOL:
-            newKey = ! CellUtil.matchingRowColumn(cell, lastCell);
-            break;
-          case NONE:
-            newKey = false;
-            break;
-          default:
-            throw new IOException("Invalid Bloom filter type: " + bloomType +
-                " (ROW or ROWCOL expected)");
-          }
-        }
-        if (newKey) {
-          /*
-           * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue.png
-           * Key = RowLen + Row + FamilyLen + Column [Family + Qualifier] + TimeStamp
-           *
-           * 2 Types of Filtering:
-           *  1. Row = Row
-           *  2. RowCol = Row + Qualifier
-           */
-          byte[] bloomKey = null;
-          // Used with ROW_COL bloom
-          KeyValue bloomKeyKV = null;
-          int bloomKeyOffset, bloomKeyLen;
-
-          switch (bloomType) {
-          case ROW:
-            bloomKey = cell.getRowArray();
-            bloomKeyOffset = cell.getRowOffset();
-            bloomKeyLen = cell.getRowLength();
-            break;
-          case ROWCOL:
-            // merge(row, qualifier)
-            // TODO: could save one buffer copy in case of compound Bloom
-            // filters when this involves creating a KeyValue
-            // TODO : Handle while writes also
-            bloomKeyKV = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(),
-                cell.getRowLength(), 
-                HConstants.EMPTY_BYTE_ARRAY, 0, 0, cell.getQualifierArray(),
-                cell.getQualifierOffset(),
-                cell.getQualifierLength());
-            bloomKey = bloomKeyKV.getBuffer();
-            bloomKeyOffset = bloomKeyKV.getKeyOffset();
-            bloomKeyLen = bloomKeyKV.getKeyLength();
-            break;
-          default:
-            throw new IOException("Invalid Bloom filter type: " + bloomType +
-                " (ROW or ROWCOL expected)");
-          }
-          generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
-          if (lastBloomKey != null) {
-            int res = 0;
-            // hbase:meta does not have blooms. So we need not have special interpretation
-            // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
-            if (bloomType == BloomType.ROW) {
-              res = Bytes.BYTES_RAWCOMPARATOR.compare(bloomKey, bloomKeyOffset, bloomKeyLen,
-                  lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen);
-            } else {
-              // TODO : Caching of kv components becomes important in these cases
-              res = CellComparator.COMPARATOR.compare(bloomKeyKV, lastBloomKeyOnlyKV);
-            }
-            if (res <= 0) {
-              throw new IOException("Non-increasing Bloom keys: "
-                  + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after "
-                  + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen));
-            }
-          }
-          lastBloomKey = bloomKey;
-          lastBloomKeyOffset = bloomKeyOffset;
-          lastBloomKeyLen = bloomKeyLen;
-          if (bloomType == BloomType.ROWCOL) {
-            lastBloomKeyOnlyKV.setKey(bloomKey, bloomKeyOffset, bloomKeyLen);
-          }
-          this.lastCell = cell;
-        }
-      }
-    }
-
-    private void appendDeleteFamilyBloomFilter(final Cell cell)
-        throws IOException {
-      if (!CellUtil.isDeleteFamily(cell) && !CellUtil.isDeleteFamilyVersion(cell)) {
-        return;
-      }
-
-      // increase the number of delete family in the store file
-      deleteFamilyCnt++;
-      if (null != this.deleteFamilyBloomFilterWriter) {
-        boolean newKey = true;
-        if (lastDeleteFamilyCell != null) {
-          // hbase:meta does not have blooms. So we need not have special interpretation
-          // of the hbase:meta cells
-          newKey = !CellUtil.matchingRows(cell, lastDeleteFamilyCell);
-        }
-        if (newKey) {
-          this.deleteFamilyBloomFilterWriter.add(cell.getRowArray(),
-              cell.getRowOffset(), cell.getRowLength());
-          this.lastDeleteFamilyCell = cell;
-        }
-      }
-    }
-
-    public void append(final Cell cell) throws IOException {
-      appendGeneralBloomfilter(cell);
-      appendDeleteFamilyBloomFilter(cell);
-      writer.append(cell);
-      trackTimestamps(cell);
-    }
-
-    public Path getPath() {
-      return this.writer.getPath();
-    }
-
-    public boolean hasGeneralBloom() {
-      return this.generalBloomFilterWriter != null;
-    }
-
-    /**
-     * For unit testing only.
-     *
-     * @return the Bloom filter used by this writer.
-     */
-    BloomFilterWriter getGeneralBloomWriter() {
-      return generalBloomFilterWriter;
-    }
-
-    private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException {
-      boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0);
-      if (haveBloom) {
-        bfw.compactBloom();
-      }
-      return haveBloom;
-    }
-
-    private boolean closeGeneralBloomFilter() throws IOException {
-      boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter);
-
-      // add the general Bloom filter writer and append file info
-      if (hasGeneralBloom) {
-        writer.addGeneralBloomFilter(generalBloomFilterWriter);
-        writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
-            Bytes.toBytes(bloomType.toString()));
-        if (lastBloomKey != null) {
-          writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
-              lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
-                  + lastBloomKeyLen));
-        }
-      }
-      return hasGeneralBloom;
-    }
-
-    private boolean closeDeleteFamilyBloomFilter() throws IOException {
-      boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter);
-
-      // add the delete family Bloom filter writer
-      if (hasDeleteFamilyBloom) {
-        writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter);
-      }
-
-      // append file info about the number of delete family kvs
-      // even if there is no delete family Bloom.
-      writer.appendFileInfo(DELETE_FAMILY_COUNT,
-          Bytes.toBytes(this.deleteFamilyCnt));
-
-      return hasDeleteFamilyBloom;
-    }
-
-    public void close() throws IOException {
-      boolean hasGeneralBloom = this.closeGeneralBloomFilter();
-      boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
-
-      writer.close();
-
-      // Log final Bloom filter statistics. This needs to be done after close()
-      // because compound Bloom filters might be finalized as part of closing.
-      if (StoreFile.LOG.isTraceEnabled()) {
-        StoreFile.LOG.trace((hasGeneralBloom ? "" : "NO ") + "General Bloom and " +
-          (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily" + " was added to HFile " +
-          getPath());
-      }
-
-    }
-
-    public void appendFileInfo(byte[] key, byte[] value) throws IOException {
-      writer.appendFileInfo(key, value);
-    }
-
-    /** For use in testing, e.g. {@link org.apache.hadoop.hbase.regionserver.CreateRandomStoreFile}
-     */
-    HFile.Writer getHFileWriter() {
-      return writer;
-    }
-  }
-
-  /**
-   * Reader for a StoreFile.
-   */
-  public static class Reader {
-    private static final Log LOG = LogFactory.getLog(Reader.class.getName());
-
-    protected BloomFilter generalBloomFilter = null;
-    protected BloomFilter deleteFamilyBloomFilter = null;
-    protected BloomType bloomFilterType;
-    private final HFile.Reader reader;
-    protected TimeRangeTracker timeRangeTracker = null;
-    protected long sequenceID = -1;
-    private byte[] lastBloomKey;
-    private long deleteFamilyCnt = -1;
-    private boolean bulkLoadResult = false;
-    private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
-    private boolean skipResetSeqId = true;
-    // Counter that is incremented every time a scanner is created on the
-    // store file.  It is decremented when the scan on the store file is
-    // done.
-    private AtomicInteger refCount = new AtomicInteger(0);
-    // Indicates if the file got compacted
-    private volatile boolean compactedAway = false;
-
-    public Reader(FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf)
-        throws IOException {
-      reader = HFile.createReader(fs, path, cacheConf, conf);
-      bloomFilterType = BloomType.NONE;
-    }
-
-    void markCompactedAway() {
-      this.compactedAway = true;
-    }
-
-    public Reader(FileSystem fs, Path path, FSDataInputStreamWrapper in, long size,
-        CacheConfig cacheConf, Configuration conf) throws IOException {
-      reader = HFile.createReader(fs, path, in, size, cacheConf, conf);
-      bloomFilterType = BloomType.NONE;
-    }
-
-    public void setReplicaStoreFile(boolean isPrimaryReplicaStoreFile) {
-      reader.setPrimaryReplicaReader(isPrimaryReplicaStoreFile);
-    }
-    public boolean isPrimaryReplicaReader() {
-      return reader.isPrimaryReplicaReader();
-    }
-
-    /**
-     * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS
-     */
-    Reader() {
-      this.reader = null;
-    }
-
-    public CellComparator getComparator() {
-      return reader.getComparator();
-    }
-
-    /**
-     * Get a scanner to scan over this StoreFile. Do not use
-     * this overload if using this scanner for compactions.
-     *
-     * @param cacheBlocks should this scanner cache blocks?
-     * @param pread use pread (for highly concurrent small readers)
-     * @return a scanner
-     */
-    public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
-                                               boolean pread) {
-      return getStoreFileScanner(cacheBlocks, pread, false,
-        // 0 is passed as readpoint because this method is only used by test
-        // where StoreFile is directly operated upon
-        0);
-    }
-
-    /**
-     * Get a scanner to scan over this StoreFile.
-     *
-     * @param cacheBlocks should this scanner cache blocks?
-     * @param pread use pread (for highly concurrent small readers)
-     * @param isCompaction is scanner being used for compaction?
-     * @return a scanner
-     */
-    public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
-                                               boolean pread,
-                                               boolean isCompaction, long readPt) {
-      // Increment the ref count
-      refCount.incrementAndGet();
-      return new StoreFileScanner(this,
-                                 getScanner(cacheBlocks, pread, isCompaction),
-                                 !isCompaction, reader.hasMVCCInfo(), readPt);
-    }
-
-    /**
-     * Decrement the ref count associated with the reader when ever a scanner associated
-     * with the reader is closed
-     */
-    void decrementRefCount() {
-      refCount.decrementAndGet();
-    }
-
-    /**
-     * @return true if the file is still used in reads
-     */
-    public boolean isReferencedInReads() {
-      return refCount.get() != 0;
-    }
- 
-    /**
-     * @return true if the file is compacted
-     */
-    public boolean isCompactedAway() {
-      return this.compactedAway;
-    }
-
-    /**
-     * @deprecated Do not write further code which depends on this call. Instead
-     * use getStoreFileScanner() which uses the StoreFileScanner class/interface
-     * which is the preferred way to scan a store with higher level concepts.
-     *
-     * @param cacheBlocks should we cache the blocks?
-     * @param pread use pread (for concurrent small readers)
-     * @return the underlying HFileScanner
-     */
-    @Deprecated
-    public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
-      return getScanner(cacheBlocks, pread, false);
-    }
-
-    /**
-     * @deprecated Do not write further code which depends on this call. Instead
-     * use getStoreFileScanner() which uses the StoreFileScanner class/interface
-     * which is the preferred way to scan a store with higher level concepts.
-     *
-     * @param cacheBlocks
-     *          should we cache the blocks?
-     * @param pread
-     *          use pread (for concurrent small readers)
-     * @param isCompaction
-     *          is scanner being used for compaction?
-     * @return the underlying HFileScanner
-     */
-    @Deprecated
-    public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
-        boolean isCompaction) {
-      return reader.getScanner(cacheBlocks, pread, isCompaction);
-    }
-
-    public void close(boolean evictOnClose) throws IOException {
-      reader.close(evictOnClose);
-    }
-
-    /**
-     * Check if this storeFile may contain keys within the TimeRange that
-     * have not expired (i.e. not older than oldestUnexpiredTS).
-     * @param timeRange the timeRange to restrict
-     * @param oldestUnexpiredTS the oldest timestamp that is not expired, as
-     *          determined by the column family's TTL
-     * @return false if queried keys definitely don't exist in this StoreFile
-     */
-    boolean passesTimerangeFilter(TimeRange timeRange, long oldestUnexpiredTS) {
-      if (timeRangeTracker == null) {
-        return true;
-      } else {
-        return timeRangeTracker.includesTimeRange(timeRange) &&
-            timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS;
-      }
-    }
-
-    /**
-     * Checks whether the given scan passes the Bloom filter (if present). Only
-     * checks Bloom filters for single-row or single-row-column scans. Bloom
-     * filter checking for multi-gets is implemented as part of the store
-     * scanner system (see {@link StoreFileScanner#seekExactly}) and uses
-     * the lower-level API {@link #passesGeneralRowBloomFilter(byte[], int, int)}
-     * and {@link #passesGeneralRowColBloomFilter(Cell)}.
-     *
-     * @param scan the scan specification. Used to determine the row, and to
-     *          check whether this is a single-row ("get") scan.
-     * @param columns the set of columns. Only used for row-column Bloom
-     *          filters.
-     * @return true if the scan with the given column set passes the Bloom
-     *         filter, or if the Bloom filter is not applicable for the scan.
-     *         False if the Bloom filter is applicable and the scan fails it.
-     */
-     boolean passesBloomFilter(Scan scan,
-        final SortedSet<byte[]> columns) {
-      // Multi-column non-get scans will use Bloom filters through the
-      // lower-level API function that this function calls.
-      if (!scan.isGetScan()) {
-        return true;
-      }
-
-      byte[] row = scan.getStartRow();
-      switch (this.bloomFilterType) {
-        case ROW:
-          return passesGeneralRowBloomFilter(row, 0, row.length);
-
-        case ROWCOL:
-          if (columns != null && columns.size() == 1) {
-            byte[] column = columns.first();
-            // create the required fake key
-            Cell kvKey = KeyValueUtil.createFirstOnRow(row, 0, row.length,
-              HConstants.EMPTY_BYTE_ARRAY, 0, 0, column, 0,
-              column.length);
-            return passesGeneralRowColBloomFilter(kvKey);
-          }
-
-          // For multi-column queries the Bloom filter is checked from the
-          // seekExact operation.
-          return true;
-
-        default:
-          return true;
-      }
-    }
-
-    public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
-        int rowLen) {
-      // Cache Bloom filter as a local variable in case it is set to null by
-      // another thread on an IO error.
-      BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
-
-      // Empty file or there is no delete family at all
-      if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
-        return false;
-      }
-
-      if (bloomFilter == null) {
-        return true;
-      }
-
-      try {
-        if (!bloomFilter.supportsAutoLoading()) {
-          return true;
-        }
-        return bloomFilter.contains(row, rowOffset, rowLen, null);
-      } catch (IllegalArgumentException e) {
-        LOG.error("Bad Delete Family bloom filter data -- proceeding without",
-            e);
-        setDeleteFamilyBloomFilterFaulty();
-      }
-
-      return true;
-    }
-
-    /**
-     * A method for checking Bloom filters. Called directly from
-     * StoreFileScanner in case of a multi-column query.
-     *
-     * @param row
-     * @param rowOffset
-     * @param rowLen
-     * @return True if passes
-     */
-    public boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) {
-      BloomFilter bloomFilter = this.generalBloomFilter;
-      if (bloomFilter == null) {
-        return true;
-      }
-
-      // Used in ROW bloom
-      byte[] key = null;
-      if (rowOffset != 0 || rowLen != row.length) {
-        throw new AssertionError(
-            "For row-only Bloom filters the row " + "must occupy the whole array");
-      }
-      key = row;
-      return checkGeneralBloomFilter(key, null, bloomFilter);
-    }
-
-    /**
-     * A method for checking Bloom filters. Called directly from
-     * StoreFileScanner in case of a multi-column query.
-     *
-     * @param cell
-     *          the cell to check if present in BloomFilter
-     * @return True if passes
-     */
-    public boolean passesGeneralRowColBloomFilter(Cell cell) {
-      BloomFilter bloomFilter = this.generalBloomFilter;
-      if (bloomFilter == null) {
-        return true;
-      }
-      // Used in ROW_COL bloom
-      Cell kvKey = null;
-      // Already if the incoming key is a fake rowcol key then use it as it is
-      if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode() && cell.getFamilyLength() == 0) {
-        kvKey = cell;
-      } else {
-        kvKey = CellUtil.createFirstOnRowCol(cell);
-      }
-      return checkGeneralBloomFilter(null, kvKey, bloomFilter);
-    }
-
-    private boolean checkGeneralBloomFilter(byte[] key, Cell kvKey, BloomFilter bloomFilter) {
-      // Empty file
-      if (reader.getTrailer().getEntryCount() == 0)
-        return false;
-      HFileBlock bloomBlock = null;
-      try {
-        boolean shouldCheckBloom;
-        ByteBuff bloom;
-        if (bloomFilter.supportsAutoLoading()) {
-          bloom = null;
-          shouldCheckBloom = true;
-        } else {
-          bloomBlock = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, true);
-          bloom = bloomBlock.getBufferWithoutHeader();
-          shouldCheckBloom = bloom != null;
-        }
-
-        if (shouldCheckBloom) {
-          boolean exists;
-
-          // Whether the primary Bloom key is greater than the last Bloom key
-          // from the file info. For row-column Bloom filters this is not yet
-          // a sufficient condition to return false.
-          boolean keyIsAfterLast = (lastBloomKey != null);
-          // hbase:meta does not have blooms. So we need not have special interpretation
-          // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
-          if (keyIsAfterLast) {
-            if (bloomFilterType == BloomType.ROW) {
-              keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) > 0);
-            } else {
-              keyIsAfterLast = (CellComparator.COMPARATOR.compare(kvKey, lastBloomKeyOnlyKV)) > 0;
-            }
-          }
-
-          if (bloomFilterType == BloomType.ROWCOL) {
-            // Since a Row Delete is essentially a DeleteFamily applied to all
-            // columns, a file might be skipped if using row+col Bloom filter.
-            // In order to ensure this file is included an additional check is
-            // required looking only for a row bloom.
-            Cell rowBloomKey = CellUtil.createFirstOnRow(kvKey);
-            // hbase:meta does not have blooms. So we need not have special interpretation
-            // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
-            if (keyIsAfterLast
-                && (CellComparator.COMPARATOR.compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0) {
-              exists = false;
-            } else {
-              exists =
-                  bloomFilter.contains(kvKey, bloom) ||
-                  bloomFilter.contains(rowBloomKey, bloom);
-            }
-          } else {
-            exists = !keyIsAfterLast
-                && bloomFilter.contains(key, 0, key.length, bloom);
-          }
-
-          return exists;
-        }
-      } catch (IOException e) {
-        LOG.error("Error reading bloom filter data -- proceeding without",
-            e);
-        setGeneralBloomFilterFaulty();
-      } catch (IllegalArgumentException e) {
-        LOG.error("Bad bloom filter data -- proceeding without", e);
-        setGeneralBloomFilterFaulty();
-      } finally {
-        // Return the bloom block so that its ref count can be decremented.
-        reader.returnBlock(bloomBlock);
-      }
-      return true;
-    }
-
-    /**
-     * Checks whether the given scan rowkey range overlaps with the current storefile's
-     * @param scan the scan specification. Used to determine the rowkey range.
-     * @return true if there is overlap, false otherwise
-     */
-    public boolean passesKeyRangeFilter(Scan scan) {
-      if (this.getFirstKey() == null || this.getLastKey() == null) {
-        // the file is empty
-        return false;
-      }
-      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;
-      return !nonOverLapping;
-    }
-
-    public Map<byte[], byte[]> loadFileInfo() throws IOException {
-      Map<byte [], byte []> fi = reader.loadFileInfo();
-
-      byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
-      if (b != null) {
-        bloomFilterType = BloomType.valueOf(Bytes.toString(b));
-      }
-
-      lastBloomKey = fi.get(LAST_BLOOM_KEY);
-      if(bloomFilterType == BloomType.ROWCOL) {
-        lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length);
-      }
-      byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
-      if (cnt != null) {
-        deleteFamilyCnt = Bytes.toLong(cnt);
-      }
-
-      return fi;
-    }
-
-    public void loadBloomfilter() {
-      this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
-      this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
-    }
-
-    private void loadBloomfilter(BlockType blockType) {
-      try {
-        if (blockType == BlockType.GENERAL_BLOOM_META) {
-          if (this.generalBloomFilter != null)
-            return; // Bloom has been loaded
-
-          DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
-          if (bloomMeta != null) {
-            // sanity check for NONE Bloom filter
-            if (bloomFilterType == BloomType.NONE) {
-              throw new IOException(
-                  "valid bloom filter type not found in FileInfo");
-            } else {
-              generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
-                  reader);
-              if (LOG.isTraceEnabled()) {
-                LOG.trace("Loaded " + bloomFilterType.toString() + " "
-                  + generalBloomFilter.getClass().getSimpleName()
-                  + " metadata for " + reader.getName());
-              }
-            }
-          }
-        } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
-          if (this.deleteFamilyBloomFilter != null)
-            return; // Bloom has been loaded
-
-          DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
-          if (bloomMeta != null) {
-            deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
-                bloomMeta, reader);
-            LOG.info("Loaded Delete Family Bloom ("
-                + deleteFamilyBloomFilter.getClass().getSimpleName()
-                + ") metadata for " + reader.getName());
-          }
-        } else {
-          throw new RuntimeException("Block Type: " + blockType.toString()
-              + "is not supported for Bloom filter");
-        }
-      } catch (IOException e) {
-        LOG.error("Error reading bloom filter meta for " + blockType
-            + " -- proceeding without", e);
-        setBloomFilterFaulty(blockType);
-      } catch (IllegalArgumentException e) {
-        LOG.error("Bad bloom filter meta " + blockType
-            + " -- proceeding without", e);
-        setBloomFilterFaulty(blockType);
-      }
-    }
-
-    private void setBloomFilterFaulty(BlockType blockType) {
-      if (blockType == BlockType.GENERAL_BLOOM_META) {
-        setGeneralBloomFilterFaulty();
-      } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
-        setDeleteFamilyBloomFilterFaulty();
-      }
-    }
-
-    /**
-     * The number of Bloom filter entries in this store file, or an estimate
-     * thereof, if the Bloom filter is not loaded. This always returns an upper
-     * bound of the number of Bloom filter entries.
-     *
-     * @return an estimate of the number of Bloom filter entries in this file
-     */
-    public long getFilterEntries() {
-      return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
-          : reader.getEntries();
-    }
-
-    public void setGeneralBloomFilterFaulty() {
-      generalBloomFilter = null;
-    }
-
-    public void setDeleteFamilyBloomFilterFaulty() {
-      this.deleteFamilyBloomFilter = null;
-    }
-
-    public Cell getLastKey() {
-      return reader.getLastKey();
-    }
-
-    public byte[] getLastRowKey() {
-      return reader.getLastRowKey();
-    }
-
-    public Cell midkey() throws IOException {
-      return reader.midkey();
-    }
-
-    public long length() {
-      return reader.length();
-    }
-
-    public long getTotalUncompressedBytes() {
-      return reader.getTrailer().getTotalUncompressedBytes();
-    }
-
-    public long getEntries() {
-      return reader.getEntries();
-    }
-
-    public long getDeleteFamilyCnt() {
-      return deleteFamilyCnt;
-    }
-
-    public Cell getFirstKey() {
-      return reader.getFirstKey();
-    }
-
-    public long indexSize() {
-      return reader.indexSize();
-    }
-
-    public BloomType getBloomFilterType() {
-      return this.bloomFilterType;
-    }
-
-    public long getSequenceID() {
-      return sequenceID;
-    }
-
-    public void setSequenceID(long sequenceID) {
-      this.sequenceID = sequenceID;
-    }
-
-    public void setBulkLoaded(boolean bulkLoadResult) {
-      this.bulkLoadResult = bulkLoadResult;
-    }
-
-    public boolean isBulkLoaded() {
-      return this.bulkLoadResult;
-    }
-
-    BloomFilter getGeneralBloomFilter() {
-      return generalBloomFilter;
-    }
-
-    long getUncompressedDataIndexSize() {
-      return reader.getTrailer().getUncompressedDataIndexSize();
-    }
-
-    public long getTotalBloomSize() {
-      if (generalBloomFilter == null)
-        return 0;
-      return generalBloomFilter.getByteSize();
-    }
-
-    public int getHFileVersion() {
-      return reader.getTrailer().getMajorVersion();
-    }
-
-    public int getHFileMinorVersion() {
-      return reader.getTrailer().getMinorVersion();
-    }
-
-    public HFile.Reader getHFileReader() {
-      return reader;
-    }
-
-    void disableBloomFilterForTesting() {
-      generalBloomFilter = null;
-      this.deleteFamilyBloomFilter = null;
-    }
-
-    public long getMaxTimestamp() {
-      return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.getMaximumTimestamp();
-    }
-
-    boolean isSkipResetSeqId() {
-      return skipResetSeqId;
-    }
-
-    void setSkipResetSeqId(boolean skipResetSeqId) {
-      this.skipResetSeqId = skipResetSeqId;
-    }
-  }
-
-  /**
    * Useful comparators for comparing StoreFiles.
    */
   public abstract static class Comparators {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
index 073adae..3c12045 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
@@ -233,7 +233,7 @@ public class StoreFileInfo {
    * @param cacheConf The cache configuration and block cache reference.
    * @return The StoreFile.Reader for the file
    */
-  public StoreFile.Reader open(final FileSystem fs,
+  public StoreFileReader open(final FileSystem fs,
       final CacheConfig cacheConf, final boolean canUseDropBehind) throws IOException {
     FSDataInputStreamWrapper in;
     FileStatus status;
@@ -257,7 +257,7 @@ public class StoreFileInfo {
     long length = status.getLen();
     hdfsBlocksDistribution = computeHDFSBlocksDistribution(fs);
 
-    StoreFile.Reader reader = null;
+    StoreFileReader reader = null;
     if (this.coprocessorHost != null) {
       reader = this.coprocessorHost.preStoreFileReaderOpen(fs, this.getPath(), in, length,
         cacheConf, reference);
@@ -267,7 +267,7 @@ public class StoreFileInfo {
         reader = new HalfStoreFileReader(fs, this.getPath(), in, length, cacheConf, reference,
           conf);
       } else {
-        reader = new StoreFile.Reader(fs, status.getPath(), in, length, cacheConf, conf);
+        reader = new StoreFileReader(fs, status.getPath(), in, length, cacheConf, conf);
       }
     }
     if (this.coprocessorHost != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/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
new file mode 100644
index 0000000..a2ad5a4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java
@@ -0,0 +1,647 @@
+/*
+ *
+ * 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 org.apache.commons.logging.Log;
+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.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.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+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.io.hfile.HFileBlock;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.util.BloomFilter;
+import org.apache.hadoop.hbase.util.BloomFilterFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Reader for a StoreFile.
+ */
+@InterfaceAudience.Private
+public class StoreFileReader {
+  private static final Log LOG = LogFactory.getLog(StoreFileReader.class.getName());
+
+  protected BloomFilter generalBloomFilter = null;
+  protected BloomFilter deleteFamilyBloomFilter = null;
+  protected BloomType bloomFilterType;
+  private final HFile.Reader reader;
+  protected TimeRangeTracker timeRangeTracker = null;
+  protected long sequenceID = -1;
+  private byte[] lastBloomKey;
+  private long deleteFamilyCnt = -1;
+  private boolean bulkLoadResult = false;
+  private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
+  private boolean skipResetSeqId = true;
+
+  public AtomicInteger getRefCount() {
+    return refCount;
+  }
+
+  // Counter that is incremented every time a scanner is created on the
+  // store file.  It is decremented when the scan on the store file is
+  // done.
+  private AtomicInteger refCount = new AtomicInteger(0);
+  // Indicates if the file got compacted
+  private volatile boolean compactedAway = false;
+
+  public StoreFileReader(FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf)
+      throws IOException {
+    reader = HFile.createReader(fs, path, cacheConf, conf);
+    bloomFilterType = BloomType.NONE;
+  }
+
+  void markCompactedAway() {
+    this.compactedAway = true;
+  }
+
+  public StoreFileReader(FileSystem fs, Path path, FSDataInputStreamWrapper in, long size,
+      CacheConfig cacheConf, Configuration conf) throws IOException {
+    reader = HFile.createReader(fs, path, in, size, cacheConf, conf);
+    bloomFilterType = BloomType.NONE;
+  }
+
+  public void setReplicaStoreFile(boolean isPrimaryReplicaStoreFile) {
+    reader.setPrimaryReplicaReader(isPrimaryReplicaStoreFile);
+  }
+  public boolean isPrimaryReplicaReader() {
+    return reader.isPrimaryReplicaReader();
+  }
+
+  /**
+   * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS
+   */
+  StoreFileReader() {
+    this.reader = null;
+  }
+
+  public CellComparator getComparator() {
+    return reader.getComparator();
+  }
+
+  /**
+   * Get a scanner to scan over this StoreFile. Do not use
+   * this overload if using this scanner for compactions.
+   *
+   * @param cacheBlocks should this scanner cache blocks?
+   * @param pread use pread (for highly concurrent small readers)
+   * @return a scanner
+   */
+  public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
+                                             boolean pread) {
+    return getStoreFileScanner(cacheBlocks, pread, false,
+      // 0 is passed as readpoint because this method is only used by test
+      // where StoreFile is directly operated upon
+      0);
+  }
+
+  /**
+   * Get a scanner to scan over this StoreFile.
+   *
+   * @param cacheBlocks should this scanner cache blocks?
+   * @param pread use pread (for highly concurrent small readers)
+   * @param isCompaction is scanner being used for compaction?
+   * @return a scanner
+   */
+  public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
+                                             boolean pread,
+                                             boolean isCompaction, long readPt) {
+    // Increment the ref count
+    refCount.incrementAndGet();
+    return new StoreFileScanner(this,
+                               getScanner(cacheBlocks, pread, isCompaction),
+                               !isCompaction, reader.hasMVCCInfo(), readPt);
+  }
+
+  /**
+   * Decrement the ref count associated with the reader when ever a scanner associated
+   * with the reader is closed
+   */
+  void decrementRefCount() {
+    refCount.decrementAndGet();
+  }
+
+  /**
+   * @return true if the file is still used in reads
+   */
+  public boolean isReferencedInReads() {
+    return refCount.get() != 0;
+  }
+
+  /**
+   * @return true if the file is compacted
+   */
+  public boolean isCompactedAway() {
+    return this.compactedAway;
+  }
+
+  /**
+   * @deprecated Do not write further code which depends on this call. Instead
+   *   use getStoreFileScanner() which uses the StoreFileScanner class/interface
+   *   which is the preferred way to scan a store with higher level concepts.
+   *
+   * @param cacheBlocks should we cache the blocks?
+   * @param pread use pread (for concurrent small readers)
+   * @return the underlying HFileScanner
+   */
+  @Deprecated
+  public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
+    return getScanner(cacheBlocks, pread, false);
+  }
+
+  /**
+   * @deprecated Do not write further code which depends on this call. Instead
+   *   use getStoreFileScanner() which uses the StoreFileScanner class/interface
+   *   which is the preferred way to scan a store with higher level concepts.
+   *
+   * @param cacheBlocks
+   *          should we cache the blocks?
+   * @param pread
+   *          use pread (for concurrent small readers)
+   * @param isCompaction
+   *          is scanner being used for compaction?
+   * @return the underlying HFileScanner
+   */
+  @Deprecated
+  public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
+      boolean isCompaction) {
+    return reader.getScanner(cacheBlocks, pread, isCompaction);
+  }
+
+  public void close(boolean evictOnClose) throws IOException {
+    reader.close(evictOnClose);
+  }
+
+  /**
+   * Check if this storeFile may contain keys within the TimeRange that
+   * have not expired (i.e. not older than oldestUnexpiredTS).
+   * @param timeRange the timeRange to restrict
+   * @param oldestUnexpiredTS the oldest timestamp that is not expired, as
+   *          determined by the column family's TTL
+   * @return false if queried keys definitely don't exist in this StoreFile
+   */
+  boolean passesTimerangeFilter(TimeRange timeRange, long oldestUnexpiredTS) {
+    if (timeRangeTracker == null) {
+      return true;
+    } else {
+      return timeRangeTracker.includesTimeRange(timeRange) &&
+          timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS;
+    }
+  }
+
+  /**
+   * Checks whether the given scan passes the Bloom filter (if present). Only
+   * checks Bloom filters for single-row or single-row-column scans. Bloom
+   * filter checking for multi-gets is implemented as part of the store
+   * scanner system (see {@link StoreFileScanner#seekExactly}) and uses
+   * the lower-level API {@link #passesGeneralRowBloomFilter(byte[], int, int)}
+   * and {@link #passesGeneralRowColBloomFilter(Cell)}.
+   *
+   * @param scan the scan specification. Used to determine the row, and to
+   *          check whether this is a single-row ("get") scan.
+   * @param columns the set of columns. Only used for row-column Bloom
+   *          filters.
+   * @return true if the scan with the given column set passes the Bloom
+   *         filter, or if the Bloom filter is not applicable for the scan.
+   *         False if the Bloom filter is applicable and the scan fails it.
+   */
+  boolean passesBloomFilter(Scan scan, final SortedSet<byte[]> columns) {
+    // Multi-column non-get scans will use Bloom filters through the
+    // lower-level API function that this function calls.
+    if (!scan.isGetScan()) {
+      return true;
+    }
+
+    byte[] row = scan.getStartRow();
+    switch (this.bloomFilterType) {
+      case ROW:
+        return passesGeneralRowBloomFilter(row, 0, row.length);
+
+      case ROWCOL:
+        if (columns != null && columns.size() == 1) {
+          byte[] column = columns.first();
+          // create the required fake key
+          Cell kvKey = KeyValueUtil.createFirstOnRow(row, 0, row.length,
+            HConstants.EMPTY_BYTE_ARRAY, 0, 0, column, 0,
+            column.length);
+          return passesGeneralRowColBloomFilter(kvKey);
+        }
+
+        // For multi-column queries the Bloom filter is checked from the
+        // seekExact operation.
+        return true;
+
+      default:
+        return true;
+    }
+  }
+
+  public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
+      int rowLen) {
+    // Cache Bloom filter as a local variable in case it is set to null by
+    // another thread on an IO error.
+    BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
+
+    // Empty file or there is no delete family at all
+    if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
+      return false;
+    }
+
+    if (bloomFilter == null) {
+      return true;
+    }
+
+    try {
+      if (!bloomFilter.supportsAutoLoading()) {
+        return true;
+      }
+      return bloomFilter.contains(row, rowOffset, rowLen, null);
+    } catch (IllegalArgumentException e) {
+      LOG.error("Bad Delete Family bloom filter data -- proceeding without",
+          e);
+      setDeleteFamilyBloomFilterFaulty();
+    }
+
+    return true;
+  }
+
+  /**
+   * A method for checking Bloom filters. Called directly from
+   * StoreFileScanner in case of a multi-column query.
+   *
+   * @return True if passes
+   */
+  public boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) {
+    BloomFilter bloomFilter = this.generalBloomFilter;
+    if (bloomFilter == null) {
+      return true;
+    }
+
+    // Used in ROW bloom
+    byte[] key = null;
+    if (rowOffset != 0 || rowLen != row.length) {
+      throw new AssertionError(
+          "For row-only Bloom filters the row " + "must occupy the whole array");
+    }
+    key = row;
+    return checkGeneralBloomFilter(key, null, bloomFilter);
+  }
+
+  /**
+   * A method for checking Bloom filters. Called directly from
+   * StoreFileScanner in case of a multi-column query.
+   *
+   * @param cell
+   *          the cell to check if present in BloomFilter
+   * @return True if passes
+   */
+  public boolean passesGeneralRowColBloomFilter(Cell cell) {
+    BloomFilter bloomFilter = this.generalBloomFilter;
+    if (bloomFilter == null) {
+      return true;
+    }
+    // Used in ROW_COL bloom
+    Cell kvKey = null;
+    // Already if the incoming key is a fake rowcol key then use it as it is
+    if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode() && cell.getFamilyLength() == 0) {
+      kvKey = cell;
+    } else {
+      kvKey = CellUtil.createFirstOnRowCol(cell);
+    }
+    return checkGeneralBloomFilter(null, kvKey, bloomFilter);
+  }
+
+  private boolean checkGeneralBloomFilter(byte[] key, Cell kvKey, BloomFilter bloomFilter) {
+    // Empty file
+    if (reader.getTrailer().getEntryCount() == 0) {
+      return false;
+    }
+    HFileBlock bloomBlock = null;
+    try {
+      boolean shouldCheckBloom;
+      ByteBuff bloom;
+      if (bloomFilter.supportsAutoLoading()) {
+        bloom = null;
+        shouldCheckBloom = true;
+      } else {
+        bloomBlock = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, true);
+        bloom = bloomBlock.getBufferWithoutHeader();
+        shouldCheckBloom = bloom != null;
+      }
+
+      if (shouldCheckBloom) {
+        boolean exists;
+
+        // Whether the primary Bloom key is greater than the last Bloom key
+        // from the file info. For row-column Bloom filters this is not yet
+        // a sufficient condition to return false.
+        boolean keyIsAfterLast = (lastBloomKey != null);
+        // hbase:meta does not have blooms. So we need not have special interpretation
+        // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
+        if (keyIsAfterLast) {
+          if (bloomFilterType == BloomType.ROW) {
+            keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) > 0);
+          } else {
+            keyIsAfterLast = (CellComparator.COMPARATOR.compare(kvKey, lastBloomKeyOnlyKV)) > 0;
+          }
+        }
+
+        if (bloomFilterType == BloomType.ROWCOL) {
+          // Since a Row Delete is essentially a DeleteFamily applied to all
+          // columns, a file might be skipped if using row+col Bloom filter.
+          // In order to ensure this file is included an additional check is
+          // required looking only for a row bloom.
+          Cell rowBloomKey = CellUtil.createFirstOnRow(kvKey);
+          // hbase:meta does not have blooms. So we need not have special interpretation
+          // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
+          if (keyIsAfterLast
+              && (CellComparator.COMPARATOR.compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0) {
+            exists = false;
+          } else {
+            exists =
+                bloomFilter.contains(kvKey, bloom) ||
+                bloomFilter.contains(rowBloomKey, bloom);
+          }
+        } else {
+          exists = !keyIsAfterLast
+              && bloomFilter.contains(key, 0, key.length, bloom);
+        }
+
+        return exists;
+      }
+    } catch (IOException e) {
+      LOG.error("Error reading bloom filter data -- proceeding without",
+          e);
+      setGeneralBloomFilterFaulty();
+    } catch (IllegalArgumentException e) {
+      LOG.error("Bad bloom filter data -- proceeding without", e);
+      setGeneralBloomFilterFaulty();
+    } finally {
+      // Return the bloom block so that its ref count can be decremented.
+      reader.returnBlock(bloomBlock);
+    }
+    return true;
+  }
+
+  /**
+   * Checks whether the given scan rowkey range overlaps with the current storefile's
+   * @param scan the scan specification. Used to determine the rowkey range.
+   * @return true if there is overlap, false otherwise
+   */
+  public boolean passesKeyRangeFilter(Scan scan) {
+    if (this.getFirstKey() == null || this.getLastKey() == null) {
+      // the file is empty
+      return false;
+    }
+    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;
+    return !nonOverLapping;
+  }
+
+  public Map<byte[], byte[]> loadFileInfo() throws IOException {
+    Map<byte [], byte []> fi = reader.loadFileInfo();
+
+    byte[] b = fi.get(StoreFile.BLOOM_FILTER_TYPE_KEY);
+    if (b != null) {
+      bloomFilterType = BloomType.valueOf(Bytes.toString(b));
+    }
+
+    lastBloomKey = fi.get(StoreFile.LAST_BLOOM_KEY);
+    if(bloomFilterType == BloomType.ROWCOL) {
+      lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length);
+    }
+    byte[] cnt = fi.get(StoreFile.DELETE_FAMILY_COUNT);
+    if (cnt != null) {
+      deleteFamilyCnt = Bytes.toLong(cnt);
+    }
+
+    return fi;
+  }
+
+  public void loadBloomfilter() {
+    this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
+    this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
+  }
+
+  public void loadBloomfilter(BlockType blockType) {
+    try {
+      if (blockType == BlockType.GENERAL_BLOOM_META) {
+        if (this.generalBloomFilter != null)
+          return; // Bloom has been loaded
+
+        DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
+        if (bloomMeta != null) {
+          // sanity check for NONE Bloom filter
+          if (bloomFilterType == BloomType.NONE) {
+            throw new IOException(
+                "valid bloom filter type not found in FileInfo");
+          } else {
+            generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
+                reader);
+            if (LOG.isTraceEnabled()) {
+              LOG.trace("Loaded " + bloomFilterType.toString() + " "
+                + generalBloomFilter.getClass().getSimpleName()
+                + " metadata for " + reader.getName());
+            }
+          }
+        }
+      } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
+        if (this.deleteFamilyBloomFilter != null)
+          return; // Bloom has been loaded
+
+        DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
+        if (bloomMeta != null) {
+          deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
+              bloomMeta, reader);
+          LOG.info("Loaded Delete Family Bloom ("
+              + deleteFamilyBloomFilter.getClass().getSimpleName()
+              + ") metadata for " + reader.getName());
+        }
+      } else {
+        throw new RuntimeException("Block Type: " + blockType.toString()
+            + "is not supported for Bloom filter");
+      }
+    } catch (IOException e) {
+      LOG.error("Error reading bloom filter meta for " + blockType
+          + " -- proceeding without", e);
+      setBloomFilterFaulty(blockType);
+    } catch (IllegalArgumentException e) {
+      LOG.error("Bad bloom filter meta " + blockType
+          + " -- proceeding without", e);
+      setBloomFilterFaulty(blockType);
+    }
+  }
+
+  private void setBloomFilterFaulty(BlockType blockType) {
+    if (blockType == BlockType.GENERAL_BLOOM_META) {
+      setGeneralBloomFilterFaulty();
+    } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
+      setDeleteFamilyBloomFilterFaulty();
+    }
+  }
+
+  /**
+   * The number of Bloom filter entries in this store file, or an estimate
+   * thereof, if the Bloom filter is not loaded. This always returns an upper
+   * bound of the number of Bloom filter entries.
+   *
+   * @return an estimate of the number of Bloom filter entries in this file
+   */
+  public long getFilterEntries() {
+    return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
+        : reader.getEntries();
+  }
+
+  public void setGeneralBloomFilterFaulty() {
+    generalBloomFilter = null;
+  }
+
+  public void setDeleteFamilyBloomFilterFaulty() {
+    this.deleteFamilyBloomFilter = null;
+  }
+
+  public Cell getLastKey() {
+    return reader.getLastKey();
+  }
+
+  public byte[] getLastRowKey() {
+    return reader.getLastRowKey();
+  }
+
+  public Cell midkey() throws IOException {
+    return reader.midkey();
+  }
+
+  public long length() {
+    return reader.length();
+  }
+
+  public long getTotalUncompressedBytes() {
+    return reader.getTrailer().getTotalUncompressedBytes();
+  }
+
+  public long getEntries() {
+    return reader.getEntries();
+  }
+
+  public long getDeleteFamilyCnt() {
+    return deleteFamilyCnt;
+  }
+
+  public Cell getFirstKey() {
+    return reader.getFirstKey();
+  }
+
+  public long indexSize() {
+    return reader.indexSize();
+  }
+
+  public BloomType getBloomFilterType() {
+    return this.bloomFilterType;
+  }
+
+  public long getSequenceID() {
+    return sequenceID;
+  }
+
+  public void setSequenceID(long sequenceID) {
+    this.sequenceID = sequenceID;
+  }
+
+  public void setBulkLoaded(boolean bulkLoadResult) {
+    this.bulkLoadResult = bulkLoadResult;
+  }
+
+  public boolean isBulkLoaded() {
+    return this.bulkLoadResult;
+  }
+
+  BloomFilter getGeneralBloomFilter() {
+    return generalBloomFilter;
+  }
+
+  long getUncompressedDataIndexSize() {
+    return reader.getTrailer().getUncompressedDataIndexSize();
+  }
+
+  public long getTotalBloomSize() {
+    if (generalBloomFilter == null)
+      return 0;
+    return generalBloomFilter.getByteSize();
+  }
+
+  public int getHFileVersion() {
+    return reader.getTrailer().getMajorVersion();
+  }
+
+  public int getHFileMinorVersion() {
+    return reader.getTrailer().getMinorVersion();
+  }
+
+  public HFile.Reader getHFileReader() {
+    return reader;
+  }
+
+  void disableBloomFilterForTesting() {
+    generalBloomFilter = null;
+    this.deleteFamilyBloomFilter = null;
+  }
+
+  public long getMaxTimestamp() {
+    return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.getMaximumTimestamp();
+  }
+
+  boolean isSkipResetSeqId() {
+    return skipResetSeqId;
+  }
+
+  void setSkipResetSeqId(boolean skipResetSeqId) {
+    this.skipResetSeqId = skipResetSeqId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7efb9ede/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 924e7f6..e7f8f88 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
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
 import org.apache.hadoop.hbase.util.Counter;
 
 /**
@@ -45,7 +44,7 @@ import org.apache.hadoop.hbase.util.Counter;
 @InterfaceAudience.LimitedPrivate("Coprocessor")
 public class StoreFileScanner implements KeyValueScanner {
   // the reader it comes from:
-  private final StoreFile.Reader reader;
+  private final StoreFileReader reader;
   private final HFileScanner hfs;
   private Cell cur = null;
   private boolean closed = false;
@@ -70,7 +69,7 @@ public class StoreFileScanner implements KeyValueScanner {
    * Implements a {@link KeyValueScanner} on top of the specified {@link HFileScanner}
    * @param hfs HFile scanner
    */
-  public StoreFileScanner(StoreFile.Reader reader, HFileScanner hfs, boolean useMVCC,
+  public StoreFileScanner(StoreFileReader reader, HFileScanner hfs, boolean useMVCC,
       boolean hasMVCC, long readPt) {
     this.readPt = readPt;
     this.reader = reader;
@@ -117,7 +116,7 @@ public class StoreFileScanner implements KeyValueScanner {
     List<StoreFileScanner> scanners = new ArrayList<StoreFileScanner>(
         files.size());
     for (StoreFile file : files) {
-      StoreFile.Reader r = file.createReader(canUseDrop);
+      StoreFileReader r = file.createReader(canUseDrop);
       r.setReplicaStoreFile(isPrimaryReplica);
       StoreFileScanner scanner = r.getStoreFileScanner(cacheBlocks, usePread,
           isCompaction, readPt);
@@ -384,7 +383,7 @@ public class StoreFileScanner implements KeyValueScanner {
     return true;
   }
 
-  Reader getReader() {
+  StoreFileReader getReader() {
     return reader;
   }
 


Mime
View raw message