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 2012/08/25 01:51:53 UTC

svn commit: r1377154 [6/6] - in /hbase/trunk/hbase-server/src: main/java/org/apache/hadoop/hbase/client/ main/java/org/apache/hadoop/hbase/filter/ main/java/org/apache/hadoop/hbase/mapreduce/ main/java/org/apache/hadoop/hbase/protobuf/ main/java/org/ap...

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java Fri Aug 24 23:51:51 2012
@@ -7322,6 +7322,491 @@ public final class HBaseProtos {
     // @@protoc_insertion_point(class_scope:TimeRange)
   }
   
+  public interface FilterOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+    
+    // required string name = 1;
+    boolean hasName();
+    String getName();
+    
+    // optional bytes serializedFilter = 2;
+    boolean hasSerializedFilter();
+    com.google.protobuf.ByteString getSerializedFilter();
+  }
+  public static final class Filter extends
+      com.google.protobuf.GeneratedMessage
+      implements FilterOrBuilder {
+    // Use Filter.newBuilder() to construct.
+    private Filter(Builder builder) {
+      super(builder);
+    }
+    private Filter(boolean noInit) {}
+    
+    private static final Filter defaultInstance;
+    public static Filter getDefaultInstance() {
+      return defaultInstance;
+    }
+    
+    public Filter getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+    
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Filter_descriptor;
+    }
+    
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Filter_fieldAccessorTable;
+    }
+    
+    private int bitField0_;
+    // required string name = 1;
+    public static final int NAME_FIELD_NUMBER = 1;
+    private java.lang.Object name_;
+    public boolean hasName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    public String getName() {
+      java.lang.Object ref = name_;
+      if (ref instanceof String) {
+        return (String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        String s = bs.toStringUtf8();
+        if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+          name_ = s;
+        }
+        return s;
+      }
+    }
+    private com.google.protobuf.ByteString getNameBytes() {
+      java.lang.Object ref = name_;
+      if (ref instanceof String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+        name_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+    
+    // optional bytes serializedFilter = 2;
+    public static final int SERIALIZEDFILTER_FIELD_NUMBER = 2;
+    private com.google.protobuf.ByteString serializedFilter_;
+    public boolean hasSerializedFilter() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    public com.google.protobuf.ByteString getSerializedFilter() {
+      return serializedFilter_;
+    }
+    
+    private void initFields() {
+      name_ = "";
+      serializedFilter_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+      
+      if (!hasName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+    
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, serializedFilter_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+    
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+    
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, serializedFilter_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+    
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+    
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter)) {
+        return super.equals(obj);
+      }
+      org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter) obj;
+      
+      boolean result = true;
+      result = result && (hasName() == other.hasName());
+      if (hasName()) {
+        result = result && getName()
+            .equals(other.getName());
+      }
+      result = result && (hasSerializedFilter() == other.hasSerializedFilter());
+      if (hasSerializedFilter()) {
+        result = result && getSerializedFilter()
+            .equals(other.getSerializedFilter());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+    
+    @java.lang.Override
+    public int hashCode() {
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasName()) {
+        hash = (37 * hash) + NAME_FIELD_NUMBER;
+        hash = (53 * hash) + getName().hashCode();
+      }
+      if (hasSerializedFilter()) {
+        hash = (37 * hash) + SERIALIZEDFILTER_FIELD_NUMBER;
+        hash = (53 * hash) + getSerializedFilter().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      return hash;
+    }
+    
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return newBuilder().mergeFrom(data, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      Builder builder = newBuilder();
+      if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+        return builder.buildParsed();
+      } else {
+        return null;
+      }
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input).buildParsed();
+    }
+    public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return newBuilder().mergeFrom(input, extensionRegistry)
+               .buildParsed();
+    }
+    
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+    
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Filter_descriptor;
+      }
+      
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_Filter_fieldAccessorTable;
+      }
+      
+      // Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+      
+      private Builder(BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+      
+      public Builder clear() {
+        super.clear();
+        name_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        serializedFilter_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+      
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDescriptor();
+      }
+      
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance();
+      }
+      
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter build() {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+      
+      private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter buildParsed()
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(
+            result).asInvalidProtocolBufferException();
+        }
+        return result;
+      }
+      
+      public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter buildPartial() {
+        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.name_ = name_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.serializedFilter_ = serializedFilter_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+      
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter) {
+          return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+      
+      public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter other) {
+        if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance()) return this;
+        if (other.hasName()) {
+          setName(other.getName());
+        }
+        if (other.hasSerializedFilter()) {
+          setSerializedFilter(other.getSerializedFilter());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+      
+      public final boolean isInitialized() {
+        if (!hasName()) {
+          
+          return false;
+        }
+        return true;
+      }
+      
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder(
+            this.getUnknownFields());
+        while (true) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              this.setUnknownFields(unknownFields.build());
+              onChanged();
+              return this;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                this.setUnknownFields(unknownFields.build());
+                onChanged();
+                return this;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              name_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              serializedFilter_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      }
+      
+      private int bitField0_;
+      
+      // required string name = 1;
+      private java.lang.Object name_ = "";
+      public boolean hasName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      public String getName() {
+        java.lang.Object ref = name_;
+        if (!(ref instanceof String)) {
+          String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+          name_ = s;
+          return s;
+        } else {
+          return (String) ref;
+        }
+      }
+      public Builder setName(String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        name_ = getDefaultInstance().getName();
+        onChanged();
+        return this;
+      }
+      void setName(com.google.protobuf.ByteString value) {
+        bitField0_ |= 0x00000001;
+        name_ = value;
+        onChanged();
+      }
+      
+      // optional bytes serializedFilter = 2;
+      private com.google.protobuf.ByteString serializedFilter_ = com.google.protobuf.ByteString.EMPTY;
+      public boolean hasSerializedFilter() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      public com.google.protobuf.ByteString getSerializedFilter() {
+        return serializedFilter_;
+      }
+      public Builder setSerializedFilter(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        serializedFilter_ = value;
+        onChanged();
+        return this;
+      }
+      public Builder clearSerializedFilter() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        serializedFilter_ = getDefaultInstance().getSerializedFilter();
+        onChanged();
+        return this;
+      }
+      
+      // @@protoc_insertion_point(builder_scope:Filter)
+    }
+    
+    static {
+      defaultInstance = new Filter(true);
+      defaultInstance.initFields();
+    }
+    
+    // @@protoc_insertion_point(class_scope:Filter)
+  }
+  
   public interface KeyValueOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
     
