You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2015/04/09 19:13:21 UTC

hbase git commit: HBASE-13429 Remove deprecated seek/reseek methods from HFileScanner.

Repository: hbase
Updated Branches:
  refs/heads/master 80dbf0665 -> 6eeb5b38e


HBASE-13429 Remove deprecated seek/reseek methods from HFileScanner.


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

Branch: refs/heads/master
Commit: 6eeb5b38e135b54bbe901ef3bdb8bc8acbeb4129
Parents: 80dbf06
Author: anoopsjohn <an...@gmail.com>
Authored: Thu Apr 9 22:42:57 2015 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Thu Apr 9 22:42:57 2015 +0530

----------------------------------------------------------------------
 .../io/encoding/BufferedDataBlockEncoder.java   |  5 --
 .../hbase/io/encoding/DataBlockEncoder.java     | 21 -----
 .../codec/prefixtree/PrefixTreeSeeker.java      | 29 -------
 .../hadoop/hbase/io/HalfStoreFileReader.java    | 36 +-------
 .../hbase/io/hfile/HFilePrettyPrinter.java      |  2 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  | 32 --------
 .../hadoop/hbase/io/hfile/HFileScanner.java     | 86 +++++++++-----------
 .../hadoop/hbase/regionserver/HStore.java       |  3 +-
 .../hbase/regionserver/StoreFileScanner.java    |  3 +-
 9 files changed, 41 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6eeb5b38/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
index be8c192..5b049fd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
@@ -677,11 +677,6 @@ abstract class BufferedDataBlockEncoder implements DataBlockEncoder {
     }
 
     @Override
