From 5876b5b662e83149bd1d726cdb7a1ac0310cc25c Mon Sep 17 00:00:00 2001 From: Zhihong Yu Date: Wed, 8 Aug 2012 14:37:34 +0000 Subject: [PATCH] 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 --- .../hadoop/hbase/filter/FuzzyRowFilter.java | 294 +++++ .../protobuf/generated/FilterProtos.java | 1001 ++++++++++++++++- .../hbase/protobuf/generated/HBaseProtos.java | 489 +++++++- hbase-server/src/main/protobuf/Filter.proto | 4 + hbase-server/src/main/protobuf/hbase.proto | 5 + .../hbase/filter/TestFuzzyRowFilter.java | 204 ++++ 6 files changed, 1949 insertions(+), 48 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java new file mode 100644 index 00000000000..5ade79f7fbf --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java @@ -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: + * + * + * + * 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> fuzzyKeysData; + private boolean done = false; + + /** + * Used internally for reflection, do NOT use it directly + */ + public FuzzyRowFilter() { + } + + public FuzzyRowFilter(List> 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 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 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 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>(count); + for (int i = 0; i < count; i++) { + byte[] keyBytes = Bytes.readByteArray(dataInput); + byte[] keyMeta = Bytes.readByteArray(dataInput); + this.fuzzyKeysData.add(new Pair(keyBytes, keyMeta)); + } + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(); + sb.append("FuzzyRowFilter"); + sb.append("{fuzzyKeysData="); + for (Pair 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; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java index 4c8d177b1a9..520f1d73950 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java @@ -1,5 +1,5 @@ // Generated by the protocol buffer compiler. DO NOT EDIT! -// source: Filter.proto +// source: filter.proto package org.apache.hadoop.hbase.protobuf.generated; @@ -3568,6 +3568,899 @@ public final class FilterProtos { // @@protoc_insertion_point(class_scope:FirstKeyOnlyFilter) } + public interface FirstKeyValueMatchingQualifiersFilterOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + public static final class FirstKeyValueMatchingQualifiersFilter extends + com.google.protobuf.GeneratedMessage + implements FirstKeyValueMatchingQualifiersFilterOrBuilder { + // Use FirstKeyValueMatchingQualifiersFilter.newBuilder() to construct. + private FirstKeyValueMatchingQualifiersFilter(Builder builder) { + super(builder); + } + private FirstKeyValueMatchingQualifiersFilter(boolean noInit) {} + + private static final FirstKeyValueMatchingQualifiersFilter defaultInstance; + public static FirstKeyValueMatchingQualifiersFilter getDefaultInstance() { + return defaultInstance; + } + + public FirstKeyValueMatchingQualifiersFilter getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FirstKeyValueMatchingQualifiersFilter_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FirstKeyValueMatchingQualifiersFilter_fieldAccessorTable; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + 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.FilterProtos.FirstKeyValueMatchingQualifiersFilter)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter other = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter 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.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter 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.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter 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.FilterProtos.FirstKeyValueMatchingQualifiersFilter 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.FilterProtos.FirstKeyValueMatchingQualifiersFilter 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.FilterProtos.FirstKeyValueMatchingQualifiersFilter parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter 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.FilterProtos.FirstKeyValueMatchingQualifiersFilter 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 + implements org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilterOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FirstKeyValueMatchingQualifiersFilter_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FirstKeyValueMatchingQualifiersFilter_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter.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(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter build() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter result = new org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + 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; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:FirstKeyValueMatchingQualifiersFilter) + } + + static { + defaultInstance = new FirstKeyValueMatchingQualifiersFilter(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:FirstKeyValueMatchingQualifiersFilter) + } + + public interface FuzzyRowFilterOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // repeated .BytesBytesPair fuzzyKeysData = 1; + java.util.List + getFuzzyKeysDataList(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getFuzzyKeysData(int index); + int getFuzzyKeysDataCount(); + java.util.List + getFuzzyKeysDataOrBuilderList(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getFuzzyKeysDataOrBuilder( + int index); + } + public static final class FuzzyRowFilter extends + com.google.protobuf.GeneratedMessage + implements FuzzyRowFilterOrBuilder { + // Use FuzzyRowFilter.newBuilder() to construct. + private FuzzyRowFilter(Builder builder) { + super(builder); + } + private FuzzyRowFilter(boolean noInit) {} + + private static final FuzzyRowFilter defaultInstance; + public static FuzzyRowFilter getDefaultInstance() { + return defaultInstance; + } + + public FuzzyRowFilter getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FuzzyRowFilter_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FuzzyRowFilter_fieldAccessorTable; + } + + // repeated .BytesBytesPair fuzzyKeysData = 1; + public static final int FUZZYKEYSDATA_FIELD_NUMBER = 1; + private java.util.List fuzzyKeysData_; + public java.util.List getFuzzyKeysDataList() { + return fuzzyKeysData_; + } + public java.util.List + getFuzzyKeysDataOrBuilderList() { + return fuzzyKeysData_; + } + public int getFuzzyKeysDataCount() { + return fuzzyKeysData_.size(); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getFuzzyKeysData(int index) { + return fuzzyKeysData_.get(index); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getFuzzyKeysDataOrBuilder( + int index) { + return fuzzyKeysData_.get(index); + } + + private void initFields() { + fuzzyKeysData_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + for (int i = 0; i < getFuzzyKeysDataCount(); i++) { + if (!getFuzzyKeysData(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + for (int i = 0; i < fuzzyKeysData_.size(); i++) { + output.writeMessage(1, fuzzyKeysData_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < fuzzyKeysData_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, fuzzyKeysData_.get(i)); + } + 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.FilterProtos.FuzzyRowFilter)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter other = (org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter) obj; + + boolean result = true; + result = result && getFuzzyKeysDataList() + .equals(other.getFuzzyKeysDataList()); + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (getFuzzyKeysDataCount() > 0) { + hash = (37 * hash) + FUZZYKEYSDATA_FIELD_NUMBER; + hash = (53 * hash) + getFuzzyKeysDataList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter 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.FilterProtos.FuzzyRowFilter parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter 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.FilterProtos.FuzzyRowFilter parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter 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.FilterProtos.FuzzyRowFilter 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.FilterProtos.FuzzyRowFilter 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.FilterProtos.FuzzyRowFilter parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter 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.FilterProtos.FuzzyRowFilter 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 + implements org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilterOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FuzzyRowFilter_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.internal_static_FuzzyRowFilter_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getFuzzyKeysDataFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (fuzzyKeysDataBuilder_ == null) { + fuzzyKeysData_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + fuzzyKeysDataBuilder_.clear(); + } + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter build() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter result = new org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter(this); + int from_bitField0_ = bitField0_; + if (fuzzyKeysDataBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + fuzzyKeysData_ = java.util.Collections.unmodifiableList(fuzzyKeysData_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.fuzzyKeysData_ = fuzzyKeysData_; + } else { + result.fuzzyKeysData_ = fuzzyKeysDataBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter.getDefaultInstance()) return this; + if (fuzzyKeysDataBuilder_ == null) { + if (!other.fuzzyKeysData_.isEmpty()) { + if (fuzzyKeysData_.isEmpty()) { + fuzzyKeysData_ = other.fuzzyKeysData_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensureFuzzyKeysDataIsMutable(); + fuzzyKeysData_.addAll(other.fuzzyKeysData_); + } + onChanged(); + } + } else { + if (!other.fuzzyKeysData_.isEmpty()) { + if (fuzzyKeysDataBuilder_.isEmpty()) { + fuzzyKeysDataBuilder_.dispose(); + fuzzyKeysDataBuilder_ = null; + fuzzyKeysData_ = other.fuzzyKeysData_; + bitField0_ = (bitField0_ & ~0x00000001); + fuzzyKeysDataBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getFuzzyKeysDataFieldBuilder() : null; + } else { + fuzzyKeysDataBuilder_.addAllMessages(other.fuzzyKeysData_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getFuzzyKeysDataCount(); i++) { + if (!getFuzzyKeysData(i).isInitialized()) { + + 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: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addFuzzyKeysData(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // repeated .BytesBytesPair fuzzyKeysData = 1; + private java.util.List fuzzyKeysData_ = + java.util.Collections.emptyList(); + private void ensureFuzzyKeysDataIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + fuzzyKeysData_ = new java.util.ArrayList(fuzzyKeysData_); + bitField0_ |= 0x00000001; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> fuzzyKeysDataBuilder_; + + public java.util.List getFuzzyKeysDataList() { + if (fuzzyKeysDataBuilder_ == null) { + return java.util.Collections.unmodifiableList(fuzzyKeysData_); + } else { + return fuzzyKeysDataBuilder_.getMessageList(); + } + } + public int getFuzzyKeysDataCount() { + if (fuzzyKeysDataBuilder_ == null) { + return fuzzyKeysData_.size(); + } else { + return fuzzyKeysDataBuilder_.getCount(); + } + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair getFuzzyKeysData(int index) { + if (fuzzyKeysDataBuilder_ == null) { + return fuzzyKeysData_.get(index); + } else { + return fuzzyKeysDataBuilder_.getMessage(index); + } + } + public Builder setFuzzyKeysData( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair value) { + if (fuzzyKeysDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureFuzzyKeysDataIsMutable(); + fuzzyKeysData_.set(index, value); + onChanged(); + } else { + fuzzyKeysDataBuilder_.setMessage(index, value); + } + return this; + } + public Builder setFuzzyKeysData( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) { + if (fuzzyKeysDataBuilder_ == null) { + ensureFuzzyKeysDataIsMutable(); + fuzzyKeysData_.set(index, builderForValue.build()); + onChanged(); + } else { + fuzzyKeysDataBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + public Builder addFuzzyKeysData(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair value) { + if (fuzzyKeysDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureFuzzyKeysDataIsMutable(); + fuzzyKeysData_.add(value); + onChanged(); + } else { + fuzzyKeysDataBuilder_.addMessage(value); + } + return this; + } + public Builder addFuzzyKeysData( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair value) { + if (fuzzyKeysDataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureFuzzyKeysDataIsMutable(); + fuzzyKeysData_.add(index, value); + onChanged(); + } else { + fuzzyKeysDataBuilder_.addMessage(index, value); + } + return this; + } + public Builder addFuzzyKeysData( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) { + if (fuzzyKeysDataBuilder_ == null) { + ensureFuzzyKeysDataIsMutable(); + fuzzyKeysData_.add(builderForValue.build()); + onChanged(); + } else { + fuzzyKeysDataBuilder_.addMessage(builderForValue.build()); + } + return this; + } + public Builder addFuzzyKeysData( + int index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder builderForValue) { + if (fuzzyKeysDataBuilder_ == null) { + ensureFuzzyKeysDataIsMutable(); + fuzzyKeysData_.add(index, builderForValue.build()); + onChanged(); + } else { + fuzzyKeysDataBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + public Builder addAllFuzzyKeysData( + java.lang.Iterable values) { + if (fuzzyKeysDataBuilder_ == null) { + ensureFuzzyKeysDataIsMutable(); + super.addAll(values, fuzzyKeysData_); + onChanged(); + } else { + fuzzyKeysDataBuilder_.addAllMessages(values); + } + return this; + } + public Builder clearFuzzyKeysData() { + if (fuzzyKeysDataBuilder_ == null) { + fuzzyKeysData_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + fuzzyKeysDataBuilder_.clear(); + } + return this; + } + public Builder removeFuzzyKeysData(int index) { + if (fuzzyKeysDataBuilder_ == null) { + ensureFuzzyKeysDataIsMutable(); + fuzzyKeysData_.remove(index); + onChanged(); + } else { + fuzzyKeysDataBuilder_.remove(index); + } + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder getFuzzyKeysDataBuilder( + int index) { + return getFuzzyKeysDataFieldBuilder().getBuilder(index); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder getFuzzyKeysDataOrBuilder( + int index) { + if (fuzzyKeysDataBuilder_ == null) { + return fuzzyKeysData_.get(index); } else { + return fuzzyKeysDataBuilder_.getMessageOrBuilder(index); + } + } + public java.util.List + getFuzzyKeysDataOrBuilderList() { + if (fuzzyKeysDataBuilder_ != null) { + return fuzzyKeysDataBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(fuzzyKeysData_); + } + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder addFuzzyKeysDataBuilder() { + return getFuzzyKeysDataFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.getDefaultInstance()); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder addFuzzyKeysDataBuilder( + int index) { + return getFuzzyKeysDataFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.getDefaultInstance()); + } + public java.util.List + getFuzzyKeysDataBuilderList() { + return getFuzzyKeysDataFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder> + getFuzzyKeysDataFieldBuilder() { + if (fuzzyKeysDataBuilder_ == null) { + fuzzyKeysDataBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPairOrBuilder>( + fuzzyKeysData_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + fuzzyKeysData_ = null; + } + return fuzzyKeysDataBuilder_; + } + + // @@protoc_insertion_point(builder_scope:FuzzyRowFilter) + } + + static { + defaultInstance = new FuzzyRowFilter(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:FuzzyRowFilter) + } + public interface InclusiveStopFilterOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -10059,6 +10952,16 @@ public final class FilterProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_FirstKeyOnlyFilter_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_FirstKeyValueMatchingQualifiersFilter_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_FirstKeyValueMatchingQualifiersFilter_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_FuzzyRowFilter_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_FuzzyRowFilter_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_InclusiveStopFilter_descriptor; private static @@ -10138,7 +11041,7 @@ public final class FilterProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\014Filter.proto\032\013hbase.proto\"%\n\024ColumnCou" + + "\n\014filter.proto\032\013hbase.proto\"%\n\024ColumnCou" + "ntGetFilter\022\r\n\005limit\030\001 \002(\005\"7\n\026ColumnPagi" + "nationFilter\022\r\n\005limit\030\001 \002(\005\022\016\n\006offset\030\002 " + "\001(\005\"$\n\022ColumnPrefixFilter\022\016\n\006prefix\030\001 \002(" + @@ -10151,30 +11054,32 @@ public final class FilterProtos { "umnQualifier\030\002 \002(\014\022\033\n\023dropDependentColum" + "n\030\003 \001(\010\"5\n\014FamilyFilter\022%\n\rcompareFilter" + "\030\001 \002(\0132\016.CompareFilter\"\024\n\022FirstKeyOnlyFi" + - "lter\")\n\023InclusiveStopFilter\022\022\n\nstopRowKe" + - "y\030\001 \002(\014\"!\n\rKeyOnlyFilter\022\020\n\010lenAsVal\030\001 \002" + - "(\010\"4\n\032MultipleColumnPrefixFilter\022\026\n\016sort" + - "edPrefixes\030\001 \003(\014\"\036\n\nPageFilter\022\020\n\010pageSi" + - "ze\030\001 \002(\003\"\036\n\014PrefixFilter\022\016\n\006prefix\030\001 \002(\014" + - "\"8\n\017QualifierFilter\022%\n\rcompareFilter\030\001 \002" + - "(\0132\016.CompareFilter\"!\n\017RandomRowFilter\022\016\n", - "\006chance\030\001 \002(\002\"2\n\tRowFilter\022%\n\rcompareFil" + - "ter\030\001 \002(\0132\016.CompareFilter\"[\n\036SingleColum" + - "nValueExcludeFilter\0229\n\027singleColumnValue" + - "Filter\030\001 \002(\0132\030.SingleColumnValueFilter\"\335" + - "\001\n\027SingleColumnValueFilter\022\024\n\014columnFami" + - "ly\030\001 \002(\014\022\027\n\017columnQualifier\030\002 \002(\014\022\037\n\tcom" + - "pareOp\030\003 \002(\0162\014.CompareType\022\022\n\ncomparator" + - "\030\004 \002(\014\022\023\n\013foundColumn\030\005 \001(\010\022\025\n\rmatchedCo" + - "lumn\030\006 \001(\010\022\027\n\017filterIfMissing\030\007 \001(\010\022\031\n\021l" + - "atestVersionOnly\030\010 \001(\010\",\n\nSkipFilter\022\036\n\006", - "filter\030\001 \002(\0132\016.NameBytesPair\"&\n\020Timestam" + - "psFilter\022\022\n\ntimestamps\030\001 \003(\003\"4\n\013ValueFil" + - "ter\022%\n\rcompareFilter\030\001 \002(\0132\016.CompareFilt" + - "er\"2\n\020WhileMatchFilter\022\036\n\006filter\030\001 \002(\0132\016" + - ".NameBytesPairBB\n*org.apache.hadoop.hbas" + - "e.protobuf.generatedB\014FilterProtosH\001\210\001\001\240" + - "\001\001" + "lter\"\'\n%FirstKeyValueMatchingQualifiersF" + + "ilter\"8\n\016FuzzyRowFilter\022&\n\rfuzzyKeysData" + + "\030\001 \003(\0132\017.BytesBytesPair\")\n\023InclusiveStop" + + "Filter\022\022\n\nstopRowKey\030\001 \002(\014\"!\n\rKeyOnlyFil" + + "ter\022\020\n\010lenAsVal\030\001 \002(\010\"4\n\032MultipleColumnP" + + "refixFilter\022\026\n\016sortedPrefixes\030\001 \003(\014\"\036\n\nP" + + "ageFilter\022\020\n\010pageSize\030\001 \002(\003\"\036\n\014PrefixFil", + "ter\022\016\n\006prefix\030\001 \002(\014\"8\n\017QualifierFilter\022%" + + "\n\rcompareFilter\030\001 \002(\0132\016.CompareFilter\"!\n" + + "\017RandomRowFilter\022\016\n\006chance\030\001 \002(\002\"2\n\tRowF" + + "ilter\022%\n\rcompareFilter\030\001 \002(\0132\016.CompareFi" + + "lter\"[\n\036SingleColumnValueExcludeFilter\0229" + + "\n\027singleColumnValueFilter\030\001 \002(\0132\030.Single" + + "ColumnValueFilter\"\335\001\n\027SingleColumnValueF" + + "ilter\022\024\n\014columnFamily\030\001 \002(\014\022\027\n\017columnQua" + + "lifier\030\002 \002(\014\022\037\n\tcompareOp\030\003 \002(\0162\014.Compar" + + "eType\022\022\n\ncomparator\030\004 \002(\014\022\023\n\013foundColumn", + "\030\005 \001(\010\022\025\n\rmatchedColumn\030\006 \001(\010\022\027\n\017filterI" + + "fMissing\030\007 \001(\010\022\031\n\021latestVersionOnly\030\010 \001(" + + "\010\",\n\nSkipFilter\022\036\n\006filter\030\001 \002(\0132\016.NameBy" + + "tesPair\"&\n\020TimestampsFilter\022\022\n\ntimestamp" + + "s\030\001 \003(\003\"4\n\013ValueFilter\022%\n\rcompareFilter\030" + + "\001 \002(\0132\016.CompareFilter\"2\n\020WhileMatchFilte" + + "r\022\036\n\006filter\030\001 \002(\0132\016.NameBytesPairBB\n*org" + + ".apache.hadoop.hbase.protobuf.generatedB" + + "\014FilterProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -10245,8 +11150,24 @@ public final class FilterProtos { new java.lang.String[] { }, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyOnlyFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyOnlyFilter.Builder.class); - internal_static_InclusiveStopFilter_descriptor = + internal_static_FirstKeyValueMatchingQualifiersFilter_descriptor = getDescriptor().getMessageTypes().get(8); + internal_static_FirstKeyValueMatchingQualifiersFilter_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_FirstKeyValueMatchingQualifiersFilter_descriptor, + new java.lang.String[] { }, + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter.class, + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder.class); + internal_static_FuzzyRowFilter_descriptor = + getDescriptor().getMessageTypes().get(9); + internal_static_FuzzyRowFilter_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_FuzzyRowFilter_descriptor, + new java.lang.String[] { "FuzzyKeysData", }, + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter.class, + org.apache.hadoop.hbase.protobuf.generated.FilterProtos.FuzzyRowFilter.Builder.class); + internal_static_InclusiveStopFilter_descriptor = + getDescriptor().getMessageTypes().get(10); internal_static_InclusiveStopFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_InclusiveStopFilter_descriptor, @@ -10254,7 +11175,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.InclusiveStopFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.InclusiveStopFilter.Builder.class); internal_static_KeyOnlyFilter_descriptor = - getDescriptor().getMessageTypes().get(9); + getDescriptor().getMessageTypes().get(11); internal_static_KeyOnlyFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_KeyOnlyFilter_descriptor, @@ -10262,7 +11183,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.KeyOnlyFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.KeyOnlyFilter.Builder.class); internal_static_MultipleColumnPrefixFilter_descriptor = - getDescriptor().getMessageTypes().get(10); + getDescriptor().getMessageTypes().get(12); internal_static_MultipleColumnPrefixFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultipleColumnPrefixFilter_descriptor, @@ -10270,7 +11191,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.MultipleColumnPrefixFilter.Builder.class); internal_static_PageFilter_descriptor = - getDescriptor().getMessageTypes().get(11); + getDescriptor().getMessageTypes().get(13); internal_static_PageFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_PageFilter_descriptor, @@ -10278,7 +11199,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.PageFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.PageFilter.Builder.class); internal_static_PrefixFilter_descriptor = - getDescriptor().getMessageTypes().get(12); + getDescriptor().getMessageTypes().get(14); internal_static_PrefixFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_PrefixFilter_descriptor, @@ -10286,7 +11207,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.PrefixFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.PrefixFilter.Builder.class); internal_static_QualifierFilter_descriptor = - getDescriptor().getMessageTypes().get(13); + getDescriptor().getMessageTypes().get(15); internal_static_QualifierFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_QualifierFilter_descriptor, @@ -10294,7 +11215,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.QualifierFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.QualifierFilter.Builder.class); internal_static_RandomRowFilter_descriptor = - getDescriptor().getMessageTypes().get(14); + getDescriptor().getMessageTypes().get(16); internal_static_RandomRowFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RandomRowFilter_descriptor, @@ -10302,7 +11223,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RandomRowFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RandomRowFilter.Builder.class); internal_static_RowFilter_descriptor = - getDescriptor().getMessageTypes().get(15); + getDescriptor().getMessageTypes().get(17); internal_static_RowFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RowFilter_descriptor, @@ -10310,7 +11231,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowFilter.Builder.class); internal_static_SingleColumnValueExcludeFilter_descriptor = - getDescriptor().getMessageTypes().get(16); + getDescriptor().getMessageTypes().get(18); internal_static_SingleColumnValueExcludeFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SingleColumnValueExcludeFilter_descriptor, @@ -10318,7 +11239,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SingleColumnValueExcludeFilter.Builder.class); internal_static_SingleColumnValueFilter_descriptor = - getDescriptor().getMessageTypes().get(17); + getDescriptor().getMessageTypes().get(19); internal_static_SingleColumnValueFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SingleColumnValueFilter_descriptor, @@ -10326,7 +11247,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SingleColumnValueFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SingleColumnValueFilter.Builder.class); internal_static_SkipFilter_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(20); internal_static_SkipFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SkipFilter_descriptor, @@ -10334,7 +11255,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SkipFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.SkipFilter.Builder.class); internal_static_TimestampsFilter_descriptor = - getDescriptor().getMessageTypes().get(19); + getDescriptor().getMessageTypes().get(21); internal_static_TimestampsFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_TimestampsFilter_descriptor, @@ -10342,7 +11263,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.TimestampsFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.TimestampsFilter.Builder.class); internal_static_ValueFilter_descriptor = - getDescriptor().getMessageTypes().get(20); + getDescriptor().getMessageTypes().get(22); internal_static_ValueFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ValueFilter_descriptor, @@ -10350,7 +11271,7 @@ public final class FilterProtos { org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ValueFilter.class, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.ValueFilter.Builder.class); internal_static_WhileMatchFilter_descriptor = - getDescriptor().getMessageTypes().get(21); + getDescriptor().getMessageTypes().get(23); internal_static_WhileMatchFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_WhileMatchFilter_descriptor, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java index 24fb93cb338..4d40b9488d0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java @@ -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 + 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; } }; diff --git a/hbase-server/src/main/protobuf/Filter.proto b/hbase-server/src/main/protobuf/Filter.proto index c665a07d2e0..e204665ff34 100644 --- a/hbase-server/src/main/protobuf/Filter.proto +++ b/hbase-server/src/main/protobuf/Filter.proto @@ -68,6 +68,10 @@ message FirstKeyOnlyFilter { message FirstKeyValueMatchingQualifiersFilter { } +message FuzzyRowFilter { + repeated BytesBytesPair fuzzyKeysData = 1; +} + message InclusiveStopFilter { required bytes stopRowKey = 1; } diff --git a/hbase-server/src/main/protobuf/hbase.proto b/hbase-server/src/main/protobuf/hbase.proto index 4b68661fa52..cc2362cbf51 100644 --- a/hbase-server/src/main/protobuf/hbase.proto +++ b/hbase-server/src/main/protobuf/hbase.proto @@ -248,3 +248,8 @@ message NameBytesPair { required string name = 1; optional bytes value = 2; } + +message BytesBytesPair { + required bytes first = 1; + required bytes second = 2; +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java new file mode 100644 index 00000000000..4faca821650 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFuzzyRowFilter.java @@ -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(); +}