You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2016/03/23 22:37:48 UTC

hbase git commit: HBASE-15392 Single Cell Get reads two HFileBlocks

Repository: hbase
Updated Branches:
  refs/heads/master 95e6d2276 -> d7a4499df


HBASE-15392 Single Cell Get reads two HFileBlocks

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
moreRowsMayExistAfterCell Exploit the fact a Scan is a Get Scan. Also save compares
if no non-default stopRow.

M hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
optimize Add doc on what is being optimized. Also, if a Get Scan, do not
optimize else we'll keep going after our row is DONE.
Another place to make use of the Get Scan fact is when we are DONE.. if
Get Scan, we can close out the scan.

M hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
Add tests for Get Scans and optimize around block loading.


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

Branch: refs/heads/master
Commit: d7a4499dfc8b3936a0eca867589fc2b23b597866
Parents: 95e6d22
Author: stack <st...@apache.org>
Authored: Fri Mar 11 15:41:26 2016 -0800
Committer: stack <st...@apache.org>
Committed: Wed Mar 23 14:37:14 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/CellComparator.java |   2 +-
 .../hbase/io/hfile/CombinedBlockCache.java      |  13 +-
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java  |   2 +-
 .../hbase/regionserver/KeyValueScanner.java     |  16 +-
 .../hbase/regionserver/ScanQueryMatcher.java    |  60 +++--
 .../hadoop/hbase/regionserver/StoreScanner.java |  85 ++++++-
 .../hbase/util/CollectionBackedScanner.java     |   3 +-
 .../hbase/regionserver/KeyValueScanFixture.java |   8 +-
 .../regionserver/TestKeyValueScanFixture.java   |   3 +-
 .../hbase/regionserver/TestStoreScanner.java    | 244 ++++++++++++++++++-
 10 files changed, 372 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
