HBASE-6477 Use PB filter definitions in RPC

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1377154 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-08-24 23:51:51 +00:00
parent ea41f302f1
commit b4be2041d4
69 changed files with 8821 additions and 1373 deletions

View File

@ -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.InterfaceAudience;
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);
}
}

View File

@ -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 OperationWithAttributes
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 OperationWithAttributes
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);
}
}
}

View File

@ -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.
*/

View File

@ -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;
/**
@ -87,30 +84,6 @@ public class RowMutations implements Row {
mutations.add(m);
}
@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());

View File

@ -31,8 +31,6 @@ 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;
@ -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 OperationWithAttributes implements Writable {
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

View File

@ -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.Bytes;
@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 WritableByteArrayComparable {
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);
}
}

View File

@ -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.Bytes;
@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 extends WritableByteArrayComparable {
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);
}
}

View File

@ -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;
@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 WritableByteArrayComparable {
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

View File

@ -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 extends FilterBase {
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 extends FilterBase {
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

View File

@ -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 extends FilterBase
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 extends FilterBase
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

View File

@ -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.Preconditions;
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 FilterBase {
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();
}
public void readFields(DataInput in) throws IOException {
this.prefix = Bytes.readByteArray(in);
/**
* @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());
}
/**
* @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) {

View File

@ -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 FilterBase {
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 FilterBase {
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;
/**
* @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);
}
this.minColumnInclusive = in.readBoolean();
boolean isMaxColumnNull = in.readBoolean();
this.maxColumn = Bytes.readByteArray(in);
if (isMaxColumnNull) {
this.maxColumn = null;
}
this.maxColumnInclusive = in.readBoolean();
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) {

View File

@ -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;
@ -71,12 +71,6 @@ public abstract class CompareFilter extends FilterBase {
protected CompareOp compareOp;
protected WritableByteArrayComparable comparator;
/**
* Writable constructor, do not use.
*/
public CompareFilter() {
}
/**
* Constructor.
* @param compareOp the compare op for row matching
@ -149,16 +143,32 @@ public abstract class CompareFilter extends FilterBase {
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

View File

@ -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
@ -52,12 +55,6 @@ public class DependentColumnFilter extends CompareFilter {
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
@ -217,28 +214,67 @@ public class DependentColumnFilter extends CompareFilter {
}
}
@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

View File

@ -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 CompareFilter {
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);
}
}

View File

@ -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);
}

View File

@ -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 implements Filter {
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;
}
}

View File

@ -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 */
@ -59,14 +61,6 @@ public class FilterList implements Filter {
private Operator operator = Operator.MUST_PASS_ALL;
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 Filter {
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

View File

@ -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 Filter {
this.filter = filter;
}
@Override
public void write(DataOutput out) throws IOException {
this.filter.write(out);
/**
* @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();
}
@Override
public void readFields(DataInput in) throws IOException {
this.filter.readFields(in);
/**
* @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 Filter {
}
}
/**
* @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);
}
}

View File

@ -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 FilterBase {
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;
}
}

View File

@ -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
@ -42,13 +49,6 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
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 FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
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);
}
}

View File

@ -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 FilterBase {
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 FilterBase {
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 FilterBase {
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;
}
}

View File

@ -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 FilterBase {
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 FilterBase {
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();
}
public void readFields(DataInput in) throws IOException {
this.stopRowKey = Bytes.readByteArray(in);
/**
* @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);
}
/**
* @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

View File

@ -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 FilterBase {
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;
}
}

View File

@ -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 extends FilterBase {
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 extends FilterBase {
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) {

View File

@ -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;
@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 WritableByteArrayComparable {
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);
}
}

View File

@ -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 >
@ -47,14 +46,6 @@ public class PageFilter extends FilterBase {
private long pageSize = Long.MAX_VALUE;
private int rowsAccepted = 0;
/**
* Default constructor, filters nothing. Required though for RPC
* deserialization.
*/
public PageFilter() {
super();
}
/**
* Constructor that takes a maximum page size.
*
@ -89,12 +80,44 @@ public class PageFilter extends FilterBase {
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

View File

@ -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 FilterBase {
this.prefix = prefix;
}
public PrefixFilter() {
super();
}
public byte[] getPrefix() {
return prefix;
}
@ -81,12 +77,44 @@ public class PrefixFilter extends FilterBase {
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();
}
public void readFields(DataInput in) throws IOException {
this.prefix = Bytes.readByteArray(in);
/**
* @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);
}
/**
* @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

View File

@ -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;
/**
@ -44,12 +50,6 @@ import java.util.ArrayList;
@InterfaceStability.Stable
public class QualifierFilter extends CompareFilter {
/**
* Writable constructor, do not use.
*/
public QualifierFilter() {
}
/**
* Constructor.
* @param op the compare op for column qualifier matching
@ -77,5 +77,54 @@ public class QualifierFilter extends CompareFilter {
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);
}
}

View File

@ -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.
@ -41,12 +42,6 @@ public class RandomRowFilter extends FilterBase {
protected float chance;
protected boolean filterOutRow;
/**
* Writable constructor, do not use.
*/
public RandomRowFilter() {
}
/**
* Create a new filter with a specified chance for a row to be included.
*
@ -114,13 +109,43 @@ public class RandomRowFilter extends FilterBase {
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();
}
}

View File

@ -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 extends WritableByteArrayComparable {
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 extends WritableByteArrayComparable {
: 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);
}
}

View File

@ -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
@ -45,13 +51,6 @@ public class RowFilter extends CompareFilter {
private boolean filterOutRow = false;
/**
* Writable constructor, do not use.
*/
public RowFilter() {
super();
}
/**
* Constructor.
* @param rowCompareOp the compare op for row matching
@ -94,4 +93,53 @@ public class RowFilter extends CompareFilter {
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);
}
}

View File

@ -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;
/**
@ -37,13 +43,6 @@ import java.util.ArrayList;
@InterfaceStability.Stable
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
@ -80,6 +79,24 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
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 SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
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);
}
}

View File

@ -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}
@ -80,12 +82,6 @@ public class SingleColumnValueFilter extends FilterBase {
private boolean filterIfMissing = false;
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
@ -127,6 +123,27 @@ public class SingleColumnValueFilter extends FilterBase {
this.comparator = comparator;
}
/**
* 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
*/
@ -285,34 +302,82 @@ public class SingleColumnValueFilter extends FilterBase {
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();
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();
}
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);
/**
* @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

View File

@ -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 FilterBase {
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 FilterBase {
return true;
}
public void write(DataOutput out) throws IOException {
out.writeUTF(this.filter.getClass().getName());
this.filter.write(out);
/**
* @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();
}
public void readFields(DataInput in) throws IOException {
String className = in.readUTF();
/**
* @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

View File

@ -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 WritableByteArrayComparable {
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 WritableByteArrayComparable {
: 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);
}
}

View File

@ -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
@ -50,13 +50,6 @@ public class TimestampsFilter extends FilterBase {
// once the timestamps fall below the minTimeStamp.
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
@ -116,23 +109,44 @@ public class TimestampsFilter extends FilterBase {
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

View File

@ -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;
/**
@ -43,12 +49,6 @@ import java.util.ArrayList;
@InterfaceStability.Stable
public class ValueFilter extends CompareFilter {
/**
* Writable constructor, do not use.
*/
public ValueFilter() {
}
/**
* Constructor.
* @param valueCompareOp the compare op for value matching
@ -73,5 +73,54 @@ public class ValueFilter extends CompareFilter {
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);
}
}

View File

@ -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 FilterBase {
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 FilterBase {
return true;
}
public void write(DataOutput out) throws IOException {
out.writeUTF(this.filter.getClass().getName());
this.filter.write(out);
/**
* @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();
}
public void readFields(DataInput in) throws IOException {
String className = in.readUTF();
/**
* @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

View File

@ -21,25 +21,20 @@ 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 WritableByteArrayComparable implements Writable, Comparabl
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

View File

@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
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.InputFormat;
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);
}
/**

View File

@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.client.Scan;
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.generated.ClientProtos.Mutate.ColumnValu
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()) {
@ -930,6 +933,79 @@ public final class ProtobufUtil {
return new Result(keyValues);
}
/**
* 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
*

View File

@ -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();
}

View File

@ -532,10 +532,10 @@ public final class ClientProtos {
boolean hasLockId();
long getLockId();
// optional .NameBytesPair filter = 5;
// optional .Filter filter = 5;
boolean hasFilter();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getFilter();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getFilterOrBuilder();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter getFilter();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder getFilterOrBuilder();
// optional .TimeRange timeRange = 6;
boolean hasTimeRange();
@ -649,16 +649,16 @@ public final class ClientProtos {
return lockId_;
}
// optional .NameBytesPair filter = 5;
// optional .Filter filter = 5;
public static final int FILTER_FIELD_NUMBER = 5;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair filter_;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter filter_;
public boolean hasFilter() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getFilter() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter getFilter() {
return filter_;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getFilterOrBuilder() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder getFilterOrBuilder() {
return filter_;
}
@ -720,7 +720,7 @@ public final class ClientProtos {
column_ = java.util.Collections.emptyList();
attribute_ = java.util.Collections.emptyList();
lockId_ = 0L;
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance();
timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance();
maxVersions_ = 1;
cacheBlocks_ = true;
@ -1093,7 +1093,7 @@ public final class ClientProtos {
lockId_ = 0L;
bitField0_ = (bitField0_ & ~0x00000008);
if (filterBuilder_ == null) {
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance();
} else {
filterBuilder_.clear();
}
@ -1376,7 +1376,7 @@ public final class ClientProtos {
break;
}
case 42: {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.newBuilder();
if (hasFilter()) {
subBuilder.mergeFrom(getFilter());
}
@ -1836,21 +1836,21 @@ public final class ClientProtos {
return this;
}
// optional .NameBytesPair filter = 5;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
// optional .Filter filter = 5;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance();
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> filterBuilder_;
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder> filterBuilder_;
public boolean hasFilter() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getFilter() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter getFilter() {
if (filterBuilder_ == null) {
return filter_;
} else {
return filterBuilder_.getMessage();
}
}
public Builder setFilter(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) {
public Builder setFilter(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter value) {
if (filterBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
@ -1864,7 +1864,7 @@ public final class ClientProtos {
return this;
}
public Builder setFilter(
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder builderForValue) {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder builderForValue) {
if (filterBuilder_ == null) {
filter_ = builderForValue.build();
onChanged();
@ -1874,12 +1874,12 @@ public final class ClientProtos {
bitField0_ |= 0x00000010;
return this;
}
public Builder mergeFilter(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) {
public Builder mergeFilter(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter value) {
if (filterBuilder_ == null) {
if (((bitField0_ & 0x00000010) == 0x00000010) &&
filter_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance()) {
filter_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance()) {
filter_ =
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder(filter_).mergeFrom(value).buildPartial();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.newBuilder(filter_).mergeFrom(value).buildPartial();
} else {
filter_ = value;
}
@ -1892,7 +1892,7 @@ public final class ClientProtos {
}
public Builder clearFilter() {
if (filterBuilder_ == null) {
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance();
onChanged();
} else {
filterBuilder_.clear();
@ -1900,12 +1900,12 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000010);
return this;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getFilterBuilder() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder getFilterBuilder() {
bitField0_ |= 0x00000010;
onChanged();
return getFilterFieldBuilder().getBuilder();
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getFilterOrBuilder() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder getFilterOrBuilder() {
if (filterBuilder_ != null) {
return filterBuilder_.getMessageOrBuilder();
} else {
@ -1913,11 +1913,11 @@ public final class ClientProtos {
}
}
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder>
getFilterFieldBuilder() {
if (filterBuilder_ == null) {
filterBuilder_ = new com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder>(
filter_,
getParentForChildren(),
isClean());
@ -3839,10 +3839,10 @@ public final class ClientProtos {
boolean hasCompareType();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType getCompareType();
// required .NameBytesPair comparator = 5;
// required .Comparator comparator = 5;
boolean hasComparator();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getComparator();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getComparatorOrBuilder();
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator getComparator();
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ComparatorOrBuilder getComparatorOrBuilder();
}
public static final class Condition extends
com.google.protobuf.GeneratedMessage
@ -3913,16 +3913,16 @@ public final class ClientProtos {
return compareType_;
}
// required .NameBytesPair comparator = 5;
// required .Comparator comparator = 5;
public static final int COMPARATOR_FIELD_NUMBER = 5;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair comparator_;
private org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator comparator_;
public boolean hasComparator() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getComparator() {
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator getComparator() {
return comparator_;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getComparatorOrBuilder() {
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ComparatorOrBuilder getComparatorOrBuilder() {
return comparator_;
}
@ -3931,7 +3931,7 @@ public final class ClientProtos {
family_ = com.google.protobuf.ByteString.EMPTY;
qualifier_ = com.google.protobuf.ByteString.EMPTY;
compareType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType.LESS;
comparator_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
comparator_ = org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.getDefaultInstance();
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@ -4216,7 +4216,7 @@ public final class ClientProtos {
compareType_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType.LESS;
bitField0_ = (bitField0_ & ~0x00000008);
if (comparatorBuilder_ == null) {
comparator_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
comparator_ = org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.getDefaultInstance();
} else {
comparatorBuilder_.clear();
}
@ -4396,7 +4396,7 @@ public final class ClientProtos {
break;
}
case 42: {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder();
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.newBuilder();
if (hasComparator()) {
subBuilder.mergeFrom(getComparator());
}
@ -4506,21 +4506,21 @@ public final class ClientProtos {
return this;
}
// required .NameBytesPair comparator = 5;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair comparator_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
// required .Comparator comparator = 5;
private org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator comparator_ = org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.getDefaultInstance();
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> comparatorBuilder_;
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ComparatorOrBuilder> comparatorBuilder_;
public boolean hasComparator() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getComparator() {
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator getComparator() {
if (comparatorBuilder_ == null) {
return comparator_;
} else {
return comparatorBuilder_.getMessage();
}
}
public Builder setComparator(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) {
public Builder setComparator(org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator value) {
if (comparatorBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
@ -4534,7 +4534,7 @@ public final class ClientProtos {
return this;
}
public Builder setComparator(
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder builderForValue) {
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.Builder builderForValue) {
if (comparatorBuilder_ == null) {
comparator_ = builderForValue.build();
onChanged();
@ -4544,12 +4544,12 @@ public final class ClientProtos {
bitField0_ |= 0x00000010;
return this;
}
public Builder mergeComparator(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) {
public Builder mergeComparator(org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator value) {
if (comparatorBuilder_ == null) {
if (((bitField0_ & 0x00000010) == 0x00000010) &&
comparator_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance()) {
comparator_ != org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.getDefaultInstance()) {
comparator_ =
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder(comparator_).mergeFrom(value).buildPartial();
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.newBuilder(comparator_).mergeFrom(value).buildPartial();
} else {
comparator_ = value;
}
@ -4562,7 +4562,7 @@ public final class ClientProtos {
}
public Builder clearComparator() {
if (comparatorBuilder_ == null) {
comparator_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
comparator_ = org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.getDefaultInstance();
onChanged();
} else {
comparatorBuilder_.clear();
@ -4570,12 +4570,12 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000010);
return this;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getComparatorBuilder() {
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.Builder getComparatorBuilder() {
bitField0_ |= 0x00000010;
onChanged();
return getComparatorFieldBuilder().getBuilder();
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getComparatorOrBuilder() {
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ComparatorOrBuilder getComparatorOrBuilder() {
if (comparatorBuilder_ != null) {
return comparatorBuilder_.getMessageOrBuilder();
} else {
@ -4583,11 +4583,11 @@ public final class ClientProtos {
}
}
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ComparatorOrBuilder>
getComparatorFieldBuilder() {
if (comparatorBuilder_ == null) {
comparatorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.Comparator.Builder, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ComparatorOrBuilder>(
comparator_,
getParentForChildren(),
isClean());
@ -8755,10 +8755,10 @@ public final class ClientProtos {
boolean hasStopRow();
com.google.protobuf.ByteString getStopRow();
// optional .NameBytesPair filter = 5;
// optional .Filter filter = 5;
boolean hasFilter();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getFilter();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getFilterOrBuilder();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter getFilter();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder getFilterOrBuilder();
// optional .TimeRange timeRange = 6;
boolean hasTimeRange();
@ -8880,16 +8880,16 @@ public final class ClientProtos {
return stopRow_;
}
// optional .NameBytesPair filter = 5;
// optional .Filter filter = 5;
public static final int FILTER_FIELD_NUMBER = 5;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair filter_;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter filter_;
public boolean hasFilter() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getFilter() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter getFilter() {
return filter_;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getFilterOrBuilder() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder getFilterOrBuilder() {
return filter_;
}
@ -8971,7 +8971,7 @@ public final class ClientProtos {
attribute_ = java.util.Collections.emptyList();
startRow_ = com.google.protobuf.ByteString.EMPTY;
stopRow_ = com.google.protobuf.ByteString.EMPTY;
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance();
timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance();
maxVersions_ = 1;
cacheBlocks_ = true;
@ -9374,7 +9374,7 @@ public final class ClientProtos {
stopRow_ = com.google.protobuf.ByteString.EMPTY;
bitField0_ = (bitField0_ & ~0x00000008);
if (filterBuilder_ == null) {
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance();
} else {
filterBuilder_.clear();
}
@ -9671,7 +9671,7 @@ public final class ClientProtos {
break;
}
case 42: {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.newBuilder();
if (hasFilter()) {
subBuilder.mergeFrom(getFilter());
}
@ -10144,21 +10144,21 @@ public final class ClientProtos {
return this;
}
// optional .NameBytesPair filter = 5;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
// optional .Filter filter = 5;
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance();
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> filterBuilder_;
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder> filterBuilder_;
public boolean hasFilter() {
return ((bitField0_ & 0x00000010) == 0x00000010);
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getFilter() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter getFilter() {
if (filterBuilder_ == null) {
return filter_;
} else {
return filterBuilder_.getMessage();
}
}
public Builder setFilter(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) {
public Builder setFilter(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter value) {
if (filterBuilder_ == null) {
if (value == null) {
throw new NullPointerException();
@ -10172,7 +10172,7 @@ public final class ClientProtos {
return this;
}
public Builder setFilter(
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder builderForValue) {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder builderForValue) {
if (filterBuilder_ == null) {
filter_ = builderForValue.build();
onChanged();
@ -10182,12 +10182,12 @@ public final class ClientProtos {
bitField0_ |= 0x00000010;
return this;
}
public Builder mergeFilter(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) {
public Builder mergeFilter(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter value) {
if (filterBuilder_ == null) {
if (((bitField0_ & 0x00000010) == 0x00000010) &&
filter_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance()) {
filter_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance()) {
filter_ =
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder(filter_).mergeFrom(value).buildPartial();
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.newBuilder(filter_).mergeFrom(value).buildPartial();
} else {
filter_ = value;
}
@ -10200,7 +10200,7 @@ public final class ClientProtos {
}
public Builder clearFilter() {
if (filterBuilder_ == null) {
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance();
filter_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.getDefaultInstance();
onChanged();
} else {
filterBuilder_.clear();
@ -10208,12 +10208,12 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000010);
return this;
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getFilterBuilder() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder getFilterBuilder() {
bitField0_ |= 0x00000010;
onChanged();
return getFilterFieldBuilder().getBuilder();
}
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getFilterOrBuilder() {
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder getFilterOrBuilder() {
if (filterBuilder_ != null) {
return filterBuilder_.getMessageOrBuilder();
} else {
@ -10221,11 +10221,11 @@ public final class ClientProtos {
}
}
private com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder>
getFilterFieldBuilder() {
if (filterBuilder_ == null) {
filterBuilder_ = new com.google.protobuf.SingleFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>(
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Filter.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.FilterOrBuilder>(
filter_,
getParentForChildren(),
isClean());
@ -21564,45 +21564,45 @@ public final class ClientProtos {
descriptor;
static {
java.lang.String[] descriptorData = {
"\n\014Client.proto\032\013hbase.proto\"+\n\006Column\022\016\n" +
"\006family\030\001 \002(\014\022\021\n\tqualifier\030\002 \003(\014\"\371\001\n\003Get" +
"\022\013\n\003row\030\001 \002(\014\022\027\n\006column\030\002 \003(\0132\007.Column\022!" +
"\n\tattribute\030\003 \003(\0132\016.NameBytesPair\022\016\n\006loc" +
"kId\030\004 \001(\004\022\036\n\006filter\030\005 \001(\0132\016.NameBytesPai" +
"r\022\035\n\ttimeRange\030\006 \001(\0132\n.TimeRange\022\026\n\013maxV" +
"ersions\030\007 \001(\r:\0011\022\031\n\013cacheBlocks\030\010 \001(\010:\004t" +
"rue\022\022\n\nstoreLimit\030\t \001(\r\022\023\n\013storeOffset\030\n" +
" \001(\r\"\037\n\006Result\022\025\n\rkeyValueBytes\030\001 \003(\014\"r\n" +
"\nGetRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpec",
"ifier\022\021\n\003get\030\002 \002(\0132\004.Get\022\030\n\020closestRowBe" +
"fore\030\003 \001(\010\022\025\n\rexistenceOnly\030\004 \001(\010\"6\n\013Get" +
"Response\022\027\n\006result\030\001 \001(\0132\007.Result\022\016\n\006exi" +
"sts\030\002 \001(\010\"\202\001\n\tCondition\022\013\n\003row\030\001 \002(\014\022\016\n\006" +
"family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022!\n\013compa" +
"reType\030\004 \002(\0162\014.CompareType\022\"\n\ncomparator" +
"\030\005 \002(\0132\016.NameBytesPair\"\306\004\n\006Mutate\022\013\n\003row" +
"\030\001 \002(\014\022&\n\nmutateType\030\002 \002(\0162\022.Mutate.Muta" +
"teType\022(\n\013columnValue\030\003 \003(\0132\023.Mutate.Col" +
"umnValue\022!\n\tattribute\030\004 \003(\0132\016.NameBytesP",
"air\022\021\n\ttimestamp\030\005 \001(\004\022\016\n\006lockId\030\006 \001(\004\022\030" +
"\n\nwriteToWAL\030\007 \001(\010:\004true\022\035\n\ttimeRange\030\n " +
"\001(\0132\n.TimeRange\032\310\001\n\013ColumnValue\022\016\n\006famil" +
"y\030\001 \002(\014\022:\n\016qualifierValue\030\002 \003(\0132\".Mutate" +
".ColumnValue.QualifierValue\032m\n\016Qualifier" +
"Value\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001(\014\022" +
"\021\n\ttimestamp\030\003 \001(\004\022&\n\ndeleteType\030\004 \001(\0162\022" +
".Mutate.DeleteType\"<\n\nMutateType\022\n\n\006APPE" +
"ND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELETE\020\003" +
"\"U\n\nDeleteType\022\026\n\022DELETE_ONE_VERSION\020\000\022\034",
"\n\030DELETE_MULTIPLE_VERSIONS\020\001\022\021\n\rDELETE_F" +
"AMILY\020\002\"i\n\rMutateRequest\022 \n\006region\030\001 \002(\013" +
"2\020.RegionSpecifier\022\027\n\006mutate\030\002 \002(\0132\007.Mut" +
"ate\022\035\n\tcondition\030\003 \001(\0132\n.Condition\"<\n\016Mu" +
"tateResponse\022\027\n\006result\030\001 \001(\0132\007.Result\022\021\n" +
"\tprocessed\030\002 \001(\010\"\252\002\n\004Scan\022\027\n\006column\030\001 \003(" +
"\0132\007.Column\022!\n\tattribute\030\002 \003(\0132\016.NameByte" +
"sPair\022\020\n\010startRow\030\003 \001(\014\022\017\n\007stopRow\030\004 \001(\014" +
"\022\036\n\006filter\030\005 \001(\0132\016.NameBytesPair\022\035\n\ttime" +
"\n\014Client.proto\032\013hbase.proto\032\020Comparator." +
"proto\"+\n\006Column\022\016\n\006family\030\001 \002(\014\022\021\n\tquali" +
"fier\030\002 \003(\014\"\362\001\n\003Get\022\013\n\003row\030\001 \002(\014\022\027\n\006colum" +
"n\030\002 \003(\0132\007.Column\022!\n\tattribute\030\003 \003(\0132\016.Na" +
"meBytesPair\022\016\n\006lockId\030\004 \001(\004\022\027\n\006filter\030\005 " +
"\001(\0132\007.Filter\022\035\n\ttimeRange\030\006 \001(\0132\n.TimeRa" +
"nge\022\026\n\013maxVersions\030\007 \001(\r:\0011\022\031\n\013cacheBloc" +
"ks\030\010 \001(\010:\004true\022\022\n\nstoreLimit\030\t \001(\r\022\023\n\013st" +
"oreOffset\030\n \001(\r\"\037\n\006Result\022\025\n\rkeyValueByt" +
"es\030\001 \003(\014\"r\n\nGetRequest\022 \n\006region\030\001 \002(\0132\020",
".RegionSpecifier\022\021\n\003get\030\002 \002(\0132\004.Get\022\030\n\020c" +
"losestRowBefore\030\003 \001(\010\022\025\n\rexistenceOnly\030\004" +
" \001(\010\"6\n\013GetResponse\022\027\n\006result\030\001 \001(\0132\007.Re" +
"sult\022\016\n\006exists\030\002 \001(\010\"\177\n\tCondition\022\013\n\003row" +
"\030\001 \002(\014\022\016\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(" +
"\014\022!\n\013compareType\030\004 \002(\0162\014.CompareType\022\037\n\n" +
"comparator\030\005 \002(\0132\013.Comparator\"\306\004\n\006Mutate" +
"\022\013\n\003row\030\001 \002(\014\022&\n\nmutateType\030\002 \002(\0162\022.Muta" +
"te.MutateType\022(\n\013columnValue\030\003 \003(\0132\023.Mut" +
"ate.ColumnValue\022!\n\tattribute\030\004 \003(\0132\016.Nam",
"eBytesPair\022\021\n\ttimestamp\030\005 \001(\004\022\016\n\006lockId\030" +
"\006 \001(\004\022\030\n\nwriteToWAL\030\007 \001(\010:\004true\022\035\n\ttimeR" +
"ange\030\n \001(\0132\n.TimeRange\032\310\001\n\013ColumnValue\022\016" +
"\n\006family\030\001 \002(\014\022:\n\016qualifierValue\030\002 \003(\0132\"" +
".Mutate.ColumnValue.QualifierValue\032m\n\016Qu" +
"alifierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005value" +
"\030\002 \001(\014\022\021\n\ttimestamp\030\003 \001(\004\022&\n\ndeleteType\030" +
"\004 \001(\0162\022.Mutate.DeleteType\"<\n\nMutateType\022" +
"\n\n\006APPEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022\n\n\006D" +
"ELETE\020\003\"U\n\nDeleteType\022\026\n\022DELETE_ONE_VERS",
"ION\020\000\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001\022\021\n\rD" +
"ELETE_FAMILY\020\002\"i\n\rMutateRequest\022 \n\006regio" +
"n\030\001 \002(\0132\020.RegionSpecifier\022\027\n\006mutate\030\002 \002(" +
"\0132\007.Mutate\022\035\n\tcondition\030\003 \001(\0132\n.Conditio" +
"n\"<\n\016MutateResponse\022\027\n\006result\030\001 \001(\0132\007.Re" +
"sult\022\021\n\tprocessed\030\002 \001(\010\"\243\002\n\004Scan\022\027\n\006colu" +
"mn\030\001 \003(\0132\007.Column\022!\n\tattribute\030\002 \003(\0132\016.N" +
"ameBytesPair\022\020\n\010startRow\030\003 \001(\014\022\017\n\007stopRo" +
"w\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007.Filter\022\035\n\ttime" +
"Range\030\006 \001(\0132\n.TimeRange\022\026\n\013maxVersions\030\007",
" \001(\r:\0011\022\031\n\013cacheBlocks\030\010 \001(\010:\004true\022\021\n\tba" +
"tchSize\030\t \001(\r\022\025\n\rmaxResultSize\030\n \001(\004\022\022\n\n" +
@ -21886,6 +21886,7 @@ public final class ClientProtos {
.internalBuildGeneratedFileFrom(descriptorData,
new com.google.protobuf.Descriptors.FileDescriptor[] {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(),
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.getDescriptor(),
}, assigner);
}

View File

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

View File

@ -2886,7 +2886,7 @@ public class HRegionServer implements ClientProtocol,
byte[] qualifier = condition.getQualifier().toByteArray();
CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
WritableByteArrayComparable comparator =
(WritableByteArrayComparable)ProtobufUtil.toObject(condition.getComparator());
ProtobufUtil.toComparator(condition.getComparator());
if (region.getCoprocessorHost() != null) {
processed = region.getCoprocessorHost().preCheckAndPut(
row, family, qualifier, compareOp, comparator, put);
@ -2915,7 +2915,7 @@ public class HRegionServer implements ClientProtocol,
byte[] qualifier = condition.getQualifier().toByteArray();
CompareOp compareOp = CompareOp.valueOf(condition.getCompareType().name());
WritableByteArrayComparable comparator =
(WritableByteArrayComparable)ProtobufUtil.toObject(condition.getComparator());
ProtobufUtil.toComparator(condition.getComparator());
if (region.getCoprocessorHost() != null) {
processed = region.getCoprocessorHost().preCheckAndDelete(
row, family, qualifier, compareOp, comparator, delete);

View File

@ -22,10 +22,14 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.DeserializationException;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.filter.FilterBase;
import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.security.User;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* <strong>NOTE: for internal use only by AccessController implementation</strong>
*
@ -68,17 +72,25 @@ class AccessControlFilter extends FilterBase {
return ReturnCode.NEXT_COL;
}
@Override
public void write(DataOutput dataOutput) throws IOException {
/**
* @return The filter serialized using pb
*/
public byte [] toByteArray() {
// no implementation, server-side use only
throw new UnsupportedOperationException(
"Serialization not supported. Intended for server-side use only.");
"Serialization not supported. Intended for server-side use only.");
}
@Override
public void readFields(DataInput dataInput) throws IOException {
/**
* @param pbBytes A pb serialized {@link AccessControlFilter} instance
* @return An instance of {@link AccessControlFilter} made from <code>bytes</code>
* @throws DeserializationException
* @see {@link #toByteArray()}
*/
public static AccessControlFilter parseFrom(final byte [] pbBytes)
throws DeserializationException {
// no implementation, server-side use only
throw new UnsupportedOperationException(
"Serialization not supported. Intended for server-side use only.");
"Serialization not supported. Intended for server-side use only.");
}
}

