You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/08/25 01:51:53 UTC
svn commit: r1377154 [2/6] - in /hbase/trunk/hbase-server/src:
main/java/org/apache/hadoop/hbase/client/
main/java/org/apache/hadoop/hbase/filter/
main/java/org/apache/hadoop/hbase/mapreduce/
main/java/org/apache/hadoop/hbase/protobuf/ main/java/org/ap...
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java Fri Aug 24 23:51:51 2012
@@ -19,18 +19,17 @@
*/
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.KeyValue;
-import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import java.util.ArrayList;
import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
/**
* A filter that will only return the key component of each KV (the value will
@@ -58,11 +57,43 @@ public class KeyOnlyFilter extends Filte
return new KeyOnlyFilter();
}
- public void write(DataOutput out) throws IOException {
- out.writeBoolean(this.lenAsVal);
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.KeyOnlyFilter.Builder builder =
+ FilterProtos.KeyOnlyFilter.newBuilder();
+ builder.setLenAsVal(this.lenAsVal);
+ return builder.build().toByteArray();
}
- public void readFields(DataInput in) throws IOException {
- this.lenAsVal = in.readBoolean();
+ /**
+ * @param pbBytes A pb serialized {@link KeyOnlyFilter} instance
+ * @return An instance of {@link KeyOnlyFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static KeyOnlyFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.KeyOnlyFilter proto;
+ try {
+ proto = FilterProtos.KeyOnlyFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new KeyOnlyFilter(proto.getLenAsVal());
+ }
+
+ /**
+ * @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 KeyOnlyFilter)) return false;
+
+ KeyOnlyFilter other = (KeyOnlyFilter)o;
+ return this.lenAsVal == other.lenAsVal;
}
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java Fri Aug 24 23:51:51 2012
@@ -19,12 +19,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 org.apache.hadoop.hbase.util.Bytes;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.DataInput;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+
import java.util.Arrays;
import java.util.Comparator;
import java.util.TreeSet;
@@ -42,10 +44,6 @@ public class MultipleColumnPrefixFilter
protected TreeSet<byte []> sortedPrefixes = createTreeSet();
private final static int MAX_LOG_PREFIXES = 5;
- public MultipleColumnPrefixFilter() {
- super();
- }
-
public MultipleColumnPrefixFilter(final byte [][] prefixes) {
if (prefixes != null) {
for (int i = 0; i < prefixes.length; i++) {
@@ -107,19 +105,52 @@ public class MultipleColumnPrefixFilter
return new MultipleColumnPrefixFilter(prefixes);
}
- public void write(DataOutput out) throws IOException {
- out.writeInt(sortedPrefixes.size());
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.MultipleColumnPrefixFilter.Builder builder =
+ FilterProtos.MultipleColumnPrefixFilter.newBuilder();
for (byte [] element : sortedPrefixes) {
- Bytes.writeByteArray(out, element);
+ if (element != null) builder.addSortedPrefixes(ByteString.copyFrom(element));
}
+ return builder.build().toByteArray();
}
- public void readFields(DataInput in) throws IOException {
- int x = in.readInt();
- this.sortedPrefixes = createTreeSet();
- for (int j = 0; j < x; j++) {
- sortedPrefixes.add(Bytes.readByteArray(in));
+ /**
+ * @param pbBytes A pb serialized {@link MultipleColumnPrefixFilter} instance
+ * @return An instance of {@link MultipleColumnPrefixFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static MultipleColumnPrefixFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.MultipleColumnPrefixFilter proto;
+ try {
+ proto = FilterProtos.MultipleColumnPrefixFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ int numPrefixes = proto.getSortedPrefixesCount();
+ byte [][] prefixes = new byte[numPrefixes][];
+ for (int i = 0; i < numPrefixes; ++i) {
+ prefixes[i] = proto.getSortedPrefixes(i).toByteArray();
}
+
+ return new MultipleColumnPrefixFilter(prefixes);
+ }
+
+ /**
+ * @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 MultipleColumnPrefixFilter)) return false;
+
+ MultipleColumnPrefixFilter other = (MultipleColumnPrefixFilter)o;
+ return this.sortedPrefixes.equals(other.sortedPrefixes);
}
public KeyValue getNextKeyHint(KeyValue kv) {
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java Fri Aug 24 23:51:51 2012
@@ -22,6 +22,10 @@ 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 com.google.protobuf.InvalidProtocolBufferException;
/**
* A binary comparator which lexicographically compares against the specified
@@ -31,9 +35,8 @@ import org.apache.hadoop.classification.
@InterfaceStability.Stable
public class NullComparator extends WritableByteArrayComparable {
- /** Nullary constructor for Writable, do not use */
public NullComparator() {
- value = new byte[0];
+ super(new byte[0]);
}
@Override
@@ -45,4 +48,42 @@ public class NullComparator extends Writ
public int compareTo(byte[] value, int offset, int length) {
throw new UnsupportedOperationException();
}
+
+ /**
+ * @return The comparator serialized using pb
+ */
+ public byte [] toByteArray() {
+ ComparatorProtos.NullComparator.Builder builder =
+ ComparatorProtos.NullComparator.newBuilder();
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link NullComparator} instance
+ * @return An instance of {@link NullComparator} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static NullComparator parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ ComparatorProtos.NullComparator proto;
+ try {
+ proto = ComparatorProtos.NullComparator.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new NullComparator();
+ }
+
+ /**
+ * @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 NullComparator)) return false;
+
+ return super.areSerializedFieldsEqual(other);
+ }
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.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.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.InvalidProtocolBufferException;
/**
* Implementation of Filter interface that limits results to a specific page
* size. It terminates scanning once the number of filter-passed rows is >
@@ -48,14 +47,6 @@ public class PageFilter extends FilterBa
private int rowsAccepted = 0;
/**
- * Default constructor, filters nothing. Required though for RPC
- * deserialization.
- */
- public PageFilter() {
- super();
- }
-
- /**
* Constructor that takes a maximum page size.
*
* @param pageSize Maximum result size.
@@ -89,12 +80,44 @@ public class PageFilter extends FilterBa
return new PageFilter(pageSize);
}
- public void readFields(final DataInput in) throws IOException {
- this.pageSize = in.readLong();
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.PageFilter.Builder builder =
+ FilterProtos.PageFilter.newBuilder();
+ builder.setPageSize(this.pageSize);
+ return builder.build().toByteArray();
}
- public void write(final DataOutput out) throws IOException {
- out.writeLong(pageSize);
+ /**
+ * @param pbBytes A pb serialized {@link PageFilter} instance
+ * @return An instance of {@link PageFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static PageFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.PageFilter proto;
+ try {
+ proto = FilterProtos.PageFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new PageFilter(proto.getPageSize());
+ }
+
+ /**
+ * @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 PageFilter)) return false;
+
+ PageFilter other = (PageFilter)o;
+ return this.getPageSize() == other.getPageSize();
}
@Override
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.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.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.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
/**
* Pass results that have same row prefix.
@@ -46,10 +46,6 @@ public class PrefixFilter extends Filter
this.prefix = prefix;
}
- public PrefixFilter() {
- super();
- }
-
public byte[] getPrefix() {
return prefix;
}
@@ -81,12 +77,44 @@ public class PrefixFilter extends Filter
return new PrefixFilter(prefix);
}
- public void write(DataOutput out) throws IOException {
- Bytes.writeByteArray(out, this.prefix);
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.PrefixFilter.Builder builder =
+ FilterProtos.PrefixFilter.newBuilder();
+ if (this.prefix != null) builder.setPrefix(ByteString.copyFrom(this.prefix));
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link PrefixFilter} instance
+ * @return An instance of {@link PrefixFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static PrefixFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.PrefixFilter proto;
+ try {
+ proto = FilterProtos.PrefixFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new PrefixFilter(proto.hasPrefix()?proto.getPrefix().toByteArray():null);
}
- 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 PrefixFilter)) return false;
+
+ PrefixFilter other = (PrefixFilter)o;
+ return Bytes.equals(this.getPrefix(), other.getPrefix());
}
@Override
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java Fri Aug 24 23:51:51 2012
@@ -22,9 +22,15 @@ 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.client.Get;
+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;
/**
@@ -45,12 +51,6 @@ import java.util.ArrayList;
public class QualifierFilter extends CompareFilter {
/**
- * Writable constructor, do not use.
- */
- public QualifierFilter() {
- }
-
- /**
* Constructor.
* @param op the compare op for column qualifier matching
* @param qualifierComparator the comparator for column qualifier matching
@@ -77,5 +77,54 @@ public class QualifierFilter extends Com
CompareOp compareOp = (CompareOp)arguments.get(0);
WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
return new QualifierFilter(compareOp, comparator);
-}
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.QualifierFilter.Builder builder =
+ FilterProtos.QualifierFilter.newBuilder();
+ builder.setCompareFilter(super.convert());
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link QualifierFilter} instance
+ * @return An instance of {@link QualifierFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static QualifierFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.QualifierFilter proto;
+ try {
+ proto = FilterProtos.QualifierFilter.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 QualifierFilter(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 QualifierFilter)) return false;
+
+ return super.areSerializedFieldsEqual(o);
+ }
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java Fri Aug 24 23:51:51 2012
@@ -20,14 +20,15 @@
package org.apache.hadoop.hbase.filter;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
import java.util.Random;
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.protobuf.InvalidProtocolBufferException;
/**
* A filter that includes rows based on a chance.
@@ -42,12 +43,6 @@ public class RandomRowFilter extends Fil
protected boolean filterOutRow;
/**
- * Writable constructor, do not use.
- */
- public RandomRowFilter() {
- }
-
- /**
* Create a new filter with a specified chance for a row to be included.
*
* @param chance
@@ -114,13 +109,43 @@ public class RandomRowFilter extends Fil
filterOutRow = false;
}
- @Override
- public void readFields(DataInput in) throws IOException {
- chance = in.readFloat();
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.RandomRowFilter.Builder builder =
+ FilterProtos.RandomRowFilter.newBuilder();
+ builder.setChance(this.chance);
+ return builder.build().toByteArray();
}
- @Override
- public void write(DataOutput out) throws IOException {
- out.writeFloat(chance);
+ /**
+ * @param pbBytes A pb serialized {@link RandomRowFilter} instance
+ * @return An instance of {@link RandomRowFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static RandomRowFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.RandomRowFilter proto;
+ try {
+ proto = FilterProtos.RandomRowFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new RandomRowFilter(proto.getChance());
+ }
+
+ /**
+ * @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 RandomRowFilter)) return false;
+
+ RandomRowFilter other = (RandomRowFilter)o;
+ return this.getChance() == other.getChance();
}
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java Fri Aug 24 23:51:51 2012
@@ -21,15 +21,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.HConstants;
+import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
+import com.google.protobuf.InvalidProtocolBufferException;
+
import java.nio.charset.Charset;
import java.nio.charset.IllegalCharsetNameException;
import java.util.regex.Pattern;
@@ -74,9 +75,6 @@ public class RegexStringComparator exten
private Pattern pattern;
- /** Nullary constructor for Writable, do not use */
- public RegexStringComparator() { }
-
/**
* Constructor
* Adds Pattern.DOTALL to the underlying Pattern
@@ -119,27 +117,59 @@ public class RegexStringComparator exten
: 1;
}
- @Override
- public void readFields(DataInput in) throws IOException {
- final String expr = in.readUTF();
- this.value = Bytes.toBytes(expr);
- int flags = in.readInt();
- this.pattern = Pattern.compile(expr, flags);
- final String charset = in.readUTF();
+ /**
+ * @return The comparator serialized using pb
+ */
+ public byte [] toByteArray() {
+ ComparatorProtos.RegexStringComparator.Builder builder =
+ ComparatorProtos.RegexStringComparator.newBuilder();
+ builder.setPattern(pattern.toString());
+ builder.setPatternFlags(pattern.flags());
+ builder.setCharset(charset.name());
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link RegexStringComparator} instance
+ * @return An instance of {@link RegexStringComparator} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static RegexStringComparator parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ ComparatorProtos.RegexStringComparator proto;
+ try {
+ proto = ComparatorProtos.RegexStringComparator.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+
+ RegexStringComparator comparator =
+ new RegexStringComparator(proto.getPattern(), proto.getPatternFlags());
+ final String charset = proto.getCharset();
if (charset.length() > 0) {
try {
- this.charset = Charset.forName(charset);
+ comparator.setCharset(Charset.forName(charset));
} catch (IllegalCharsetNameException e) {
LOG.error("invalid charset", e);
}
}
+ return comparator;
}
- @Override
- public void write(DataOutput out) throws IOException {
- out.writeUTF(pattern.toString());
- out.writeInt(pattern.flags());
- out.writeUTF(charset.name());
+ /**
+ * @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 RegexStringComparator)) return false;
+
+ RegexStringComparator comparator = (RegexStringComparator)other;
+ return super.areSerializedFieldsEqual(comparator)
+ && this.pattern.toString().equals(comparator.pattern.toString())
+ && this.pattern.flags() == comparator.pattern.flags()
+ && this.charset.equals(comparator.charset);
}
-
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java Fri Aug 24 23:51:51 2012
@@ -20,12 +20,18 @@
package org.apache.hadoop.hbase.filter;
+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.client.Scan;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used to filter based on the key. It takes an operator
@@ -46,13 +52,6 @@ public class RowFilter extends CompareFi
private boolean filterOutRow = false;
/**
- * Writable constructor, do not use.
- */
- public RowFilter() {
- super();
- }
-
- /**
* Constructor.
* @param rowCompareOp the compare op for row matching
* @param rowComparator the comparator for row matching
@@ -94,4 +93,53 @@ public class RowFilter extends CompareFi
WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
return new RowFilter(compareOp, comparator);
}
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.RowFilter.Builder builder =
+ FilterProtos.RowFilter.newBuilder();
+ builder.setCompareFilter(super.convert());
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link RowFilter} instance
+ * @return An instance of {@link RowFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static RowFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.RowFilter proto;
+ try {
+ proto = FilterProtos.RowFilter.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 RowFilter(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 RowFilter)) return false;
+
+ return super.areSerializedFieldsEqual(o);
+ }
}
\ No newline at end of file
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java Fri Aug 24 23:51:51 2012
@@ -22,9 +22,15 @@ 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.filter.CompareFilter.CompareOp;
+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;
/**
@@ -38,13 +44,6 @@ import java.util.ArrayList;
public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
/**
- * Writable constructor, do not use.
- */
- public SingleColumnValueExcludeFilter() {
- super();
- }
-
- /**
* Constructor for binary compare of the value of a single column. If the
* column is found and the condition passes, all columns of the row will be
* emitted; except for the tested column value. If the column is not found or
@@ -80,6 +79,24 @@ public class SingleColumnValueExcludeFil
super(family, qualifier, compareOp, comparator);
}
+ /**
+ * Constructor for protobuf deserialization only.
+ * @param family
+ * @param qualifier
+ * @param compareOp
+ * @param comparator
+ * @param foundColumn
+ * @param matchedColumn
+ * @param filterIfMissing
+ * @param latestVersionOnly
+ */
+ protected SingleColumnValueExcludeFilter(final byte[] family, final byte [] qualifier,
+ final CompareOp compareOp, WritableByteArrayComparable comparator, final boolean foundColumn,
+ final boolean matchedColumn, final boolean filterIfMissing, final boolean latestVersionOnly) {
+ super(family,qualifier,compareOp,comparator,foundColumn,
+ matchedColumn,filterIfMissing,latestVersionOnly);
+ }
+
public ReturnCode filterKeyValue(KeyValue keyValue) {
ReturnCode superRetCode = super.filterKeyValue(keyValue);
if (superRetCode == ReturnCode.INCLUDE) {
@@ -102,7 +119,61 @@ public class SingleColumnValueExcludeFil
if (filterArguments.size() == 6) {
filter.setFilterIfMissing(tempFilter.getFilterIfMissing());
filter.setLatestVersionOnly(tempFilter.getLatestVersionOnly());
-}
+ }
return filter;
}
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.SingleColumnValueExcludeFilter.Builder builder =
+ FilterProtos.SingleColumnValueExcludeFilter.newBuilder();
+ builder.setSingleColumnValueFilter(super.convert());
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link SingleColumnValueExcludeFilter} instance
+ * @return An instance of {@link SingleColumnValueExcludeFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static SingleColumnValueExcludeFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.SingleColumnValueExcludeFilter proto;
+ try {
+ proto = FilterProtos.SingleColumnValueExcludeFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+
+ FilterProtos.SingleColumnValueFilter parentProto = proto.getSingleColumnValueFilter();
+ final CompareOp compareOp =
+ CompareOp.valueOf(parentProto.getCompareOp().name());
+ final WritableByteArrayComparable comparator;
+ try {
+ comparator = ProtobufUtil.toComparator(parentProto.getComparator());
+ } catch (IOException ioe) {
+ throw new DeserializationException(ioe);
+ }
+
+ return new SingleColumnValueExcludeFilter(
+ parentProto.hasColumnFamily()?parentProto.getColumnFamily().toByteArray():null,
+ parentProto.hasColumnQualifier()?parentProto.getColumnQualifier().toByteArray():null,
+ compareOp, comparator, parentProto.getFoundColumn(),parentProto.getMatchedColumn(),
+ parentProto.getFilterIfMissing(),parentProto.getLatestVersionOnly());
+ }
+
+ /**
+ * @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 SingleColumnValueExcludeFilter)) return false;
+
+ return super.areSerializedFieldsEqual(o);
+ }
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java Fri Aug 24 23:51:51 2012
@@ -24,20 +24,22 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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.client.Scan;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-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.Arrays;
-import java.util.List;
import java.util.ArrayList;
import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
/**
* This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp}
@@ -81,12 +83,6 @@ public class SingleColumnValueFilter ext
private boolean latestVersionOnly = true;
/**
- * Writable constructor, do not use.
- */
- public SingleColumnValueFilter() {
- }
-
- /**
* Constructor for binary compare of the value of a single column. If the
* column is found and the condition passes, all columns of the row will be
* emitted. If the condition fails, the row will not be emitted.
@@ -128,6 +124,27 @@ public class SingleColumnValueFilter ext
}
/**
+ * Constructor for protobuf deserialization only.
+ * @param family
+ * @param qualifier
+ * @param compareOp
+ * @param comparator
+ * @param foundColumn
+ * @param matchedColumn
+ * @param filterIfMissing
+ * @param latestVersionOnly
+ */
+ protected SingleColumnValueFilter(final byte[] family, final byte [] qualifier,
+ final CompareOp compareOp, WritableByteArrayComparable comparator, final boolean foundColumn,
+ final boolean matchedColumn, final boolean filterIfMissing, final boolean latestVersionOnly) {
+ this(family,qualifier,compareOp,comparator);
+ this.foundColumn = foundColumn;
+ this.matchedColumn = matchedColumn;
+ this.filterIfMissing = filterIfMissing;
+ this.latestVersionOnly = latestVersionOnly;
+ }
+
+ /**
* @return operator
*/
public CompareOp getOperator() {
@@ -285,34 +302,82 @@ public class SingleColumnValueFilter ext
return filter;
}
- public void readFields(final DataInput in) throws IOException {
- this.columnFamily = Bytes.readByteArray(in);
- if(this.columnFamily.length == 0) {
- this.columnFamily = null;
+ FilterProtos.SingleColumnValueFilter convert() {
+ FilterProtos.SingleColumnValueFilter.Builder builder =
+ FilterProtos.SingleColumnValueFilter.newBuilder();
+ if (this.columnFamily != null) {
+ builder.setColumnFamily(ByteString.copyFrom(this.columnFamily));
}
- this.columnQualifier = Bytes.readByteArray(in);
- if(this.columnQualifier.length == 0) {
- this.columnQualifier = null;
+ if (this.columnQualifier != null) {
+ builder.setColumnQualifier(ByteString.copyFrom(this.columnQualifier));
}
- this.compareOp = CompareOp.valueOf(in.readUTF());
- this.comparator =
- (WritableByteArrayComparable)HbaseObjectWritable.readObject(in, null);
- this.foundColumn = in.readBoolean();
- this.matchedColumn = in.readBoolean();
- this.filterIfMissing = in.readBoolean();
- this.latestVersionOnly = in.readBoolean();
- }
-
- public void write(final DataOutput out) throws IOException {
- Bytes.writeByteArray(out, this.columnFamily);
- Bytes.writeByteArray(out, this.columnQualifier);
- out.writeUTF(compareOp.name());
- HbaseObjectWritable.writeObject(out, comparator,
- WritableByteArrayComparable.class, null);
- out.writeBoolean(foundColumn);
- out.writeBoolean(matchedColumn);
- out.writeBoolean(filterIfMissing);
- out.writeBoolean(latestVersionOnly);
+ HBaseProtos.CompareType compareOp = CompareType.valueOf(this.compareOp.name());
+ builder.setCompareOp(compareOp);
+ builder.setComparator(ProtobufUtil.toComparator(this.comparator));
+ builder.setFoundColumn(this.foundColumn);
+ builder.setMatchedColumn(this.matchedColumn);
+ builder.setFilterIfMissing(this.filterIfMissing);
+ builder.setLatestVersionOnly(this.latestVersionOnly);
+
+ return builder.build();
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ return convert().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link SingleColumnValueFilter} instance
+ * @return An instance of {@link SingleColumnValueFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static SingleColumnValueFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.SingleColumnValueFilter proto;
+ try {
+ proto = FilterProtos.SingleColumnValueFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+
+ final CompareOp compareOp =
+ CompareOp.valueOf(proto.getCompareOp().name());
+ final WritableByteArrayComparable comparator;
+ try {
+ comparator = ProtobufUtil.toComparator(proto.getComparator());
+ } catch (IOException ioe) {
+ throw new DeserializationException(ioe);
+ }
+
+ return new SingleColumnValueFilter(
+ proto.hasColumnFamily()?proto.getColumnFamily().toByteArray():null,
+ proto.hasColumnQualifier()?proto.getColumnQualifier().toByteArray():null,
+ compareOp, comparator, proto.getFoundColumn(),proto.getMatchedColumn(),
+ proto.getFilterIfMissing(),proto.getLatestVersionOnly());
+ }
+
+ /**
+ * @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 SingleColumnValueFilter)) return false;
+
+ SingleColumnValueFilter other = (SingleColumnValueFilter)o;
+ return Bytes.equals(this.getFamily(), other.getFamily())
+ && Bytes.equals(this.getQualifier(), other.getQualifier())
+ && this.compareOp.equals(other.compareOp)
+ && this.getComparator().areSerializedFieldsEqual(other.getComparator())
+ && this.foundColumn == other.foundColumn
+ && this.matchedColumn == other.matchedColumn
+ && this.getFilterIfMissing() == other.getFilterIfMissing()
+ && this.getLatestVersionOnly() == other.getLatestVersionOnly();
}
@Override
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java Fri Aug 24 23:51:51 2012
@@ -22,7 +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.KeyValue;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import com.google.protobuf.InvalidProtocolBufferException;
import java.io.DataInput;
import java.io.DataOutput;
@@ -54,10 +59,6 @@ public class SkipFilter extends FilterBa
private boolean filterRow = false;
private Filter filter;
- public SkipFilter() {
- super();
- }
-
public SkipFilter(Filter filter) {
this.filter = filter;
}
@@ -94,23 +95,48 @@ public class SkipFilter extends FilterBa
return true;
}
- public void write(DataOutput out) throws IOException {
- out.writeUTF(this.filter.getClass().getName());
- this.filter.write(out);
- }
-
- public void readFields(DataInput in) throws IOException {
- String className = in.readUTF();
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.SkipFilter.Builder builder =
+ FilterProtos.SkipFilter.newBuilder();
+ builder.setFilter(ProtobufUtil.toFilter(this.filter));
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link SkipFilter} instance
+ * @return An instance of {@link SkipFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static SkipFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.SkipFilter proto;
try {
- this.filter = (Filter)(Class.forName(className).newInstance());
- this.filter.readFields(in);
- } catch (InstantiationException e) {
- throw new RuntimeException("Failed deserialize.", e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Failed deserialize.", e);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Failed deserialize.", e);
+ proto = FilterProtos.SkipFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
}
+ try {
+ return new SkipFilter(ProtobufUtil.toFilter(proto.getFilter()));
+ } catch (IOException ioe) {
+ throw new DeserializationException(ioe);
+ }
+ }
+
+ /**
+ * @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 SkipFilter)) return false;
+
+ SkipFilter other = (SkipFilter)o;
+ return getFilter().areSerializedFieldsEqual(other.getFilter());
}
@Override
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SubstringComparator.java Fri Aug 24 23:51:51 2012
@@ -21,11 +21,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 java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
+import com.google.protobuf.InvalidProtocolBufferException;
+
/**
* This comparator is for use with SingleColumnValueFilter, for filtering based on
@@ -48,11 +49,6 @@ public class SubstringComparator extends
private String substr;
- /** Nullary constructor for Writable, do not use */
- public SubstringComparator() {
- super();
- }
-
/**
* Constructor
* @param substr the substring
@@ -73,16 +69,45 @@ public class SubstringComparator extends
: 1;
}
- @Override
- public void readFields(DataInput in) throws IOException {
- String substr = in.readUTF();
- this.value = Bytes.toBytes(substr);
- this.substr = substr;
+ /**
+ * @return The comparator serialized using pb
+ */
+ public byte [] toByteArray() {
+ ComparatorProtos.SubstringComparator.Builder builder =
+ ComparatorProtos.SubstringComparator.newBuilder();
+ builder.setSubstr(this.substr);
+ return builder.build().toByteArray();
}
- @Override
- public void write(DataOutput out) throws IOException {
- out.writeUTF(substr);
+ /**
+ * @param pbBytes A pb serialized {@link SubstringComparator} instance
+ * @return An instance of {@link SubstringComparator} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static SubstringComparator parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ ComparatorProtos.SubstringComparator proto;
+ try {
+ proto = ComparatorProtos.SubstringComparator.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new SubstringComparator(proto.getSubstr());
+ }
+
+ /**
+ * @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 SubstringComparator)) return false;
+
+ SubstringComparator comparator = (SubstringComparator)other;
+ return super.areSerializedFieldsEqual(comparator)
+ && this.substr.equals(comparator.substr);
}
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java Fri Aug 24 23:51:51 2012
@@ -17,18 +17,18 @@
*/
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.List;
import java.util.TreeSet;
-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;
/**
* Filter that returns only cells whose timestamp (version) is
@@ -51,13 +51,6 @@ public class TimestampsFilter extends Fi
long minTimeStamp = Long.MAX_VALUE;
/**
- * Used during deserialization. Do not use otherwise.
- */
- public TimestampsFilter() {
- super();
- }
-
- /**
* Constructor for filter that retains only those
* cells whose timestamp (version) is in the specified
* list of timestamps.
@@ -116,23 +109,44 @@ public class TimestampsFilter extends Fi
return new TimestampsFilter(timestamps);
}
- @Override
- public void readFields(DataInput in) throws IOException {
- int numTimestamps = in.readInt();
- this.timestamps = new TreeSet<Long>();
- for (int idx = 0; idx < numTimestamps; idx++) {
- this.timestamps.add(in.readLong());
- }
- init();
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.TimestampsFilter.Builder builder =
+ FilterProtos.TimestampsFilter.newBuilder();
+ builder.addAllTimestamps(this.timestamps);
+ return builder.build().toByteArray();
}
- @Override
- public void write(DataOutput out) throws IOException {
- int numTimestamps = this.timestamps.size();
- out.writeInt(numTimestamps);
- for (Long timestamp : this.timestamps) {
- out.writeLong(timestamp);
+ /**
+ * @param pbBytes A pb serialized {@link TimestampsFilter} instance
+ * @return An instance of {@link TimestampsFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static TimestampsFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.TimestampsFilter proto;
+ try {
+ proto = FilterProtos.TimestampsFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
}
+ return new TimestampsFilter(proto.getTimestampsList());
+ }
+
+ /**
+ * @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 TimestampsFilter)) return false;
+
+ TimestampsFilter other = (TimestampsFilter)o;
+ return this.getTimestamps().equals(other.getTimestamps());
}
@Override
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.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;
/**
@@ -44,12 +50,6 @@ import java.util.ArrayList;
public class ValueFilter extends CompareFilter {
/**
- * Writable constructor, do not use.
- */
- public ValueFilter() {
- }
-
- /**
* Constructor.
* @param valueCompareOp the compare op for value matching
* @param valueComparator the comparator for value matching
@@ -73,5 +73,54 @@ public class ValueFilter extends Compare
CompareOp compareOp = (CompareOp)arguments.get(0);
WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
return new ValueFilter(compareOp, comparator);
-}
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.ValueFilter.Builder builder =
+ FilterProtos.ValueFilter.newBuilder();
+ builder.setCompareFilter(super.convert());
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link ValueFilter} instance
+ * @return An instance of {@link ValueFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static ValueFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.ValueFilter proto;
+ try {
+ proto = FilterProtos.ValueFilter.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 ValueFilter(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 ValueFilter)) return false;
+
+ return super.areSerializedFieldsEqual(o);
+ }
}
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java Fri Aug 24 23:51:51 2012
@@ -22,7 +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.KeyValue;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import com.google.protobuf.InvalidProtocolBufferException;
import java.io.DataInput;
import java.io.DataOutput;
@@ -43,10 +48,6 @@ public class WhileMatchFilter extends Fi
private boolean filterAllRemaining = false;
private Filter filter;
- public WhileMatchFilter() {
- super();
- }
-
public WhileMatchFilter(Filter filter) {
this.filter = filter;
}
@@ -94,23 +95,48 @@ public class WhileMatchFilter extends Fi
return true;
}
- public void write(DataOutput out) throws IOException {
- out.writeUTF(this.filter.getClass().getName());
- this.filter.write(out);
- }
-
- public void readFields(DataInput in) throws IOException {
- String className = in.readUTF();
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.WhileMatchFilter.Builder builder =
+ FilterProtos.WhileMatchFilter.newBuilder();
+ builder.setFilter(ProtobufUtil.toFilter(this.filter));
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link WhileMatchFilter} instance
+ * @return An instance of {@link WhileMatchFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static WhileMatchFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.WhileMatchFilter proto;
try {
- this.filter = (Filter)(Class.forName(className).newInstance());
- this.filter.readFields(in);
- } catch (InstantiationException e) {
- throw new RuntimeException("Failed deserialize.", e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Failed deserialize.", e);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Failed deserialize.", e);
+ proto = FilterProtos.WhileMatchFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
}
+ try {
+ return new WhileMatchFilter(ProtobufUtil.toFilter(proto.getFilter()));
+ } catch (IOException ioe) {
+ throw new DeserializationException(ioe);
+ }
+ }
+
+ /**
+ * @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 WhileMatchFilter)) return false;
+
+ WhileMatchFilter other = (WhileMatchFilter)o;
+ return getFilter().areSerializedFieldsEqual(other.getFilter());
}
@Override
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java Fri Aug 24 23:51:51 2012
@@ -21,26 +21,21 @@ 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 org.apache.hadoop.io.Writable;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
+import com.google.protobuf.ByteString;
-/** Base class, combines Comparable<byte []> and Writable. */
+
+/** Base class for byte array comparators */
@InterfaceAudience.Public
@InterfaceStability.Stable
-public abstract class WritableByteArrayComparable implements Writable, Comparable<byte[]> {
+public abstract class WritableByteArrayComparable implements Comparable<byte[]> {
byte[] value;
/**
- * Nullary constructor, for Writable
- */
- public WritableByteArrayComparable() { }
-
- /**
* Constructor.
* @param value the value to compare against
*/
@@ -52,14 +47,40 @@ public abstract class WritableByteArrayC
return value;
}
- @Override
- public void readFields(DataInput in) throws IOException {
- value = Bytes.readByteArray(in);
+ /**
+ * @return The comparator serialized using pb
+ */
+ public abstract byte [] toByteArray();
+
+ ComparatorProtos.ByteArrayComparable convert() {
+ ComparatorProtos.ByteArrayComparable.Builder builder =
+ ComparatorProtos.ByteArrayComparable.newBuilder();
+ if (value != null) builder.setValue(ByteString.copyFrom(value));
+ return builder.build();
}
- @Override
- public void write(DataOutput out) throws IOException {
- Bytes.writeByteArray(out, value);
+ /**
+ * @param pbBytes A pb serialized {@link WritableByteArrayComparable} instance
+ * @return An instance of {@link WritableByteArrayComparable} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see {@link #toByteArray()}
+ */
+ public static WritableByteArrayComparable parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ throw new DeserializationException(
+ "parseFrom called on base WritableByteArrayComparable, but should be called on derived type");
+ }
+
+ /**
+ * @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 o) {
+ if (o == this) return true;
+ if (!(o instanceof WritableByteArrayComparable)) return false;
+
+ return Bytes.equals(this.getValue(), o.getValue());
}
@Override
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java Fri Aug 24 23:51:51 2012
@@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.HBaseConf
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes;
@@ -55,6 +57,8 @@ import org.apache.hadoop.mapreduce.Input
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.util.StringUtils;
+import com.google.protobuf.InvalidProtocolBufferException;
+
/**
* Utility for {@link TableMapper} and {@link TableReducer}
*/
@@ -239,10 +243,8 @@ public class TableMapReduceUtil {
* @throws IOException When writing the scan fails.
*/
static String convertScanToString(Scan scan) throws IOException {
- ByteArrayOutputStream out = new ByteArrayOutputStream();
- DataOutputStream dos = new DataOutputStream(out);
- scan.write(dos);
- return Base64.encodeBytes(out.toByteArray());
+ ClientProtos.Scan proto = ProtobufUtil.toScan(scan);
+ return Base64.encodeBytes(proto.toByteArray());
}
/**
@@ -253,11 +255,15 @@ public class TableMapReduceUtil {
* @throws IOException When reading the scan instance fails.
*/
static Scan convertStringToScan(String base64) throws IOException {
- ByteArrayInputStream bis = new ByteArrayInputStream(Base64.decode(base64));
- DataInputStream dis = new DataInputStream(bis);
- Scan scan = new Scan();
- scan.readFields(dis);
- return scan;
+ byte [] decoded = Base64.decode(base64);
+ ClientProtos.Scan scan;
+ try {
+ scan = ClientProtos.Scan.parseFrom(decoded);
+ } catch (InvalidProtocolBufferException ipbe) {
+ throw new IOException(ipbe);
+ }
+
+ return ProtobufUtil.toScan(scan);
}
/**
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Fri Aug 24 23:51:51 2012
@@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.client.Sc
import org.apache.hadoop.hbase.client.coprocessor.Exec;
import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
@@ -101,6 +102,8 @@ import org.apache.hadoop.hbase.protobuf.
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType;
+import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
@@ -371,8 +374,8 @@ public final class ProtobufUtil {
get.setTimeRange(minStamp, maxStamp);
}
if (proto.hasFilter()) {
- NameBytesPair filter = proto.getFilter();
- get.setFilter((Filter)toObject(filter));
+ HBaseProtos.Filter filter = proto.getFilter();
+ get.setFilter(ProtobufUtil.toFilter(filter));
}
for (NameBytesPair attribute: proto.getAttributeList()) {
get.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
@@ -608,7 +611,7 @@ public final class ProtobufUtil {
scanBuilder.setStopRow(ByteString.copyFrom(stopRow));
}
if (scan.hasFilter()) {
- scanBuilder.setFilter(ProtobufUtil.toParameter(scan.getFilter()));
+ scanBuilder.setFilter(ProtobufUtil.toFilter(scan.getFilter()));
}
Column.Builder columnBuilder = Column.newBuilder();
for (Map.Entry<byte[],NavigableSet<byte []>>
@@ -677,8 +680,8 @@ public final class ProtobufUtil {
scan.setTimeRange(minStamp, maxStamp);
}
if (proto.hasFilter()) {
- NameBytesPair filter = proto.getFilter();
- scan.setFilter((Filter)toObject(filter));
+ HBaseProtos.Filter filter = proto.getFilter();
+ scan.setFilter(ProtobufUtil.toFilter(filter));
}
if (proto.hasBatchSize()) {
scan.setBatch(proto.getBatchSize());
@@ -759,7 +762,7 @@ public final class ProtobufUtil {
builder.setLockId(get.getLockId());
}
if (get.getFilter() != null) {
- builder.setFilter(ProtobufUtil.toParameter(get.getFilter()));
+ builder.setFilter(ProtobufUtil.toFilter(get.getFilter()));
}
TimeRange timeRange = get.getTimeRange();
if (!timeRange.isAllTime()) {
@@ -931,6 +934,79 @@ public final class ProtobufUtil {
}
/**
+ * Convert a WritableByteArrayComparable to a protocol buffer Comparator
+ *
+ * @param comparator the WritableByteArrayComparable to convert
+ * @return the converted protocol buffer Comparator
+ */
+ public static ComparatorProtos.Comparator toComparator(WritableByteArrayComparable comparator) {
+ ComparatorProtos.Comparator.Builder builder = ComparatorProtos.Comparator.newBuilder();
+ builder.setName(comparator.getClass().getName());
+ builder.setSerializedComparator(ByteString.copyFrom(comparator.toByteArray()));
+ return builder.build();
+ }
+
+ /**
+ * Convert a protocol buffer Comparator to a WritableByteArrayComparable
+ *
+ * @param proto the protocol buffer Comparator to convert
+ * @return the converted WritableByteArrayComparable
+ */
+ public static WritableByteArrayComparable toComparator(ComparatorProtos.Comparator proto)
+ throws IOException {
+ String type = proto.getName();
+ String funcName = "parseFrom";
+ byte [] value = proto.getSerializedComparator().toByteArray();
+ try {
+ Class<? extends WritableByteArrayComparable> c =
+ (Class<? extends WritableByteArrayComparable>)(Class.forName(type));
+ Method parseFrom = c.getMethod(funcName, byte[].class);
+ if (parseFrom == null) {
+ throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
+ }
+ return (WritableByteArrayComparable)parseFrom.invoke(null, value);
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ }
+
+ /**
+ * Convert a protocol buffer Filter to a client Filter
+ *
+ * @param proto the protocol buffer Filter to convert
+ * @return the converted Filter
+ */
+ public static Filter toFilter(HBaseProtos.Filter proto) throws IOException {
+ String type = proto.getName();
+ final byte [] value = proto.getSerializedFilter().toByteArray();
+ String funcName = "parseFrom";
+ try {
+ Class<? extends Filter> c =
+ (Class<? extends Filter>)Class.forName(type);
+ Method parseFrom = c.getMethod(funcName, byte[].class);
+ if (parseFrom == null) {
+ throw new IOException("Unable to locate function: " + funcName + " in type: " + type);
+ }
+ return (Filter)parseFrom.invoke(c, value);
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ }
+
+ /**
+ * Convert a client Filter to a protocol buffer Filter
+ *
+ * @param filter the Filter to convert
+ * @return the converted protocol buffer Filter
+ */
+ public static HBaseProtos.Filter toFilter(Filter filter) {
+ HBaseProtos.Filter.Builder builder = HBaseProtos.Filter.newBuilder();
+ builder.setName(filter.getClass().getName());
+ builder.setSerializedFilter(ByteString.copyFrom(filter.toByteArray()));
+ return builder.build();
+ }
+
+ /**
* Get the HLog entries from a list of protocol buffer WALEntry
*
* @param protoList the list of protocol buffer WALEntry
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java?rev=1377154&r1=1377153&r2=1377154&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java Fri Aug 24 23:51:51 2012
@@ -868,7 +868,7 @@ public final class RequestConverter {
builder.setRow(ByteString.copyFrom(row));
builder.setFamily(ByteString.copyFrom(family));
builder.setQualifier(ByteString.copyFrom(qualifier));
- builder.setComparator(ProtobufUtil.toParameter(comparator));
+ builder.setComparator(ProtobufUtil.toComparator(comparator));
builder.setCompareType(compareType);
return builder.build();
}