index a5e26cf..4a5c0b7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
@@ -433,7 +433,7 @@ public class CellComparator implements Comparator<Cell>, Serializable {
   /**
    * Used to compare two cells based on the column hint provided. This is specifically
    * used when we need to optimize the seeks based on the next indexed key. This is an
-   * advance usage API specifically needed for some optimizations.
+   * advanced usage API specifically needed for some optimizations.
    * @param nextIndexedCell the next indexed cell 
    * @param currentCell the cell to be compared
    * @param foff the family offset of the currentCell

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
index 22bffee..666b357 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
@@ -63,8 +63,8 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
   @Override
   public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory,
       final boolean cacheDataInL1) {
-    boolean isMetaBlock = buf.getBlockType().getCategory() != BlockCategory.DATA;
-    if (isMetaBlock || cacheDataInL1) {
+    boolean metaBlock = buf.getBlockType().getCategory() != BlockCategory.DATA;
+    if (metaBlock || cacheDataInL1) {
       lruCache.cacheBlock(cacheKey, buf, inMemory, cacheDataInL1);
     } else {
       l2Cache.cacheBlock(cacheKey, buf, inMemory, false);
@@ -81,12 +81,9 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
       boolean repeat, boolean updateCacheMetrics) {
     // TODO: is there a hole here, or just awkwardness since in the lruCache getBlock
     // we end up calling l2Cache.getBlock.
-    if (lruCache.containsBlock(cacheKey)) {
-      return lruCache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
-    }
-    Cacheable result = l2Cache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
-
-    return result;
+    return lruCache.containsBlock(cacheKey)?
+        lruCache.getBlock(cacheKey, caching, repeat, updateCacheMetrics):
+        l2Cache.getBlock(cacheKey, caching, repeat, updateCacheMetrics);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index d310d13..9ab46cf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -416,7 +416,7 @@ public class HFileWriterImpl implements HFile.Writer {
     // No opportunity for optimization. Just return right key.
     return right;
   }
-  
+
   /**
    * @param leftArray
    * @param leftOffset

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
index 3df284c..ed86a83 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/KeyValueScanner.java
@@ -30,6 +30,9 @@ import org.apache.hadoop.hbase.client.Scan;
  * Scanner that returns the next KeyValue.
  */
 @InterfaceAudience.Private
+// TODO: Change name from KeyValueScanner to CellScanner only we already have a simple CellScanner
+// so this should be something else altogether, a decoration on our base CellScanner. TODO.
+// This class shows in CPs so do it all in one swell swoop. HBase-2.0.0.
 public interface KeyValueScanner extends Shipper, Closeable {
   /**
    * The byte array represents for NO_NEXT_INDEXED_KEY;
@@ -136,11 +139,11 @@ public interface KeyValueScanner extends Shipper, Closeable {
    * peek KeyValue of scanner has the same row with specified Cell,
    * otherwise seek the scanner at the first Cell of the row which is the
    * previous row of specified KeyValue
-   * 
+   *
    * @param key seek KeyValue
    * @return true if the scanner is at the valid KeyValue, false if such
    *         KeyValue does not exist
-   * 
+   *
    */
   public boolean backwardSeek(Cell key) throws IOException;
 
@@ -155,7 +158,7 @@ public interface KeyValueScanner extends Shipper, Closeable {
 
   /**
    * Seek the scanner at the first KeyValue of last row
-   * 
+   *
    * @return true if scanner has values left, false if the underlying data is
    *         empty
    * @throws IOException
@@ -163,8 +166,9 @@ public interface KeyValueScanner extends Shipper, Closeable {
   public boolean seekToLastRow() throws IOException;
 
   /**
-   * @return the next key in the index (the key to seek to the next block)
-   * if known, or null otherwise
+   * @return the next key in the index, usually the first key of next block OR a key that falls
+   * between last key of current block and first key of next block..
+   * see HFileWriterImpl#getMidpoint, or null if not known.
    */
   public Cell getNextIndexedKey();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
index c220b5c..706fc5b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java
@@ -93,7 +93,7 @@ public class ScanQueryMatcher {
   /* row is not private for tests */
   /** Row the query is on */
   Cell curCell;
-  
+
   /**
    * Oldest put in any of the involved store files
    * Used to decide whether it is ok to delete
@@ -119,7 +119,7 @@ public class ScanQueryMatcher {
    * first column.
    * */
   private boolean hasNullColumn = true;
-  
+
   private RegionCoprocessorHost regionCoprocessorHost= null;
 
   // By default, when hbase.hstore.time.to.purge.deletes is 0ms, a delete
@@ -140,22 +140,22 @@ public class ScanQueryMatcher {
   // currently influencing. This is because Puts, that this delete can
   // influence.  may appear out of order.
   private final long timeToPurgeDeletes;
-  
+
   private final boolean isUserScan;
 
   private final boolean isReversed;
 
   /**
+   * True if we are doing a 'Get' Scan. Every Get is actually a one-row Scan.
+   */
+  private final boolean get;
+
+  /**
    * Construct a QueryMatcher for a scan
-   * @param scan
    * @param scanInfo The store's immutable scan info
-   * @param columns
    * @param scanType Type of the scan
    * @param earliestPutTs Earliest put seen in any of the store files.
-   * @param oldestUnexpiredTS the oldest timestamp we are interested in,
-   *  based on TTL
-   * @param regionCoprocessorHost 
-   * @throws IOException 
+   * @param oldestUnexpiredTS the oldest timestamp we are interested in, based on TTL
    */
   public ScanQueryMatcher(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       ScanType scanType, long readPointToUse, long earliestPutTs, long oldestUnexpiredTS,
@@ -166,6 +166,7 @@ public class ScanQueryMatcher {
     } else {
       this.tr = timeRange;
     }
+    this.get = scan.isGetScan();
     this.rowComparator = scanInfo.getComparator();
     this.regionCoprocessorHost = regionCoprocessorHost;
     this.deletes =  instantiateDeleteTracker();
@@ -234,8 +235,8 @@ public class ScanQueryMatcher {
    * @param now the current server time
    * @param dropDeletesFromRow The inclusive left bound of the range; can be EMPTY_START_ROW.
    * @param dropDeletesToRow The exclusive right bound of the range; can be EMPTY_END_ROW.
-   * @param regionCoprocessorHost 
-   * @throws IOException 
+   * @param regionCoprocessorHost
+   * @throws IOException
    */
   public ScanQueryMatcher(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,
       long readPointToUse, long earliestPutTs, long oldestUnexpiredTS, long now,
@@ -280,7 +281,7 @@ public class ScanQueryMatcher {
    *      caused by a data corruption.
    */
   public MatchCode match(Cell cell) throws IOException {
-      if (filter != null && filter.filterAllRemaining()) {
+    if (filter != null && filter.filterAllRemaining()) {
       return MatchCode.DONE_SCAN;
     }
     if (curCell != null) {
@@ -324,7 +325,7 @@ public class ScanQueryMatcher {
     // check if the cell is expired by cell TTL
     if (HStore.isCellTTLExpired(cell, this.oldestUnexpiredTS, this.now)) {
       return MatchCode.SKIP;
-    }    
+    }
 
     /*
      * The delete logic is pretty complicated now.
@@ -359,10 +360,10 @@ public class ScanQueryMatcher {
         }
         // Can't early out now, because DelFam come before any other keys
       }
-     
+
       if ((!isUserScan)
           && timeToPurgeDeletes > 0
-          && (EnvironmentEdgeManager.currentTime() - timestamp) 
+          && (EnvironmentEdgeManager.currentTime() - timestamp)
             <= timeToPurgeDeletes) {
         return MatchCode.INCLUDE;
       } else if (retainDeletesInOutput || mvccVersion > maxReadPointToTrackVersions) {
@@ -503,22 +504,27 @@ public class ScanQueryMatcher {
     }
   }
 
+  /**
+   * @return Returns false if we know there are no more rows to be scanned (We've reached the
+   * <code>stopRow</code> or we are scanning on row only because this Scan is for a Get, etc.
+   */
   public boolean moreRowsMayExistAfter(Cell kv) {
-    if (this.isReversed) {
-      if (rowComparator.compareRows(kv, stopRow, 0, stopRow.length) <= 0) {
-        return false;
-      } else {
-        return true;
-      }
-    }
-    if (!Bytes.equals(stopRow , HConstants.EMPTY_END_ROW) &&
-        rowComparator.compareRows(kv, stopRow, 0, stopRow.length) >= 0) {
-      // KV >= STOPROW
-      // then NO there is nothing left.
+    // If a 'get' Scan -- we are doing a Get (every Get is a single-row Scan in implementation) --
+    // then we are looking at one row only, the one specified in the Get coordinate..so we know
+    // for sure that there are no more rows on this Scan
+    if (this.get) {
       return false;
-    } else {
+    }
+    // If no stopRow, return that there may be more rows. The tests that follow depend on a
+    // non-empty, non-default stopRow so this little test below short-circuits out doing the
+    // following compares.
+    if (this.stopRow == null || this.stopRow == HConstants.EMPTY_BYTE_ARRAY) {
       return true;
     }
+    return this.isReversed?
+      rowComparator.compareRows(kv, stopRow, 0, stopRow.length) > 0:
+      Bytes.equals(stopRow, HConstants.EMPTY_END_ROW) ||
+        rowComparator.compareRows(kv, stopRow, 0, stopRow.length) < 0;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 2f0d284..8dd3d7d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -133,7 +133,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
   protected List<KeyValueScanner> currentScanners = new ArrayList<KeyValueScanner>();
   // flush update lock
   private ReentrantLock flushLock = new ReentrantLock();
-  
+
   protected final long readPt;
 
   // used by the injection framework to test race between StoreScanner construction and compaction
@@ -600,6 +600,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
         continue;
 
       case DONE:
+        // Optimization for Gets! If DONE, no more to get on this row, early exit!
+        if (this.scan.isGetScan()) {
+          // Then no more to this row... exit.
+          close(false);// Do all cleanup except heap.close()
+          return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
+        }
         matcher.curCell = null;
         return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
 
@@ -649,11 +655,60 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
   }
 
-  /*
-   * See if we should actually SEEK or rather just SKIP to the next Cell.
-   * (see HBASE-13109)
+  /**
+   * See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109).
+   * This method works together with ColumnTrackers and Filters. ColumnTrackers may issue SEEK
+   * hints, such as seek to next column, next row, or seek to an arbitrary seek key.
+   * This method intercepts these qcodes and decides whether a seek is the most efficient _actual_
+   * way to get us to the requested cell (SEEKs are more expensive than SKIP, SKIP, SKIP inside the
+   * current, loaded block).
+   * It does this by looking at the next indexed key of the current HFile. This key
+   * is then compared with the _SEEK_ key, where a SEEK key is an artificial 'last possible key
+   * on the row' (only in here, we avoid actually creating a SEEK key; in the compare we work with
+   * the current Cell but compare as though it were a seek key; see down in
+   * matcher.compareKeyForNextRow, etc). If the compare gets us onto the
+   * next block we *_SEEK, otherwise we just INCLUDE or SKIP, and let the ColumnTrackers or Filters
+   * go through the next Cell, and so on)
+   *
+   * <p>The ColumnTrackers and Filters must behave correctly in all cases, i.e. if they are past the
+   * Cells they care about they must issues a SKIP or SEEK.
+   *
+   * <p>Other notes:
+   * <ul>
+   * <li>Rows can straddle block boundaries</li>
+   * <li>Versions of columns can straddle block boundaries (i.e. column C1 at T1 might be in a
+   * different block than column C1 at T2)</li>
+   * <li>We want to SKIP and INCLUDE if the chance is high that we'll find the desired Cell after a
+   * few SKIPs...</li>
+   * <li>We want to INCLUDE_AND_SEEK and SEEK when the chance is high that we'll be able to seek
+   * past many Cells, especially if we know we need to go to the next block.</li>
+   * </ul>
+   * <p>A good proxy (best effort) to determine whether INCLUDE/SKIP is better than SEEK is whether
+   * we'll likely end up seeking to the next block (or past the next block) to get our next column.
+   * Example:
+   * <pre>
+   * |    BLOCK 1              |     BLOCK 2                   |
+   * |  r1/c1, r1/c2, r1/c3    |    r1/c4, r1/c5, r2/c1        |
+   *                                   ^         ^
+   *                                   |         |
+   *                           Next Index Key   SEEK_NEXT_ROW (before r2/c1)
+   *
+   *
+   * |    BLOCK 1                       |     BLOCK 2                      |
+   * |  r1/c1/t5, r1/c1/t4, r1/c1/t3    |    r1/c1/t2, r1/c1/T1, r1/c2/T3  |
+   *                                            ^              ^
+   *                                            |              |
+   *                                    Next Index Key        SEEK_NEXT_COL
+   * </pre>
+   * Now imagine we want columns c1 and c3 (see first diagram above), the 'Next Index Key' of r1/c4
+   * is > r1/c3 so we should seek to get to the c1 on the next row, r2. In second case, say we only
+   * want one version of c1, after we have it, a SEEK_COL will be issued to get to c2. Looking at
+   * the 'Next Index Key', it would land us in the next block, so we should SEEK. In other scenarios
+   * where the SEEK will not land us in the next block, it is very likely better to issues a series
+   * of SKIPs.
    */
-  private ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
+  @VisibleForTesting
+  protected ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
     switch(qcode) {
     case INCLUDE_AND_SEEK_NEXT_COL:
     case SEEK_NEXT_COL:
@@ -668,10 +723,16 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     case INCLUDE_AND_SEEK_NEXT_ROW:
     case SEEK_NEXT_ROW:
     {
-      Cell nextIndexedKey = getNextIndexedKey();
-      if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
-          && matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0) {
-        return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
+      // If it is a Get Scan, then we know that we are done with this row; there are no more
+      // rows beyond the current one: don't try to optimize. We are DONE. Return the *_NEXT_ROW
+      // qcode as is. When the caller gets these flags on a Get Scan, it knows it can shut down the
+      // Scan.
+      if (!this.scan.isGetScan()) {
+        Cell nextIndexedKey = getNextIndexedKey();
+        if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY
+            && matcher.compareKeyForNextRow(nextIndexedKey, cell) > 0) {
+          return qcode == MatchCode.SEEK_NEXT_ROW ? MatchCode.SKIP : MatchCode.INCLUDE;
+        }
       }
       break;
     }
@@ -809,10 +870,10 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     // check the var without any lock. Suppose even if we see the old
     // value here still it is ok to continue because we will not be resetting
     // the heap but will continue with the referenced memstore's snapshot. For compactions
-    // any way we don't need the updateReaders at all to happen as we still continue with 
+    // any way we don't need the updateReaders at all to happen as we still continue with
     // the older files
     if (flushed) {
-      // If there is a flush and the current scan is notified on the flush ensure that the 
+      // If there is a flush and the current scan is notified on the flush ensure that the
       // scan's heap gets reset and we do a seek on the newly flushed file.
       if(!this.closing) {
         this.lastTop = this.peek();
@@ -842,7 +903,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
     if (scanners.isEmpty()) return;
     int storeFileScannerCount = scanners.size();
     CountDownLatch latch = new CountDownLatch(storeFileScannerCount);
-    List<ParallelSeekHandler> handlers = 
+    List<ParallelSeekHandler> handlers =
         new ArrayList<ParallelSeekHandler>(storeFileScannerCount);
     for (KeyValueScanner scanner : scanners) {
       if (scanner instanceof StoreFileScanner) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
index 9fc068f..4720880 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CollectionBackedScanner.java
@@ -30,8 +30,7 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.regionserver.NonReversedNonLazyKeyValueScanner;
 
 /**
- * Utility scanner that wraps a sortable collection and serves
- * as a KeyValueScanner.
+ * Utility scanner that wraps a sortable collection and serves as a KeyValueScanner.
  */
 @InterfaceAudience.Private
 public class CollectionBackedScanner extends NonReversedNonLazyKeyValueScanner {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
index 3f87a00..a4e7f9b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/KeyValueScanFixture.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.regionserver;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.CollectionBackedScanner;
@@ -33,9 +34,8 @@ import org.apache.hadoop.hbase.util.CollectionBackedScanner;
  * to be a store file scanner.
  */
 public class KeyValueScanFixture extends CollectionBackedScanner {
-  public KeyValueScanFixture(CellComparator comparator,
-                             KeyValue... incData) {
-    super(comparator, incData);
+  public KeyValueScanFixture(CellComparator comparator, Cell... cells) {
+    super(comparator, cells);
   }
 
   public static List<KeyValueScanner> scanFixture(KeyValue[] ... kvArrays) {
@@ -45,4 +45,4 @@ public class KeyValueScanFixture extends CollectionBackedScanner {
     }
     return scanners;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
index a8c2c65..0e96682 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestKeyValueScanFixture.java
@@ -46,8 +46,7 @@ public class TestKeyValueScanFixture extends TestCase {
         KeyValueTestUtil.create("RowB", "family", "qf1",
             10, KeyValue.Type.Put, "value-10")
     };
-    KeyValueScanner scan = new KeyValueScanFixture(
-        CellComparator.COMPARATOR, kvs);
+    KeyValueScanner scan = new KeyValueScanFixture(CellComparator.COMPARATOR, kvs);
 
     KeyValue kv = KeyValueUtil.createFirstOnRow(Bytes.toBytes("RowA"));
     // should seek to this:

http://git-wip-us.apache.org/repos/asf/hbase/blob/d7a4499d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
index 92c85aa..4c594b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java
@@ -20,6 +20,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture;
+import static org.junit.Assert.*;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -27,16 +28,21 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.NavigableSet;
 import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
+import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -53,16 +59,113 @@ import org.junit.rules.TestRule;
 // Can't be small as it plays with EnvironmentEdgeManager
 @Category({RegionServerTests.class, MediumTests.class})
 public class TestStoreScanner {
+  private static final Log LOG = LogFactory.getLog(TestStoreScanner.class);
   @Rule public TestName name = new TestName();
   @Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
       withLookingForStuckThread(true).build();
   private static final String CF_STR = "cf";
-  final byte [] CF = Bytes.toBytes(CF_STR);
+  private static final byte [] CF = Bytes.toBytes(CF_STR);
   static Configuration CONF = HBaseConfiguration.create();
   private ScanInfo scanInfo = new ScanInfo(CONF, CF, 0, Integer.MAX_VALUE,
       Long.MAX_VALUE, KeepDeletedCells.FALSE, 0, CellComparator.COMPARATOR);
   private ScanType scanType = ScanType.USER_SCAN;
 
+  /**
+   * From here on down, we have a bunch of defines and specific CELL_GRID of Cells. The
+   * CELL_GRID then has a Scanner that can fake out 'block' transitions. All this elaborate
+   * setup is for tests that ensure we don't overread, and that the
+   * {@link StoreScanner#optimize(org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode,
+   * Cell)} is not overly enthusiastic.
+   */
+  private static final byte [] ZERO = new byte [] {'0'};
+  private static final byte [] ZERO_POINT_ZERO = new byte [] {'0', '.', '0'};
+  private static final byte [] ONE = new byte [] {'1'};
+  private static final byte [] TWO = new byte [] {'2'};
+  private static final byte [] TWO_POINT_TWO = new byte [] {'2', '.', '2'};
+  private static final byte [] THREE = new byte [] {'3'};
+  private static final byte [] FOUR = new byte [] {'4'};
+  private static final byte [] FIVE = new byte [] {'5'};
+  private static final byte [] VALUE = new byte [] {'v'};
+  private static final int CELL_GRID_BLOCK2_BOUNDARY = 4;
+  private static final int CELL_GRID_BLOCK3_BOUNDARY = 11;
+  private static final int CELL_GRID_BLOCK4_BOUNDARY = 15;
+  private static final int CELL_GRID_BLOCK5_BOUNDARY = 19;
+
+  /**
+   * Five rows by four columns distinguished by column qualifier (column qualifier is one of the
+   * four rows... ONE, TWO, etc.). Exceptions are a weird row after TWO; it is TWO_POINT_TWO.
+   * And then row FOUR has five columns finishing w/ row FIVE having a single column.
+   * We will use this to test scan does the right thing as it
+   * we do Gets, StoreScanner#optimize, and what we do on (faked) block boundaries.
+   */
+  private static final Cell [] CELL_GRID = new Cell [] {
+    CellUtil.createCell(ONE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(ONE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(ONE, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(ONE, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 4 CELL_GRID_BLOCK2_BOUNDARY
+    CellUtil.createCell(TWO, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO_POINT_TWO, CF, ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO_POINT_TWO, CF, ZERO_POINT_ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(TWO_POINT_TWO, CF, FIVE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 11! CELL_GRID_BLOCK3_BOUNDARY
+    CellUtil.createCell(THREE, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(THREE, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(THREE, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(THREE, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 15 CELL_GRID_BLOCK4_BOUNDARY
+    CellUtil.createCell(FOUR, CF, ONE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FOUR, CF, TWO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FOUR, CF, THREE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FOUR, CF, FOUR, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    // Offset 19 CELL_GRID_BLOCK5_BOUNDARY
+    CellUtil.createCell(FOUR, CF, FIVE, 1L, KeyValue.Type.Put.getCode(), VALUE),
+    CellUtil.createCell(FIVE, CF, ZERO, 1L, KeyValue.Type.Put.getCode(), VALUE),
+  };
+
+  /**
+   * A StoreScanner for our CELL_GRID above. Fakes the block transitions. Does counts of
+   * calls to optimize and counts of when optimize actually did an optimize.
+   */
+  private static class CellGridStoreScanner extends StoreScanner {
+    // Count of how often optimize is called and of how often it does an optimize.
+    final AtomicInteger count = new AtomicInteger(0);
+    final AtomicInteger optimization = new AtomicInteger(0);
+
+    CellGridStoreScanner(final Scan scan, ScanInfo scanInfo, ScanType scanType)
+    throws IOException {
+      super(scan, scanInfo, scanType, scan.getFamilyMap().get(CF),
+        Arrays.<KeyValueScanner>asList(
+          new KeyValueScanner[] {new KeyValueScanFixture(CellComparator.COMPARATOR, CELL_GRID)}));
+    }
+
+    protected ScanQueryMatcher.MatchCode optimize(ScanQueryMatcher.MatchCode qcode, Cell cell) {
+      count.incrementAndGet();
+      ScanQueryMatcher.MatchCode after = super.optimize(qcode, cell);
+      LOG.info("Cell=" + cell + ", nextIndex=" + CellUtil.toString(getNextIndexedKey(), false) +
+          ", before=" + qcode + ", after=" + after);
+      if (qcode != after) {
+        optimization.incrementAndGet();
+      }
+      return after;
+    };
+
+    @Override
+    public Cell getNextIndexedKey() {
+      // Fake block boundaries by having index of next block change as we go through scan.
+      return count.get() > CELL_GRID_BLOCK4_BOUNDARY?
+          CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK5_BOUNDARY]):
+            count.get() > CELL_GRID_BLOCK3_BOUNDARY?
+                CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK4_BOUNDARY]):
+                  count.get() > CELL_GRID_BLOCK2_BOUNDARY?
+                      CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK3_BOUNDARY]):
+                        CellUtil.createFirstOnRow(CELL_GRID[CELL_GRID_BLOCK2_BOUNDARY]);
+    }
+  };
+
   /*
    * Test utility for building a NavigableSet for scanners.
    * @param strCols
@@ -78,6 +181,145 @@ public class TestStoreScanner {
   }
 
   @Test
+  public void testFullRowGetDoesNotOverreadWhenRowInsideOneBlock() throws IOException {
+    // Do a Get against row two. Row two is inside a block that starts with row TWO but ends with
+    // row TWO_POINT_TWO. We should read one block only.
+    Get get = new Get(TWO);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      while (scanner.next(results)) {
+        continue;
+      }
+      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
+      // TWO_POINT_TWO row does not have a a column ONE.
+      Assert.assertEquals(4, results.size());
+      // We should have gone the optimize route 5 times totally... an INCLUDE for the four cells
+      // in the row plus the DONE on the end.
+      Assert.assertEquals(5, scanner.count.get());
+      // For a full row Get, there should be no opportunity for scanner optimization.
+      Assert.assertEquals(0, scanner.optimization.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  @Test
+  public void testFullRowSpansBlocks() throws IOException {
+    // Do a Get against row FOUR. It spans two blocks.
+    Get get = new Get(FOUR);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      while (scanner.next(results)) {
+        continue;
+      }
+      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
+      // TWO_POINT_TWO row does not have a a column ONE.
+      Assert.assertEquals(5, results.size());
+      // We should have gone the optimize route 6 times totally... an INCLUDE for the five cells
+      // in the row plus the DONE on the end.
+      Assert.assertEquals(6, scanner.count.get());
+      // For a full row Get, there should be no opportunity for scanner optimization.
+      Assert.assertEquals(0, scanner.optimization.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  /**
+   * Test optimize in StoreScanner. Test that we skip to the next 'block' when we it makes sense
+   * reading the block 'index'.
+   * @throws IOException
+   */
+  @Test
+  public void testOptimize() throws IOException {
+    Scan scan = new Scan();
+    // A scan that just gets the first qualifier on each row of the CELL_GRID
+    scan.addColumn(CF, ONE);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      while (scanner.next(results)) {
+        continue;
+      }
+      // Should be four results of column 1 (though there are 5 rows in the CELL_GRID -- the
+      // TWO_POINT_TWO row does not have a a column ONE.
+      Assert.assertEquals(4, results.size());
+      for (Cell cell: results) {
+        assertTrue(Bytes.equals(ONE, 0, ONE.length,
+            cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
+      }
+      Assert.assertTrue("Optimize should do some optimizations", scanner.optimization.get() > 0);
+    } finally {
+      scanner.close();
+    }
+  }
+
+  /**
+   * Ensure the optimize Scan method in StoreScanner does not get in the way of a Get doing minimum
+   * work... seeking to start of block and then SKIPPING until we find the wanted Cell.
+   * This 'simple' scenario mimics case of all Cells fitting inside a single HFileBlock.
+   * See HBASE-15392. This test is a little cryptic. Takes a bit of staring to figure what it up to.
+   * @throws IOException
+   */
+  @Test
+  public void testOptimizeAndGet() throws IOException {
+    // First test a Get of two columns in the row R2. Every Get is a Scan. Get columns named
+    // R2 and R3.
+    Get get = new Get(TWO);
+    get.addColumn(CF, TWO);
+    get.addColumn(CF, THREE);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      // For a Get there should be no more next's after the first call.
+      Assert.assertEquals(false, scanner.next(results));
+      // Should be one result only.
+      Assert.assertEquals(2, results.size());
+      // And we should have gone through optimize twice only.
+      Assert.assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
+        3, scanner.count.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  /**
+   * Ensure that optimize does not cause the Get to do more seeking than required. Optimize
+   * (see HBASE-15392) was causing us to seek all Cells in a block when a Get Scan if the next block
+   * index/start key was a different row to the current one. A bug. We'd call next too often
+   * because we had to exhaust all Cells in the current row making us load the next block just to
+   * discard what we read there. This test is a little cryptic. Takes a bit of staring to figure
+   * what it up to.
+   * @throws IOException
+   */
+  @Test
+  public void testOptimizeAndGetWithFakedNextBlockIndexStart() throws IOException {
+    // First test a Get of second column in the row R2. Every Get is a Scan. Second column has a
+    // qualifier of R2.
+    Get get = new Get(THREE);
+    get.addColumn(CF, TWO);
+    Scan scan = new Scan(get);
+    CellGridStoreScanner scanner = new CellGridStoreScanner(scan, this.scanInfo, this.scanType);
+    try {
+      List<Cell> results = new ArrayList<Cell>();
+      // For a Get there should be no more next's after the first call.
+      Assert.assertEquals(false, scanner.next(results));
+      // Should be one result only.
+      Assert.assertEquals(1, results.size());
+      // And we should have gone through optimize twice only.
+      Assert.assertEquals("First qcode is SEEK_NEXT_COL and second INCLUDE_AND_SEEK_NEXT_ROW",
+        2, scanner.count.get());
+    } finally {
+      scanner.close();
+    }
+  }
+
+  @Test
   public void testScanTimeRange() throws IOException {
     String r1 = "R1";
     // returns only 1 of these 2 even though same timestamp