hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r1377154 [1/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...
Date Fri, 24 Aug 2012 23:51:53 GMT
Author: stack
Date: Fri Aug 24 23:51:51 2012
New Revision: 1377154

URL: http://svn.apache.org/viewvc?rev=1377154&view=rev
Log:
HBASE-6477 Use PB filter definitions in RPC

Added:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ComparatorProtos.java
    hbase/trunk/hbase-server/src/main/protobuf/Comparator.proto
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestComparatorSerialization.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterSerialization.java
Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Append.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Get.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Row.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Scan.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java
    hbase/trunk/hbase-server/src/main/protobuf/Client.proto
    hbase/trunk/hbase-server/src/main/protobuf/Filter.proto
    hbase/trunk/hbase-server/src/main/protobuf/hbase.proto
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSerialization.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAttributes.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestGet.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScan.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java
    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
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInclusiveStopFilter.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPageFilter.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestPrefixFilter.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestRandomRowFilter.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Append.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Append.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Append.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Append.java Fri Aug 24 23:51:51 2012
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -29,7 +26,6 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Writable;
 
 /**
  * Performs Append operations on a single row.
@@ -67,9 +63,6 @@ public class Append extends Mutation {
     return v == null ? true : Bytes.toBoolean(v);
   }
 
-  /** Constructor for Writable.  DO NOT USE */
-  public Append() {}
-
   /**
    * Create a Append operation for the specified row.
    * <p>
@@ -98,60 +91,4 @@ public class Append extends Mutation {
     return this;
   }
 
-  @Override
-  public void readFields(final DataInput in)
-  throws IOException {
-    int version = in.readByte();
-    if (version > APPEND_VERSION) {
-      throw new IOException("version not supported: "+version);
-    }
-    this.row = Bytes.readByteArray(in);
-    this.ts = in.readLong();
-    this.lockId = in.readLong();
-    this.writeToWAL = in.readBoolean();
-    int numFamilies = in.readInt();
-    if (!this.familyMap.isEmpty()) this.familyMap.clear();
-    for(int i=0;i<numFamilies;i++) {
-      byte [] family = Bytes.readByteArray(in);
-      int numKeys = in.readInt();
-      List<KeyValue> keys = new ArrayList<KeyValue>(numKeys);
-      int totalLen = in.readInt();
-      byte [] buf = new byte[totalLen];
-      int offset = 0;
-      for (int j = 0; j < numKeys; j++) {
-        int keyLength = in.readInt();
-        in.readFully(buf, offset, keyLength);
-        keys.add(new KeyValue(buf, offset, keyLength));
-        offset += keyLength;
-      }
-      this.familyMap.put(family, keys);
-    }
-    readAttributes(in);
-  }
-
-  @Override
-  public void write(final DataOutput out)
-  throws IOException {
-    out.writeByte(APPEND_VERSION);
-    Bytes.writeByteArray(out, this.row);
-    out.writeLong(this.ts);
-    out.writeLong(this.lockId);
-    out.writeBoolean(this.writeToWAL);
-    out.writeInt(familyMap.size());
-    for (Map.Entry<byte [], List<KeyValue>> entry : familyMap.entrySet()) {
-      Bytes.writeByteArray(out, entry.getKey());
-      List<KeyValue> keys = entry.getValue();
-      out.writeInt(keys.size());
-      int totalLen = 0;
-      for(KeyValue kv : keys) {
-        totalLen += kv.getLength();
-      }
-      out.writeInt(totalLen);
-      for(KeyValue kv : keys) {
-        out.writeInt(kv.getLength());
-        out.write(kv.getBuffer(), kv.getOffset(), kv.getLength());
-      }
-    }
-    writeAttributes(out);
-  }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Get.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Get.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Get.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Get.java Fri Aug 24 23:51:51 2012
@@ -26,11 +26,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableFactories;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -68,7 +64,7 @@ import java.util.TreeSet;
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class Get extends OperationWithAttributes
-  implements Writable, Row, Comparable<Row> {
+  implements Row, Comparable<Row> {
 
   private static final byte VERSION_WITHOUT_PAGINATION = (byte) 2;
   private static final byte VERSION_WITH_PAGINATION = (byte) 3;
@@ -93,9 +89,6 @@ public class Get extends OperationWithAt
     return VERSION_WITHOUT_PAGINATION;
   }
 
-  /** Constructor for Writable.  DO NOT USE */
-  public Get() {}
-
   /**
    * Create a Get operation for the specified row.
    * <p>
@@ -440,95 +433,5 @@ public class Get extends OperationWithAt
   public int compareTo(Row other) {
     return Bytes.compareTo(this.getRow(), other.getRow());
   }
-  
-  //Writable
-  public void readFields(final DataInput in)
-  throws IOException {
-    int version = in.readByte();
-    if (version > GET_VERSION) {
-      throw new IOException("unsupported version");
-    }
-    this.row = Bytes.readByteArray(in);
-    this.lockId = in.readLong();
-    this.maxVersions = in.readInt();
-    if (version >= VERSION_WITH_PAGINATION) {
-      this.storeLimit = in.readInt();
-      this.storeOffset = in.readInt();
-    }
-    boolean hasFilter = in.readBoolean();
-    if (hasFilter) {
-      this.filter = (Filter)createForName(Bytes.toString(Bytes.readByteArray(in)));
-      this.filter.readFields(in);
-    }
-    this.cacheBlocks = in.readBoolean();
-    this.tr = new TimeRange();
-    tr.readFields(in);
-    int numFamilies = in.readInt();
-    this.familyMap =
-      new TreeMap<byte [],NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
-    for(int i=0; i<numFamilies; i++) {
-      byte [] family = Bytes.readByteArray(in);
-      boolean hasColumns = in.readBoolean();
-      NavigableSet<byte []> set = null;
-      if(hasColumns) {
-        int numColumns = in.readInt();
-        set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
-        for(int j=0; j<numColumns; j++) {
-          byte [] qualifier = Bytes.readByteArray(in);
-          set.add(qualifier);
-        }
-      }
-      this.familyMap.put(family, set);
-    }
-    readAttributes(in);
-  }
 
-  public void write(final DataOutput out)
-  throws IOException {
-    byte version = getVersion();
-    out.writeByte(version);
-    Bytes.writeByteArray(out, this.row);
-    out.writeLong(this.lockId);
-    out.writeInt(this.maxVersions);
-    if (version >= VERSION_WITH_PAGINATION) {
-      out.writeInt(this.storeLimit);
-      out.writeInt(this.storeOffset);
-    }
-    if(this.filter == null) {
-      out.writeBoolean(false);
-    } else {
-      out.writeBoolean(true);
-      Bytes.writeByteArray(out, Bytes.toBytes(filter.getClass().getName()));
-      filter.write(out);
-    }
-    out.writeBoolean(this.cacheBlocks);
-    tr.write(out);
-    out.writeInt(familyMap.size());
-    for(Map.Entry<byte [], NavigableSet<byte []>> entry :
-      familyMap.entrySet()) {
-      Bytes.writeByteArray(out, entry.getKey());
-      NavigableSet<byte []> columnSet = entry.getValue();
-      if(columnSet == null) {
-        out.writeBoolean(false);
-      } else {
-        out.writeBoolean(true);
-        out.writeInt(columnSet.size());
-        for(byte [] qualifier : columnSet) {
-          Bytes.writeByteArray(out, qualifier);
-        }
-      }
-    }
-    writeAttributes(out);
-  }
-
-  @SuppressWarnings("unchecked")
-  private Writable createForName(String className) {
-    try {
-      Class<? extends Writable> clazz =
-        (Class<? extends Writable>) Class.forName(className);
-      return WritableFactories.newInstance(clazz, new Configuration());
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException("Can't find class " + className);
-    }
-  }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Row.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Row.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Row.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Row.java Fri Aug 24 23:51:51 2012
@@ -21,14 +21,13 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.WritableComparable;
 
 /**
  * Has a row.
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public interface Row extends WritableComparable<Row> {
+public interface Row extends Comparable<Row> {
   /**
    * @return The row.
    */

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/RowMutations.java Fri Aug 24 23:51:51 2012
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -28,7 +26,6 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.io.HbaseObjectWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -88,30 +85,6 @@ public class RowMutations implements Row
   }
 
   @Override