@@ -10549,6 +11034,11 @@ public final class HBaseProtos {
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_TimeRange_fieldAccessorTable;
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_Filter_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_Filter_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_KeyValue_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -10619,22 +11109,24 @@ public final class HBaseProtos {
       "\022\"\n\014coprocessors\030\006 \003(\0132\014.Coprocessor\022\027\n\017",
       "reportStartTime\030\007 \001(\004\022\025\n\rreportEndTime\030\010" +
       " \001(\004\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 " +
-      "\001(\004\"w\n\010KeyValue\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002" +
-      " \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\021\n\ttimestamp\030\004 \001" +
-      "(\004\022\031\n\007keyType\030\005 \001(\0162\010.KeyType\022\r\n\005value\030\006" +
-      " \001(\014\"?\n\nServerName\022\020\n\010hostName\030\001 \002(\t\022\014\n\004" +
-      "port\030\002 \001(\r\022\021\n\tstartCode\030\003 \001(\004\"\033\n\013Coproce" +
-      "ssor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004" +
-      "name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPa" +
-      "ir\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016Bytes",
-      "BytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014" +
-      "*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQU" +
-      "AL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATE" +
-      "R_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006*_\n\007" +
-      "KeyType\022\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020" +
-      "\010\022\021\n\rDELETE_COLUMN\020\014\022\021\n\rDELETE_FAMILY\020\016\022" +
-      "\014\n\007MAXIMUM\020\377\001B>\n*org.apache.hadoop.hbase" +
-      ".protobuf.generatedB\013HBaseProtosH\001\240\001\001"
+      "\001(\004\"0\n\006Filter\022\014\n\004name\030\001 \002(\t\022\030\n\020serialize" +
+      "dFilter\030\002 \001(\014\"w\n\010KeyValue\022\013\n\003row\030\001 \002(\014\022\016" +
+      "\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\021\n\ttim" +
+      "estamp\030\004 \001(\004\022\031\n\007keyType\030\005 \001(\0162\010.KeyType\022" +
+      "\r\n\005value\030\006 \001(\014\"?\n\nServerName\022\020\n\010hostName" +
+      "\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\021\n\tstartCode\030\003 \001(\004\"" +
+      "\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStri" +
+      "ngPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\rN",
+      "ameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001(" +
+      "\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006se" +
+      "cond\030\002 \002(\014*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rL" +
+      "ESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003" +
+      "\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005N" +
+      "O_OP\020\006*_\n\007KeyType\022\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022" +
+      "\n\n\006DELETE\020\010\022\021\n\rDELETE_COLUMN\020\014\022\021\n\rDELETE" +
+      "_FAMILY\020\016\022\014\n\007MAXIMUM\020\377\001B>\n*org.apache.ha" +
+      "doop.hbase.protobuf.generatedB\013HBaseProt" +
+      "osH\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -10713,8 +11205,16 @@ public final class HBaseProtos {
               new java.lang.String[] { "From", "To", },
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.class,
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.Builder.class);
-          internal_static_KeyValue_descriptor =
+          internal_static_Filter_descriptor =
             getDescriptor().getMessageTypes().get(7);
+          internal_static_Filter_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_Filter_descriptor,
+              new java.lang.String[] { "Name", "SerializedFilter", },
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.class,
+              org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder.class);
+          internal_static_KeyValue_descriptor =
+            getDescriptor().getMessageTypes().get(8);
           internal_static_KeyValue_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_KeyValue_descriptor,
@@ -10722,7 +11222,7 @@ public final class HBaseProtos {
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.class,
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.KeyValue.Builder.class);
           internal_static_ServerName_descriptor =
-            getDescriptor().getMessageTypes().get(8);
+            getDescriptor().getMessageTypes().get(9);
           internal_static_ServerName_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_ServerName_descriptor,
@@ -10730,7 +11230,7 @@ public final class HBaseProtos {
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.class,
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder.class);
           internal_static_Coprocessor_descriptor =
-            getDescriptor().getMessageTypes().get(9);
+            getDescriptor().getMessageTypes().get(10);
           internal_static_Coprocessor_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_Coprocessor_descriptor,
@@ -10738,7 +11238,7 @@ public final class HBaseProtos {
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor.class,
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor.Builder.class);
           internal_static_NameStringPair_descriptor =
-            getDescriptor().getMessageTypes().get(10);
+            getDescriptor().getMessageTypes().get(11);
           internal_static_NameStringPair_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_NameStringPair_descriptor,
@@ -10746,7 +11246,7 @@ public final class HBaseProtos {
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair.class,
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair.Builder.class);
           internal_static_NameBytesPair_descriptor =
-            getDescriptor().getMessageTypes().get(11);
+            getDescriptor().getMessageTypes().get(12);
           internal_static_NameBytesPair_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_NameBytesPair_descriptor,
@@ -10754,7 +11254,7 @@ public final class HBaseProtos {
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.class,
               org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder.class);
           internal_static_BytesBytesPair_descriptor =
-            getDescriptor().getMessageTypes().get(12);
+            getDescriptor().getMessageTypes().get(13);
           internal_static_BytesBytesPair_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_BytesBytesPair_descriptor,

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Fri Aug 24 23:51:51 2012
@@ -2886,7 +2886,7 @@ public class  HRegionServer implements C
           byte[] qualifier = condition.getQualifier().toByteArray();
           CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
           WritableByteArrayComparable comparator =
