HBASE-8707 Add LongComparator for filter (Liu Shaohui)
This commit is contained in:
parent
d64ade4fde
commit
b0eaa92f5b
|
@ -0,0 +1,82 @@
|
||||||
|
/*
|
||||||
|
* 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 com.google.protobuf.InvalidProtocolBufferException;
|
||||||
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A long comparator which numerical compares against the specified byte array
|
||||||
|
*/
|
||||||
|
|
||||||
|
public class LongComparator extends ByteArrayComparable {
|
||||||
|
private Long longValue;
|
||||||
|
|
||||||
|
public LongComparator(long value) {
|
||||||
|
super(Bytes.toBytes(value));
|
||||||
|
this.longValue = value;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compareTo(byte[] value, int offset, int length) {
|
||||||
|
Long that = Bytes.toLong(value, offset, length);
|
||||||
|
return this.longValue.compareTo(that);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The comparator serialized using pb
|
||||||
|
*/
|
||||||
|
public byte [] toByteArray() {
|
||||||
|
ComparatorProtos.LongComparator.Builder builder =
|
||||||
|
ComparatorProtos.LongComparator.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 org.apache.hadoop.hbase.exceptions.DeserializationException
|
||||||
|
* @see #toByteArray
|
||||||
|
*/
|
||||||
|
public static LongComparator parseFrom(final byte [] pbBytes)
|
||||||
|
throws DeserializationException {
|
||||||
|
ComparatorProtos.LongComparator proto;
|
||||||
|
try {
|
||||||
|
proto = ComparatorProtos.LongComparator.parseFrom(pbBytes);
|
||||||
|
} catch (InvalidProtocolBufferException e) {
|
||||||
|
throw new DeserializationException(e);
|
||||||
|
}
|
||||||
|
return new LongComparator(Bytes.toLong(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(LongComparator other) {
|
||||||
|
if (other == this) return true;
|
||||||
|
if (!(other instanceof LongComparator)) return false;
|
||||||
|
|
||||||
|
return super.areSerializedFieldsEqual(other);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,37 @@
|
||||||
|
/*
|
||||||
|
* 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 org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.junit.Test;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
public class TestLongComparator {
|
||||||
|
private long values[] = { Long.MIN_VALUE, -10000000000L, -1000000L, 0L, 1000000L, 10000000000L,
|
||||||
|
Long.MAX_VALUE };
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSimple() {
|
||||||
|
for (int i = 1; i < values.length ; i++) {
|
||||||
|
for (int j = 0; j < i; j++) {
|
||||||
|
LongComparator cp = new LongComparator(values[i]);
|
||||||
|
assertEquals(1, cp.compareTo(Bytes.toBytes(values[j])));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -1608,6 +1608,559 @@ public final class ComparatorProtos {
|
||||||
// @@protoc_insertion_point(class_scope:BinaryComparator)
|
// @@protoc_insertion_point(class_scope:BinaryComparator)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public interface LongComparatorOrBuilder
|
||||||
|
extends com.google.protobuf.MessageOrBuilder {
|
||||||
|
|
||||||
|
// required .ByteArrayComparable comparable = 1;
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
boolean hasComparable();
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable getComparable();
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder getComparableOrBuilder();
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* Protobuf type {@code LongComparator}
|
||||||
|
*/
|
||||||
|
public static final class LongComparator extends
|
||||||
|
com.google.protobuf.GeneratedMessage
|
||||||
|
implements LongComparatorOrBuilder {
|
||||||
|
// Use LongComparator.newBuilder() to construct.
|
||||||
|
private LongComparator(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
|
||||||
|
super(builder);
|
||||||
|
this.unknownFields = builder.getUnknownFields();
|
||||||
|
}
|
||||||
|
private LongComparator(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
|
||||||
|
|
||||||
|
private static final LongComparator defaultInstance;
|
||||||
|
public static LongComparator getDefaultInstance() {
|
||||||
|
return defaultInstance;
|
||||||
|
}
|
||||||
|
|
||||||
|
public LongComparator getDefaultInstanceForType() {
|
||||||
|
return defaultInstance;
|
||||||
|
}
|
||||||
|
|
||||||
|
private final com.google.protobuf.UnknownFieldSet unknownFields;
|
||||||
|
@java.lang.Override
|
||||||
|
public final com.google.protobuf.UnknownFieldSet
|
||||||
|
getUnknownFields() {
|
||||||
|
return this.unknownFields;
|
||||||
|
}
|
||||||
|
private LongComparator(
|
||||||
|
com.google.protobuf.CodedInputStream input,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
initFields();
|
||||||
|
int mutable_bitField0_ = 0;
|
||||||
|
com.google.protobuf.UnknownFieldSet.Builder unknownFields =
|
||||||
|
com.google.protobuf.UnknownFieldSet.newBuilder();
|
||||||
|
try {
|
||||||
|
boolean done = false;
|
||||||
|
while (!done) {
|
||||||
|
int tag = input.readTag();
|
||||||
|
switch (tag) {
|
||||||
|
case 0:
|
||||||
|
done = true;
|
||||||
|
break;
|
||||||
|
default: {
|
||||||
|
if (!parseUnknownField(input, unknownFields,
|
||||||
|
extensionRegistry, tag)) {
|
||||||
|
done = true;
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case 10: {
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder subBuilder = null;
|
||||||
|
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||||
|
subBuilder = comparable_.toBuilder();
|
||||||
|
}
|
||||||
|
comparable_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.PARSER, extensionRegistry);
|
||||||
|
if (subBuilder != null) {
|
||||||
|
subBuilder.mergeFrom(comparable_);
|
||||||
|
comparable_ = subBuilder.buildPartial();
|
||||||
|
}
|
||||||
|
bitField0_ |= 0x00000001;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
|
||||||
|
throw e.setUnfinishedMessage(this);
|
||||||
|
} catch (java.io.IOException e) {
|
||||||
|
throw new com.google.protobuf.InvalidProtocolBufferException(
|
||||||
|
e.getMessage()).setUnfinishedMessage(this);
|
||||||
|
} finally {
|
||||||
|
this.unknownFields = unknownFields.build();
|
||||||
|
makeExtensionsImmutable();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
public static final com.google.protobuf.Descriptors.Descriptor
|
||||||
|
getDescriptor() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.internal_static_LongComparator_descriptor;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||||
|
internalGetFieldAccessorTable() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.internal_static_LongComparator_fieldAccessorTable
|
||||||
|
.ensureFieldAccessorsInitialized(
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator.class, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator.Builder.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static com.google.protobuf.Parser<LongComparator> PARSER =
|
||||||
|
new com.google.protobuf.AbstractParser<LongComparator>() {
|
||||||
|
public LongComparator parsePartialFrom(
|
||||||
|
com.google.protobuf.CodedInputStream input,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
return new LongComparator(input, extensionRegistry);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
@java.lang.Override
|
||||||
|
public com.google.protobuf.Parser<LongComparator> getParserForType() {
|
||||||
|
return PARSER;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int bitField0_;
|
||||||
|
// required .ByteArrayComparable comparable = 1;
|
||||||
|
public static final int COMPARABLE_FIELD_NUMBER = 1;
|
||||||
|
private org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable comparable_;
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public boolean hasComparable() {
|
||||||
|
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable getComparable() {
|
||||||
|
return comparable_;
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder getComparableOrBuilder() {
|
||||||
|
return comparable_;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initFields() {
|
||||||
|
comparable_ = org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.getDefaultInstance();
|
||||||
|
}
|
||||||
|
private byte memoizedIsInitialized = -1;
|
||||||
|
public final boolean isInitialized() {
|
||||||
|
byte isInitialized = memoizedIsInitialized;
|
||||||
|
if (isInitialized != -1) return isInitialized == 1;
|
||||||
|
|
||||||
|
if (!hasComparable()) {
|
||||||
|
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.writeMessage(1, comparable_);
|
||||||
|
}
|
||||||
|
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
|
||||||
|
.computeMessageSize(1, comparable_);
|
||||||
|
}
|
||||||
|
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.ComparatorProtos.LongComparator)) {
|
||||||
|
return super.equals(obj);
|
||||||
|
}
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator other = (org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator) obj;
|
||||||
|
|
||||||
|
boolean result = true;
|
||||||
|
result = result && (hasComparable() == other.hasComparable());
|
||||||
|
if (hasComparable()) {
|
||||||
|
result = result && getComparable()
|
||||||
|
.equals(other.getComparable());
|
||||||
|
}
|
||||||
|
result = result &&
|
||||||
|
getUnknownFields().equals(other.getUnknownFields());
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int memoizedHashCode = 0;
|
||||||
|
@java.lang.Override
|
||||||
|
public int hashCode() {
|
||||||
|
if (memoizedHashCode != 0) {
|
||||||
|
return memoizedHashCode;
|
||||||
|
}
|
||||||
|
int hash = 41;
|
||||||
|
hash = (19 * hash) + getDescriptorForType().hashCode();
|
||||||
|
if (hasComparable()) {
|
||||||
|
hash = (37 * hash) + COMPARABLE_FIELD_NUMBER;
|
||||||
|
hash = (53 * hash) + getComparable().hashCode();
|
||||||
|
}
|
||||||
|
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||||
|
memoizedHashCode = hash;
|
||||||
|
return hash;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
|
||||||
|
com.google.protobuf.ByteString data)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
return PARSER.parseFrom(data);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
|
||||||
|
com.google.protobuf.ByteString data,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
return PARSER.parseFrom(data, extensionRegistry);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseFrom(byte[] data)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
return PARSER.parseFrom(data);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
|
||||||
|
byte[] data,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||||
|
return PARSER.parseFrom(data, extensionRegistry);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseFrom(java.io.InputStream input)
|
||||||
|
throws java.io.IOException {
|
||||||
|
return PARSER.parseFrom(input);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
|
||||||
|
java.io.InputStream input,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws java.io.IOException {
|
||||||
|
return PARSER.parseFrom(input, extensionRegistry);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseDelimitedFrom(java.io.InputStream input)
|
||||||
|
throws java.io.IOException {
|
||||||
|
return PARSER.parseDelimitedFrom(input);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseDelimitedFrom(
|
||||||
|
java.io.InputStream input,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws java.io.IOException {
|
||||||
|
return PARSER.parseDelimitedFrom(input, extensionRegistry);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
|
||||||
|
com.google.protobuf.CodedInputStream input)
|
||||||
|
throws java.io.IOException {
|
||||||
|
return PARSER.parseFrom(input);
|
||||||
|
}
|
||||||
|
public static org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parseFrom(
|
||||||
|
com.google.protobuf.CodedInputStream input,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws java.io.IOException {
|
||||||
|
return PARSER.parseFrom(input, extensionRegistry);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Builder newBuilder() { return Builder.create(); }
|
||||||
|
public Builder newBuilderForType() { return newBuilder(); }
|
||||||
|
public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator 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;
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* Protobuf type {@code LongComparator}
|
||||||
|
*/
|
||||||
|
public static final class Builder extends
|
||||||
|
com.google.protobuf.GeneratedMessage.Builder<Builder>
|
||||||
|
implements org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparatorOrBuilder {
|
||||||
|
public static final com.google.protobuf.Descriptors.Descriptor
|
||||||
|
getDescriptor() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.internal_static_LongComparator_descriptor;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||||
|
internalGetFieldAccessorTable() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.internal_static_LongComparator_fieldAccessorTable
|
||||||
|
.ensureFieldAccessorsInitialized(
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator.class, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator.Builder.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Construct using org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator.newBuilder()
|
||||||
|
private Builder() {
|
||||||
|
maybeForceBuilderInitialization();
|
||||||
|
}
|
||||||
|
|
||||||
|
private Builder(
|
||||||
|
com.google.protobuf.GeneratedMessage.BuilderParent parent) {
|
||||||
|
super(parent);
|
||||||
|
maybeForceBuilderInitialization();
|
||||||
|
}
|
||||||
|
private void maybeForceBuilderInitialization() {
|
||||||
|
if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
|
||||||
|
getComparableFieldBuilder();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
private static Builder create() {
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder clear() {
|
||||||
|
super.clear();
|
||||||
|
if (comparableBuilder_ == null) {
|
||||||
|
comparable_ = org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.getDefaultInstance();
|
||||||
|
} else {
|
||||||
|
comparableBuilder_.clear();
|
||||||
|
}
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000001);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder clone() {
|
||||||
|
return create().mergeFrom(buildPartial());
|
||||||
|
}
|
||||||
|
|
||||||
|
public com.google.protobuf.Descriptors.Descriptor
|
||||||
|
getDescriptorForType() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.internal_static_LongComparator_descriptor;
|
||||||
|
}
|
||||||
|
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator getDefaultInstanceForType() {
|
||||||
|
return org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator.getDefaultInstance();
|
||||||
|
}
|
||||||
|
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator build() {
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator result = buildPartial();
|
||||||
|
if (!result.isInitialized()) {
|
||||||
|
throw newUninitializedMessageException(result);
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator buildPartial() {
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator result = new org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator(this);
|
||||||
|
int from_bitField0_ = bitField0_;
|
||||||
|
int to_bitField0_ = 0;
|
||||||
|
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
||||||
|
to_bitField0_ |= 0x00000001;
|
||||||
|
}
|
||||||
|
if (comparableBuilder_ == null) {
|
||||||
|
result.comparable_ = comparable_;
|
||||||
|
} else {
|
||||||
|
result.comparable_ = comparableBuilder_.build();
|
||||||
|
}
|
||||||
|
result.bitField0_ = to_bitField0_;
|
||||||
|
onBuilt();
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder mergeFrom(com.google.protobuf.Message other) {
|
||||||
|
if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator) {
|
||||||
|
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator)other);
|
||||||
|
} else {
|
||||||
|
super.mergeFrom(other);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator other) {
|
||||||
|
if (other == org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator.getDefaultInstance()) return this;
|
||||||
|
if (other.hasComparable()) {
|
||||||
|
mergeComparable(other.getComparable());
|
||||||
|
}
|
||||||
|
this.mergeUnknownFields(other.getUnknownFields());
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public final boolean isInitialized() {
|
||||||
|
if (!hasComparable()) {
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder mergeFrom(
|
||||||
|
com.google.protobuf.CodedInputStream input,
|
||||||
|
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
|
||||||
|
throws java.io.IOException {
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator parsedMessage = null;
|
||||||
|
try {
|
||||||
|
parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
|
||||||
|
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
|
||||||
|
parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.LongComparator) e.getUnfinishedMessage();
|
||||||
|
throw e;
|
||||||
|
} finally {
|
||||||
|
if (parsedMessage != null) {
|
||||||
|
mergeFrom(parsedMessage);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
private int bitField0_;
|
||||||
|
|
||||||
|
// required .ByteArrayComparable comparable = 1;
|
||||||
|
private org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable comparable_ = org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.getDefaultInstance();
|
||||||
|
private com.google.protobuf.SingleFieldBuilder<
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder> comparableBuilder_;
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public boolean hasComparable() {
|
||||||
|
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable getComparable() {
|
||||||
|
if (comparableBuilder_ == null) {
|
||||||
|
return comparable_;
|
||||||
|
} else {
|
||||||
|
return comparableBuilder_.getMessage();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public Builder setComparable(org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable value) {
|
||||||
|
if (comparableBuilder_ == null) {
|
||||||
|
if (value == null) {
|
||||||
|
throw new NullPointerException();
|
||||||
|
}
|
||||||
|
comparable_ = value;
|
||||||
|
onChanged();
|
||||||
|
} else {
|
||||||
|
comparableBuilder_.setMessage(value);
|
||||||
|
}
|
||||||
|
bitField0_ |= 0x00000001;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public Builder setComparable(
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder builderForValue) {
|
||||||
|
if (comparableBuilder_ == null) {
|
||||||
|
comparable_ = builderForValue.build();
|
||||||
|
onChanged();
|
||||||
|
} else {
|
||||||
|
comparableBuilder_.setMessage(builderForValue.build());
|
||||||
|
}
|
||||||
|
bitField0_ |= 0x00000001;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public Builder mergeComparable(org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable value) {
|
||||||
|
if (comparableBuilder_ == null) {
|
||||||
|
if (((bitField0_ & 0x00000001) == 0x00000001) &&
|
||||||
|
comparable_ != org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.getDefaultInstance()) {
|
||||||
|
comparable_ =
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.newBuilder(comparable_).mergeFrom(value).buildPartial();
|
||||||
|
} else {
|
||||||
|
comparable_ = value;
|
||||||
|
}
|
||||||
|
onChanged();
|
||||||
|
} else {
|
||||||
|
comparableBuilder_.mergeFrom(value);
|
||||||
|
}
|
||||||
|
bitField0_ |= 0x00000001;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public Builder clearComparable() {
|
||||||
|
if (comparableBuilder_ == null) {
|
||||||
|
comparable_ = org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.getDefaultInstance();
|
||||||
|
onChanged();
|
||||||
|
} else {
|
||||||
|
comparableBuilder_.clear();
|
||||||
|
}
|
||||||
|
bitField0_ = (bitField0_ & ~0x00000001);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder getComparableBuilder() {
|
||||||
|
bitField0_ |= 0x00000001;
|
||||||
|
onChanged();
|
||||||
|
return getComparableFieldBuilder().getBuilder();
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
public org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder getComparableOrBuilder() {
|
||||||
|
if (comparableBuilder_ != null) {
|
||||||
|
return comparableBuilder_.getMessageOrBuilder();
|
||||||
|
} else {
|
||||||
|
return comparable_;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* <code>required .ByteArrayComparable comparable = 1;</code>
|
||||||
|
*/
|
||||||
|
private com.google.protobuf.SingleFieldBuilder<
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder>
|
||||||
|
getComparableFieldBuilder() {
|
||||||
|
if (comparableBuilder_ == null) {
|
||||||
|
comparableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
|
||||||
|
org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparable.Builder, org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos.ByteArrayComparableOrBuilder>(
|
||||||
|
comparable_,
|
||||||
|
getParentForChildren(),
|
||||||
|
isClean());
|
||||||
|
comparable_ = null;
|
||||||
|
}
|
||||||
|
return comparableBuilder_;
|
||||||
|
}
|
||||||
|
|
||||||
|
// @@protoc_insertion_point(builder_scope:LongComparator)
|
||||||
|
}
|
||||||
|
|
||||||
|
static {
|
||||||
|
defaultInstance = new LongComparator(true);
|
||||||
|
defaultInstance.initFields();
|
||||||
|
}
|
||||||
|
|
||||||
|
// @@protoc_insertion_point(class_scope:LongComparator)
|
||||||
|
}
|
||||||
|
|
||||||
public interface BinaryPrefixComparatorOrBuilder
|
public interface BinaryPrefixComparatorOrBuilder
|
||||||
extends com.google.protobuf.MessageOrBuilder {
|
extends com.google.protobuf.MessageOrBuilder {
|
||||||
|
|
||||||
|
@ -4735,6 +5288,11 @@ public final class ComparatorProtos {
|
||||||
private static
|
private static
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||||
internal_static_BinaryComparator_fieldAccessorTable;
|
internal_static_BinaryComparator_fieldAccessorTable;
|
||||||
|
private static com.google.protobuf.Descriptors.Descriptor
|
||||||
|
internal_static_LongComparator_descriptor;
|
||||||
|
private static
|
||||||
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||||
|
internal_static_LongComparator_fieldAccessorTable;
|
||||||
private static com.google.protobuf.Descriptors.Descriptor
|
private static com.google.protobuf.Descriptors.Descriptor
|
||||||
internal_static_BinaryPrefixComparator_descriptor;
|
internal_static_BinaryPrefixComparator_descriptor;
|
||||||
private static
|
private static
|
||||||
|
@ -4773,18 +5331,19 @@ public final class ComparatorProtos {
|
||||||
"\030\001 \002(\t\022\035\n\025serialized_comparator\030\002 \001(\014\"$\n" +
|
"\030\001 \002(\t\022\035\n\025serialized_comparator\030\002 \001(\014\"$\n" +
|
||||||
"\023ByteArrayComparable\022\r\n\005value\030\001 \001(\014\"<\n\020B" +
|
"\023ByteArrayComparable\022\r\n\005value\030\001 \001(\014\"<\n\020B" +
|
||||||
"inaryComparator\022(\n\ncomparable\030\001 \002(\0132\024.By" +
|
"inaryComparator\022(\n\ncomparable\030\001 \002(\0132\024.By" +
|
||||||
"teArrayComparable\"B\n\026BinaryPrefixCompara" +
|
"teArrayComparable\":\n\016LongComparator\022(\n\nc" +
|
||||||
|
"omparable\030\001 \002(\0132\024.ByteArrayComparable\"B\n" +
|
||||||
|
"\026BinaryPrefixComparator\022(\n\ncomparable\030\001 " +
|
||||||
|
"\002(\0132\024.ByteArrayComparable\"\216\001\n\rBitCompara" +
|
||||||
"tor\022(\n\ncomparable\030\001 \002(\0132\024.ByteArrayCompa" +
|
"tor\022(\n\ncomparable\030\001 \002(\0132\024.ByteArrayCompa" +
|
||||||
"rable\"\216\001\n\rBitComparator\022(\n\ncomparable\030\001 " +
|
"rable\022,\n\nbitwise_op\030\002 \002(\0162\030.BitComparato",
|
||||||
"\002(\0132\024.ByteArrayComparable\022,\n\nbitwise_op\030" +
|
"r.BitwiseOp\"%\n\tBitwiseOp\022\007\n\003AND\020\001\022\006\n\002OR\020" +
|
||||||
"\002 \002(\0162\030.BitComparator.BitwiseOp\"%\n\tBitwi" +
|
"\002\022\007\n\003XOR\020\003\"\020\n\016NullComparator\"`\n\025RegexStr" +
|
||||||
"seOp\022\007\n\003AND\020\001\022\006\n\002OR\020\002\022\007\n\003XOR\020\003\"\020\n\016NullCo",
|
"ingComparator\022\017\n\007pattern\030\001 \002(\t\022\025\n\rpatter" +
|
||||||
"mparator\"`\n\025RegexStringComparator\022\017\n\007pat" +
|
"n_flags\030\002 \002(\005\022\017\n\007charset\030\003 \002(\t\022\016\n\006engine" +
|
||||||
"tern\030\001 \002(\t\022\025\n\rpattern_flags\030\002 \002(\005\022\017\n\007cha" +
|
"\030\004 \001(\t\"%\n\023SubstringComparator\022\016\n\006substr\030" +
|
||||||
"rset\030\003 \002(\t\022\016\n\006engine\030\004 \001(\t\"%\n\023SubstringC" +
|
"\001 \002(\tBF\n*org.apache.hadoop.hbase.protobu" +
|
||||||
"omparator\022\016\n\006substr\030\001 \002(\tBF\n*org.apache." +
|
"f.generatedB\020ComparatorProtosH\001\210\001\001\240\001\001"
|
||||||
"hadoop.hbase.protobuf.generatedB\020Compara" +
|
|
||||||
"torProtosH\001\210\001\001\240\001\001"
|
|
||||||
};
|
};
|
||||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||||
|
@ -4809,32 +5368,38 @@ public final class ComparatorProtos {
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_BinaryComparator_descriptor,
|
internal_static_BinaryComparator_descriptor,
|
||||||
new java.lang.String[] { "Comparable", });
|
new java.lang.String[] { "Comparable", });
|
||||||
internal_static_BinaryPrefixComparator_descriptor =
|
internal_static_LongComparator_descriptor =
|
||||||
getDescriptor().getMessageTypes().get(3);
|
getDescriptor().getMessageTypes().get(3);
|
||||||
|
internal_static_LongComparator_fieldAccessorTable = new
|
||||||
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
|
internal_static_LongComparator_descriptor,
|
||||||
|
new java.lang.String[] { "Comparable", });
|
||||||
|
internal_static_BinaryPrefixComparator_descriptor =
|
||||||
|
getDescriptor().getMessageTypes().get(4);
|
||||||
internal_static_BinaryPrefixComparator_fieldAccessorTable = new
|
internal_static_BinaryPrefixComparator_fieldAccessorTable = new
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_BinaryPrefixComparator_descriptor,
|
internal_static_BinaryPrefixComparator_descriptor,
|
||||||
new java.lang.String[] { "Comparable", });
|
new java.lang.String[] { "Comparable", });
|
||||||
internal_static_BitComparator_descriptor =
|
internal_static_BitComparator_descriptor =
|
||||||
getDescriptor().getMessageTypes().get(4);
|
getDescriptor().getMessageTypes().get(5);
|
||||||
internal_static_BitComparator_fieldAccessorTable = new
|
internal_static_BitComparator_fieldAccessorTable = new
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_BitComparator_descriptor,
|
internal_static_BitComparator_descriptor,
|
||||||
new java.lang.String[] { "Comparable", "BitwiseOp", });
|
new java.lang.String[] { "Comparable", "BitwiseOp", });
|
||||||
internal_static_NullComparator_descriptor =
|
internal_static_NullComparator_descriptor =
|
||||||
getDescriptor().getMessageTypes().get(5);
|
getDescriptor().getMessageTypes().get(6);
|
||||||
internal_static_NullComparator_fieldAccessorTable = new
|
internal_static_NullComparator_fieldAccessorTable = new
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_NullComparator_descriptor,
|
internal_static_NullComparator_descriptor,
|
||||||
new java.lang.String[] { });
|
new java.lang.String[] { });
|
||||||
internal_static_RegexStringComparator_descriptor =
|
internal_static_RegexStringComparator_descriptor =
|
||||||
getDescriptor().getMessageTypes().get(6);
|
getDescriptor().getMessageTypes().get(7);
|
||||||
internal_static_RegexStringComparator_fieldAccessorTable = new
|
internal_static_RegexStringComparator_fieldAccessorTable = new
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_RegexStringComparator_descriptor,
|
internal_static_RegexStringComparator_descriptor,
|
||||||
new java.lang.String[] { "Pattern", "PatternFlags", "Charset", "Engine", });
|
new java.lang.String[] { "Pattern", "PatternFlags", "Charset", "Engine", });
|
||||||
internal_static_SubstringComparator_descriptor =
|
internal_static_SubstringComparator_descriptor =
|
||||||
getDescriptor().getMessageTypes().get(7);
|
getDescriptor().getMessageTypes().get(8);
|
||||||
internal_static_SubstringComparator_fieldAccessorTable = new
|
internal_static_SubstringComparator_fieldAccessorTable = new
|
||||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||||
internal_static_SubstringComparator_descriptor,
|
internal_static_SubstringComparator_descriptor,
|
||||||
|
|
|
@ -39,6 +39,10 @@ message BinaryComparator {
|
||||||
required ByteArrayComparable comparable = 1;
|
required ByteArrayComparable comparable = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message LongComparator {
|
||||||
|
required ByteArrayComparable comparable = 1;
|
||||||
|
}
|
||||||
|
|
||||||
message BinaryPrefixComparator {
|
message BinaryPrefixComparator {
|
||||||
required ByteArrayComparable comparable = 1;
|
required ByteArrayComparable comparable = 1;
|
||||||
}
|
}
|
||||||
|
|
|
@ -75,6 +75,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||||
import org.apache.hadoop.hbase.filter.Filter;
|
import org.apache.hadoop.hbase.filter.Filter;
|
||||||
import org.apache.hadoop.hbase.filter.FilterList;
|
import org.apache.hadoop.hbase.filter.FilterList;
|
||||||
import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
|
import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
|
||||||
|
import org.apache.hadoop.hbase.filter.LongComparator;
|
||||||
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
||||||
import org.apache.hadoop.hbase.filter.QualifierFilter;
|
import org.apache.hadoop.hbase.filter.QualifierFilter;
|
||||||
import org.apache.hadoop.hbase.filter.RegexStringComparator;
|
import org.apache.hadoop.hbase.filter.RegexStringComparator;
|
||||||
|
@ -749,6 +750,37 @@ public class TestFromClientSide {
|
||||||
scanner.close();
|
scanner.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterWithLongCompartor() throws Exception {
|
||||||
|
byte [] TABLE = Bytes.toBytes("testFilterWithLongCompartor");
|
||||||
|
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
|
||||||
|
byte [][] ROWS = makeN(ROW, 10);
|
||||||
|
byte [][] values = new byte[10][];
|
||||||
|
for (int i = 0; i < 10; i ++) {
|
||||||
|
values[i] = Bytes.toBytes(100L * i);
|
||||||
|
}
|
||||||
|
for(int i = 0; i < 10; i ++) {
|
||||||
|
Put put = new Put(ROWS[i]);
|
||||||
|
put.setDurability(Durability.SKIP_WAL);
|
||||||
|
put.add(FAMILY, QUALIFIER, values[i]);
|
||||||
|
ht.put(put);
|
||||||
|
}
|
||||||
|
Scan scan = new Scan();
|
||||||
|
scan.addFamily(FAMILY);
|
||||||
|
Filter filter = new SingleColumnValueFilter(FAMILY, QUALIFIER, CompareOp.GREATER,
|
||||||
|
new LongComparator(500));
|
||||||
|
scan.setFilter(filter);
|
||||||
|
ResultScanner scanner = ht.getScanner(scan);
|
||||||
|
int expectedIndex = 0;
|
||||||
|
for(Result result : ht.getScanner(scan)) {
|
||||||
|
assertEquals(result.size(), 1);
|
||||||
|
assertTrue(Bytes.toLong(result.getValue(FAMILY, QUALIFIER)) > 500);
|
||||||
|
expectedIndex++;
|
||||||
|
}
|
||||||
|
assertEquals(expectedIndex, 4);
|
||||||
|
scanner.close();
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testKeyOnlyFilter() throws Exception {
|
public void testKeyOnlyFilter() throws Exception {
|
||||||
byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter");
|
byte [] TABLE = Bytes.toBytes("testKeyOnlyFilter");
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
|
@ -95,6 +96,25 @@ public class TestSingleColumnValueFilter {
|
||||||
new RegexStringComparator(pattern.pattern(), pattern.flags()));
|
new RegexStringComparator(pattern.pattern(), pattern.flags()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLongComparator() throws IOException {
|
||||||
|
Filter filter = new SingleColumnValueFilter(COLUMN_FAMILY,
|
||||||
|
COLUMN_QUALIFIER, CompareOp.GREATER, new LongComparator(100L));
|
||||||
|
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||||
|
Bytes.toBytes(1L));
|
||||||
|
assertTrue("less than", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||||
|
filter.reset();
|
||||||
|
|
||||||
|
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||||
|
Bytes.toBytes(100L));
|
||||||
|
assertTrue("Equals 100", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||||
|
filter.reset();
|
||||||
|
|
||||||
|
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||||
|
Bytes.toBytes(120L));
|
||||||
|
assertTrue("include 120", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||||
|
}
|
||||||
|
|
||||||
private void basicFilterTests(SingleColumnValueFilter filter)
|
private void basicFilterTests(SingleColumnValueFilter filter)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
||||||
|
|
Loading…
Reference in New Issue