Author: apurtell
Date: Wed Sep  8 20:26:46 2010
New Revision: 995240

URL: http://svn.apache.org/viewvc?rev=995240&view=rev
Log:
HBASE-2942 Custom filters should not require registration in HBaseObjectWritable


Modified:
    hbase/branches/0.20/CHANGES.txt
    
hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
    
hbase/branches/0.20/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java

Modified: hbase/branches/0.20/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/hbase/branches/0.20/CHANGES.txt?rev=995240&r1=995239&r2=995240&view=diff
==============================================================================
--- hbase/branches/0.20/CHANGES.txt (original)
+++ hbase/branches/0.20/CHANGES.txt Wed Sep  8 20:26:46 2010
@@ -5,6 +5,8 @@ Release 0.20.7 - Unreleased
    HBASE-2927  BaseScanner gets stale HRegionInfo in some race cases
    HBASE-2897  [stargate] RowResultGenerator should handle 
                NoSuchColumnFamilyException (Andrew Purtell via J-D)
+   HBASE-2942  Custom filters should not require registration in
+               HBaseObjectWritable (Gary Helmling via Andrew Purtell)
 
   IMPROVEMENTS
 

Modified: 
hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
URL: 
http://svn.apache.org/viewvc/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java?rev=995240&r1=995239&r2=995240&view=diff
==============================================================================
--- 
hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
 (original)
+++ 
hbase/branches/0.20/src/java/org/apache/hadoop/hbase/io/HbaseObjectWritable.java
 Wed Sep  8 20:26:46 2010
@@ -246,6 +246,11 @@ public class HbaseObjectWritable impleme
   throws IOException {
     Byte code = CLASS_TO_CODE.get(c);
     if (code == null) {
+      if (Writable.class.isAssignableFrom(c)) {
+        code = CLASS_TO_CODE.get(Writable.class);
+      }
+    }
+    if (code == null) {
       LOG.error("Unsupported type " + c);
       StackTraceElement[] els = new Exception().getStackTrace();
       for(StackTraceElement elem : els) {

Modified: 
hbase/branches/0.20/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java
URL: 
http://svn.apache.org/viewvc/hbase/branches/0.20/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java?rev=995240&r1=995239&r2=995240&view=diff
==============================================================================
--- 
hbase/branches/0.20/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java
 (original)
+++ 
hbase/branches/0.20/src/test/org/apache/hadoop/hbase/io/TestHbaseObjectWritable.java
 Wed Sep  8 20:26:46 2010
@@ -21,7 +21,9 @@ package org.apache.hadoop.hbase.io;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
 import java.io.DataInputStream;
+import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
@@ -30,10 +32,14 @@ import junit.framework.TestCase;
 
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.RowFilterInterface;
 import org.apache.hadoop.hbase.filter.StopRowFilter;
 import org.apache.hadoop.hbase.filter.PrefixFilter;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparator;
 
 public class TestHbaseObjectWritable extends TestCase {
@@ -85,6 +91,29 @@ public class TestHbaseObjectWritable ext
       PrefixFilter.class);
     assertTrue(obj instanceof PrefixFilter);
   }
+
+  public void testCustomWritable() throws Exception {
+    HBaseConfiguration conf = new HBaseConfiguration();
+
+    // test proper serialization of un-encoded custom writables
+    CustomWritable custom = new CustomWritable("test phrase");
+    Object obj = doType(conf, custom, CustomWritable.class);
+    assertTrue(obj instanceof Writable);
+    assertTrue(obj instanceof CustomWritable);
+    assertEquals("test phrase", ((CustomWritable)obj).getValue());
+
+    // 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);
+    assertTrue(obj instanceof FilterList);
+    assertNotNull(((FilterList)obj).getFilters());
+    assertEquals(1, ((FilterList)obj).getFilters().size());
+    Filter child = ((FilterList)obj).getFilters().get(0);
+    assertTrue(child instanceof CustomFilter);
+    assertEquals("mykey", ((CustomFilter)child).getKey());
+  }
   
   private Object doType(final HBaseConfiguration conf, final Object value,
       final Class<?> clazz)
@@ -100,4 +129,65 @@ public class TestHbaseObjectWritable ext
     dis.close();
     return product;
   }
-}
\ No newline at end of file
+
+  public static class CustomWritable implements Writable {
+    private String value = null;
+
+    public CustomWritable() {
+    }
+
+    public CustomWritable(String val) {
+      this.value = val;
+    }
+
+    public String getValue() { return value; }
+
+    public void write(DataOutput out) throws IOException {
+      Text.writeString(out, this.value);
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      this.value = Text.readString(in);
+    }
+  }
+
+  public static class CustomFilter implements Filter {
+    private String key = null;
+
+    public CustomFilter() {
+    }
+
+    public CustomFilter(String key) {
+      this.key = key;
+    }
+
+    public String getKey() { return key; }
+
+    public void write(DataOutput out) throws IOException {
+      Text.writeString(out, this.key);
+    }
+
+    public void readFields(DataInput in) throws IOException {
+      this.key = Text.readString(in);
+    }
+
+       public boolean filterAllRemaining() {
+               return false;
+       }
+
+       public ReturnCode filterKeyValue(KeyValue v) {
+               return null;
+       }
+
+       public boolean filterRow() {
+               return false;
+       }
+
+       public boolean filterRowKey(byte[] buffer, int offset, int length) {
+               return false;
+       }
+
+       public void reset() {
+       }
+  }
+}


Reply via email to