-            (WritableByteArrayComparable)ProtobufUtil.toObject(condition.getComparator());
+            ProtobufUtil.toComparator(condition.getComparator());
           if (region.getCoprocessorHost() != null) {
             processed = region.getCoprocessorHost().preCheckAndPut(
               row, family, qualifier, compareOp, comparator, put);
@@ -2915,7 +2915,7 @@ public class  HRegionServer implements C
           byte[] qualifier = condition.getQualifier().toByteArray();
           CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
           WritableByteArrayComparable comparator =
-            (WritableByteArrayComparable)ProtobufUtil.toObject(condition.getComparator());
+            ProtobufUtil.toComparator(condition.getComparator());
           if (region.getCoprocessorHost() != null) {
             processed = region.getCoprocessorHost().preCheckAndDelete(
               row, family, qualifier, compareOp, comparator, delete);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java Fri Aug 24 23:51:51 2012
@@ -22,10 +22,14 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.security.User;
 
+import com.google.protobuf.InvalidProtocolBufferException;
+
 /**
  * <strong>NOTE: for internal use only by AccessController implementation</strong>
  *
@@ -68,17 +72,25 @@ class AccessControlFilter extends Filter
     return ReturnCode.NEXT_COL;
   }
 
-  @Override
-  public void write(DataOutput dataOutput) throws IOException {
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
     // no implementation, server-side use only
     throw new UnsupportedOperationException(
-        "Serialization not supported.  Intended for server-side use only.");
+      "Serialization not supported.  Intended for server-side use only.");
   }
 
-  @Override
-  public void readFields(DataInput dataInput) throws IOException {
+  /**
+   * @param pbBytes A pb serialized {@link AccessControlFilter} instance
+   * @return An instance of {@link AccessControlFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static AccessControlFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
     // no implementation, server-side use only
     throw new UnsupportedOperationException(
-        "Serialization not supported.  Intended for server-side use only.");
+      "Serialization not supported.  Intended for server-side use only.");
   }
 }

Modified: hbase/trunk/hbase-server/src/main/protobuf/Client.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/protobuf/Client.proto?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/protobuf/Client.proto (original)
+++ hbase/trunk/hbase-server/src/main/protobuf/Client.proto Fri Aug 24 23:51:51 2012
@@ -25,6 +25,7 @@ option java_generate_equals_and_hash = t
 option optimize_for = SPEED;
 
 import "hbase.proto";
+import "Comparator.proto";
 
 /**
  * Container for a list of column qualifier names of a family.
@@ -42,7 +43,7 @@ message Get {
   repeated Column column = 2;
   repeated NameBytesPair attribute = 3;
   optional uint64 lockId = 4;
-  optional NameBytesPair filter = 5;
+  optional Filter filter = 5;
   optional TimeRange timeRange = 6;
   optional uint32 maxVersions = 7 [default = 1];
   optional bool cacheBlocks = 8 [default = true];
@@ -98,7 +99,7 @@ message Condition {
   required bytes family = 2;
   required bytes qualifier = 3;
   required CompareType compareType = 4;
-  required NameBytesPair comparator = 5;
+  required Comparator comparator = 5;
 }
 
 /**
@@ -181,7 +182,7 @@ message Scan {
   repeated NameBytesPair attribute = 2;
   optional bytes startRow = 3;
   optional bytes stopRow = 4;
-  optional NameBytesPair filter = 5;
+  optional Filter filter = 5;
   optional TimeRange timeRange = 6;
   optional uint32 maxVersions = 7 [default = 1];
   optional bool cacheBlocks = 8 [default = true];

Added: hbase/trunk/hbase-server/src/main/protobuf/Comparator.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/protobuf/Comparator.proto?rev=1377154&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/protobuf/Comparator.proto (added)
+++ hbase/trunk/hbase-server/src/main/protobuf/Comparator.proto Fri Aug 24 23:51:51 2012
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// This file contains protocol buffers that are used for filters
+
+option java_package = "org.apache.hadoop.hbase.protobuf.generated";
+option java_outer_classname = "ComparatorProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+// This file contains protocol buffers that are used for comparators (e.g. in filters)
+
+message Comparator {
+  required string name = 1;
+  optional bytes serializedComparator = 2;
+}
+
+message ByteArrayComparable {
+  optional bytes value = 1;
+}
+
+message BinaryComparator {
+  required ByteArrayComparable comparable = 1;
+}
+
+message BinaryPrefixComparator {
+  required ByteArrayComparable comparable = 1;
+}
+
+message BitComparator {
+  required ByteArrayComparable comparable = 1;
+  required BitwiseOp bitwiseOp = 2;
+
+  enum BitwiseOp {
+    AND = 1;
+    OR = 2;
+    XOR = 3;
+  }
+}
+
+message NullComparator {
+}
+
+message RegexStringComparator {
+  required string pattern = 1;
+  required int32 patternFlags = 2;
+  required string charset = 3;
+}
+
+message SubstringComparator {
+  required string substr = 1;
+}

Modified: hbase/trunk/hbase-server/src/main/protobuf/Filter.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/protobuf/Filter.proto?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/protobuf/Filter.proto (original)
+++ hbase/trunk/hbase-server/src/main/protobuf/Filter.proto Fri Aug 24 23:51:51 2012
@@ -25,6 +25,7 @@ option java_generate_equals_and_hash = t
 option optimize_for = SPEED;
 
 import "hbase.proto";
+import "Comparator.proto";
 
 message ColumnCountGetFilter {
   required int32 limit = 1;
@@ -40,32 +41,47 @@ message ColumnPrefixFilter {
 }
 
 message ColumnRangeFilter {
-  required bytes minColumn = 1;
+  optional bytes minColumn = 1;
   optional bool minColumnInclusive = 2;
-  required bytes maxColumn = 3;
+  optional bytes maxColumn = 3;
   optional bool maxColumnInclusive = 4;
 }
 
 message CompareFilter {
   required CompareType compareOp = 1;
-  required bytes comparator = 2;
-  
+  optional Comparator comparator = 2;
 }
 
 message DependentColumnFilter {
-  required bytes columnFamily = 1;
-  required bytes columnQualifier = 2;
-  optional bool dropDependentColumn = 3;
+  required CompareFilter compareFilter = 1;
+  optional bytes columnFamily = 2;
+  optional bytes columnQualifier = 3;
+  optional bool dropDependentColumn = 4;
 }
 
 message FamilyFilter {
   required CompareFilter compareFilter = 1;
 }
 
+message FilterList {
+  required Operator operator = 1;
+  repeated Filter filters = 2;
+
+  enum Operator {
+    MUST_PASS_ALL = 1;
+    MUST_PASS_ONE = 2;
+  }
+}
+
+message FilterWrapper {
+  required Filter filter = 1;
+}
+
 message FirstKeyOnlyFilter {
 }
 
 message FirstKeyValueMatchingQualifiersFilter {
+  repeated bytes qualifiers = 1;
 }
 
 message FuzzyRowFilter {
@@ -73,7 +89,7 @@ message FuzzyRowFilter {
 }
 
 message InclusiveStopFilter {
-  required bytes stopRowKey = 1;
+  optional bytes stopRowKey = 1;
 }
 
 message KeyOnlyFilter {
@@ -89,7 +105,7 @@ message PageFilter {
 }
 
 message PrefixFilter {
-  required bytes prefix = 1;
+  optional bytes prefix = 1;
 }
 
 message QualifierFilter {
@@ -109,10 +125,10 @@ message SingleColumnValueExcludeFilter {
 }
 
 message SingleColumnValueFilter {
-  required bytes columnFamily = 1;
-  required bytes columnQualifier = 2;
+  optional bytes columnFamily = 1;
+  optional bytes columnQualifier = 2;
   required CompareType compareOp = 3;
-  required bytes comparator = 4;
+  required Comparator comparator = 4;
   optional bool foundColumn = 5;
   optional bool matchedColumn = 6;
   optional bool filterIfMissing = 7;
@@ -120,7 +136,7 @@ message SingleColumnValueFilter {
 }
 
 message SkipFilter {
-  required NameBytesPair filter = 1;
+  required Filter filter = 1;
 }
 
 message TimestampsFilter {
@@ -132,5 +148,5 @@ message ValueFilter {
 }
 
 message WhileMatchFilter {
-  required NameBytesPair filter = 1;
+  required Filter filter = 1;
 }

Modified: hbase/trunk/hbase-server/src/main/protobuf/hbase.proto
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/protobuf/hbase.proto?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/protobuf/hbase.proto (original)
+++ hbase/trunk/hbase-server/src/main/protobuf/hbase.proto Fri Aug 24 23:51:51 2012
@@ -185,6 +185,11 @@ message TimeRange {
   optional uint64 to = 2;
 }
 
+message Filter {
+  required string name = 1;
+  optional bytes serializedFilter = 2;
+}
+
 /* Comparison operators */
 enum CompareType {
   LESS = 0;

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java Fri Aug 24 23:51:51 2012
@@ -47,6 +47,8 @@ import org.apache.hadoop.hbase.filter.Ro
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.HbaseMapWritable;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.DataInputBuffer;
@@ -69,8 +71,8 @@ public class TestSerialization {
   @Test public void testCompareFilter() throws Exception {
     Filter f = new RowFilter(CompareOp.EQUAL,
       new BinaryComparator(Bytes.toBytes("testRowOne-2")));
-    byte [] bytes = Writables.getBytes(f);
-    Filter ff = (Filter)Writables.getWritable(bytes, new RowFilter());
+    byte [] bytes = f.toByteArray();
+    Filter ff = RowFilter.parseFrom(bytes);
     assertNotNull(ff);
   }
 
@@ -263,8 +265,8 @@ public class TestSerialization {
     get.setTimeRange(ts, ts+1);
     get.setMaxVersions(maxVersions);
 
-    byte[] sb = Writables.getBytes(get);
-    Get desGet = (Get)Writables.getWritable(sb, new Get());
+    ClientProtos.Get getProto = ProtobufUtil.toGet(get);
+    Get desGet = ProtobufUtil.toGet(getProto);
 
     assertTrue(Bytes.equals(get.getRow(), desGet.getRow()));
     Set<byte[]> set = null;
@@ -304,8 +306,8 @@ public class TestSerialization {
     scan.setTimeRange(ts, ts+1);
     scan.setMaxVersions(maxVersions);
 
-    byte[] sb = Writables.getBytes(scan);
-    Scan desScan = (Scan)Writables.getWritable(sb, new Scan());
+    ClientProtos.Scan scanProto = ProtobufUtil.toScan(scan);
+    Scan desScan = ProtobufUtil.toScan(scanProto);
 
     assertTrue(Bytes.equals(scan.getStartRow(), desScan.getStartRow()));
     assertTrue(Bytes.equals(scan.getStopRow(), desScan.getStopRow()));
@@ -327,8 +329,8 @@ public class TestSerialization {
       final String name = "testScan";
       byte [] prefix = Bytes.toBytes(name);
       scan.setFilter(new PrefixFilter(prefix));
-      sb = Writables.getBytes(scan);
-      desScan = (Scan)Writables.getWritable(sb, new Scan());
+      scanProto = ProtobufUtil.toScan(scan);
+      desScan = ProtobufUtil.toScan(scanProto);
       Filter f = desScan.getFilter();
       assertTrue(f instanceof PrefixFilter);
     }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java Fri Aug 24 23:51:51 2012
@@ -156,7 +156,7 @@ public class TestAttributes {
 
   @Test
   public void testGetId() {
-    Get get = new Get();
+    Get get = new Get(null);
     Assert.assertNull("Make sure id is null if unset", get.toMap().get("id"));
     get.setId("myId");
     Assert.assertEquals("myId", get.toMap().get("id"));
@@ -164,7 +164,7 @@ public class TestAttributes {
 
   @Test
   public void testAppendId() {
-    Append append = new Append();
+    Append append = new Append(Bytes.toBytes("testRow"));
     Assert.assertNull("Make sure id is null if unset", append.toMap().get("id"));
     append.setId("myId");
     Assert.assertEquals("myId", append.toMap().get("id"));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGet.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGet.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGet.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGet.java Fri Aug 24 23:51:51 2012
@@ -29,6 +29,8 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.Test;
@@ -39,20 +41,14 @@ import org.junit.experimental.categories
 public class TestGet {
   @Test
   public void testAttributesSerialization() throws IOException {
-    Get get = new Get();
+    Get get = new Get(Bytes.toBytes("row"));
     get.setAttribute("attribute1", Bytes.toBytes("value1"));
     get.setAttribute("attribute2", Bytes.toBytes("value2"));
     get.setAttribute("attribute3", Bytes.toBytes("value3"));
 
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-    DataOutput out = new DataOutputStream(byteArrayOutputStream);
-    get.write(out);
-
-    Get get2 = new Get();
-    Assert.assertTrue(get2.getAttributesMap().isEmpty());
-
-    get2.readFields(new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray())));
+    ClientProtos.Get getProto = ProtobufUtil.toGet(get);
 
+    Get get2 = ProtobufUtil.toGet(getProto);
     Assert.assertNull(get2.getAttribute("absent"));
     Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), get2.getAttribute("attribute1")));
     Assert.assertTrue(Arrays.equals(Bytes.toBytes("value2"), get2.getAttribute("attribute2")));
@@ -62,7 +58,7 @@ public class TestGet {
 
   @Test
   public void testGetAttributes() {
-    Get get = new Get();
+    Get get = new Get(null);
     Assert.assertTrue(get.getAttributesMap().isEmpty());
     Assert.assertNull(get.getAttribute("absent"));
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScan.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScan.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScan.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScan.java Fri Aug 24 23:51:51 2012
@@ -29,6 +29,8 @@ import java.io.IOException;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Assert;
 import org.junit.Test;
@@ -44,14 +46,9 @@ public class TestScan {
     scan.setAttribute("attribute2", Bytes.toBytes("value2"));
     scan.setAttribute("attribute3", Bytes.toBytes("value3"));
 
-    ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-    DataOutput out = new DataOutputStream(byteArrayOutputStream);
-    scan.write(out);
+    ClientProtos.Scan scanProto = ProtobufUtil.toScan(scan);
 
-    Scan scan2 = new Scan();
-    Assert.assertTrue(scan2.getAttributesMap().isEmpty());
-
-    scan2.readFields(new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray())));
+    Scan scan2 = ProtobufUtil.toScan(scanProto);
 
     Assert.assertNull(scan2.getAttribute("absent"));
     Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), scan2.getAttribute("attribute1")));

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java Fri Aug 24 23:51:51 2012
@@ -37,6 +37,8 @@ import org.apache.hadoop.hbase.client.Sc
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.client.coprocessor.Exec;
 import org.apache.hadoop.hbase.io.HbaseObjectWritable;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -47,6 +49,8 @@ import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import com.google.protobuf.ByteString;
+
 /**
  * TestEndpoint: test cases to verify coprocessor Endpoint
  */
@@ -181,29 +185,6 @@ public class TestCoprocessorEndpoint {
     table.close();
   }
 
-  @Test
-  public void testExecDeserialization() throws IOException {
-    DataOutputBuffer dob = new DataOutputBuffer();
-    dob.writeUTF(methodName);
-    dob.writeInt(1);
-    Scan scan = new Scan();
-    HbaseObjectWritable.writeObject(dob, scan, Scan.class, new Configuration());
-    dob.writeUTF("org.apache.hadoop.hbase.client.Scan");
-    Bytes.writeByteArray(dob, new byte[]{'a'});
-    // this is the dynamic protocol name
-    dob.writeUTF(protocolName);
-
-    DataInputBuffer dib = new DataInputBuffer();
-    dib.reset(dob.getData(), dob.getLength());
-
-    Exec after = new Exec();
-    after.setConf(HBaseConfiguration.create());
-    after.readFields(dib);
-    // no error thrown
-    assertEquals(after.getProtocolName(), protocolName);
-    assertEquals(after.getMethodName(), methodName);
-  }
-
   private static byte[][] makeN(byte[] base, int n) {
     byte[][] ret = new byte[n][];
     for (int i = 0; i < n; i++) {

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=1377154&r1=1377153&r2=1377154&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 Fri Aug 24 23:51:51 2012
@@ -26,6 +26,8 @@ import java.io.DataOutputStream;
 
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import org.junit.Before;
@@ -58,18 +60,10 @@ public class TestColumnPaginationFilter
     }
 
     private Filter serializationTest(Filter filter) throws Exception {
-        ByteArrayOutputStream stream = new ByteArrayOutputStream();
-        DataOutputStream out = new DataOutputStream(stream);
-        filter.write(out);
-        out.close();
-        byte[] buffer = stream.toByteArray();
-
-        DataInputStream in =
-            new DataInputStream(new ByteArrayInputStream(buffer));
-        Filter newFilter = new ColumnPaginationFilter();
-        newFilter.readFields(in);
+      HBaseProtos.Filter filterProto = ProtobufUtil.toFilter(filter);
+      Filter newFilter = ProtobufUtil.toFilter(filterProto);
 
-        return newFilter;
+      return newFilter;
     }
 
 

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java?rev=1377154&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java Fri Aug 24 23:51:51 2012
@@ -0,0 +1,84 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestComparatorSerialization {
+
+  @Test
+  public void testBinaryComparator() throws Exception {
+    BinaryComparator binaryComparator = new BinaryComparator(Bytes.toBytes("binaryComparator"));
+    assertTrue(binaryComparator.areSerializedFieldsEqual(
+      ProtobufUtil.toComparator(ProtobufUtil.toComparator(binaryComparator))));
+  }
+
+  @Test
+  public void testBinaryPrefixComparator() throws Exception {
+    BinaryPrefixComparator binaryPrefixComparator =
+      new BinaryPrefixComparator(Bytes.toBytes("binaryPrefixComparator"));
+    assertTrue(binaryPrefixComparator.areSerializedFieldsEqual(
+      ProtobufUtil.toComparator(ProtobufUtil.toComparator(binaryPrefixComparator))));
+  }
+
+  @Test
+  public void testBitComparator() throws Exception {
+    BitComparator bitComparator =
+      new BitComparator(Bytes.toBytes("bitComparator"), BitComparator.BitwiseOp.XOR);
+    assertTrue(bitComparator.areSerializedFieldsEqual(
+      ProtobufUtil.toComparator(ProtobufUtil.toComparator(bitComparator))));
+  }
+
+  @Test
+  public void testNullComparator() throws Exception {
+    NullComparator nullComparator = new NullComparator();
+    assertTrue(nullComparator.areSerializedFieldsEqual(
+      ProtobufUtil.toComparator(ProtobufUtil.toComparator(nullComparator))));
+  }
+
+  @Test
+  public void testRegexStringComparator() throws Exception {
+    // test without specifying flags
+    RegexStringComparator regexStringComparator = new RegexStringComparator(".+-2");
+    assertTrue(regexStringComparator.areSerializedFieldsEqual(
+      ProtobufUtil.toComparator(ProtobufUtil.toComparator(regexStringComparator))));
+
+    // test with specifying flags
+    regexStringComparator =
+      new RegexStringComparator("regex", Pattern.CASE_INSENSITIVE | Pattern.DOTALL);
+  }
+
+  @Test
+  public void testSubstringComparator() throws Exception {
+    SubstringComparator substringComparator = new SubstringComparator("substr");
+    assertTrue(substringComparator.areSerializedFieldsEqual(
+      ProtobufUtil.toComparator(ProtobufUtil.toComparator(substringComparator))));
+  }
+
+}

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=1377154&r1=1377153&r2=1377154&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 Fri Aug 24 23:51:51 2012
@@ -1659,12 +1659,7 @@ public class TestFilter {
     private static Thread ipcHandlerThread = null;
     
     @Override
-    public void readFields(DataInput arg0) throws IOException {
-    }
-
-    @Override
-    public void write(DataOutput arg0) throws IOException {
-    }
+    public byte [] toByteArray() {return null;}
 
     @Override
     public boolean filterRow() {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java Fri Aug 24 23:51:51 2012
@@ -32,12 +32,15 @@ import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNull;
 
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -57,20 +60,20 @@ public class TestFilterList {
 
   @Test
   public void testAddFilter() throws Exception {
-    Filter filter1 = new RowFilter();
-    Filter filter2 = new RowFilter();
+    Filter filter1 = new FirstKeyOnlyFilter();
+    Filter filter2 = new FirstKeyOnlyFilter();
 
     FilterList filterList = new FilterList(filter1, filter2);
-    filterList.addFilter(new RowFilter());
+    filterList.addFilter(new FirstKeyOnlyFilter());
 
     filterList = new FilterList(Arrays.asList(filter1, filter2));
-    filterList.addFilter(new RowFilter());
+    filterList.addFilter(new FirstKeyOnlyFilter());
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2);
-    filterList.addFilter(new RowFilter());
+    filterList.addFilter(new FirstKeyOnlyFilter());
 
     filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(filter1, filter2));
-    filterList.addFilter(new RowFilter());
+    filterList.addFilter(new FirstKeyOnlyFilter());
 
   }
 
@@ -81,11 +84,19 @@ public class TestFilterList {
    */
   @Test
   public void testMPONE() throws Exception {
+    mpOneTest(getFilterMPONE());
+  }
+
+  private Filter getFilterMPONE() {
     List<Filter> filters = new ArrayList<Filter>();
     filters.add(new PageFilter(MAX_PAGES));
     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
     Filter filterMPONE =
-        new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
+      new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
+    return filterMPONE;
+  }
+
+  private void mpOneTest(Filter filterMPONE) throws Exception {
     /* Filter must do all below steps:
      * <ul>
      * <li>{@link #reset()}</li>
@@ -141,11 +152,19 @@ public class TestFilterList {
    */
   @Test
   public void testMPALL() throws Exception {
+    mpAllTest(getMPALLFilter());
+  }
+
+  private Filter getMPALLFilter() {
     List<Filter> filters = new ArrayList<Filter>();
     filters.add(new PageFilter(MAX_PAGES));
     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
     Filter filterMPALL =
       new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+    return filterMPALL;
+  }
+
+  private void mpAllTest(Filter filterMPALL) throws Exception {
     /* Filter must do all below steps:
      * <ul>
      * <li>{@link #reset()}</li>
@@ -184,11 +203,19 @@ public class TestFilterList {
    */
   @Test
   public void testOrdering() throws Exception {
+    orderingTest(getOrderingFilter());
+  }
+
+  public Filter getOrderingFilter() {
     List<Filter> filters = new ArrayList<Filter>();
     filters.add(new PrefixFilter(Bytes.toBytes("yyy")));
     filters.add(new PageFilter(MAX_PAGES));
     Filter filterMPONE =
-        new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
+      new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
+    return filterMPONE;
+  }
+
+  public void orderingTest(Filter filterMPONE) throws Exception {
     /* Filter must do all below steps:
      * <ul>
      * <li>{@link #reset()}</li>
@@ -248,18 +275,15 @@ public class TestFilterList {
       new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
 
     // Decompose filterMPALL to bytes.
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(stream);
-    filterMPALL.write(out);
-    out.close();
-    byte[] buffer = stream.toByteArray();
+    byte[] buffer = filterMPALL.toByteArray();
 
     // Recompose filterMPALL.
-    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
-    FilterList newFilter = new FilterList();
-    newFilter.readFields(in);
+    FilterList newFilter = FilterList.parseFrom(buffer);
 
-    // TODO: Run TESTS!!!
+    // Run tests
+    mpOneTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getFilterMPONE())));
+    mpAllTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getMPALLFilter())));
+    orderingTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getOrderingFilter())));
   }
 
   /**
@@ -274,10 +298,7 @@ public class TestFilterList {
 
     Filter filterNoHint = new FilterBase() {
       @Override
-      public void readFields(DataInput arg0) throws IOException {}
-
-      @Override
-      public void write(DataOutput arg0) throws IOException {}
+      public byte [] toByteArray() {return null;}
     };
 
     Filter filterMinHint = new FilterBase() {
@@ -287,10 +308,7 @@ public class TestFilterList {
       }
 
       @Override
-      public void readFields(DataInput arg0) throws IOException {}
-
-      @Override
-      public void write(DataOutput arg0) throws IOException {}
+      public byte [] toByteArray() {return null;}
     };
 
     Filter filterMaxHint = new FilterBase() {
@@ -300,10 +318,7 @@ public class TestFilterList {
       }
 
       @Override
-      public void readFields(DataInput arg0) throws IOException {}
-
-      @Override
-      public void write(DataOutput arg0) throws IOException {}
+      public byte [] toByteArray() {return null;}
     };
 
     // MUST PASS ONE

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java?rev=1377154&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java Fri Aug 24 23:51:51 2012
@@ -0,0 +1,322 @@
+/**
+ * Copyright The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.LinkedList;
+import java.util.TreeSet;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestFilterSerialization {
+
+  @Test
+  public void testColumnCountGetFilter() throws Exception {
+    ColumnCountGetFilter columnCountGetFilter = new ColumnCountGetFilter(1);
+    assertTrue(columnCountGetFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnCountGetFilter))));
+  }
+
+  @Test
+  public void testColumnPaginationFilter() throws Exception {
+    ColumnPaginationFilter columnPaginationFilter = new ColumnPaginationFilter(1,7);
+    assertTrue(columnPaginationFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnPaginationFilter))));
+  }
+
+  @Test
+  public void testColumnPrefixFilter() throws Exception {
+    // empty string
+    ColumnPrefixFilter columnPrefixFilter = new ColumnPrefixFilter(Bytes.toBytes(""));
+    assertTrue(columnPrefixFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnPrefixFilter))));
+
+    // non-empty string
+    columnPrefixFilter = new ColumnPrefixFilter(Bytes.toBytes(""));
+    assertTrue(columnPrefixFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnPrefixFilter))));
+  }
+
+  @Test
+  public void testColumnRangeFilter() throws Exception {
+    // null columns
+    ColumnRangeFilter columnRangeFilter = new ColumnRangeFilter(null, true, null, false);
+    assertTrue(columnRangeFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnRangeFilter))));
+
+    // non-null columns
+    columnRangeFilter = new ColumnRangeFilter(Bytes.toBytes("a"), false, Bytes.toBytes("b"), true);
+    assertTrue(columnRangeFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(columnRangeFilter))));
+  }
+
+  @Test
+  public void testDependentColumnFilter() throws Exception {
+    // null column qualifier/family
+    DependentColumnFilter dependentColumnFilter = new DependentColumnFilter(null, null);
+    assertTrue(dependentColumnFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(dependentColumnFilter))));
+
+    // non-null column qualifier/family
+    dependentColumnFilter = new DependentColumnFilter(Bytes.toBytes("family"),
+      Bytes.toBytes("qual"), true, CompareOp.GREATER_OR_EQUAL,
+      new BitComparator(Bytes.toBytes("bitComparator"), BitComparator.BitwiseOp.OR));
+    assertTrue(dependentColumnFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(dependentColumnFilter))));
+  }
+
+  @Test
+  public void testFamilyFilter() throws Exception {
+    FamilyFilter familyFilter = new FamilyFilter(CompareFilter.CompareOp.EQUAL,
+      new BinaryPrefixComparator(Bytes.toBytes("testValueOne")));
+    assertTrue(familyFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(familyFilter))));
+  }
+
+  @Test
+  public void testFilterList() throws Exception {
+    // empty filter list
+    FilterList filterList = new FilterList(new LinkedList<Filter>());
+    assertTrue(filterList.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(filterList))));
+
+    // non-empty filter list
+    LinkedList<Filter> list = new LinkedList<Filter>();
+    list.add(new ColumnCountGetFilter(1));
+    list.add(new RowFilter(CompareFilter.CompareOp.EQUAL,
+      new SubstringComparator("testFilterList")));
+    assertTrue(filterList.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(filterList))));
+  }
+
+  @Test
+  public void testFilterWrapper() throws Exception {
+    FilterWrapper filterWrapper =
+      new FilterWrapper(
+        new ColumnRangeFilter(Bytes.toBytes("e"), false, Bytes.toBytes("f"), true));
+    assertTrue(filterWrapper.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(filterWrapper))));
+  }
+
+  @Test
+  public void testFirstKeyValueMatchingQualifiersFilter() throws Exception {
+    // empty qualifiers set
+    TreeSet<byte []> set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+    FirstKeyValueMatchingQualifiersFilter firstKeyValueMatchingQualifiersFilter =
+      new FirstKeyValueMatchingQualifiersFilter(set);
+    assertTrue(firstKeyValueMatchingQualifiersFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(firstKeyValueMatchingQualifiersFilter))));
+
+    // non-empty qualifiers set
+    set.add(Bytes.toBytes("col0"));
+    set.add(Bytes.toBytes("col1"));
+    firstKeyValueMatchingQualifiersFilter =
+      new FirstKeyValueMatchingQualifiersFilter(set);
+    assertTrue(firstKeyValueMatchingQualifiersFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(firstKeyValueMatchingQualifiersFilter))));
+  }
+
+  @Test
+  public void testFirstKeyOnlyFilter() throws Exception {
+    FirstKeyOnlyFilter firstKeyOnlyFilter = new FirstKeyOnlyFilter();
+    assertTrue(firstKeyOnlyFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(firstKeyOnlyFilter))));
+  }
+
+  @Test
+  public void testFuzzyRowFilter() throws Exception {
+    LinkedList<Pair<byte[], byte[]>> fuzzyList = new LinkedList<Pair<byte[], byte[]>>();
+    fuzzyList.add(new Pair<byte[], byte[]>(Bytes.toBytes("999"),new byte[] {0, 0, 1}));
+    fuzzyList.add(new Pair<byte[], byte[]>(Bytes.toBytes("abcd"),new byte[] {1, 0, 1, 1}));
+    FuzzyRowFilter fuzzyRowFilter = new FuzzyRowFilter(fuzzyList);
+    assertTrue(fuzzyRowFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(fuzzyRowFilter))));
+  }
+
+  @Test
+  public void testInclusiveStopFilter() throws Exception {
+    // InclusveStopFilter with null stopRowKey
+    InclusiveStopFilter inclusiveStopFilter = new InclusiveStopFilter(null);
+    assertTrue(inclusiveStopFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(inclusiveStopFilter))));
+
+    // InclusveStopFilter with non-null stopRowKey
+    inclusiveStopFilter = new InclusiveStopFilter(Bytes.toBytes("inclusiveStopFilter"));
+    assertTrue(inclusiveStopFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(inclusiveStopFilter))));
+  }
+
+  @Test
+  public void testKeyOnlyFilter() throws Exception {
+    // KeyOnlyFilter with lenAsVal
+    KeyOnlyFilter keyOnlyFilter = new KeyOnlyFilter(true);
+    assertTrue(keyOnlyFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(keyOnlyFilter))));
+
+    // KeyOnlyFilter without lenAsVal
+    keyOnlyFilter = new KeyOnlyFilter();
+    assertTrue(keyOnlyFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(keyOnlyFilter))));
+  }
+
+  @Test
+  public void testMultipleColumnPrefixFilter() throws Exception {
+    // empty array
+    byte [][] prefixes = null;
+    MultipleColumnPrefixFilter multipleColumnPrefixFilter =
+      new MultipleColumnPrefixFilter(prefixes);
+    assertTrue(multipleColumnPrefixFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(multipleColumnPrefixFilter))));
+
+    // non-empty array
+    prefixes = new byte[2][];
+    prefixes[0] = Bytes.toBytes("a");
+    prefixes[1] = Bytes.toBytes("");
+    multipleColumnPrefixFilter = new MultipleColumnPrefixFilter(prefixes);
+    assertTrue(multipleColumnPrefixFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(multipleColumnPrefixFilter))));
+  }
+
+  @Test
+  public void testPageFilter() throws Exception {
+    PageFilter pageFilter = new PageFilter(6);
+    assertTrue(pageFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(pageFilter))));
+  }
+
+  @Test
+  public void testPrefixFilter() throws Exception {
+    // null prefix
+    PrefixFilter prefixFilter = new PrefixFilter(null);
+    assertTrue(prefixFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(prefixFilter))));
+
+    // non-null prefix
+    prefixFilter = new PrefixFilter(Bytes.toBytes("abc"));
+    assertTrue(prefixFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(prefixFilter))));
+  }
+
+  @Test
+  public void testQualifierFilter() throws Exception {
+    QualifierFilter qualifierFilter = new QualifierFilter(CompareFilter.CompareOp.EQUAL,
+      new NullComparator());
+    assertTrue(qualifierFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(qualifierFilter))));
+  }
+
+  @Test
+  public void testRandomRowFilter() throws Exception {
+    RandomRowFilter randomRowFilter = new RandomRowFilter((float)0.1);
+    assertTrue(randomRowFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(randomRowFilter))));
+  }
+
+  @Test
+  public void testRowFilter() throws Exception {
+    RowFilter rowFilter = new RowFilter(CompareFilter.CompareOp.EQUAL,
+      new SubstringComparator("testRowFilter"));
+    assertTrue(rowFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(rowFilter))));
+  }
+
+  @Test
+  public void testSingleColumnValueExcludeFilter() throws Exception {
+    // null family/column SingleColumnValueExcludeFilter
+    SingleColumnValueExcludeFilter singleColumnValueExcludeFilter =
+      new SingleColumnValueExcludeFilter(null, null,
+      CompareFilter.CompareOp.GREATER_OR_EQUAL, Bytes.toBytes("value"));
+    assertTrue(singleColumnValueExcludeFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(singleColumnValueExcludeFilter))));
+
+    // non-null family/column SingleColumnValueFilter
+    singleColumnValueExcludeFilter =
+      new SingleColumnValueExcludeFilter(Bytes.toBytes("fam"), Bytes.toBytes("qual"),
+      CompareFilter.CompareOp.LESS_OR_EQUAL, new NullComparator(), false, true, false, false);
+    assertTrue(singleColumnValueExcludeFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(singleColumnValueExcludeFilter))));
+  }
+
+  @Test
+  public void testSingleColumnValueFilter() throws Exception {
+    // null family/column SingleColumnValueFilter
+    SingleColumnValueFilter singleColumnValueFilter =
+      new SingleColumnValueFilter(null, null,
+      CompareFilter.CompareOp.LESS, Bytes.toBytes("value"));
+    assertTrue(singleColumnValueFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(singleColumnValueFilter))));
+
+    // non-null family/column SingleColumnValueFilter
+    singleColumnValueFilter =
+      new SingleColumnValueFilter(Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
+      CompareFilter.CompareOp.NOT_EQUAL, new NullComparator(), true, false, true, true);
+    assertTrue(singleColumnValueFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(singleColumnValueFilter))));
+  }
+
+  @Test
+  public void testSkipFilter() throws Exception {
+    SkipFilter skipFilter = new SkipFilter(new PageFilter(6));
+    assertTrue(skipFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(skipFilter))));
+  }
+
+  @Test
+  public void testTimestampsFilter() throws Exception {
+    // Empty timestamp list
+    TimestampsFilter timestampsFilter = new TimestampsFilter(new LinkedList<Long>());
+    assertTrue(timestampsFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(timestampsFilter))));
+
+    // Non-empty timestamp list
+    LinkedList<Long> list = new LinkedList<Long>();
+    list.add(new Long(System.currentTimeMillis()));
+    list.add(new Long(System.currentTimeMillis()));
+    timestampsFilter = new TimestampsFilter(list);
+    assertTrue(timestampsFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(timestampsFilter))));
+  }
+
+  @Test
+  public void testValueFilter() throws Exception {
+    ValueFilter valueFilter = new ValueFilter(CompareFilter.CompareOp.NO_OP,
+      new BinaryComparator(Bytes.toBytes("testValueOne")));
+    assertTrue(valueFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(valueFilter))));
+  }
+
+  @Test
+  public void testWhileMatchFilter() throws Exception {
+    WhileMatchFilter whileMatchFilter =
+      new WhileMatchFilter(
+        new ColumnRangeFilter(Bytes.toBytes("c"), false, Bytes.toBytes("d"), true));
+    assertTrue(whileMatchFilter.areSerializedFieldsEqual(
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(whileMatchFilter))));
+  }
+
+}

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java Fri Aug 24 23:51:51 2012
@@ -32,6 +32,7 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 /**
  * Tests the inclusive stop row filter
@@ -65,16 +66,10 @@ public class TestInclusiveStopFilter {
   @Test
   public void testSerialization() throws Exception {
     // Decompose mainFilter to bytes.
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(stream);
-    mainFilter.write(out);
-    out.close();
-    byte[] buffer = stream.toByteArray();
+    byte[] buffer = mainFilter.toByteArray();
 
     // Recompose mainFilter.
-    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
-    Filter newFilter = new InclusiveStopFilter();
-    newFilter.readFields(in);
+    Filter newFilter = InclusiveStopFilter.parseFrom(buffer);
 
     // Ensure the serialization preserved the filter by running a full test.
     stopRowTests(newFilter);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java Fri Aug 24 23:51:51 2012
@@ -55,15 +55,9 @@ public class TestPageFilter {
   public void testSerialization() throws Exception {
     Filter f = new PageFilter(ROW_LIMIT);
     // Decompose mainFilter to bytes.
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(stream);
-    f.write(out);
-    out.close();
-    byte[] buffer = stream.toByteArray();
+    byte[] buffer = f.toByteArray();
     // Recompose mainFilter.
-    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
-    Filter newFilter = new PageFilter();
-    newFilter.readFields(in);
+    Filter newFilter = PageFilter.parseFrom(buffer);
 
     // Ensure the serialization preserved the filter by running a full test.
     pageSizeTests(newFilter);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java Fri Aug 24 23:51:51 2012
@@ -69,16 +69,10 @@ public class TestPrefixFilter {
   @Test
   public void testSerialization() throws Exception {
     // Decompose mainFilter to bytes.
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(stream);
-    mainFilter.write(out);
-    out.close();
-    byte[] buffer = stream.toByteArray();
+    byte[] buffer = mainFilter.toByteArray();
 
     // Recompose filter.
-    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
-    Filter newFilter = new PrefixFilter();
-    newFilter.readFields(in);
+    Filter newFilter = PrefixFilter.parseFrom(buffer);
 
     // Ensure the serialization preserved the filter by running all test.
     prefixRowTests(newFilter);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java Fri Aug 24 23:51:51 2012
@@ -81,16 +81,10 @@ public class TestRandomRowFilter {
   private RandomRowFilter serializationTest(RandomRowFilter filter)
       throws Exception {
     // Decompose filter to bytes.
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(stream);
-    filter.write(out);
-    out.close();
-    byte[] buffer = stream.toByteArray();
+    byte[] buffer = filter.toByteArray();
 
     // Recompose filter.
-    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
-    RandomRowFilter newFilter = new RandomRowFilter();
-    newFilter.readFields(in);
+    RandomRowFilter newFilter = RandomRowFilter.parseFrom(buffer);
 
     return newFilter;
   }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java Fri Aug 24 23:51:51 2012
@@ -156,17 +156,10 @@ public class TestSingleColumnValueFilter
   private Filter serializationTest(Filter filter)
       throws Exception {
     // Decompose filter to bytes.
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(stream);
-    filter.write(out);
-    out.close();
-    byte[] buffer = stream.toByteArray();
+    byte[] buffer = filter.toByteArray();
 
     // Recompose filter.
-    DataInputStream in =
-      new DataInputStream(new ByteArrayInputStream(buffer));
-    Filter newFilter = new SingleColumnValueFilter();
-    newFilter.readFields(in);
+    Filter newFilter = SingleColumnValueFilter.parseFrom(buffer);
     return newFilter;
   }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java Fri Aug 24 23:51:51 2012
@@ -36,6 +36,7 @@ import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -83,6 +84,9 @@ import org.apache.hadoop.hbase.filter.Sk
 import org.apache.hadoop.hbase.filter.ValueFilter;
 import org.apache.hadoop.hbase.filter.WhileMatchFilter;
 import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
@@ -97,6 +101,7 @@ import org.junit.Assert;
 import org.junit.experimental.categories.Category;
 
 import com.google.common.collect.Lists;
+import com.google.protobuf.InvalidProtocolBufferException;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcController;
 
@@ -214,8 +219,8 @@ public class TestHbaseObjectWritable ext
     assertTrue(obj instanceof ArrayList);
     Assert.assertArrayEquals(list.toArray(), ((ArrayList)obj).toArray() );
     // Check that filters can be serialized
-    obj = doType(conf, new PrefixFilter(HConstants.EMPTY_BYTE_ARRAY),
-      PrefixFilter.class);
+    obj =
+      ProtobufUtil.toFilter(ProtobufUtil.toFilter(new PrefixFilter(HConstants.EMPTY_BYTE_ARRAY)));
     assertTrue(obj instanceof PrefixFilter);
   }
 
@@ -228,12 +233,14 @@ public class TestHbaseObjectWritable ext
     assertTrue(obj instanceof Writable);
     assertTrue(obj instanceof CustomWritable);
     assertEquals("test phrase", ((CustomWritable)obj).getValue());
+  }
 
+  public void testCustomFilter() throws Exception {
     // test proper serialization of a custom filter
     CustomFilter filt = new CustomFilter("mykey");
     FilterList filtlist = new FilterList(FilterList.Operator.MUST_PASS_ALL);
     filtlist.addFilter(filt);
-    obj = doType(conf, filtlist, FilterList.class);
+    Filter obj = ProtobufUtil.toFilter(ProtobufUtil.toFilter(filtlist));
     assertTrue(obj instanceof FilterList);
     assertNotNull(((FilterList)obj).getFilters());
     assertEquals(1, ((FilterList)obj).getFilters().size());
@@ -416,6 +423,26 @@ public class TestHbaseObjectWritable ext
     public void readFields(DataInput in) throws IOException {
       this.key = Text.readString(in);
     }
+
+    public byte [] toByteArray() {
+      // rather than write a PB definition for this type,
+      // just reuse one that works
+      HBaseProtos.NameBytesPair.Builder builder =
+        HBaseProtos.NameBytesPair.newBuilder();
+      builder.setName(this.key);
+      return builder.build().toByteArray();
+    }
+
+    public static CustomFilter parseFrom(final byte [] pbBytes)
+    throws DeserializationException {
+      HBaseProtos.NameBytesPair proto;
+      try {
+        proto = HBaseProtos.NameBytesPair.parseFrom(pbBytes);
+      } catch (InvalidProtocolBufferException e) {
+        throw new DeserializationException(e);
+      }
+      return new CustomFilter(proto.getName());
+    }
   }
 
   /**