-    public int seekToKeyInBlock(byte[] key, int offset, int length, boolean seekBefore) {
-      return seekToKeyInBlock(new KeyValue.KeyOnlyKeyValue(key, offset, length), seekBefore);
-    }
-
-    @Override
     public int seekToKeyInBlock(Cell seekCell, boolean seekBefore) {
       int rowCommonPrefix = 0;
       int familyCommonPrefix = 0;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eeb5b38/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java
index 872c22c..8073e54 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/DataBlockEncoder.java
@@ -182,27 +182,6 @@ public interface DataBlockEncoder {
      * previous block if the requested key turns out to be the first key of the
      * current block.</li>
      * </ul>
-     * @param key byte array containing the key
-     * @param offset key position the array
-     * @param length key length in bytes
-     * @param seekBefore find the key strictly less than the given key in case
-     *          of an exact match. Does not matter in case of an inexact match.
-     * @return 0 on exact match, 1 on inexact match.
-     */
-    @Deprecated
-    int seekToKeyInBlock(
-      byte[] key, int offset, int length, boolean seekBefore
-    );
-    /**
-     * Moves the seeker position within the current block to:
-     * <ul>
-     * <li>the last key that that is less than or equal to the given key if
-     * <code>seekBefore</code> is false</li>
-     * <li>the last key that is strictly less than the given key if <code>
-     * seekBefore</code> is true. The caller is responsible for loading the
-     * previous block if the requested key turns out to be the first key of the
-     * current block.</li>
-     * </ul>
      * @param key - Cell to which the seek should happen
      * @param seekBefore find the key strictly less than the given key in case
      *          of an exact match. Does not matter in case of an inexact match.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eeb5b38/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
----------------------------------------------------------------------
diff --git a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
index b95055c..d9852be 100644
--- a/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
+++ b/hbase-prefix-tree/src/main/java/org/apache/hadoop/hbase/codec/prefixtree/PrefixTreeSeeker.java
@@ -140,35 +140,6 @@ public class PrefixTreeSeeker implements EncodedSeeker {
 
   private static final boolean USE_POSITION_BEFORE = false;
 
-  /**
-   * Seek forward only (should be called reseekToKeyInBlock?).
-   * <p/>
-   * If the exact key is found look at the seekBefore variable and:<br/>
-   * - if true: go to the previous key if it's true<br/>
-   * - if false: stay on the exact key
-   * <p/>
-   * If the exact key is not found, then go to the previous key *if possible*, but remember to
-   * leave the scanner in a valid state if possible.
-   * <p/>
-   * @param keyOnlyBytes KeyValue format of a Cell's key at which to position the seeker
-   * @param offset offset into the keyOnlyBytes array
-   * @param length number of bytes of the keyOnlyBytes array to use
-   * @param forceBeforeOnExactMatch if an exact match is found and seekBefore=true, back up 1 Cell
-   * @return 0 if the seeker is on the exact key<br/>
-   *         1 if the seeker is not on the key for any reason, including seekBefore being true
-   */
-  @Override
-  public int seekToKeyInBlock(byte[] keyOnlyBytes, int offset, int length,
-      boolean forceBeforeOnExactMatch) {
-    if (USE_POSITION_BEFORE) {
-      return seekToOrBeforeUsingPositionAtOrBefore(keyOnlyBytes, offset, length,
-          forceBeforeOnExactMatch);
-    } else {
-      return seekToOrBeforeUsingPositionAtOrAfter(keyOnlyBytes, offset, length,
-          forceBeforeOnExactMatch);
-    }
-  }
-
   /*
    * Support both of these options since the underlying PrefixTree supports both.  Possibly
    * expand the EncodedSeeker to utilize them both.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eeb5b38/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
index 43bbab5..58502bb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java
@@ -174,17 +174,6 @@ public class HalfStoreFileReader extends StoreFile.Reader {
       }
 
       @Override
-      public boolean seekBefore(byte[] key) throws IOException {
-        return seekBefore(key, 0, key.length);
-      }
-
-      @Override
-      public boolean seekBefore(byte [] key, int offset, int length)
-      throws IOException {
-        return seekBefore(new KeyValue.KeyOnlyKeyValue(key, offset, length));
-      }
-
-      @Override
       public boolean seekTo() throws IOException {
         if (top) {
           int r = this.delegate.seekTo(new KeyValue.KeyOnlyKeyValue(splitkey, 0, splitkey.length));
@@ -212,29 +201,6 @@ public class HalfStoreFileReader extends StoreFile.Reader {
             splitkey, 0, splitkey.length) < 0;
       }
 
-      @Override
-      public int seekTo(byte[] key) throws IOException {
-        return seekTo(key, 0, key.length);
-      }
-
-      @Override
-      public int seekTo(byte[] key, int offset, int length) throws IOException {
-        return seekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
-      }
-
-      @Override
-      public int reseekTo(byte[] key) throws IOException {
-        return reseekTo(key, 0, key.length);
-      }
-
-      @Override
-      public int reseekTo(byte[] key, int offset, int length)
-      throws IOException {
-        //This function is identical to the corresponding seekTo function except
-        //that we call reseekTo (and not seekTo) on the delegate.
-        return reseekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
-      }
-
       public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
         return this.delegate.getReader();
       }
@@ -338,7 +304,7 @@ public class HalfStoreFileReader extends StoreFile.Reader {
     // Get a scanner that caches the block and that uses pread.
     HFileScanner scanner = getScanner(true, true);
     try {
-      if (scanner.seekBefore(this.splitkey)) {
+      if (scanner.seekBefore(this.splitCell)) {
         return Bytes.toBytes(scanner.getKey());
       }
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eeb5b38/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
index 7b92df9..41779a7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
@@ -264,7 +264,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
       if (this.isSeekToRow) {
         // seek to the first kv on this row
         shouldScanKeysValues =
-          (scanner.seekTo(KeyValueUtil.createFirstOnRow(this.row).getKey()) != -1);
+          (scanner.seekTo(KeyValueUtil.createFirstOnRow(this.row)) != -1);
       } else {
         shouldScanKeysValues = scanner.seekTo();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eeb5b38/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 933ad22..1e84e6a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -478,21 +478,6 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
     }
 
     @Override
-    public int seekTo(byte[] key) throws IOException {
-      return seekTo(key, 0, key.length);
-    }
-
-    @Override
-    public boolean seekBefore(byte[] key) throws IOException {
-      return seekBefore(key, 0, key.length);
-    }
-
-    @Override
-    public int reseekTo(byte[] key) throws IOException {
-      return reseekTo(key, 0, key.length);
-    }
-
-    @Override
     public HFile.Reader getReader() {
       return reader;
     }
@@ -652,18 +637,6 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
     }
 
     @Override
-    public int seekTo(byte[] key, int offset, int length) throws IOException {
-      // Always rewind to the first key of the block, because the given key
-      // might be before or after the current key.
-      return seekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
-    }
-
-    @Override
-    public int reseekTo(byte[] key, int offset, int length) throws IOException {
-      return reseekTo(new KeyValue.KeyOnlyKeyValue(key, offset, length));
-    }
-
-    @Override
     public int seekTo(Cell key) throws IOException {
       return seekTo(key, true);
     }
@@ -724,11 +697,6 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
     }
 
     @Override
-    public boolean seekBefore(byte[] key, int offset, int length) throws IOException {
-      return seekBefore(new KeyValue.KeyOnlyKeyValue(key, offset, length));
-    }
-
-    @Override
     public boolean seekBefore(Cell key) throws IOException {
       HFileBlock seekToBlock = reader.getDataBlockIndexReader().seekToDataBlock(key, block,
           cacheBlocks, pread, isCompaction, reader.getEffectiveEncodingInCache(isCompaction));

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eeb5b38/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
index deaa2c0..2b6e011 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileScanner.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.Cell;
  *
  * <p>A scanner doesn't always have a key/value that it is pointing to
  * when it is first created and before
- * {@link #seekTo()}/{@link #seekTo(byte[])} are called.
+ * {@link #seekTo()}/{@link #seekTo(Cell)} are called.
  * In this case, {@link #getKey()}/{@link #getValue()} returns null.  At most
  * other times, a key and value will be available.  The general pattern is that
  * you position the Scanner using the seekTo variants and then getKey and
@@ -39,69 +39,57 @@ import org.apache.hadoop.hbase.Cell;
 @InterfaceAudience.Private
 public interface HFileScanner {
   /**
-   * SeekTo or just before the passed <code>key</code>.  Examine the return
-   * code to figure whether we found the key or not.
-   * Consider the key stream of all the keys in the file,
-   * <code>k[0] .. k[n]</code>, where there are n keys in the file.
-   * @param key Key to find.
-   * @return -1, if key < k[0], no position;
-   * 0, such that k[i] = key and scanner is left in position i; and
-   * 1, such that k[i] < key, and scanner is left in position i.
-   * The scanner will position itself between k[i] and k[i+1] where
-   * k[i] < key <= k[i+1].
-   * If there is no key k[i+1] greater than or equal to the input key, then the
+   * SeekTo or just before the passed <code>cell</code>.  Examine the return
+   * code to figure whether we found the cell or not.
+   * Consider the cell stream of all the cells in the file,
+   * <code>c[0] .. c[n]</code>, where there are n cells in the file.
+   * @param cell
+   * @return -1, if cell < c[0], no position;
+   * 0, such that c[i] = cell and scanner is left in position i; and
+   * 1, such that c[i] < cell, and scanner is left in position i.
+   * The scanner will position itself between c[i] and c[i+1] where
+   * c[i] < cell <= c[i+1].
+   * If there is no cell c[i+1] greater than or equal to the input cell, then the
    * scanner will position itself at the end of the file and next() will return
    * false when it is called.
    * @throws IOException
    */
-  @Deprecated
-  int seekTo(byte[] key) throws IOException;
-  @Deprecated
-  int seekTo(byte[] key, int offset, int length) throws IOException;
+  int seekTo(Cell cell) throws IOException;
 
-  int seekTo(Cell c) throws IOException;
   /**
-   * Reseek to or just before the passed <code>key</code>. Similar to seekTo
+   * Reseek to or just before the passed <code>cell</code>. Similar to seekTo
    * except that this can be called even if the scanner is not at the beginning
    * of a file.
-   * This can be used to seek only to keys which come after the current position
+   * This can be used to seek only to cells which come after the current position
    * of the scanner.
-   * Consider the key stream of all the keys in the file,
-   * <code>k[0] .. k[n]</code>, where there are n keys in the file after
+   * Consider the cell stream of all the cells in the file,
+   * <code>c[0] .. c[n]</code>, where there are n cellc in the file after
    * current position of HFileScanner.
-   * The scanner will position itself between k[i] and k[i+1] where
-   * k[i] < key <= k[i+1].
-   * If there is no key k[i+1] greater than or equal to the input key, then the
+   * The scanner will position itself between c[i] and c[i+1] where
+   * c[i] < cell <= c[i+1].
+   * If there is no cell c[i+1] greater than or equal to the input cell, then the
    * scanner will position itself at the end of the file and next() will return
    * false when it is called.
-   * @param key Key to find (should be non-null)
-   * @return -1, if key < k[0], no position;
-   * 0, such that k[i] = key and scanner is left in position i; and
-   * 1, such that k[i] < key, and scanner is left in position i.
+   * @param cell Cell to find (should be non-null)
+   * @return -1, if cell < c[0], no position;
+   * 0, such that c[i] = cell and scanner is left in position i; and
+   * 1, such that c[i] < cell, and scanner is left in position i.
    * @throws IOException
    */
-  @Deprecated
-  int reseekTo(byte[] key) throws IOException;
-  @Deprecated
-  int reseekTo(byte[] key, int offset, int length) throws IOException;
+  int reseekTo(Cell cell) throws IOException;
 
-  int reseekTo(Cell c) throws IOException;
   /**
-   * Consider the key stream of all the keys in the file,
-   * <code>k[0] .. k[n]</code>, where there are n keys in the file.
-   * @param key Key to find
-   * @return false if key <= k[0] or true with scanner in position 'i' such
-   * that: k[i] < key.  Furthermore: there may be a k[i+1], such that
-   * k[i] < key <= k[i+1] but there may also NOT be a k[i+1], and next() will
+   * Consider the cell stream of all the cells in the file,
+   * <code>c[0] .. c[n]</code>, where there are n cells in the file.
+   * @param cell Cell to find
+   * @return false if cell <= c[0] or true with scanner in position 'i' such
+   * that: c[i] < cell.  Furthermore: there may be a c[i+1], such that
+   * c[i] < cell <= c[i+1] but there may also NOT be a c[i+1], and next() will
    * return false (EOF).
    * @throws IOException
    */
-  @Deprecated
-  boolean seekBefore(byte[] key) throws IOException;
-  @Deprecated
-  boolean seekBefore(byte[] key, int offset, int length) throws IOException;
+  boolean seekBefore(Cell cell) throws IOException;
 
-  boolean seekBefore(Cell kv) throws IOException;
   /**
    * Positions this scanner at the start of the file.
    * @return False if empty file; i.e. a call to next would return false and
@@ -117,14 +105,14 @@ public interface HFileScanner {
   boolean next() throws IOException;
   /**
    * Gets a buffer view to the current key. You must call
-   * {@link #seekTo(byte[])} before this method.
+   * {@link #seekTo(Cell)} before this method.
    * @return byte buffer for the key. The limit is set to the key size, and the
    * position is 0, the start of the buffer view.
    */
   ByteBuffer getKey();
   /**
    * Gets a buffer view to the current value.  You must call
-   * {@link #seekTo(byte[])} before this method.
+   * {@link #seekTo(Cell)} before this method.
    *
    * @return byte buffer for the value. The limit is set to the value size, and
    * the position is 0, the start of the buffer view.
@@ -136,13 +124,13 @@ public interface HFileScanner {
   Cell getKeyValue();
   /**
    * Convenience method to get a copy of the key as a string - interpreting the
-   * bytes as UTF8. You must call {@link #seekTo(byte[])} before this method.
+   * bytes as UTF8. You must call {@link #seekTo(Cell)} before this method.
    * @return key as a string
    */
   String getKeyString();
   /**
    * Convenience method to get a copy of the value as a string - interpreting
-   * the bytes as UTF8. You must call {@link #seekTo(byte[])} before this method.
+   * the bytes as UTF8. You must call {@link #seekTo(Cell)} before this method.
    * @return value as a string
    */
   String getValueString();
@@ -152,7 +140,7 @@ public interface HFileScanner {
   HFile.Reader getReader();
   /**
    * @return True is scanner has had one of the seek calls invoked; i.e.
-   * {@link #seekBefore(byte[])} or {@link #seekTo()} or {@link #seekTo(byte[])}.
+   * {@link #seekBefore(Cell)} or {@link #seekTo()} or {@link #seekTo(Cell)}.
    * Otherwise returns false.
    */
   boolean isSeeked();

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eeb5b38/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 686df49..042deed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -1870,8 +1870,7 @@ public class HStore implements Store {
     // Unlikely that there'll be an instance of actual first row in table.
     if (walkForwardInSingleRow(scanner, firstOnRow, state)) return true;
     // If here, need to start backing up.
-    while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(),
-       firstOnRow.getKeyLength())) {
+    while (scanner.seekBefore(firstOnRow)) {
       Cell kv = scanner.getKeyValue();
       if (!state.isTargetTable(kv)) break;
       if (!state.isBetterCandidate(kv)) break;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eeb5b38/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 a8ee091..22fd46e 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
@@ -422,8 +422,7 @@ public class StoreFileScanner implements KeyValueScanner {
         KeyValue seekKey = KeyValueUtil.createFirstOnRow(key.getRowArray(), key.getRowOffset(),
             key.getRowLength());
         if (seekCount != null) seekCount.incrementAndGet();
-        if (!hfs.seekBefore(seekKey.getBuffer(), seekKey.getKeyOffset(),
-            seekKey.getKeyLength())) {
+        if (!hfs.seekBefore(seekKey)) {
           close();
           return false;
         }