View File

@ -25,6 +25,7 @@ option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
import "hbase.proto";
import "Comparator.proto";
/**
* Container for a list of column qualifier names of a family.
@ -42,7 +43,7 @@ message Get {
repeated Column column = 2;
repeated NameBytesPair attribute = 3;
optional uint64 lockId = 4;
optional NameBytesPair filter = 5;
optional Filter filter = 5;
optional TimeRange timeRange = 6;
optional uint32 maxVersions = 7 [default = 1];
optional bool cacheBlocks = 8 [default = true];
@ -98,7 +99,7 @@ message Condition {
required bytes family = 2;
required bytes qualifier = 3;
required CompareType compareType = 4;
required NameBytesPair comparator = 5;
required Comparator comparator = 5;
}
/**
@ -181,7 +182,7 @@ message Scan {
repeated NameBytesPair attribute = 2;
optional bytes startRow = 3;
optional bytes stopRow = 4;
optional NameBytesPair filter = 5;
optional Filter filter = 5;
optional TimeRange timeRange = 6;
optional uint32 maxVersions = 7 [default = 1];
optional bool cacheBlocks = 8 [default = true];

View File

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

View File

@ -25,6 +25,7 @@ option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
import "hbase.proto";
import "Comparator.proto";
message ColumnCountGetFilter {
required int32 limit = 1;
@ -40,32 +41,47 @@ message ColumnPrefixFilter {
}
message ColumnRangeFilter {
required bytes minColumn = 1;
optional bytes minColumn = 1;
optional bool minColumnInclusive = 2;
required bytes maxColumn = 3;
optional bytes maxColumn = 3;
optional bool maxColumnInclusive = 4;
}
message CompareFilter {
required CompareType compareOp = 1;
required bytes comparator = 2;
optional Comparator comparator = 2;
}
message DependentColumnFilter {
required bytes columnFamily = 1;
required bytes columnQualifier = 2;
optional bool dropDependentColumn = 3;
required CompareFilter compareFilter = 1;
optional bytes columnFamily = 2;
optional bytes columnQualifier = 3;
optional bool dropDependentColumn = 4;
}
message FamilyFilter {
required CompareFilter compareFilter = 1;
}
message FilterList {
required Operator operator = 1;
repeated Filter filters = 2;
enum Operator {
MUST_PASS_ALL = 1;
MUST_PASS_ONE = 2;
}
}
message FilterWrapper {
required Filter filter = 1;
}
message FirstKeyOnlyFilter {
}
message FirstKeyValueMatchingQualifiersFilter {
repeated bytes qualifiers = 1;
}
message FuzzyRowFilter {
@ -73,7 +89,7 @@ message FuzzyRowFilter {
}
message InclusiveStopFilter {
required bytes stopRowKey = 1;
optional bytes stopRowKey = 1;
}
message KeyOnlyFilter {
@ -89,7 +105,7 @@ message PageFilter {
}
message PrefixFilter {
required bytes prefix = 1;
optional bytes prefix = 1;
}
message QualifierFilter {
@ -109,10 +125,10 @@ message SingleColumnValueExcludeFilter {
}
message SingleColumnValueFilter {
required bytes columnFamily = 1;
required bytes columnQualifier = 2;
optional bytes columnFamily = 1;
optional bytes columnQualifier = 2;
required CompareType compareOp = 3;
required bytes comparator = 4;
required Comparator comparator = 4;
optional bool foundColumn = 5;
optional bool matchedColumn = 6;
optional bool filterIfMissing = 7;
@ -120,7 +136,7 @@ message SingleColumnValueFilter {
}
message SkipFilter {
required NameBytesPair filter = 1;
required Filter filter = 1;
}
message TimestampsFilter {
@ -132,5 +148,5 @@ message ValueFilter {
}
message WhileMatchFilter {
required NameBytesPair filter = 1;
required Filter filter = 1;
}

View File

@ -185,6 +185,11 @@ message TimeRange {
optional uint64 to = 2;
}
message Filter {
required string name = 1;
optional bytes serializedFilter = 2;
}
/* Comparison operators */
enum CompareType {
LESS = 0;

View File

@ -47,6 +47,8 @@ import org.apache.hadoop.hbase.filter.RowFilter;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.io.HbaseMapWritable;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.io.DataInputBuffer;
@ -69,8 +71,8 @@ public class TestSerialization {
@Test public void testCompareFilter() throws Exception {
Filter f = new RowFilter(CompareOp.EQUAL,
new BinaryComparator(Bytes.toBytes("testRowOne-2")));
byte [] bytes = Writables.getBytes(f);
Filter ff = (Filter)Writables.getWritable(bytes, new RowFilter());
byte [] bytes = f.toByteArray();
Filter ff = RowFilter.parseFrom(bytes);
assertNotNull(ff);
}
@ -263,8 +265,8 @@ public class TestSerialization {
get.setTimeRange(ts, ts+1);
get.setMaxVersions(maxVersions);
byte[] sb = Writables.getBytes(get);
Get desGet = (Get)Writables.getWritable(sb, new Get());
ClientProtos.Get getProto = ProtobufUtil.toGet(get);
Get desGet = ProtobufUtil.toGet(getProto);
assertTrue(Bytes.equals(get.getRow(), desGet.getRow()));
Set<byte[]> set = null;
@ -304,8 +306,8 @@ public class TestSerialization {
scan.setTimeRange(ts, ts+1);
scan.setMaxVersions(maxVersions);
byte[] sb = Writables.getBytes(scan);
Scan desScan = (Scan)Writables.getWritable(sb, new Scan());
ClientProtos.Scan scanProto = ProtobufUtil.toScan(scan);
Scan desScan = ProtobufUtil.toScan(scanProto);
assertTrue(Bytes.equals(scan.getStartRow(), desScan.getStartRow()));
assertTrue(Bytes.equals(scan.getStopRow(), desScan.getStopRow()));
@ -327,8 +329,8 @@ public class TestSerialization {
final String name = "testScan";
byte [] prefix = Bytes.toBytes(name);
scan.setFilter(new PrefixFilter(prefix));
sb = Writables.getBytes(scan);
desScan = (Scan)Writables.getWritable(sb, new Scan());
scanProto = ProtobufUtil.toScan(scan);
desScan = ProtobufUtil.toScan(scanProto);
Filter f = desScan.getFilter();
assertTrue(f instanceof PrefixFilter);
}

View File

@ -156,7 +156,7 @@ public class TestAttributes {
@Test
public void testGetId() {
Get get = new Get();
Get get = new Get(null);
Assert.assertNull("Make sure id is null if unset", get.toMap().get("id"));
get.setId("myId");
Assert.assertEquals("myId", get.toMap().get("id"));
@ -164,7 +164,7 @@ public class TestAttributes {
@Test
public void testAppendId() {
Append append = new Append();
Append append = new Append(Bytes.toBytes("testRow"));
Assert.assertNull("Make sure id is null if unset", append.toMap().get("id"));
append.setId("myId");
Assert.assertEquals("myId", append.toMap().get("id"));

View File

@ -29,6 +29,8 @@ import java.io.IOException;
import java.util.Arrays;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert;
import org.junit.Test;
@ -39,20 +41,14 @@ import org.junit.experimental.categories.Category;
public class TestGet {
@Test
public void testAttributesSerialization() throws IOException {
Get get = new Get();
Get get = new Get(Bytes.toBytes("row"));
get.setAttribute("attribute1", Bytes.toBytes("value1"));
get.setAttribute("attribute2", Bytes.toBytes("value2"));
get.setAttribute("attribute3", Bytes.toBytes("value3"));
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
DataOutput out = new DataOutputStream(byteArrayOutputStream);
get.write(out);
Get get2 = new Get();
Assert.assertTrue(get2.getAttributesMap().isEmpty());
get2.readFields(new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray())));
ClientProtos.Get getProto = ProtobufUtil.toGet(get);
Get get2 = ProtobufUtil.toGet(getProto);
Assert.assertNull(get2.getAttribute("absent"));
Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), get2.getAttribute("attribute1")));
Assert.assertTrue(Arrays.equals(Bytes.toBytes("value2"), get2.getAttribute("attribute2")));
@ -62,7 +58,7 @@ public class TestGet {
@Test
public void testGetAttributes() {
Get get = new Get();
Get get = new Get(null);
Assert.assertTrue(get.getAttributesMap().isEmpty());
Assert.assertNull(get.getAttribute("absent"));

View File

@ -29,6 +29,8 @@ import java.io.IOException;
import java.util.Arrays;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert;
import org.junit.Test;
@ -44,14 +46,9 @@ public class TestScan {
scan.setAttribute("attribute2", Bytes.toBytes("value2"));
scan.setAttribute("attribute3", Bytes.toBytes("value3"));
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
DataOutput out = new DataOutputStream(byteArrayOutputStream);
scan.write(out);
ClientProtos.Scan scanProto = ProtobufUtil.toScan(scan);
Scan scan2 = new Scan();
Assert.assertTrue(scan2.getAttributesMap().isEmpty());
scan2.readFields(new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray())));
Scan scan2 = ProtobufUtil.toScan(scanProto);
Assert.assertNull(scan2.getAttribute("absent"));
Assert.assertTrue(Arrays.equals(Bytes.toBytes("value1"), scan2.getAttribute("attribute1")));

View File

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

View File

@ -26,6 +26,8 @@ import java.io.DataOutputStream;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Before;
@ -58,18 +60,10 @@ public class TestColumnPaginationFilter
}
private Filter serializationTest(Filter filter) throws Exception {
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(stream);
filter.write(out);
out.close();
byte[] buffer = stream.toByteArray();
HBaseProtos.Filter filterProto = ProtobufUtil.toFilter(filter);
Filter newFilter = ProtobufUtil.toFilter(filterProto);
DataInputStream in =
new DataInputStream(new ByteArrayInputStream(buffer));
Filter newFilter = new ColumnPaginationFilter();
newFilter.readFields(in);
return newFilter;
return newFilter;
}

View File

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

View File

@ -1659,12 +1659,7 @@ public class TestFilter {
private static Thread ipcHandlerThread = null;
@Override
public void readFields(DataInput arg0) throws IOException {
}
@Override
public void write(DataOutput arg0) throws IOException {
}
public byte [] toByteArray() {return null;}
@Override
public boolean filterRow() {

View File

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

View File

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

View File

@ -32,6 +32,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
/**
* Tests the inclusive stop row filter
@ -65,16 +66,10 @@ public class TestInclusiveStopFilter {
@Test
public void testSerialization() throws Exception {
// Decompose mainFilter to bytes.
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(stream);
mainFilter.write(out);
out.close();
byte[] buffer = stream.toByteArray();
byte[] buffer = mainFilter.toByteArray();
// Recompose mainFilter.
DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
Filter newFilter = new InclusiveStopFilter();
newFilter.readFields(in);
Filter newFilter = InclusiveStopFilter.parseFrom(buffer);
// Ensure the serialization preserved the filter by running a full test.
stopRowTests(newFilter);

View File

@ -55,15 +55,9 @@ public class TestPageFilter {
public void testSerialization() throws Exception {
Filter f = new PageFilter(ROW_LIMIT);
// Decompose mainFilter to bytes.
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(stream);
f.write(out);
out.close();
byte[] buffer = stream.toByteArray();
byte[] buffer = f.toByteArray();
// Recompose mainFilter.
DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
Filter newFilter = new PageFilter();
newFilter.readFields(in);
Filter newFilter = PageFilter.parseFrom(buffer);
// Ensure the serialization preserved the filter by running a full test.
pageSizeTests(newFilter);

View File

@ -69,16 +69,10 @@ public class TestPrefixFilter {
@Test
public void testSerialization() throws Exception {
// Decompose mainFilter to bytes.
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(stream);
mainFilter.write(out);
out.close();
byte[] buffer = stream.toByteArray();
byte[] buffer = mainFilter.toByteArray();
// Recompose filter.
DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
Filter newFilter = new PrefixFilter();
newFilter.readFields(in);
Filter newFilter = PrefixFilter.parseFrom(buffer);
// Ensure the serialization preserved the filter by running all test.
prefixRowTests(newFilter);

View File

@ -81,16 +81,10 @@ public class TestRandomRowFilter {
private RandomRowFilter serializationTest(RandomRowFilter filter)
throws Exception {
// Decompose filter to bytes.
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(stream);
filter.write(out);
out.close();
byte[] buffer = stream.toByteArray();
byte[] buffer = filter.toByteArray();
// Recompose filter.
DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
RandomRowFilter newFilter = new RandomRowFilter();
newFilter.readFields(in);
RandomRowFilter newFilter = RandomRowFilter.parseFrom(buffer);
return newFilter;
}

View File

@ -156,17 +156,10 @@ public class TestSingleColumnValueFilter {
private Filter serializationTest(Filter filter)
throws Exception {
// Decompose filter to bytes.
ByteArrayOutputStream stream = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(stream);
filter.write(out);
out.close();
byte[] buffer = stream.toByteArray();
byte[] buffer = filter.toByteArray();
// Recompose filter.
DataInputStream in =
new DataInputStream(new ByteArrayInputStream(buffer));
Filter newFilter = new SingleColumnValueFilter();
newFilter.readFields(in);
Filter newFilter = SingleColumnValueFilter.parseFrom(buffer);
return newFilter;
}

View File

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