-  public void readFields(final DataInput in) throws IOException {
-    int version = in.readByte();
-    if (version > VERSION) {
-      throw new IOException("version not supported");
-    }
-    this.row = Bytes.readByteArray(in);
-    int numMutations = in.readInt();
-    mutations.clear();
-    for(int i = 0; i < numMutations; i++) {
-      mutations.add((Mutation) HbaseObjectWritable.readObject(in, null));
-    }
-  }
-
-  @Override
-  public void write(final DataOutput out) throws IOException {
-    out.writeByte(VERSION);
-    Bytes.writeByteArray(out, this.row);
-    out.writeInt(mutations.size());
-    for (Mutation m : mutations) {
-      HbaseObjectWritable.writeObject(out, m, m.getClass(), null);
-    }
-  }
-
-  @Override
   public int compareTo(Row i) {
     return Bytes.compareTo(this.getRow(), i.getRow());
   }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Scan.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Scan.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Scan.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/Scan.java Fri Aug 24 23:51:51 2012
@@ -31,8 +31,6 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -87,7 +85,7 @@ import java.util.TreeSet;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class Scan extends OperationWithAttributes implements Writable {
+public class Scan extends OperationWithAttributes {
   private static final String RAW_ATTR = "_raw_";
   private static final String ISOLATION_LEVEL = "_isolationlevel_";
 
@@ -630,105 +628,6 @@ public class Scan extends OperationWithA
     return map;
   }
 
-  @SuppressWarnings("unchecked")
-  private Writable createForName(String className) {
-    try {
-      Class<? extends Writable> clazz =
-        (Class<? extends Writable>) Class.forName(className);
-      return WritableFactories.newInstance(clazz, new Configuration());
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException("Can't find class " + className);
-    }
-  }
-
-  //Writable
-  public void readFields(final DataInput in)
-  throws IOException {
-    int version = in.readByte();
-    if (version > (int)SCAN_VERSION) {
-      throw new IOException("version not supported");
-    }
-    this.startRow = Bytes.readByteArray(in);
-    this.stopRow = Bytes.readByteArray(in);
-    this.maxVersions = in.readInt();
-    this.batch = in.readInt();
-    if (version >= VERSION_WITH_PAGINATION) {
-      this.storeLimit = in.readInt();
-      this.storeOffset = in.readInt();
-    }
-    this.caching = in.readInt();
-    this.cacheBlocks = in.readBoolean();
-    if(in.readBoolean()) {
-      this.filter = (Filter)createForName(Bytes.toString(Bytes.readByteArray(in)));
-      this.filter.readFields(in);
-    }
-    this.tr = new TimeRange();
-    tr.readFields(in);
-    int numFamilies = in.readInt();
-    this.familyMap =
-      new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
-    for(int i=0; i<numFamilies; i++) {
-      byte [] family = Bytes.readByteArray(in);
-      int numColumns = in.readInt();
-      TreeSet<byte []> set = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
-      for(int j=0; j<numColumns; j++) {
-        byte [] qualifier = Bytes.readByteArray(in);
-        set.add(qualifier);
-      }
-      this.familyMap.put(family, set);
-    }
-
-    if (version >= VERSION_WITH_ATTRIBUTES) {
-      readAttributes(in);
-    }
-    if (version >= VERSION_WITH_RESULT_SIZE) {
-      this.maxResultSize = in.readLong();
-    }
-  }
-
-  public void write(final DataOutput out)
-  throws IOException {
-    byte version = getVersion();
-    out.writeByte(version);
-    Bytes.writeByteArray(out, this.startRow);
-    Bytes.writeByteArray(out, this.stopRow);
-    out.writeInt(this.maxVersions);
-    out.writeInt(this.batch);
-    if (version >= VERSION_WITH_PAGINATION) {
-      out.writeInt(this.storeLimit);
-      out.writeInt(this.storeOffset);
-    }
-    out.writeInt(this.caching);
-    out.writeBoolean(this.cacheBlocks);
-    if(this.filter == null) {
-      out.writeBoolean(false);
-    } else {
-      out.writeBoolean(true);
-      Bytes.writeByteArray(out, Bytes.toBytes(filter.getClass().getName()));
-      filter.write(out);
-    }
-    tr.write(out);
-    out.writeInt(familyMap.size());
-    for(Map.Entry<byte [], NavigableSet<byte []>> entry : familyMap.entrySet()) {
-      Bytes.writeByteArray(out, entry.getKey());
-      NavigableSet<byte []> columnSet = entry.getValue();
-      if(columnSet != null){
-        out.writeInt(columnSet.size());
-        for(byte [] qualifier : columnSet) {
-          Bytes.writeByteArray(out, qualifier);
-        }
-      } else {
-        out.writeInt(0);
-      }
-    }
-    if (version >= VERSION_WITH_ATTRIBUTES) {
-      writeAttributes(out);
-    }
-    if (version >= VERSION_WITH_RESULT_SIZE) {
-      out.writeLong(maxResultSize);
-    }
-  }
-
   /**
    * Enable/disable "raw" mode for this scan.
    * If "raw" is enabled the scan will return all

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryComparator.java Fri Aug 24 23:51:51 2012
@@ -22,8 +22,12 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
+import com.google.protobuf.InvalidProtocolBufferException;
+
 /**
  * A binary comparator which lexicographically compares against the specified
  * byte array using {@link org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.
@@ -32,9 +36,6 @@ import org.apache.hadoop.hbase.util.Byte
 @InterfaceStability.Stable
 public class BinaryComparator extends WritableByteArrayComparable {
 
-  /** Nullary constructor for Writable, do not use */
-  public BinaryComparator() { }
-
   /**
    * Constructor
    * @param value value
@@ -47,4 +48,43 @@ public class BinaryComparator extends Wr
   public int compareTo(byte [] value, int offset, int length) {
     return Bytes.compareTo(this.value, 0, this.value.length, value, offset, length);
   }
+
+  /**
+   * @return The comparator serialized using pb
+   */
+  public byte [] toByteArray() {
+    ComparatorProtos.BinaryComparator.Builder builder =
+      ComparatorProtos.BinaryComparator.newBuilder();
+    builder.setComparable(super.convert());
+    return builder.build().toByteArray();
+  }
+
+  /**
+   * @param pbBytes A pb serialized {@link BinaryComparator} instance
+   * @return An instance of {@link BinaryComparator} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static BinaryComparator parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    ComparatorProtos.BinaryComparator proto;
+    try {
+      proto = ComparatorProtos.BinaryComparator.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return new BinaryComparator(proto.getComparable().getValue().toByteArray());
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the comparator that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(WritableByteArrayComparable other) {
+    if (other == this) return true;
+    if (!(other instanceof BinaryComparator)) return false;
+
+    return super.areSerializedFieldsEqual(other);
+  }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BinaryPrefixComparator.java Fri Aug 24 23:51:51 2012
@@ -22,8 +22,12 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
+import com.google.protobuf.InvalidProtocolBufferException;
+
 /**
  * A comparator which compares against a specified byte array, but only compares
  * up to the length of this byte array. For the rest it is similar to
@@ -33,9 +37,6 @@ import org.apache.hadoop.hbase.util.Byte
 @InterfaceStability.Stable
 public class BinaryPrefixComparator extends WritableByteArrayComparable {
 
-  /** Nullary constructor for Writable, do not use */
-  public BinaryPrefixComparator() { }
-
   /**
    * Constructor
    * @param value value
@@ -49,4 +50,43 @@ public class BinaryPrefixComparator exte
     return Bytes.compareTo(this.value, 0, this.value.length, value, offset,
         this.value.length <= length ? this.value.length : length);
   }
+
+  /**
+   * @return The comparator serialized using pb
+   */
+  public byte [] toByteArray() {
+    ComparatorProtos.BinaryPrefixComparator.Builder builder =
+      ComparatorProtos.BinaryPrefixComparator.newBuilder();
+    builder.setComparable(super.convert());
+    return builder.build().toByteArray();
+  }
+
+  /**
+   * @param pbBytes A pb serialized {@link BinaryPrefixComparator} instance
+   * @return An instance of {@link BinaryPrefixComparator} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static BinaryPrefixComparator parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    ComparatorProtos.BinaryPrefixComparator proto;
+    try {
+      proto = ComparatorProtos.BinaryPrefixComparator.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return new BinaryPrefixComparator(proto.getComparable().getValue().toByteArray());
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the comparator that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(WritableByteArrayComparable other) {
+    if (other == this) return true;
+    if (!(other instanceof BinaryPrefixComparator)) return false;
+
+    return super.areSerializedFieldsEqual(other);
+  }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/BitComparator.java Fri Aug 24 23:51:51 2012
@@ -20,12 +20,12 @@
 
 package org.apache.hadoop.hbase.filter;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
+
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A bit comparator which performs the specified bitwise operation on each of the bytes
@@ -35,9 +35,6 @@ import org.apache.hadoop.classification.
 @InterfaceStability.Stable
 public class BitComparator extends WritableByteArrayComparable {
 
-  /** Nullary constructor for Writable, do not use */
-  public BitComparator() {}
-
   /** Bit operators. */
   public enum BitwiseOp {
     /** and */
@@ -66,16 +63,49 @@ public class BitComparator extends Writa
     return bitOperator;
   }
 
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    bitOperator = BitwiseOp.valueOf(in.readUTF());
+  /**
+   * @return The comparator serialized using pb
+   */
+  public byte [] toByteArray() {
+    ComparatorProtos.BitComparator.Builder builder =
+      ComparatorProtos.BitComparator.newBuilder();
+    builder.setComparable(super.convert());
+    ComparatorProtos.BitComparator.BitwiseOp bitwiseOpPb =
+      ComparatorProtos.BitComparator.BitwiseOp.valueOf(bitOperator.name());
+    builder.setBitwiseOp(bitwiseOpPb);
+    return builder.build().toByteArray();
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    out.writeUTF(bitOperator.name());
+  /**
+   * @param pbBytes A pb serialized {@link BitComparator} instance
+   * @return An instance of {@link BitComparator} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static BitComparator parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    ComparatorProtos.BitComparator proto;
+    try {
+      proto = ComparatorProtos.BitComparator.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    BitwiseOp bitwiseOp = BitwiseOp.valueOf(proto.getBitwiseOp().name());
+    return new BitComparator(proto.getComparable().getValue().toByteArray(),bitwiseOp);
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the comparator that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(WritableByteArrayComparable other) {
+    if (other == this) return true;
+    if (!(other instanceof BitComparator)) return false;
+
+    BitComparator comparator = (BitComparator)other;
+    return super.areSerializedFieldsEqual(other)
+      && this.getOperator().equals(comparator.getOperator());
   }
 
   @Override

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java Fri Aug 24 23:51:51 2012
@@ -22,14 +22,14 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
 
 import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * Simple filter that returns first N columns on row only.
@@ -43,14 +43,6 @@ public class ColumnCountGetFilter extend
   private int limit = 0;
   private int count = 0;
 
-  /**
-   * Used during serialization.
-   * Do not use.
-   */
-  public ColumnCountGetFilter() {
-    super();
-  }
-
   public ColumnCountGetFilter(final int n) {
     Preconditions.checkArgument(n >= 0, "limit be positive %s", n);
     this.limit = n;
@@ -83,14 +75,44 @@ public class ColumnCountGetFilter extend
     return new ColumnCountGetFilter(limit);
   }
 
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.limit = in.readInt();
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.ColumnCountGetFilter.Builder builder =
+      FilterProtos.ColumnCountGetFilter.newBuilder();
+    builder.setLimit(this.limit);
+    return builder.build().toByteArray();
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(this.limit);
+  /**
+   * @param pbBytes A pb serialized {@link ColumnCountGetFilter} instance
+   * @return An instance of {@link ColumnCountGetFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static ColumnCountGetFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.ColumnCountGetFilter proto;
+    try {
+      proto = FilterProtos.ColumnCountGetFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return new ColumnCountGetFilter(proto.getLimit());
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof ColumnCountGetFilter)) return false;
+
+    ColumnCountGetFilter other = (ColumnCountGetFilter)o;
+    return this.getLimit() == other.getLimit();
   }
 
   @Override

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java Fri Aug 24 23:51:51 2012
@@ -19,15 +19,16 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
 import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset.
@@ -42,14 +43,6 @@ public class ColumnPaginationFilter exte
   private int offset = 0;
   private int count = 0;
 
-    /**
-     * Used during serialization. Do not use.
-     */
-  public ColumnPaginationFilter()
-  {
-    super();
-  }
-
   public ColumnPaginationFilter(final int limit, final int offset)
   {
     Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
@@ -99,16 +92,45 @@ public class ColumnPaginationFilter exte
     return new ColumnPaginationFilter(limit, offset);
   }
 
-  public void readFields(DataInput in) throws IOException
-  {
-    this.limit = in.readInt();
-    this.offset = in.readInt();
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.ColumnPaginationFilter.Builder builder =
+      FilterProtos.ColumnPaginationFilter.newBuilder();
+    builder.setLimit(this.limit);
+    builder.setOffset(this.offset);
+    return builder.build().toByteArray();
   }
 
-  public void write(DataOutput out) throws IOException
-  {
-    out.writeInt(this.limit);
-    out.writeInt(this.offset);
+  /**
+   * @param pbBytes A pb serialized {@link ColumnPaginationFilter} instance
+   * @return An instance of {@link ColumnPaginationFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static ColumnPaginationFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.ColumnPaginationFilter proto;
+    try {
+      proto = FilterProtos.ColumnPaginationFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return new ColumnPaginationFilter(proto.getLimit(),proto.getOffset());
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof ColumnPaginationFilter)) return false;
+
+    ColumnPaginationFilter other = (ColumnPaginationFilter)o;
+    return this.getLimit() == other.getLimit() && this.getOffset() == other.getOffset();
   }
 
   @Override

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java Fri Aug 24 23:51:51 2012
@@ -22,15 +22,16 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.DataInput;
 import java.util.ArrayList;
 
 import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used for selecting only those keys with columns that matches
@@ -42,10 +43,6 @@ import com.google.common.base.Preconditi
 public class ColumnPrefixFilter extends FilterBase {
   protected byte [] prefix = null;
 
-  public ColumnPrefixFilter() {
-    super();
-  }
-
   public ColumnPrefixFilter(final byte [] prefix) {
     this.prefix = prefix;
   }
@@ -92,12 +89,44 @@ public class ColumnPrefixFilter extends 
     return new ColumnPrefixFilter(columnPrefix);
   }
 
-  public void write(DataOutput out) throws IOException {
-    Bytes.writeByteArray(out, this.prefix);
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.ColumnPrefixFilter.Builder builder =
+      FilterProtos.ColumnPrefixFilter.newBuilder();
+    if (this.prefix != null) builder.setPrefix(ByteString.copyFrom(this.prefix));
+    return builder.build().toByteArray();
+  }
+
+  /**
+   * @param pbBytes A pb serialized {@link ColumnPrefixFilter} instance
+   * @return An instance of {@link ColumnPrefixFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static ColumnPrefixFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.ColumnPrefixFilter proto;
+    try {
+      proto = FilterProtos.ColumnPrefixFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return new ColumnPrefixFilter(proto.getPrefix().toByteArray());
   }
 
-  public void readFields(DataInput in) throws IOException {
-    this.prefix = Bytes.readByteArray(in);
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+   if (o == this) return true;
+   if (!(o instanceof ColumnPrefixFilter)) return false;
+
+   ColumnPrefixFilter other = (ColumnPrefixFilter)o;
+    return Bytes.equals(this.getPrefix(), other.getPrefix());
   }
 
   public KeyValue getNextKeyHint(KeyValue kv) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java Fri Aug 24 23:51:51 2012
@@ -22,15 +22,16 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.DataInput;
 import java.util.ArrayList;
 
 import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This filter is used for selecting only those keys with columns that are
@@ -52,9 +53,6 @@ public class ColumnRangeFilter extends F
   protected byte[] maxColumn = null;
   protected boolean maxColumnInclusive = false;
 
-  public ColumnRangeFilter() {
-    super();
-  }
   /**
    * Create a filter to select those keys with columns that are between minColumn
    * and maxColumn.
@@ -166,38 +164,53 @@ public class ColumnRangeFilter extends F
                                  maxColumn, maxColumnInclusive);
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    // need to write out a flag for null value separately. Otherwise,
-    // we will not be able to differentiate empty string and null
-    out.writeBoolean(this.minColumn == null);
-    Bytes.writeByteArray(out, this.minColumn);
-    out.writeBoolean(this.minColumnInclusive);
-
-    out.writeBoolean(this.maxColumn == null);
-    Bytes.writeByteArray(out, this.maxColumn);
-    out.writeBoolean(this.maxColumnInclusive);
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.ColumnRangeFilter.Builder builder =
+      FilterProtos.ColumnRangeFilter.newBuilder();
+    if (this.minColumn != null) builder.setMinColumn(ByteString.copyFrom(this.minColumn));
+    builder.setMinColumnInclusive(this.minColumnInclusive);
+    if (this.maxColumn != null) builder.setMaxColumn(ByteString.copyFrom(this.maxColumn));
+    builder.setMaxColumnInclusive(this.maxColumnInclusive);
+    return builder.build().toByteArray();
   }
 
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    boolean isMinColumnNull = in.readBoolean();
-    this.minColumn = Bytes.readByteArray(in);
-
-    if (isMinColumnNull) {
-      this.minColumn = null;
-    }
-
-    this.minColumnInclusive = in.readBoolean();
-
-    boolean isMaxColumnNull = in.readBoolean();
-    this.maxColumn = Bytes.readByteArray(in);
-    if (isMaxColumnNull) {
-      this.maxColumn = null;
-    }
-    this.maxColumnInclusive = in.readBoolean();
+  /**
+   * @param pbBytes A pb serialized {@link ColumnRangeFilter} instance
+   * @return An instance of {@link ColumnRangeFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static ColumnRangeFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.ColumnRangeFilter proto;
+    try {
+      proto = FilterProtos.ColumnRangeFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return new ColumnRangeFilter(proto.hasMinColumn()?proto.getMinColumn().toByteArray():null,
+      proto.getMinColumnInclusive(),proto.hasMaxColumn()?proto.getMaxColumn().toByteArray():null,
+      proto.getMaxColumnInclusive());
   }
 
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+   if (o == this) return true;
+   if (!(o instanceof ColumnRangeFilter)) return false;
+
+   ColumnRangeFilter other = (ColumnRangeFilter)o;
+   return Bytes.equals(this.getMinColumn(),other.getMinColumn())
+     && this.getMinColumnInclusive() == other.getMinColumnInclusive()
+     && Bytes.equals(this.getMaxColumn(), other.getMaxColumn())
+     && this.getMaxColumnInclusive() == other.getMaxColumnInclusive();
+  }
 
   @Override
   public KeyValue getNextKeyHint(KeyValue kv) {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java Fri Aug 24 23:51:51 2012
@@ -22,12 +22,12 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hbase.io.HbaseObjectWritable;
+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.protobuf.generated.HBaseProtos.CompareType;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
 
 import com.google.common.base.Preconditions;
@@ -72,12 +72,6 @@ public abstract class CompareFilter exte
   protected WritableByteArrayComparable comparator;
 
   /**
-   * Writable constructor, do not use.
-   */
-  public CompareFilter() {
-  }
-
-  /**
    * Constructor.
    * @param compareOp the compare op for row matching
    * @param comparator the comparator for row matching
@@ -149,16 +143,32 @@ public abstract class CompareFilter exte
     return arguments;
   }
 
-  public void readFields(DataInput in) throws IOException {
-    compareOp = CompareOp.valueOf(in.readUTF());
-    comparator = (WritableByteArrayComparable)
-      HbaseObjectWritable.readObject(in, null);
+  /**
+   * @return A pb instance to represent this instance.
+   */
+  FilterProtos.CompareFilter convert() {
+    FilterProtos.CompareFilter.Builder builder =
+      FilterProtos.CompareFilter.newBuilder();
+    HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name());
+    builder.setCompareOp(compareOp);
+    if (this.comparator != null) builder.setComparator(ProtobufUtil.toComparator(this.comparator));
+    return builder.build();
   }
 
-  public void write(DataOutput out) throws IOException {
-    out.writeUTF(compareOp.name());
-    HbaseObjectWritable.writeObject(out, comparator,
-      WritableByteArrayComparable.class, null);
+  /**
+   *
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof CompareFilter)) return false;
+
+    CompareFilter other = (CompareFilter)o;
+    return this.getOperator().equals(other.getOperator()) &&
+      (this.getComparator() == other.getComparator()
+        || this.getComparator().areSerializedFieldsEqual(other.getComparator()));
   }
 
   @Override

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java Fri Aug 24 23:51:51 2012
@@ -19,8 +19,6 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -30,10 +28,15 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter for adding inter-column timestamp matching
@@ -53,12 +56,6 @@ public class DependentColumnFilter exten
   protected Set<Long> stampSet = new HashSet<Long>();
   
   /**
-   * Should only be used for writable
-   */
-  public DependentColumnFilter() {
-  }
-  
-  /**
    * Build a dependent column filter with value checking
    * dependent column varies will be compared using the supplied
    * compareOp and comparator, for usage of which
@@ -217,28 +214,67 @@ public class DependentColumnFilter exten
     }
   }
 
-  @Override
-  public void readFields(DataInput in) throws IOException {
-	super.readFields(in);
-    this.columnFamily = Bytes.readByteArray(in);
-	if(this.columnFamily.length == 0) {
-	  this.columnFamily = null;
-	}
-    
-    this.columnQualifier = Bytes.readByteArray(in);
-    if(this.columnQualifier.length == 0) {
-      this.columnQualifier = null;
-    }	
-    
-    this.dropDependentColumn = in.readBoolean();
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.DependentColumnFilter.Builder builder =
+      FilterProtos.DependentColumnFilter.newBuilder();
+    builder.setCompareFilter(super.convert());
+    if (this.columnFamily != null) {
+      builder.setColumnFamily(ByteString.copyFrom(this.columnFamily));
+    }
+    if (this.columnQualifier != null) {
+      builder.setColumnQualifier(ByteString.copyFrom(this.columnQualifier));
+    }
+    builder.setDropDependentColumn(this.dropDependentColumn);
+    return builder.build().toByteArray();
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    Bytes.writeByteArray(out, this.columnFamily);
-    Bytes.writeByteArray(out, this.columnQualifier);
-    out.writeBoolean(this.dropDependentColumn);    
+  /**
+   * @param pbBytes A pb serialized {@link DependentColumnFilter} instance
+   * @return An instance of {@link DependentColumnFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static DependentColumnFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.DependentColumnFilter proto;
+    try {
+      proto = FilterProtos.DependentColumnFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    final CompareOp valueCompareOp =
+      CompareOp.valueOf(proto.getCompareFilter().getCompareOp().name());
+    WritableByteArrayComparable valueComparator = null;
+    try {
+      if (proto.getCompareFilter().hasComparator()) {
+        valueComparator = ProtobufUtil.toComparator(proto.getCompareFilter().getComparator());
+      }
+    } catch (IOException ioe) {
+      throw new DeserializationException(ioe);
+    }
+    return new DependentColumnFilter(
+      proto.hasColumnFamily()?proto.getColumnFamily().toByteArray():null,
+      proto.hasColumnQualifier()?proto.getColumnQualifier().toByteArray():null,
+      proto.getDropDependentColumn(), valueCompareOp, valueComparator);
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof DependentColumnFilter)) return false;
+
+    DependentColumnFilter other = (DependentColumnFilter)o;
+    return other != null && super.areSerializedFieldsEqual(other)
+      && Bytes.equals(this.getFamily(), other.getFamily())
+      && Bytes.equals(this.getQualifier(), other.getQualifier())
+      && this.dropDependentColumn() == other.dropDependentColumn();
   }
 
   @Override

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java Fri Aug 24 23:51:51 2012
@@ -22,8 +22,14 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
+import com.google.protobuf.InvalidProtocolBufferException;
+
+import java.io.IOException;
 import java.util.ArrayList;
 
 /**
@@ -42,11 +48,6 @@ import java.util.ArrayList;
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class FamilyFilter extends CompareFilter {
-  /**
-   * Writable constructor, do not use.
-   */
-  public FamilyFilter() {
-  }
 
   /**
    * Constructor.
@@ -76,5 +77,55 @@ public class FamilyFilter extends Compar
     CompareOp compareOp = (CompareOp)arguments.get(0);
     WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
     return new FamilyFilter(compareOp, comparator);
-}
+  }
+
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.FamilyFilter.Builder builder =
+      FilterProtos.FamilyFilter.newBuilder();
+    builder.setCompareFilter(super.convert());
+    return builder.build().toByteArray();
+  }
+
+  /**
+   * @param pbBytes A pb serialized {@link FamilyFilter} instance
+   * @return An instance of {@link FamilyFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static FamilyFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.FamilyFilter proto;
+    try {
+      proto = FilterProtos.FamilyFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    final CompareOp valueCompareOp =
+      CompareOp.valueOf(proto.getCompareFilter().getCompareOp().name());
+    WritableByteArrayComparable valueComparator = null;
+    try {
+      if (proto.getCompareFilter().hasComparator()) {
+        valueComparator = ProtobufUtil.toComparator(proto.getCompareFilter().getComparator());
+      }
+    } catch (IOException ioe) {
+      throw new DeserializationException(ioe);
+    }
+    return new FamilyFilter(valueCompareOp,valueComparator);
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof FamilyFilter)) return false;
+
+    FamilyFilter other = (FamilyFilter)o;
+    return super.areSerializedFieldsEqual(other);
+ }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java Fri Aug 24 23:51:51 2012
@@ -22,8 +22,8 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.io.Writable;
 
 import java.util.List;
 
@@ -52,11 +52,11 @@ import java.util.List;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public interface Filter extends Writable {
+public abstract class Filter {
   /**
    * Reset the state of the filter between rows.
    */
-  public void reset();
+  abstract public void reset();
 
   /**
    * Filters a row based on the row key. If this returns true, the entire
@@ -68,14 +68,14 @@ public interface Filter extends Writable
    * @param length length of the row key
    * @return true, remove entire row, false, include the row (maybe).
    */
-  public boolean filterRowKey(byte [] buffer, int offset, int length);
+  abstract public boolean filterRowKey(byte [] buffer, int offset, int length);
 
   /**
    * If this returns true, the scan will terminate.
    *
    * @return true to end scan, false to continue.
    */
-  public boolean filterAllRemaining();
+  abstract public boolean filterAllRemaining();
 
   /**
    * A way to filter based on the column family, column qualifier and/or the
@@ -91,7 +91,7 @@ public interface Filter extends Writable
    * @return code as described below
    * @see Filter.ReturnCode
    */
-  public ReturnCode filterKeyValue(final KeyValue v);
+  abstract public ReturnCode filterKeyValue(final KeyValue v);
 
   /**
    * Give the filter a chance to transform the passed KeyValue.
@@ -106,7 +106,7 @@ public interface Filter extends Writable
    * @param v the KeyValue in question
    * @return the changed KeyValue
    */
-  public KeyValue transform(final KeyValue v);
+  abstract public KeyValue transform(final KeyValue v);
 
   /**
    * Return codes for filterValue().
@@ -140,14 +140,14 @@ public interface Filter extends Writable
    * Modifications to the list will carry on
    * @param kvs the list of keyvalues to be filtered
    */
-  public void filterRow(List<KeyValue> kvs);
+  abstract public void filterRow(List<KeyValue> kvs);
 
   /**
    * @return True if this filter actively uses filterRow(List) or filterRow().
    * Primarily used to check for conflicts with scans(such as scans
    * that do not read a full row at a time)
    */
-  public boolean hasFilterRow();
+  abstract public boolean hasFilterRow();
 
   /**
    * Last chance to veto row based on previous {@link #filterKeyValue(KeyValue)}
@@ -156,7 +156,7 @@ public interface Filter extends Writable
    * (for example).
    * @return true to exclude row, false to include row.
    */
-  public boolean filterRow();
+  abstract public boolean filterRow();
 
   /**
    * If the filter returns the match code SEEK_NEXT_USING_HINT, then
@@ -166,5 +166,28 @@ public interface Filter extends Writable
    * @return KeyValue which must be next seeked. return null if the filter is
    * not sure which key to seek to next.
    */
-  public KeyValue getNextKeyHint(final KeyValue currentKV);
+  abstract public KeyValue getNextKeyHint(final KeyValue currentKV);
+
+  /**
+   * @return The filter serialized using pb
+   */
+  abstract public byte [] toByteArray();
+
+  /**
+   * @param pbBytes A pb serialized {@link Filter} instance
+   * @return An instance of {@link Filter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException {
+    throw new DeserializationException(
+      "parseFrom called on base Filter, but should be called on derived type");
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  abstract boolean areSerializedFieldsEqual(Filter other);
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java Fri Aug 24 23:51:51 2012
@@ -36,7 +36,7 @@ import java.util.ArrayList;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public abstract class FilterBase implements Filter {
+public abstract class FilterBase extends Filter {
 
   /**
    * Filters that are purely stateless and do nothing in their reset() methods can inherit
@@ -150,4 +150,15 @@ public abstract class FilterBase impleme
   public String toString() {
     return this.getClass().getSimpleName();
   }
+
+  /**
+   * Default implementation so that writers of custom filters aren't forced to implement.
+   *
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter other) {
+    return true;
+  }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java Fri Aug 24 23:51:51 2012
@@ -22,13 +22,15 @@ package org.apache.hadoop.hbase.filter;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.io.HbaseObjectWritable;
-import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+
+import com.google.protobuf.InvalidProtocolBufferException;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -45,7 +47,7 @@ import java.util.List;
  */
 @InterfaceAudience.Public
 @InterfaceStability.Stable
-public class FilterList implements Filter {
+public class FilterList extends Filter {
   /** set operator */
   public static enum Operator {
     /** !AND */
@@ -60,14 +62,6 @@ public class FilterList implements Filte
   private List<Filter> filters = new ArrayList<Filter>();
 
   /**
-   * Default constructor, filters nothing. Required though for RPC
-   * deserialization.
-   */
-  public FilterList() {
-    super();
-  }
-
-  /**
    * Constructor that takes a set of {@link Filter}s. The default operator
    * MUST_PASS_ALL is assumed.
    *
@@ -278,25 +272,58 @@ public class FilterList implements Filte
     return  operator == Operator.MUST_PASS_ONE;
   }
 
-  public void readFields(final DataInput in) throws IOException {
-    byte opByte = in.readByte();
-    operator = Operator.values()[opByte];
-    int size = in.readInt();
-    if (size > 0) {
-      filters = new ArrayList<Filter>(size);
-      for (int i = 0; i < size; i++) {
-        Filter filter = (Filter)HbaseObjectWritable.readObject(in, conf);
-        filters.add(filter);
-      }
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.FilterList.Builder builder =
+      FilterProtos.FilterList.newBuilder();
+    builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
+    for (Filter filter : filters) {
+      builder.addFilters(ProtobufUtil.toFilter(filter));
     }
+    return builder.build().toByteArray();
   }
 
-  public void write(final DataOutput out) throws IOException {
-    out.writeByte(operator.ordinal());
-    out.writeInt(filters.size());
-    for (Filter filter : filters) {
-      HbaseObjectWritable.writeObject(out, filter, Writable.class, conf);
+  /**
+   * @param pbBytes A pb serialized {@link FilterList} instance
+   * @return An instance of {@link FilterList} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static FilterList parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.FilterList proto;
+    try {
+      proto = FilterProtos.FilterList.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+
+    List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
+    try {
+      for (HBaseProtos.Filter filter : proto.getFiltersList()) {
+        rowFilters.add(ProtobufUtil.toFilter(filter));
+      }
+    } catch (IOException ioe) {
+      throw new DeserializationException(ioe);
     }
+    return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters);
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof FilterList)) return false;
+
+    FilterList other = (FilterList)o;
+    return this.getOperator().equals(other.getOperator()) &&
+      ((this.getFilters() == other.getFilters())
+      || this.getFilters().equals(other.getFilters()));
   }
 
   @Override

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java Fri Aug 24 23:51:51 2012
@@ -19,15 +19,18 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * This is a Filter wrapper class which is used in the server side. Some filter
@@ -38,7 +41,7 @@ import org.apache.hadoop.hbase.KeyValue;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class FilterWrapper implements Filter {
+public class FilterWrapper extends Filter {
   Filter filter = null;
 
   public FilterWrapper( Filter filter ) {
@@ -49,14 +52,35 @@ public class FilterWrapper implements Fi
     this.filter = filter;
   }
 
-  @Override
-  public void write(DataOutput out) throws IOException {
-    this.filter.write(out);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.filter.readFields(in);
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.FilterWrapper.Builder builder =
+      FilterProtos.FilterWrapper.newBuilder();
+    builder.setFilter(ProtobufUtil.toFilter(this.filter));
+    return builder.build().toByteArray();
+  }
+
+  /**
+   * @param pbBytes A pb serialized {@link FilterWrapper} instance
+   * @return An instance of {@link FilterWrapper} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static FilterWrapper parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.FilterWrapper proto;
+    try {
+      proto = FilterProtos.FilterWrapper.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    try {
+      return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter()));
+    } catch (IOException ioe) {
+      throw new DeserializationException(ioe);
+    }
   }
 
   @Override
@@ -112,4 +136,16 @@ public class FilterWrapper implements Fi
     }
   }
 
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof FilterWrapper)) return false;
+
+    FilterWrapper other = (FilterWrapper)o;
+    return this.filter.areSerializedFieldsEqual(other.filter);
+  }
 }
\ No newline at end of file

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java Fri Aug 24 23:51:51 2012
@@ -21,15 +21,14 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.DataInput;
-import java.util.List;
 import java.util.ArrayList;
 
 import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A filter that will only return the first KV from each row.
@@ -75,9 +74,42 @@ public class FirstKeyOnlyFilter extends 
     this.foundKV = value;
   }
 
-  public void write(DataOutput out) throws IOException {
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.FirstKeyOnlyFilter.Builder builder =
+      FilterProtos.FirstKeyOnlyFilter.newBuilder();
+    return builder.build().toByteArray();
   }
 
-  public void readFields(DataInput in) throws IOException {
+  /**
+   * @param pbBytes A pb serialized {@link FirstKeyOnlyFilter} instance
+   * @return An instance of {@link FirstKeyOnlyFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static FirstKeyOnlyFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.FirstKeyOnlyFilter proto;
+    try {
+      proto = FilterProtos.FirstKeyOnlyFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+
+    return new FirstKeyOnlyFilter();
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof FirstKeyOnlyFilter)) return false;
+
+    return true;
   }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java Fri Aug 24 23:51:51 2012
@@ -20,11 +20,18 @@ package org.apache.hadoop.hbase.filter;
 
 import java.util.Collections;
 import java.util.Set;
+import java.util.TreeSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.mapreduce.RowCounter;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * The filter looks for the given columns in KeyValue. Once there is a match for
@@ -43,13 +50,6 @@ public class FirstKeyValueMatchingQualif
   private Set<byte []> qualifiers;
 
   /**
-   * This constructor should not be used.
-   */
-  public FirstKeyValueMatchingQualifiersFilter() {
-    qualifiers = Collections.emptySet();
-  }
-
-  /**
    * Constructor which takes a set of columns. As soon as first KeyValue
    * matching any of these columns is found, filter moves to next row.
    * 
@@ -77,4 +77,50 @@ public class FirstKeyValueMatchingQualif
     return false;
   }
 
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder =
+      FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder();
+    for (byte[] qualifier : qualifiers) {
+      if (qualifier != null) builder.addQualifiers(ByteString.copyFrom(qualifier));
+    }
+    return builder.build().toByteArray();
+  }
+
+  /**
+   * @param pbBytes A pb serialized {@link FirstKeyValueMatchingQualifiersFilter} instance
+   * @return An instance of {@link FirstKeyValueMatchingQualifiersFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static FirstKeyValueMatchingQualifiersFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.FirstKeyValueMatchingQualifiersFilter proto;
+    try {
+      proto = FilterProtos.FirstKeyValueMatchingQualifiersFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+
+    TreeSet<byte []> qualifiers = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+    for (ByteString qualifier : proto.getQualifiersList()) {
+      qualifiers.add(qualifier.toByteArray());
+    }
+    return new FirstKeyValueMatchingQualifiersFilter(qualifiers);
+  }
+
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof FirstKeyValueMatchingQualifiersFilter)) return false;
+
+    FirstKeyValueMatchingQualifiersFilter other = (FirstKeyValueMatchingQualifiersFilter)o;
+    return this.qualifiers.equals(other.qualifiers);
+  }
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java Fri Aug 24 23:51:51 2012
@@ -17,19 +17,22 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+
 /**
  * Filters data based on fuzzy row key. Performs fast-forwards during scanning.
  * It takes pairs (row key, fuzzy info) to match row keys. Where fuzzy info is
@@ -64,12 +67,6 @@ public class FuzzyRowFilter extends Filt
   private List<Pair<byte[], byte[]>> fuzzyKeysData;
   private boolean done = false;
 
-  /**
-   * Used internally for reflection, do NOT use it directly
-   */
-  public FuzzyRowFilter() {
-  }
-
   public FuzzyRowFilter(List<Pair<byte[], byte[]>> fuzzyKeysData) {
     this.fuzzyKeysData = fuzzyKeysData;
   }
@@ -134,24 +131,44 @@ public class FuzzyRowFilter extends Filt
     return done;
   }
 
-  @Override
-  public void write(DataOutput dataOutput) throws IOException {
-    dataOutput.writeInt(this.fuzzyKeysData.size());
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.FuzzyRowFilter.Builder builder =
+      FilterProtos.FuzzyRowFilter.newBuilder();
     for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
-      Bytes.writeByteArray(dataOutput, fuzzyData.getFirst());
-      Bytes.writeByteArray(dataOutput, fuzzyData.getSecond());
+      BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
+      bbpBuilder.setFirst(ByteString.copyFrom(fuzzyData.getFirst()));
+      bbpBuilder.setSecond(ByteString.copyFrom(fuzzyData.getSecond()));
+      builder.addFuzzyKeysData(bbpBuilder);
     }
+    return builder.build().toByteArray();
   }
 
-  @Override
-  public void readFields(DataInput dataInput) throws IOException {
-    int count = dataInput.readInt();
-    this.fuzzyKeysData = new ArrayList<Pair<byte[], byte[]>>(count);
-    for (int i = 0; i < count; i++) {
-      byte[] keyBytes = Bytes.readByteArray(dataInput);
-      byte[] keyMeta = Bytes.readByteArray(dataInput);
-      this.fuzzyKeysData.add(new Pair<byte[], byte[]>(keyBytes, keyMeta));
+  /**
+   * @param pbBytes A pb serialized {@link FuzzyRowFilter} instance
+   * @return An instance of {@link FuzzyRowFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static FuzzyRowFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.FuzzyRowFilter proto;
+    try {
+      proto = FilterProtos.FuzzyRowFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    int count = proto.getFuzzyKeysDataCount();
+    ArrayList<Pair<byte[], byte[]>> fuzzyKeysData= new ArrayList<Pair<byte[], byte[]>>(count);
+    for (int i = 0; i < count; ++i) {
+      BytesBytesPair current = proto.getFuzzyKeysData(i);
+      byte[] keyBytes = current.getFirst().toByteArray();
+      byte[] keyMeta = current.getSecond().toByteArray();
+      fuzzyKeysData.add(new Pair<byte[], byte[]>(keyBytes, keyMeta));
     }
+    return new FuzzyRowFilter(fuzzyKeysData);
   }
 
   @Override
@@ -291,4 +308,26 @@ public class FuzzyRowFilter extends Filt
     return result;
   }
 
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof FuzzyRowFilter)) return false;
+
+    FuzzyRowFilter other = (FuzzyRowFilter)o;
+    if (this.fuzzyKeysData.size() != other.fuzzyKeysData.size()) return false;
+    for (int i = 0; i < fuzzyKeysData.size(); ++i) {
+      Pair<byte[], byte[]> thisData = this.fuzzyKeysData.get(i);
+      Pair<byte[], byte[]> otherData = other.fuzzyKeysData.get(i);
+      if (!(Bytes.equals(thisData.getFirst(), otherData.getFirst())
+        && Bytes.equals(thisData.getSecond(), otherData.getSecond()))) {
+        return false;
+      }
+    }
+    return true;
+  }
+
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java Fri Aug 24 23:51:51 2012
@@ -22,16 +22,16 @@ package org.apache.hadoop.hbase.filter;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
 import java.util.ArrayList;
 
 import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * A Filter that stops after the given row.  There is no "RowStopFilter" because
@@ -45,10 +45,6 @@ public class InclusiveStopFilter extends
   private byte [] stopRowKey;
   private boolean done = false;
 
-  public InclusiveStopFilter() {
-    super();
-  }
-
   public InclusiveStopFilter(final byte [] stopRowKey) {
     this.stopRowKey = stopRowKey;
   }
@@ -86,12 +82,44 @@ public class InclusiveStopFilter extends
     return new InclusiveStopFilter(stopRowKey);
   }
 
-  public void write(DataOutput out) throws IOException {
-    Bytes.writeByteArray(out, this.stopRowKey);
+  /**
+   * @return The filter serialized using pb
+   */
+  public byte [] toByteArray() {
+    FilterProtos.InclusiveStopFilter.Builder builder =
+      FilterProtos.InclusiveStopFilter.newBuilder();
+    if (this.stopRowKey != null) builder.setStopRowKey(ByteString.copyFrom(this.stopRowKey));
+    return builder.build().toByteArray();
+  }
+
+  /**
+   * @param pbBytes A pb serialized {@link InclusiveStopFilter} instance
+   * @return An instance of {@link InclusiveStopFilter} made from <code>bytes</code>
+   * @throws DeserializationException
+   * @see {@link #toByteArray()}
+   */
+  public static InclusiveStopFilter parseFrom(final byte [] pbBytes)
+  throws DeserializationException {
+    FilterProtos.InclusiveStopFilter proto;
+    try {
+      proto = FilterProtos.InclusiveStopFilter.parseFrom(pbBytes);
+    } catch (InvalidProtocolBufferException e) {
+      throw new DeserializationException(e);
+    }
+    return new InclusiveStopFilter(proto.hasStopRowKey()?proto.getStopRowKey().toByteArray():null);
   }
 
-  public void readFields(DataInput in) throws IOException {
-    this.stopRowKey = Bytes.readByteArray(in);
+  /**
+   * @param other
+   * @return true if and only if the fields of the filter that are serialized
+   * are equal to the corresponding fields in other.  Used for testing.
+   */
+  boolean areSerializedFieldsEqual(Filter o) {
+    if (o == this) return true;
+    if (!(o instanceof InclusiveStopFilter)) return false;
+
+    InclusiveStopFilter other = (InclusiveStopFilter)o;
+    return Bytes.equals(this.getStopRowKey(), other.getStopRowKey());
   }
 
   @Override



Mime
View raw message