HBASE-2942 Custom filters should not require registration in HBaseObjectWritable
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@995220 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
3b16358da3
commit
6e2d002734
|
@ -889,6 +889,8 @@ Release 0.21.0 - Unreleased
|
|||
HBASE-2857 HBaseAdmin.tableExists() should not require a full meta scan
|
||||
HBASE-2962 Add missing methods to HTableInterface (and HTable)
|
||||
(Lars Francke via Stack)
|
||||
HBASE-2942 Custom filters should not require registration in
|
||||
HBaseObjectWritable (Gary Helmling via Andrew Purtell)
|
||||
|
||||
NEW FEATURES
|
||||
HBASE-1961 HBase EC2 scripts
|
||||
|
|
|
@ -294,6 +294,9 @@ public class HbaseObjectWritable implements Writable, Configurable {
|
|||
if ( List.class.isAssignableFrom(c)) {
|
||||
code = CLASS_TO_CODE.get(List.class);
|
||||
}
|
||||
else if (Writable.class.isAssignableFrom(c)) {
|
||||
code = CLASS_TO_CODE.get(Writable.class);
|
||||
}
|
||||
}
|
||||
if (code == null) {
|
||||
LOG.error("Unsupported type " + c);
|
||||
|
|
|
@ -19,21 +19,21 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.io;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.*;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import junit.framework.TestCase;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.filter.FilterList;
|
||||
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;
|
||||
import org.junit.Assert;
|
||||
|
||||
|
@ -99,7 +99,30 @@ public class TestHbaseObjectWritable extends TestCase {
|
|||
assertTrue(obj instanceof PrefixFilter);
|
||||
}
|
||||
|
||||
private Object doType(final HBaseConfiguration conf, final Object value,
|
||||
public void testCustomWritable() throws Exception {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
|
||||
// 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 Configuration conf, final Object value,
|
||||
final Class<?> clazz)
|
||||
throws IOException {
|
||||
ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
|
||||
|
@ -114,4 +137,47 @@ public class TestHbaseObjectWritable extends TestCase {
|
|||
return product;
|
||||
}
|
||||
|
||||
public static class CustomWritable implements Writable {
|
||||
private String value = null;
|
||||
|
||||
public CustomWritable() {
|
||||
}
|
||||
|
||||
public CustomWritable(String val) {
|
||||
this.value = val;
|
||||
}
|
||||
|
||||
public String getValue() { return value; }
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
Text.writeString(out, this.value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
this.value = Text.readString(in);
|
||||
}
|
||||
}
|
||||
|
||||
public static class CustomFilter extends FilterBase {
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue