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 2013/04/23 19:50:51 UTC

svn commit: r1471070 - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ hbase-protocol/src/main/protobuf/ hbase-server/src/test/java/org/apache/hadoop/h...

Author: stack
Date: Tue Apr 23 17:50:50 2013
New Revision: 1471070

URL: http://svn.apache.org/r1471070
Log:
HBASE-8284 Allow String Offset(s) in ColumnPaginationFilter for bookmark based pagination

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
    hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
    hbase/trunk/hbase-protocol/src/main/protobuf/Filter.proto
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java?rev=1471070&r1=1471069&r2=1471070&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java Tue Apr 23 17:50:50 2013
@@ -19,12 +19,14 @@
 package org.apache.hadoop.hbase.filter;
 
 import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
 import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.util.Bytes;
 
 import java.util.ArrayList;
 
@@ -39,9 +41,19 @@ import java.util.ArrayList;
 public class ColumnPaginationFilter extends FilterBase
 {
   private int limit = 0;
-  private int offset = 0;
+  private int offset = -1;
+  private byte[] columnOffset = null;
   private int count = 0;
 
+  /**
+   * Initializes filter with an integer offset and limit. The offset is arrived at
+   * scanning sequentially and skipping entries. @limit number of columns are
+   * then retrieved. If multiple column families are involved, the columns may be spread
+   * across them.
+   *
+   * @param limit Max number of columns to return.
+   * @param offset The integer offset where to start pagination.
+   */
   public ColumnPaginationFilter(final int limit, final int offset)
   {
     Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
@@ -51,6 +63,25 @@ public class ColumnPaginationFilter exte
   }
 
   /**
+   * Initializes filter with a string/bookmark based offset and limit. The offset is arrived
+   * at, by seeking to it using scanner hints. If multiple column families are involved,
+   * pagination starts at the first column family which contains @columnOffset. Columns are
+   * then retrieved sequentially upto @limit number of columns which maybe spread across
+   * multiple column families, depending on how the scan is setup.
+   *
+   * @param limit Max number of columns to return.
+   * @param columnOffset The string/bookmark offset on where to start pagination.
+   */
+  public ColumnPaginationFilter(final int limit, final byte[] columnOffset) {
+    Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
+    Preconditions.checkArgument(columnOffset != null,
+                                "columnOffset must be non-null %s",
+                                columnOffset);
+    this.limit = limit;
+    this.columnOffset = columnOffset;
+  }
+
+  /**
    * @return limit
    */
   public int getLimit() {
@@ -64,18 +95,56 @@ public class ColumnPaginationFilter exte
     return offset;
   }
 
+  /**
+   * @return columnOffset
+   */
+  public byte[] getColumnOffset() {
+    return columnOffset;
+  }
+
   @Override
   public ReturnCode filterKeyValue(KeyValue v)
   {
-    if(count >= offset + limit)
-    {
-      return ReturnCode.NEXT_ROW;
+    if (columnOffset != null) {
+      if (count >= limit) {
+        return ReturnCode.NEXT_ROW;
+      }
+      byte[] buffer = v.getBuffer();
+      if (buffer == null) {
+        return ReturnCode.SEEK_NEXT_USING_HINT;
+      }
+      int cmp = 0;
+      // Only compare if no KV's have been seen so far.
+      if (count == 0) {
+        cmp = Bytes.compareTo(buffer,
+                              v.getQualifierOffset(),
+                              v.getQualifierLength(),
+                              this.columnOffset,
+                              0,
+                              this.columnOffset.length);
+      }
+      if (cmp < 0) {
+        return ReturnCode.SEEK_NEXT_USING_HINT;
+      } else {
+        count++;
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+      }
+    } else {
+      if (count >= offset + limit) {
+        return ReturnCode.NEXT_ROW;
+      }
+
+      ReturnCode code = count < offset ? ReturnCode.NEXT_COL :
+                                         ReturnCode.INCLUDE_AND_NEXT_COL;
+      count++;
+      return code;
     }
+  }
 
-    ReturnCode code = count < offset ? ReturnCode.NEXT_COL :
-                                       ReturnCode.INCLUDE_AND_NEXT_COL;
-    count++;
-    return code;
+  public KeyValue getNextKeyHint(KeyValue kv) {
+    return KeyValue.createFirstOnRow(
+        kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
+        kv.getFamilyOffset(), kv.getFamilyLength(), columnOffset, 0, columnOffset.length);
   }
 
   @Override
@@ -99,7 +168,12 @@ public class ColumnPaginationFilter exte
     FilterProtos.ColumnPaginationFilter.Builder builder =
       FilterProtos.ColumnPaginationFilter.newBuilder();
     builder.setLimit(this.limit);
-    builder.setOffset(this.offset);
+    if (this.offset >= 0) {
+      builder.setOffset(this.offset);
+    }
+    if (this.columnOffset != null) {
+      builder.setColumnOffset(ByteString.copyFrom(this.columnOffset));
+    }
     return builder.build().toByteArray();
   }
 
@@ -117,6 +191,10 @@ public class ColumnPaginationFilter exte
     } catch (InvalidProtocolBufferException e) {
       throw new DeserializationException(e);
     }
