HBASE-6509 Implement fast-forwarding FuzzyRowFilter to allow filtering rows e.g. by "???alex?b" (Alex Baranau)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1370788 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
5d1007ef3f
commit
5876b5b662
|
@ -0,0 +1,294 @@
|
|||
/**
|
||||
* 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 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.KeyValue;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
/**
|
||||
* 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
|
||||
* a byte array with 0 or 1 as its values:
|
||||
* <ul>
|
||||
* <li>
|
||||
* 0 - means that this byte in provided row key is fixed, i.e. row key's byte at same position
|
||||
* must match
|
||||
* </li>
|
||||
* <li>
|
||||
* 1 - means that this byte in provided row key is NOT fixed, i.e. row key's byte at this
|
||||
* position can be different from the one in provided row key
|
||||
* </li>
|
||||
* </ul>
|
||||
*
|
||||
*
|
||||
* Example:
|
||||
* Let's assume row key format is userId_actionId_year_month. Length of userId is fixed
|
||||
* and is 4, length of actionId is 2 and year and month are 4 and 2 bytes long respectively.
|
||||
*
|
||||
* Let's assume that we need to fetch all users that performed certain action (encoded as "99")
|
||||
* in Jan of any year. Then the pair (row key, fuzzy info) would be the following:
|
||||
* row key = "????_99_????_01" (one can use any value instead of "?")
|
||||
* fuzzy info = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00"
|
||||
*
|
||||
* I.e. fuzzy info tells the matching mask is "????_99_????_01", where at ? can be any value.
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
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;
|
||||
}
|
||||
|
||||
// TODO: possible improvement: save which fuzzy row key to use when providing a hint
|
||||
@Override
|
||||
public ReturnCode filterKeyValue(KeyValue kv) {
|
||||
byte[] rowKey = kv.getRow();
|
||||
// assigning "worst" result first and looking for better options
|
||||
SatisfiesCode bestOption = SatisfiesCode.NO_NEXT;
|
||||
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
|
||||
SatisfiesCode satisfiesCode =
|
||||
satisfies(rowKey, fuzzyData.getFirst(), fuzzyData.getSecond());
|
||||
if (satisfiesCode == SatisfiesCode.YES) {
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
if (satisfiesCode == SatisfiesCode.NEXT_EXISTS) {
|
||||
bestOption = SatisfiesCode.NEXT_EXISTS;
|
||||
}
|
||||
}
|
||||
|
||||
if (bestOption == SatisfiesCode.NEXT_EXISTS) {
|
||||
return ReturnCode.SEEK_NEXT_USING_HINT;
|
||||
}
|
||||
|
||||
// the only unhandled SatisfiesCode is NO_NEXT, i.e. we are done
|
||||
done = true;
|
||||
return ReturnCode.NEXT_ROW;
|
||||
}
|
||||
|
||||
@Override
|
||||
public KeyValue getNextKeyHint(KeyValue currentKV) {
|
||||
byte[] rowKey = currentKV.getRow();
|
||||
byte[] nextRowKey = null;
|
||||
// Searching for the "smallest" row key that satisfies at least one fuzzy row key
|
||||
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
|
||||
byte[] nextRowKeyCandidate = getNextForFuzzyRule(rowKey,
|
||||
fuzzyData.getFirst(), fuzzyData.getSecond());
|
||||
if (nextRowKeyCandidate == null) {
|
||||
continue;
|
||||
}
|
||||
if (nextRowKey == null || Bytes.compareTo(nextRowKeyCandidate, nextRowKey) < 0) {
|
||||
nextRowKey = nextRowKeyCandidate;
|
||||
}
|
||||
}
|
||||
|
||||
if (nextRowKey == null) {
|
||||
// SHOULD NEVER happen
|
||||
// TODO: is there a better way than throw exception? (stop the scanner?)
|
||||
throw new IllegalStateException("No next row key that satisfies fuzzy exists when" +
|
||||
" getNextKeyHint() is invoked." +
|
||||
" Filter: " + this.toString() +
|
||||
" currentKV: " + currentKV.toString());
|
||||
}
|
||||
|
||||
return KeyValue.createFirstOnRow(nextRowKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean filterAllRemaining() {
|
||||
return done;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput dataOutput) throws IOException {
|
||||
dataOutput.writeInt(this.fuzzyKeysData.size());
|
||||
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
|
||||
Bytes.writeByteArray(dataOutput, fuzzyData.getFirst());
|
||||
Bytes.writeByteArray(dataOutput, fuzzyData.getSecond());
|
||||
}
|
||||
}
|
||||
|
||||
@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));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder();
|
||||
sb.append("FuzzyRowFilter");
|
||||
sb.append("{fuzzyKeysData=");
|
||||
for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
|
||||
sb.append('{').append(Bytes.toStringBinary(fuzzyData.getFirst())).append(":");
|
||||
sb.append(Bytes.toStringBinary(fuzzyData.getSecond())).append('}');
|
||||
}
|
||||
sb.append("}, ");
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
// Utility methods
|
||||
|
||||
static enum SatisfiesCode {
|
||||
// row satisfies fuzzy rule
|
||||
YES,
|
||||
// row doesn't satisfy fuzzy rule, but there's possible greater row that does
|
||||
NEXT_EXISTS,
|
||||
// row doesn't satisfy fuzzy rule and there's no greater row that does
|
||||
NO_NEXT
|
||||
}
|
||||
|
||||
static SatisfiesCode satisfies(byte[] row,
|
||||
byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
|
||||
return satisfies(row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
|
||||
}
|
||||
|
||||
private static SatisfiesCode satisfies(byte[] row, int offset, int length,
|
||||
byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
|
||||
if (row == null) {
|
||||
// do nothing, let scan to proceed
|
||||
return SatisfiesCode.YES;
|
||||
}
|
||||
|
||||
boolean nextRowKeyCandidateExists = false;
|
||||
|
||||
for (int i = 0; i < fuzzyKeyMeta.length && i < length; i++) {
|
||||
// First, checking if this position is fixed and not equals the given one
|
||||
boolean byteAtPositionFixed = fuzzyKeyMeta[i] == 0;
|
||||
boolean fixedByteIncorrect = byteAtPositionFixed && fuzzyKeyBytes[i] != row[i + offset];
|
||||
if (fixedByteIncorrect) {
|
||||
// in this case there's another row that satisfies fuzzy rule and bigger than this row
|
||||
if (nextRowKeyCandidateExists) {
|
||||
return SatisfiesCode.NEXT_EXISTS;
|
||||
}
|
||||
|
||||
// If this row byte is less than fixed then there's a byte array bigger than
|
||||
// this row and which satisfies the fuzzy rule. Otherwise there's no such byte array:
|
||||
// this row is simply bigger than any byte array that satisfies the fuzzy rule
|
||||
boolean rowByteLessThanFixed = (row[i + offset] & 0xFF) < (fuzzyKeyBytes[i] & 0xFF);
|
||||
return rowByteLessThanFixed ? SatisfiesCode.NEXT_EXISTS : SatisfiesCode.NO_NEXT;
|
||||
}
|
||||
|
||||
// Second, checking if this position is not fixed and byte value is not the biggest. In this
|
||||
// case there's a byte array bigger than this row and which satisfies the fuzzy rule. To get
|
||||
// bigger byte array that satisfies the rule we need to just increase this byte
|
||||
// (see the code of getNextForFuzzyRule below) by one.
|
||||
// Note: if non-fixed byte is already at biggest value, this doesn't allow us to say there's
|
||||
// bigger one that satisfies the rule as it can't be increased.
|
||||
if (fuzzyKeyMeta[i] == 1 && !isMax(fuzzyKeyBytes[i])) {
|
||||
nextRowKeyCandidateExists = true;
|
||||
}
|
||||
}
|
||||
|
||||
return SatisfiesCode.YES;
|
||||
}
|
||||
|
||||
private static boolean isMax(byte fuzzyKeyByte) {
|
||||
return (fuzzyKeyByte & 0xFF) == 255;
|
||||
}
|
||||
|
||||
static byte[] getNextForFuzzyRule(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
|
||||
return getNextForFuzzyRule(row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return greater byte array than given (row) which satisfies the fuzzy rule if it exists,
|
||||
* null otherwise
|
||||
*/
|
||||
private static byte[] getNextForFuzzyRule(byte[] row, int offset, int length,
|
||||
byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
|
||||
// To find out the next "smallest" byte array that satisfies fuzzy rule and "greater" than
|
||||
// the given one we do the following:
|
||||
// 1. setting values on all "fixed" positions to the values from fuzzyKeyBytes
|
||||
// 2. if during the first step given row did not increase, then we increase the value at
|
||||
// the first "non-fixed" position (where it is not maximum already)
|
||||
|
||||
// It is easier to perform this by using fuzzyKeyBytes copy and setting "non-fixed" position
|
||||
// values than otherwise.
|
||||
byte[] result = Arrays.copyOf(fuzzyKeyBytes,
|
||||
length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length);
|
||||
int toInc = -1;
|
||||
|
||||
boolean increased = false;
|
||||
for (int i = 0; i < result.length; i++) {
|
||||
if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) {
|
||||
result[i] = row[offset + i];
|
||||
if (!isMax(row[i])) {
|
||||
// this is "non-fixed" position and is not at max value, hence we can increase it
|
||||
toInc = i;
|
||||
}
|
||||
} else if (i < fuzzyKeyMeta.length && fuzzyKeyMeta[i] == 0) {
|
||||
if ((row[i + offset] & 0xFF) < (fuzzyKeyBytes[i] & 0xFF)) {
|
||||
// if setting value for any fixed position increased the original array,
|
||||
// we are OK
|
||||
increased = true;
|
||||
break;
|
||||
}
|
||||
if ((row[i + offset] & 0xFF) > (fuzzyKeyBytes[i] & 0xFF)) {
|
||||
// if setting value for any fixed position makes array "smaller", then just stop:
|
||||
// in case we found some non-fixed position to increase we will do it, otherwise
|
||||
// there's no "next" row key that satisfies fuzzy rule and "greater" than given row
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!increased) {
|
||||
if (toInc < 0) {
|
||||
return null;
|
||||
}
|
||||
result[toInc]++;
|
||||
|
||||
// Setting all "non-fixed" positions to zeroes to the right of the one we increased so
|
||||
// that found "next" row key is the smallest possible
|
||||
for (int i = toInc + 1; i < result.length; i++) {
|
||||
if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) {
|
||||
result[i] = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -10044,6 +10044,465 @@ public final class HBaseProtos {
|
|||
// @@protoc_insertion_point(class_scope:NameBytesPair)
|
||||
}
|
||||
|
||||
public interface BytesBytesPairOrBuilder
|
||||
extends com.google.protobuf.MessageOrBuilder {
|
||||
|
||||
// required bytes first = 1;
|
||||
boolean hasFirst();
|
||||
com.google.protobuf.ByteString getFirst();
|
||||
|
||||
// required bytes second = 2;
|
||||
boolean hasSecond();
|
||||
com.google.protobuf.ByteString getSecond();
|
||||
}
|
||||
public static final class BytesBytesPair extends
|
||||
com.google.protobuf.GeneratedMessage
|
||||
implements BytesBytesPairOrBuilder {
|
||||
// Use BytesBytesPair.newBuilder() to construct.
|
||||
private BytesBytesPair(Builder builder) {
|
||||
super(builder);
|
||||
}
|
||||
private BytesBytesPair(boolean noInit) {}
|
||||
|
||||
private static final BytesBytesPair defaultInstance;
|
||||
public static BytesBytesPair getDefaultInstance() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public BytesBytesPair getDefaultInstanceForType() {
|
||||
return defaultInstance;
|
||||
}
|
||||
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_BytesBytesPair_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_BytesBytesPair_fieldAccessorTable;
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
// required bytes first = 1;
|
||||
public static final int FIRST_FIELD_NUMBER = 1;
|
||||
private com.google.protobuf.ByteString first_;
|
||||
public boolean hasFirst() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public com.google.protobuf.ByteString getFirst() {
|
||||
return first_;
|
||||
}
|
||||
|
||||
// required bytes second = 2;
|
||||
public static final int SECOND_FIELD_NUMBER = 2;
|
||||
private com.google.protobuf.ByteString second_;
|
||||
public boolean hasSecond() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
public com.google.protobuf.ByteString getSecond() {
|
||||
return second_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
first_ = com.google.protobuf.ByteString.EMPTY;
|
||||
second_ = com.google.protobuf.ByteString.EMPTY;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
byte isInitialized = memoizedIsInitialized;
|
||||
if (isInitialized != -1) return isInitialized == 1;
|
||||
|
||||
if (!hasFirst()) {
|
||||
memoizedIsInitialized = 0;
|
||||
return false;
|
||||
}
|
||||
if (!hasSecond()) {
|
||||
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, first_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
output.writeBytes(2, second_);
|
||||
}
|
||||
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, first_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBytesSize(2, second_);
|
||||
}
|
||||
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.BytesBytesPair)) {
|
||||
return super.equals(obj);
|
||||
}
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair other = (org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair) obj;
|
||||
|
||||
boolean result = true;
|
||||
result = result && (hasFirst() == other.hasFirst());
|
||||
if (hasFirst()) {
|
||||
result = result && getFirst()
|
||||
.equals(other.getFirst());
|
||||
}
|
||||
result = result && (hasSecond() == other.hasSecond());
|
||||
if (hasSecond()) {
|
||||
result = result && getSecond()
|
||||
.equals(other.getSecond());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
}
|
||||
|
||||
@java.lang.Override
|
||||
public int hashCode() {
|
||||
int hash = 41;
|
||||
hash = (19 * hash) + getDescriptorForType().hashCode();
|
||||
if (hasFirst()) {
|
||||
hash = (37 * hash) + FIRST_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getFirst().hashCode();
|
||||
}
|
||||
if (hasSecond()) {
|
||||
hash = (37 * hash) + SECOND_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getSecond().hashCode();
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
return hash;
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair 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.BytesBytesPair 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.BytesBytesPair parseFrom(byte[] data)
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
return newBuilder().mergeFrom(data).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair 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.BytesBytesPair parseFrom(java.io.InputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair 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.BytesBytesPair 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.BytesBytesPair 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.BytesBytesPair parseFrom(
|
||||
com.google.protobuf.CodedInputStream input)
|
||||
throws java.io.IOException {
|
||||
return newBuilder().mergeFrom(input).buildParsed();
|
||||
}
|
||||
public static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair 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.BytesBytesPair 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.BytesBytesPairOrBuilder {
|
||||
public static final com.google.protobuf.Descriptors.Descriptor
|
||||
getDescriptor() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_BytesBytesPair_descriptor;
|
||||
}
|
||||
|
||||
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internalGetFieldAccessorTable() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.internal_static_BytesBytesPair_fieldAccessorTable;
|
||||
}
|
||||
|
||||
// Construct using org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.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();
|
||||
first_ = com.google.protobuf.ByteString.EMPTY;
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
second_ = 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.BytesBytesPair.getDescriptor();
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getDefaultInstanceForType() {
|
||||
return org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.getDefaultInstance();
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair build() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair buildParsed()
|
||||
throws com.google.protobuf.InvalidProtocolBufferException {
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair result = buildPartial();
|
||||
if (!result.isInitialized()) {
|
||||
throw newUninitializedMessageException(
|
||||
result).asInvalidProtocolBufferException();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair buildPartial() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair result = new org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair(this);
|
||||
int from_bitField0_ = bitField0_;
|
||||
int to_bitField0_ = 0;
|
||||
if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
to_bitField0_ |= 0x00000001;
|
||||
}
|
||||
result.first_ = first_;
|
||||
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
to_bitField0_ |= 0x00000002;
|
||||
}
|
||||
result.second_ = second_;
|
||||
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.BytesBytesPair) {
|
||||
return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair)other);
|
||||
} else {
|
||||
super.mergeFrom(other);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair other) {
|
||||
if (other == org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.getDefaultInstance()) return this;
|
||||
if (other.hasFirst()) {
|
||||
setFirst(other.getFirst());
|
||||
}
|
||||
if (other.hasSecond()) {
|
||||
setSecond(other.getSecond());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
||||
public final boolean isInitialized() {
|
||||
if (!hasFirst()) {
|
||||
|
||||
return false;
|
||||
}
|
||||
if (!hasSecond()) {
|
||||
|
||||
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;
|
||||
first_ = input.readBytes();
|
||||
break;
|
||||
}
|
||||
case 18: {
|
||||
bitField0_ |= 0x00000002;
|
||||
second_ = input.readBytes();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
|
||||
// required bytes first = 1;
|
||||
private com.google.protobuf.ByteString first_ = com.google.protobuf.ByteString.EMPTY;
|
||||
public boolean hasFirst() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
public com.google.protobuf.ByteString getFirst() {
|
||||
return first_;
|
||||
}
|
||||
public Builder setFirst(com.google.protobuf.ByteString value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000001;
|
||||
first_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public Builder clearFirst() {
|
||||
bitField0_ = (bitField0_ & ~0x00000001);
|
||||
first_ = getDefaultInstance().getFirst();
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// required bytes second = 2;
|
||||
private com.google.protobuf.ByteString second_ = com.google.protobuf.ByteString.EMPTY;
|
||||
public boolean hasSecond() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
public com.google.protobuf.ByteString getSecond() {
|
||||
return second_;
|
||||
}
|
||||
public Builder setSecond(com.google.protobuf.ByteString value) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
bitField0_ |= 0x00000002;
|
||||
second_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
public Builder clearSecond() {
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
second_ = getDefaultInstance().getSecond();
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:BytesBytesPair)
|
||||
}
|
||||
|
||||
static {
|
||||
defaultInstance = new BytesBytesPair(true);
|
||||
defaultInstance.initFields();
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(class_scope:BytesBytesPair)
|
||||
}
|
||||
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_TableSchema_descriptor;
|
||||
private static
|
||||
|
@ -10114,6 +10573,11 @@ public final class HBaseProtos {
|
|||
private static
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_NameBytesPair_fieldAccessorTable;
|
||||
private static com.google.protobuf.Descriptors.Descriptor
|
||||
internal_static_BytesBytesPair_descriptor;
|
||||
private static
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable
|
||||
internal_static_BytesBytesPair_fieldAccessorTable;
|
||||
|
||||
public static com.google.protobuf.Descriptors.FileDescriptor
|
||||
getDescriptor() {
|
||||
|
@ -10162,14 +10626,15 @@ public final class HBaseProtos {
|
|||
"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*r\n\013Compa",
|
||||
"reType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005E" +
|
||||
"QUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUA" +
|
||||
"L\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_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\rDELE" +
|
||||
"TE_COLUMN\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIMU" +
|
||||
"M\020\377\001B>\n*org.apache.hadoop.hbase.protobuf" +
|
||||
".generatedB\013HBaseProtosH\001\240\001\001"
|
||||
"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"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -10288,6 +10753,14 @@ public final class HBaseProtos {
|
|||
new java.lang.String[] { "Name", "Value", },
|
||||
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);
|
||||
internal_static_BytesBytesPair_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_BytesBytesPair_descriptor,
|
||||
new java.lang.String[] { "First", "Second", },
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.class,
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder.class);
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -68,6 +68,10 @@ message FirstKeyOnlyFilter {
|
|||
message FirstKeyValueMatchingQualifiersFilter {
|
||||
}
|
||||
|
||||
message FuzzyRowFilter {
|
||||
repeated BytesBytesPair fuzzyKeysData = 1;
|
||||
}
|
||||
|
||||
message InclusiveStopFilter {
|
||||
required bytes stopRowKey = 1;
|
||||
}
|
||||
|
|
|
@ -248,3 +248,8 @@ message NameBytesPair {
|
|||
required string name = 1;
|
||||
optional bytes value = 2;
|
||||
}
|
||||
|
||||
message BytesBytesPair {
|
||||
required bytes first = 1;
|
||||
required bytes second = 2;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,204 @@
|
|||
/**
|
||||
* 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.SmallTests;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestFuzzyRowFilter {
|
||||
@Test
|
||||
public void testSatisfies() {
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS,
|
||||
FuzzyRowFilter.satisfies(new byte[]{1, (byte) -128, 0, 0, 1}, // row to check
|
||||
new byte[]{1, 0, 1}, // fuzzy row
|
||||
new byte[]{0, 1, 0})); // mask
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.YES,
|
||||
FuzzyRowFilter.satisfies(new byte[]{1, (byte) -128, 1, 0, 1},
|
||||
new byte[]{1, 0, 1},
|
||||
new byte[]{0, 1, 0}));
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS,
|
||||
FuzzyRowFilter.satisfies(new byte[]{1, (byte) -128, 2, 0, 1},
|
||||
new byte[]{1, 0, 1},
|
||||
new byte[]{0, 1, 0}));
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NO_NEXT,
|
||||
FuzzyRowFilter.satisfies(new byte[]{2, 3, 1, 1, 1},
|
||||
new byte[]{1, 0, 1},
|
||||
new byte[]{0, 1, 0}));
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.YES,
|
||||
FuzzyRowFilter.satisfies(new byte[]{1, 2, 1, 3, 3},
|
||||
new byte[]{1, 2, 0, 3},
|
||||
new byte[]{0, 0, 1, 0}));
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS,
|
||||
FuzzyRowFilter.satisfies(new byte[]{1, 1, 1, 3, 0}, // row to check
|
||||
new byte[]{1, 2, 0, 3}, // fuzzy row
|
||||
new byte[]{0, 0, 1, 0})); // mask
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS,
|
||||
FuzzyRowFilter.satisfies(new byte[]{1, 1, 1, 3, 0},
|
||||
new byte[]{1, (byte) 245, 0, 3},
|
||||
new byte[]{0, 0, 1, 0}));
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NO_NEXT,
|
||||
FuzzyRowFilter.satisfies(new byte[]{1, (byte) 245, 1, 3, 0},
|
||||
new byte[]{1, 1, 0, 3},
|
||||
new byte[]{0, 0, 1, 0}));
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NO_NEXT,
|
||||
FuzzyRowFilter.satisfies(new byte[]{1, 3, 1, 3, 0},
|
||||
new byte[]{1, 2, 0, 3},
|
||||
new byte[]{0, 0, 1, 0}));
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NO_NEXT,
|
||||
FuzzyRowFilter.satisfies(new byte[]{2, 1, 1, 1, 0},
|
||||
new byte[]{1, 2, 0, 3},
|
||||
new byte[]{0, 0, 1, 0}));
|
||||
|
||||
Assert.assertEquals(FuzzyRowFilter.SatisfiesCode.NEXT_EXISTS,
|
||||
FuzzyRowFilter.satisfies(new byte[]{1, 2, 1, 0, 1},
|
||||
new byte[]{0, 1, 2},
|
||||
new byte[]{1, 0, 0}));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetNextForFuzzyRule() {
|
||||
assertNext(
|
||||
new byte[]{0, 1, 2}, // fuzzy row
|
||||
new byte[]{1, 0, 0}, // mask
|
||||
new byte[]{1, 2, 1, 0, 1}, // current
|
||||
new byte[]{2, 1, 2, 0, 0}); // expected next
|
||||
|
||||
assertNext(
|
||||
new byte[]{0, 1, 2}, // fuzzy row
|
||||
new byte[]{1, 0, 0}, // mask
|
||||
new byte[]{1, 1, 2, 0, 1}, // current
|
||||
new byte[]{1, 1, 2, 0, 2}); // expected next
|
||||
|
||||
assertNext(
|
||||
new byte[]{0, 1, 0, 2, 0}, // fuzzy row
|
||||
new byte[]{1, 0, 1, 0, 1}, // mask
|
||||
new byte[]{1, 0, 2, 0, 1}, // current
|
||||
new byte[]{1, 1, 0, 2, 0}); // expected next
|
||||
|
||||
assertNext(
|
||||
new byte[]{1, 0, 1},
|
||||
new byte[]{0, 1, 0},
|
||||
new byte[]{1, (byte) 128, 2, 0, 1},
|
||||
new byte[]{1, (byte) 129, 1, 0, 0});
|
||||
|
||||
assertNext(
|
||||
new byte[]{0, 1, 0, 1},
|
||||
new byte[]{1, 0, 1, 0},
|
||||
new byte[]{5, 1, 0, 1},
|
||||
new byte[]{5, 1, 1, 1});
|
||||
|
||||
assertNext(
|
||||
new byte[]{0, 1, 0, 1},
|
||||
new byte[]{1, 0, 1, 0},
|
||||
new byte[]{5, 1, 0, 1, 1},
|
||||
new byte[]{5, 1, 0, 1, 2});
|
||||
|
||||
assertNext(
|
||||
new byte[]{0, 1, 0, 0}, // fuzzy row
|
||||
new byte[]{1, 0, 1, 1}, // mask
|
||||
new byte[]{5, 1, (byte) 255, 1}, // current
|
||||
new byte[]{5, 1, (byte) 255, 2}); // expected next
|
||||
|
||||
assertNext(
|
||||
new byte[]{0, 1, 0, 1}, // fuzzy row
|
||||
new byte[]{1, 0, 1, 0}, // mask
|
||||
new byte[]{5, 1, (byte) 255, 1}, // current
|
||||
new byte[]{6, 1, 0, 1}); // expected next
|
||||
|
||||
assertNext(
|
||||
new byte[]{0, 1, 0, 1}, // fuzzy row
|
||||
new byte[]{1, 0, 1, 0}, // mask
|
||||
new byte[]{5, 1, (byte) 255, 0}, // current
|
||||
new byte[]{5, 1, (byte) 255, 1}); // expected next
|
||||
|
||||
assertNext(
|
||||
new byte[]{5, 1, 1, 0},
|
||||
new byte[]{0, 0, 1, 1},
|
||||
new byte[]{5, 1, (byte) 255, 1},
|
||||
new byte[]{5, 1, (byte) 255, 2});
|
||||
|
||||
assertNext(
|
||||
new byte[]{1, 1, 1, 1},
|
||||
new byte[]{0, 0, 1, 1},
|
||||
new byte[]{1, 1, 2, 2},
|
||||
new byte[]{1, 1, 2, 3});
|
||||
|
||||
assertNext(
|
||||
new byte[]{1, 1, 1, 1},
|
||||
new byte[]{0, 0, 1, 1},
|
||||
new byte[]{1, 1, 3, 2},
|
||||
new byte[]{1, 1, 3, 3});
|
||||
|
||||
assertNext(
|
||||
new byte[]{1, 1, 1, 1},
|
||||
new byte[]{1, 1, 1, 1},
|
||||
new byte[]{1, 1, 2, 3},
|
||||
new byte[]{1, 1, 2, 4});
|
||||
|
||||
assertNext(
|
||||
new byte[]{1, 1, 1, 1},
|
||||
new byte[]{1, 1, 1, 1},
|
||||
new byte[]{1, 1, 3, 2},
|
||||
new byte[]{1, 1, 3, 3});
|
||||
|
||||
assertNext(
|
||||
new byte[]{1, 1, 0, 0},
|
||||
new byte[]{0, 0, 1, 1},
|
||||
new byte[]{0, 1, 3, 2},
|
||||
new byte[]{1, 1, 0, 0});
|
||||
|
||||
// No next for this one
|
||||
Assert.assertNull(FuzzyRowFilter.getNextForFuzzyRule(
|
||||
new byte[]{2, 3, 1, 1, 1}, // row to check
|
||||
new byte[]{1, 0, 1}, // fuzzy row
|
||||
new byte[]{0, 1, 0})); // mask
|
||||
Assert.assertNull(FuzzyRowFilter.getNextForFuzzyRule(
|
||||
new byte[]{1, (byte) 245, 1, 3, 0},
|
||||
new byte[]{1, 1, 0, 3},
|
||||
new byte[]{0, 0, 1, 0}));
|
||||
Assert.assertNull(FuzzyRowFilter.getNextForFuzzyRule(
|
||||
new byte[]{1, 3, 1, 3, 0},
|
||||
new byte[]{1, 2, 0, 3},
|
||||
new byte[]{0, 0, 1, 0}));
|
||||
Assert.assertNull(FuzzyRowFilter.getNextForFuzzyRule(
|
||||
new byte[]{2, 1, 1, 1, 0},
|
||||
new byte[]{1, 2, 0, 3},
|
||||
new byte[]{0, 0, 1, 0}));
|
||||
}
|
||||
|
||||
private void assertNext(byte[] fuzzyRow, byte[] mask, byte[] current, byte[] expected) {
|
||||
byte[] nextForFuzzyRule = FuzzyRowFilter.getNextForFuzzyRule(current, fuzzyRow, mask);
|
||||
Assert.assertArrayEquals(expected, nextForFuzzyRule);
|
||||
}
|
||||
|
||||
@org.junit.Rule
|
||||
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
|
||||
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
|
||||
}
|
Loading…
Reference in New Issue