You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2016/03/14 20:01:13 UTC

phoenix git commit: PHOENIX-2756 - FilteredKeyValueScanner should not implement KeyValueScanner

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 7d67e205d -> 92f2551db


PHOENIX-2756 - FilteredKeyValueScanner should not implement KeyValueScanner


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 92f2551db3339d30857eff3fd38896400d10d348
Parents: 7d67e20
Author: rahul gidwani <rg...@salesforce.com>
Authored: Thu Mar 10 14:45:26 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Mon Mar 14 11:55:59 2016 -0700

----------------------------------------------------------------------
 .../index/scanner/FilteredKeyValueScanner.java  | 72 +++-----------------
 .../phoenix/hbase/index/scanner/Scanner.java    |  4 +-
 .../hbase/index/scanner/ScannerBuilder.java     |  6 +-
 3 files changed, 14 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/92f2551d/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
index 1f16bef..69761ba 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
@@ -19,11 +19,10 @@
 package org.apache.phoenix.hbase.index.scanner;
 
 import java.io.IOException;
-import java.util.SortedSet;
 
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
 import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
@@ -34,7 +33,7 @@ import org.apache.phoenix.hbase.index.covered.KeyValueStore;
  * here because we are only concerned with a single MemStore for the index; we don't need to worry about multiple column
  * families or minimizing seeking through file - we just want to iterate the kvs quickly, in-memory.
  */
-public class FilteredKeyValueScanner implements KeyValueScanner {
+public class FilteredKeyValueScanner implements Scanner {
 
     private KeyValueScanner delegate;
     private Filter filter;
@@ -49,7 +48,7 @@ public class FilteredKeyValueScanner implements KeyValueScanner {
     }
 
     @Override
-    public KeyValue peek() {
+    public Cell peek() {
         return delegate.peek();
     }
 
@@ -61,16 +60,16 @@ public class FilteredKeyValueScanner implements KeyValueScanner {
      *         filters.
      */
     @Override
-    public KeyValue next() throws IOException {
+    public Cell next() throws IOException {
         seekToNextUnfilteredKeyValue();
         return delegate.next();
     }
 
     @Override
-    public boolean seek(KeyValue key) throws IOException {
+    public boolean seek(Cell key) throws IOException {
         if (filter.filterAllRemaining()) { return false; }
         // see if we can seek to the next key
-        if (!delegate.seek(key)) { return false; }
+        if (!delegate.seek(KeyValueUtil.ensureKeyValue(key))) { return false; }
 
         return seekToNextUnfilteredKeyValue();
     }
@@ -102,67 +101,14 @@ public class FilteredKeyValueScanner implements KeyValueScanner {
         }
     }
 
-    @Override
-    public boolean reseek(KeyValue key) throws IOException {
-        this.delegate.reseek(key);
+    public boolean reseek(Cell key) throws IOException {
+        this.delegate.reseek(KeyValueUtil.ensureKeyValue(key));
         return this.seekToNextUnfilteredKeyValue();
     }
 
     @Override
-    public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom) throws IOException {
-        return this.reseek(kv);
-    }
-
-    @Override
-    public boolean isFileScanner() {
-        return false;
-    }
-
-    @Override
-    public long getSequenceID() {
-        return this.delegate.getSequenceID();
-    }
-
-    @Override
-    public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns, long oldestUnexpiredTS) {
-        throw new UnsupportedOperationException(this.getClass().getName()
-                + " doesn't support checking to see if it should use a scanner!");
-    }
-
-    @Override
-    public boolean realSeekDone() {
-        return this.delegate.realSeekDone();
-    }
-
-    @Override
-    public void enforceSeek() throws IOException {
-        this.delegate.enforceSeek();
-    }
-
-    @Override
     public void close() {
         this.delegate.close();
     }
 
-    @Override
-    public boolean backwardSeek(KeyValue arg0) throws IOException {
-        return this.delegate.backwardSeek(arg0);
-    }
-
-    @Override
-    public boolean seekToLastRow() throws IOException {
-        return this.delegate.seekToLastRow();
-    }
-
-    @Override
-    public boolean seekToPreviousRow(KeyValue arg0) throws IOException {
-        return this.delegate.seekToPreviousRow(arg0);
-    }
-
-    // Added for compatibility with HBASE-13109
-    // Once we drop support for older versions, add an @override annotation here
-    // and figure out how to get the next indexed key
-    public byte[] getNextIndexedKey() {
-        return null; // indicate that we cannot use the optimization
-    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92f2551d/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/Scanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/Scanner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/Scanner.java
index 524df84..9454de5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/Scanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/Scanner.java
@@ -34,7 +34,7 @@ public interface Scanner extends Closeable {
    * @return the next keyvalue in the scanner or <tt>null</tt> if there is no next {@link KeyValue}
    * @throws IOException if there is an underlying error reading the data
    */
-  public KeyValue next() throws IOException;
+  public Cell next() throws IOException;
 
   /**
    * Seek to immediately before the given {@link KeyValue}. If that exact {@link KeyValue} is
@@ -52,5 +52,5 @@ public interface Scanner extends Closeable {
    * @return the next {@link KeyValue} or <tt>null</tt> if there are no more values in <tt>this</tt>
    * @throws IOException if there is an error reading the underlying data.
    */
-  public KeyValue peek() throws IOException;
+  public Cell peek() throws IOException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/92f2551d/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
index 10f2e1b..4333de6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
@@ -127,7 +127,7 @@ public class ScannerBuilder {
     return new Scanner() {
 
       @Override
-      public KeyValue next() {
+      public Cell next() {
         try {
           return kvScanner.next();
         } catch (IOException e) {
@@ -139,7 +139,7 @@ public class ScannerBuilder {
       public boolean seek(Cell next) throws IOException {
         // check to see if the next kv is after the current key, in which case we can use reseek,
         // which will be more efficient
-        KeyValue peek = kvScanner.peek();
+        Cell peek = kvScanner.peek();
         // there is another value and its before the requested one - we can do a reseek!
         if (peek != null) {
           int compare = KeyValue.COMPARATOR.compare(peek, next);
@@ -154,7 +154,7 @@ public class ScannerBuilder {
       }
 
       @Override
-      public KeyValue peek() throws IOException {
+      public Cell peek() throws IOException {
         return kvScanner.peek();
       }