+    if (proto.hasColumnOffset()) {
+      return new ColumnPaginationFilter(proto.getLimit(),
+                                        proto.getColumnOffset().toByteArray());
+    }
     return new ColumnPaginationFilter(proto.getLimit(),proto.getOffset());
   }
 
@@ -130,11 +208,19 @@ public class ColumnPaginationFilter exte
     if (!(o instanceof ColumnPaginationFilter)) return false;
 
     ColumnPaginationFilter other = (ColumnPaginationFilter)o;
+    if (this.columnOffset != null) {
+      return this.getLimit() == this.getLimit() &&
+          Bytes.equals(this.getColumnOffset(), other.getColumnOffset());
+    }
     return this.getLimit() == other.getLimit() && this.getOffset() == other.getOffset();
   }
 
   @Override
   public String toString() {
+    if (this.columnOffset != null) {
+      return (this.getClass().getSimpleName() + "(" + this.limit + ", " +
+          Bytes.toStringBinary(this.columnOffset) + ")");
+    }
     return String.format("%s (%d, %d)", this.getClass().getSimpleName(),
         this.limit, this.offset);
   }

Modified: hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java?rev=1471070&r1=1471069&r2=1471070&view=diff
==============================================================================
--- hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java (original)
+++ hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java Tue Apr 23 17:50:50 2013
@@ -397,6 +397,10 @@ public final class FilterProtos {
     // optional int32 offset = 2;
     boolean hasOffset();
     int getOffset();
+    
+    // optional bytes columnOffset = 3;
+    boolean hasColumnOffset();
+    com.google.protobuf.ByteString getColumnOffset();
   }
   public static final class ColumnPaginationFilter extends
       com.google.protobuf.GeneratedMessage
@@ -447,9 +451,20 @@ public final class FilterProtos {
       return offset_;
     }
     
+    // optional bytes columnOffset = 3;
+    public static final int COLUMNOFFSET_FIELD_NUMBER = 3;
+    private com.google.protobuf.ByteString columnOffset_;
+    public boolean hasColumnOffset() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    public com.google.protobuf.ByteString getColumnOffset() {
+      return columnOffset_;
+    }
+    
     private void initFields() {
       limit_ = 0;
       offset_ = 0;
+      columnOffset_ = com.google.protobuf.ByteString.EMPTY;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -473,6 +488,9 @@ public final class FilterProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeInt32(2, offset_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, columnOffset_);
+      }
       getUnknownFields().writeTo(output);
     }
     
@@ -490,6 +508,10 @@ public final class FilterProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(2, offset_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, columnOffset_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -523,6 +545,11 @@ public final class FilterProtos {
         result = result && (getOffset()
             == other.getOffset());
       }
+      result = result && (hasColumnOffset() == other.hasColumnOffset());
+      if (hasColumnOffset()) {
+        result = result && getColumnOffset()
+            .equals(other.getColumnOffset());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -540,6 +567,10 @@ public final class FilterProtos {
         hash = (37 * hash) + OFFSET_FIELD_NUMBER;
         hash = (53 * hash) + getOffset();
       }
+      if (hasColumnOffset()) {
+        hash = (37 * hash) + COLUMNOFFSET_FIELD_NUMBER;
+        hash = (53 * hash) + getColumnOffset().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       return hash;
     }
@@ -660,6 +691,8 @@ public final class FilterProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         offset_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
+        columnOffset_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
       
@@ -706,6 +739,10 @@ public final class FilterProtos {
           to_bitField0_ |= 0x00000002;
         }
         result.offset_ = offset_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.columnOffset_ = columnOffset_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -728,6 +765,9 @@ public final class FilterProtos {
         if (other.hasOffset()) {
           setOffset(other.getOffset());
         }
+        if (other.hasColumnOffset()) {
+          setColumnOffset(other.getColumnOffset());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -773,6 +813,11 @@ public final class FilterProtos {
               offset_ = input.readInt32();
               break;
             }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              columnOffset_ = input.readBytes();
+              break;
+            }
           }
         }
       }
@@ -821,6 +866,30 @@ public final class FilterProtos {
         return this;
       }
       
+      // optional bytes columnOffset = 3;
+      private com.google.protobuf.ByteString columnOffset_ = com.google.protobuf.ByteString.EMPTY;
+      public boolean hasColumnOffset() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      public com.google.protobuf.ByteString getColumnOffset() {
+        return columnOffset_;
+      }
+      public Builder setColumnOffset(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        columnOffset_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearColumnOffset() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        columnOffset_ = getDefaultInstance().getColumnOffset();
+        onChanged();
+        return this;
+      }
+      
       // @@protoc_insertion_point(builder_scope:ColumnPaginationFilter)
     }
     
@@ -12547,49 +12616,49 @@ public final class FilterProtos {
     java.lang.String[] descriptorData = {
       "\n\014Filter.proto\032\013hbase.proto\032\020Comparator." +
       "proto\"%\n\024ColumnCountGetFilter\022\r\n\005limit\030\001" +
-      " \002(\005\"7\n\026ColumnPaginationFilter\022\r\n\005limit\030" +
-      "\001 \002(\005\022\016\n\006offset\030\002 \001(\005\"$\n\022ColumnPrefixFil" +
-      "ter\022\016\n\006prefix\030\001 \002(\014\"q\n\021ColumnRangeFilter" +
-      "\022\021\n\tminColumn\030\001 \001(\014\022\032\n\022minColumnInclusiv" +
-      "e\030\002 \001(\010\022\021\n\tmaxColumn\030\003 \001(\014\022\032\n\022maxColumnI" +
-      "nclusive\030\004 \001(\010\"Q\n\rCompareFilter\022\037\n\tcompa" +
-      "reOp\030\001 \002(\0162\014.CompareType\022\037\n\ncomparator\030\002" +
-      " \001(\0132\013.Comparator\"\212\001\n\025DependentColumnFil",
-      "ter\022%\n\rcompareFilter\030\001 \002(\0132\016.CompareFilt" +
-      "er\022\024\n\014columnFamily\030\002 \001(\014\022\027\n\017columnQualif" +
-      "ier\030\003 \001(\014\022\033\n\023dropDependentColumn\030\004 \001(\010\"5" +
-      "\n\014FamilyFilter\022%\n\rcompareFilter\030\001 \002(\0132\016." +
-      "CompareFilter\"\200\001\n\nFilterList\022&\n\010operator" +
-      "\030\001 \002(\0162\024.FilterList.Operator\022\030\n\007filters\030" +
-      "\002 \003(\0132\007.Filter\"0\n\010Operator\022\021\n\rMUST_PASS_" +
-      "ALL\020\001\022\021\n\rMUST_PASS_ONE\020\002\"(\n\rFilterWrappe" +
-      "r\022\027\n\006filter\030\001 \002(\0132\007.Filter\"\024\n\022FirstKeyOn" +
-      "lyFilter\";\n%FirstKeyValueMatchingQualifi",
-      "ersFilter\022\022\n\nqualifiers\030\001 \003(\014\"8\n\016FuzzyRo" +
-      "wFilter\022&\n\rfuzzyKeysData\030\001 \003(\0132\017.BytesBy" +
-      "tesPair\")\n\023InclusiveStopFilter\022\022\n\nstopRo" +
-      "wKey\030\001 \001(\014\"!\n\rKeyOnlyFilter\022\020\n\010lenAsVal\030" +
-      "\001 \002(\010\"4\n\032MultipleColumnPrefixFilter\022\026\n\016s" +
-      "ortedPrefixes\030\001 \003(\014\"\036\n\nPageFilter\022\020\n\010pag" +
-      "eSize\030\001 \002(\003\"\036\n\014PrefixFilter\022\016\n\006prefix\030\001 " +
-      "\001(\014\"8\n\017QualifierFilter\022%\n\rcompareFilter\030" +
-      "\001 \002(\0132\016.CompareFilter\"!\n\017RandomRowFilter" +
-      "\022\016\n\006chance\030\001 \002(\002\"2\n\tRowFilter\022%\n\rcompare",
-      "Filter\030\001 \002(\0132\016.CompareFilter\"[\n\036SingleCo" +
-      "lumnValueExcludeFilter\0229\n\027singleColumnVa" +
-      "lueFilter\030\001 \002(\0132\030.SingleColumnValueFilte" +
-      "r\"\276\001\n\027SingleColumnValueFilter\022\024\n\014columnF" +
-      "amily\030\001 \001(\014\022\027\n\017columnQualifier\030\002 \001(\014\022\037\n\t" +
-      "compareOp\030\003 \002(\0162\014.CompareType\022\037\n\ncompara" +
-      "tor\030\004 \002(\0132\013.Comparator\022\027\n\017filterIfMissin" +
-      "g\030\005 \001(\010\022\031\n\021latestVersionOnly\030\006 \001(\010\"%\n\nSk" +
-      "ipFilter\022\027\n\006filter\030\001 \002(\0132\007.Filter\"*\n\020Tim" +
-      "estampsFilter\022\026\n\ntimestamps\030\001 \003(\003B\002\020\001\"4\n",
-      "\013ValueFilter\022%\n\rcompareFilter\030\001 \002(\0132\016.Co" +
-      "mpareFilter\"+\n\020WhileMatchFilter\022\027\n\006filte" +
-      "r\030\001 \002(\0132\007.FilterBB\n*org.apache.hadoop.hb" +
-      "ase.protobuf.generatedB\014FilterProtosH\001\210\001" +
-      "\001\240\001\001"
+      " \002(\005\"M\n\026ColumnPaginationFilter\022\r\n\005limit\030" +
+      "\001 \002(\005\022\016\n\006offset\030\002 \001(\005\022\024\n\014columnOffset\030\003 " +
+      "\001(\014\"$\n\022ColumnPrefixFilter\022\016\n\006prefix\030\001 \002(" +
+      "\014\"q\n\021ColumnRangeFilter\022\021\n\tminColumn\030\001 \001(" +
+      "\014\022\032\n\022minColumnInclusive\030\002 \001(\010\022\021\n\tmaxColu" +
+      "mn\030\003 \001(\014\022\032\n\022maxColumnInclusive\030\004 \001(\010\"Q\n\r" +
+      "CompareFilter\022\037\n\tcompareOp\030\001 \002(\0162\014.Compa" +
+      "reType\022\037\n\ncomparator\030\002 \001(\0132\013.Comparator\"",
+      "\212\001\n\025DependentColumnFilter\022%\n\rcompareFilt" +
+      "er\030\001 \002(\0132\016.CompareFilter\022\024\n\014columnFamily" +
+      "\030\002 \001(\014\022\027\n\017columnQualifier\030\003 \001(\014\022\033\n\023dropD" +
+      "ependentColumn\030\004 \001(\010\"5\n\014FamilyFilter\022%\n\r" +
+      "compareFilter\030\001 \002(\0132\016.CompareFilter\"\200\001\n\n" +
+      "FilterList\022&\n\010operator\030\001 \002(\0162\024.FilterLis" +
+      "t.Operator\022\030\n\007filters\030\002 \003(\0132\007.Filter\"0\n\010" +
+      "Operator\022\021\n\rMUST_PASS_ALL\020\001\022\021\n\rMUST_PASS" +
+      "_ONE\020\002\"(\n\rFilterWrapper\022\027\n\006filter\030\001 \002(\0132" +
+      "\007.Filter\"\024\n\022FirstKeyOnlyFilter\";\n%FirstK",
+      "eyValueMatchingQualifiersFilter\022\022\n\nquali" +
+      "fiers\030\001 \003(\014\"8\n\016FuzzyRowFilter\022&\n\rfuzzyKe" +
+      "ysData\030\001 \003(\0132\017.BytesBytesPair\")\n\023Inclusi" +
+      "veStopFilter\022\022\n\nstopRowKey\030\001 \001(\014\"!\n\rKeyO" +
+      "nlyFilter\022\020\n\010lenAsVal\030\001 \002(\010\"4\n\032MultipleC" +
+      "olumnPrefixFilter\022\026\n\016sortedPrefixes\030\001 \003(" +
+      "\014\"\036\n\nPageFilter\022\020\n\010pageSize\030\001 \002(\003\"\036\n\014Pre" +
+      "fixFilter\022\016\n\006prefix\030\001 \001(\014\"8\n\017QualifierFi" +
+      "lter\022%\n\rcompareFilter\030\001 \002(\0132\016.CompareFil" +
+      "ter\"!\n\017RandomRowFilter\022\016\n\006chance\030\001 \002(\002\"2",
+      "\n\tRowFilter\022%\n\rcompareFilter\030\001 \002(\0132\016.Com" +
+      "pareFilter\"[\n\036SingleColumnValueExcludeFi" +
+      "lter\0229\n\027singleColumnValueFilter\030\001 \002(\0132\030." +
+      "SingleColumnValueFilter\"\276\001\n\027SingleColumn" +
+      "ValueFilter\022\024\n\014columnFamily\030\001 \001(\014\022\027\n\017col" +
+      "umnQualifier\030\002 \001(\014\022\037\n\tcompareOp\030\003 \002(\0162\014." +
+      "CompareType\022\037\n\ncomparator\030\004 \002(\0132\013.Compar" +
+      "ator\022\027\n\017filterIfMissing\030\005 \001(\010\022\031\n\021latestV" +
+      "ersionOnly\030\006 \001(\010\"%\n\nSkipFilter\022\027\n\006filter" +
+      "\030\001 \002(\0132\007.Filter\"*\n\020TimestampsFilter\022\026\n\nt",
+      "imestamps\030\001 \003(\003B\002\020\001\"4\n\013ValueFilter\022%\n\rco" +
+      "mpareFilter\030\001 \002(\0132\016.CompareFilter\"+\n\020Whi" +
+      "leMatchFilter\022\027\n\006filter\030\001 \002(\0132\007.FilterBB" +
+      "\n*org.apache.hadoop.hbase.protobuf.gener" +
+      "atedB\014FilterProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -12609,7 +12678,7 @@ public final class FilterProtos {
           internal_static_ColumnPaginationFilter_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ColumnPaginationFilter_descriptor,
-              new java.lang.String[] { "Limit", "Offset", },
+              new java.lang.String[] { "Limit", "Offset", "ColumnOffset", },
               org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ColumnPaginationFilter.class,
               org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ColumnPaginationFilter.Builder.class);
           internal_static_ColumnPrefixFilter_descriptor =

Modified: hbase/trunk/hbase-protocol/src/main/protobuf/Filter.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-protocol/src/main/protobuf/Filter.proto?rev=1471070&r1=1471069&r2=1471070&view=diff
==============================================================================
--- hbase/trunk/hbase-protocol/src/main/protobuf/Filter.proto (original)
+++ hbase/trunk/hbase-protocol/src/main/protobuf/Filter.proto Tue Apr 23 17:50:50 2013
@@ -34,6 +34,7 @@ message ColumnCountGetFilter {
 message ColumnPaginationFilter {
   required int32 limit = 1;
   optional int32 offset = 2;
+  optional bytes columnOffset = 3;
 }
 
 message ColumnPrefixFilter {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java?rev=1471070&r1=1471069&r2=1471070&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestColumnPaginationFilter.java Tue Apr 23 17:50:50 2013
@@ -45,17 +45,23 @@ public class TestColumnPaginationFilter
     private static final byte[] ROW = Bytes.toBytes("row_1_test");
     private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test");
     private static final byte[] VAL_1 = Bytes.toBytes("a");
-    private static final byte [] COLUMN_QUALIFIER = Bytes.toBytes("foo");
+    private static final byte[] COLUMN_QUALIFIER = Bytes.toBytes("foo");
 
+    private Filter columnPaginationFilterOffset;
     private Filter columnPaginationFilter;
 
     @Before
     public void setUp() throws Exception {
         columnPaginationFilter = getColumnPaginationFilter();
-
+        columnPaginationFilterOffset = getColumnPaginationFilterOffset();
     }
+
     private Filter getColumnPaginationFilter() {
-        return new ColumnPaginationFilter(1,0);
+        return new ColumnPaginationFilter(1, 0);
+    }
+
+    private Filter getColumnPaginationFilterOffset() {
+        return new ColumnPaginationFilter(1, COLUMN_QUALIFIER);
     }
 
     private Filter serializationTest(Filter filter) throws Exception {
@@ -87,6 +93,9 @@ public class TestColumnPaginationFilter
     public void testSerialization() throws Exception {
       Filter newFilter = serializationTest(columnPaginationFilter);
       basicFilterTests((ColumnPaginationFilter)newFilter);
+
+      Filter newFilterOffset = serializationTest(columnPaginationFilterOffset);
+      basicFilterTests((ColumnPaginationFilter)newFilterOffset);
     }
 
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java?rev=1471070&r1=1471069&r2=1471070&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java Tue Apr 23 17:50:50 2013
@@ -1429,7 +1429,7 @@ public class TestFilter {
           " total but already scanned " + (results.size() + idx) +
           (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"),
           kvs.length >= idx + results.size());
-      for(KeyValue kv : results) {
+      for (KeyValue kv : results) {
         LOG.info("row=" + row + ", result=" + kv.toString() +
             ", match=" + kvs[idx].toString());
         assertTrue("Row mismatch",
@@ -1496,6 +1496,90 @@ public class TestFilter {
         kvs.length, idx);
   }
 
+  public void testColumnPaginationFilterColumnOffset() throws Exception {
+    KeyValue [] expectedKVs = {
+      // testRowOne-0
+      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+      // testRowOne-2
+      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+      // testRowOne-3
+      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
+      // testRowTwo-0
+      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+      // testRowTwo-2
+      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+      // testRowTwo-3
+      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
+    };
+    KeyValue [] expectedKVs1 = {
+      // testRowTwo-0
+      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+      // testRowTwo-2
+      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+      // testRowTwo-3
+      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1])
+    };
+    KeyValue [] expectedKVs2 = {
+      // testRowTwo-0
+      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
+      // testRowTwo-2
+      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
+      // testRowTwo-3
+      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
+      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1])
+    };
+    KeyValue [] expectedKVs3 = {
+      // testRowTwo-0
+      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
+      // testRowTwo-2
+      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
+      // testRowTwo-3
+      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
+      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
+    };
+    Scan s = new Scan();
+
+    // Page size 1.
+    long expectedRows = 6;
+    long expectedKeys = 1;
+    s.setFilter(new ColumnPaginationFilter(1, QUALIFIERS_ONE[1]));
+    verifyScan(s, expectedRows, expectedKeys);
+    this.verifyScanFull(s, expectedKVs);
+
+    // Page size 2.
+    expectedRows = 3;
+    expectedKeys = 2;
+    s = new Scan();
+    s.setFilter(new ColumnPaginationFilter(2, QUALIFIERS_TWO[2]));
+    verifyScan(s, expectedRows, expectedKeys);
+    this.verifyScanFull(s, expectedKVs1);
+
+    // Page size 3 across multiple column families.
+    expectedRows = 3;
+    expectedKeys = 3;
+    s = new Scan();
+    s.setFilter(new ColumnPaginationFilter(3, QUALIFIERS_TWO[2]));
+    verifyScan(s, expectedRows, expectedKeys);
+    this.verifyScanFull(s, expectedKVs2);
+
+    // Page size 2 restricted to one column family.
+    expectedRows = 3;
+    expectedKeys = 2;
+    s = new Scan();
+    s.addFamily(FAMILIES[1]);
+    s.setFilter(new ColumnPaginationFilter(2, QUALIFIERS_TWO[2]));
+    this.verifyScanFull(s, expectedKVs3);
+  }
 
   @Test
   public void testColumnPaginationFilter() throws Exception {
@@ -1522,7 +1606,6 @@ public class TestFilter {
         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1])
       };
 
-
       // Set of KVs (page: 3; pageSize: 1)  - the third set of 1 column per row
       KeyValue [] expectedKVs2 = {
         // testRowOne-0
@@ -1594,7 +1677,7 @@ public class TestFilter {
       expectedRows = 0;
       verifyScan(s, expectedRows, 0);
       this.verifyScanFull(s, expectedKVs4);
-    }
+  }
 
   @Test
   public void testKeyOnlyFilter() throws Exception {