HBASE-26091 Remove FirstKeyValueMatchingQualifiersFilter (#3497)

Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
GeorryHuang 2021-07-25 21:12:20 +08:00 committed by GitHub
parent 0294c73f1f
commit 8ae394285a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 13 additions and 206 deletions

View File

@ -18,31 +18,15 @@
package org.apache.hadoop.hbase.filter;
import java.util.Objects;
import java.util.Set;
import java.util.TreeSet;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
/**
* The filter looks for the given columns in KeyValue. Once there is a match for
* any one of the columns, it returns ReturnCode.NEXT_ROW for remaining
* KeyValues in the row.
* <p>
* Note : It may emit KVs which do not have the given columns in them, if
* these KVs happen to occur before a KV which does have a match. Given this
* caveat, this filter is only useful for special cases
* like org.apache.hadoop.hbase.mapreduce.RowCounter.
* <p>
* This filter was deprecated in 2.0.0 and should be removed in 3.0.0. We keep the code here
* to prevent the proto serialization exceptions puzzle those users who use older version clients
* to communicate with newer version servers.
*
* @deprecated Deprecated in 2.0.0 and will be removed in 3.0.0.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-13347">HBASE-13347</a>
*/
@ -50,50 +34,6 @@ import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
@Deprecated
public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
private Set<byte []> qualifiers;
/**
* Constructor which takes a set of columns. As soon as first KeyValue
* matching any of these columns is found, filter moves to next row.
*
* @param qualifiers the set of columns to me matched.
*/
public FirstKeyValueMatchingQualifiersFilter(Set<byte []> qualifiers) {
this.qualifiers = qualifiers;
}
@Override
public ReturnCode filterCell(final Cell c) {
if (hasFoundKV()) {
return ReturnCode.NEXT_ROW;
} else if (hasOneMatchingQualifier(c)) {
setFoundKV(true);
}
return ReturnCode.INCLUDE;
}
private boolean hasOneMatchingQualifier(Cell c) {
for (byte[] q : qualifiers) {
if (CellUtil.matchingQualifier(c, q)) {
return true;
}
}
return false;
}
/**
* @return The filter serialized using pb
*/
@Override
public byte [] toByteArray() {
FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder =
FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder();
for (byte[] qualifier : qualifiers) {
if (qualifier != null) builder.addQualifiers(UnsafeByteOperations.unsafeWrap(qualifier));
}
return builder.build().toByteArray();
}
/**
* @param pbBytes A pb serialized {@link FirstKeyValueMatchingQualifiersFilter} instance
* @return An instance of {@link FirstKeyValueMatchingQualifiersFilter} made from <code>bytes</code>
@ -102,41 +42,7 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
*/
public static FirstKeyValueMatchingQualifiersFilter parseFrom(final byte [] pbBytes)
throws DeserializationException {
FilterProtos.FirstKeyValueMatchingQualifiersFilter proto;
try {
proto = FilterProtos.FirstKeyValueMatchingQualifiersFilter.parseFrom(pbBytes);
} catch (InvalidProtocolBufferException e) {
throw new DeserializationException(e);
}
TreeSet<byte []> qualifiers = new TreeSet<>(Bytes.BYTES_COMPARATOR);
for (ByteString qualifier : proto.getQualifiersList()) {
qualifiers.add(qualifier.toByteArray());
}
return new FirstKeyValueMatchingQualifiersFilter(qualifiers);
}
/**
* @param o the other filter to compare with
* @return true if and only if the fields of the filter that are serialized
* are equal to the corresponding fields in other. Used for testing.
*/
@Override
boolean areSerializedFieldsEqual(Filter o) {
if (o == this) return true;
if (!(o instanceof FirstKeyValueMatchingQualifiersFilter)) return false;
FirstKeyValueMatchingQualifiersFilter other = (FirstKeyValueMatchingQualifiersFilter)o;
return this.qualifiers.equals(other.qualifiers);
}
@Override
public boolean equals(Object obj) {
return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj);
}
@Override
public int hashCode() {
return Objects.hash(this.qualifiers);
throw new DeserializationException(
"Stop using FirstKeyValueMatchingQualifiersFilter, which has been permanently removed");
}
}

View File

@ -89,6 +89,8 @@ message FirstKeyOnlyFilter {
}
message FirstKeyValueMatchingQualifiersFilter {
//Just deprecate it to guarantee old client could communicate with server
option deprecated = true;
repeated bytes qualifiers = 1;
}

View File

@ -21,14 +21,13 @@ import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
@ -795,9 +794,10 @@ public class TestPartialResultsFromClientSide {
testPartialResultsWithColumnFilter(new ColumnRangeFilter(Bytes.toBytes("testQualifer1"), true,
Bytes.toBytes("testQualifier7"), true));
Set<byte[]> qualifiers = new LinkedHashSet<>();
qualifiers.add(Bytes.toBytes("testQualifier5"));
testPartialResultsWithColumnFilter(new FirstKeyValueMatchingQualifiersFilter(qualifiers));
//Throw an Exception to the old version client to remind them not to use this filter anymore
assertThrows("Stop using", DoNotRetryIOException.class,
() -> testPartialResultsWithColumnFilter(
new FirstKeyValueMatchingQualifiersFilter()));
}
public void testPartialResultsWithColumnFilter(Filter filter) throws Exception {

View File

@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.LinkedList;
import java.util.List;
import java.util.TreeSet;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange;
@ -132,25 +131,6 @@ public class TestFilterSerialization {
ProtobufUtil.toFilter(ProtobufUtil.toFilter(filterWrapper))));
}
@SuppressWarnings("deprecation")
@Test
public void testFirstKeyValueMatchingQualifiersFilter() throws Exception {
// empty qualifiers set
TreeSet<byte []> set = new TreeSet<>(Bytes.BYTES_COMPARATOR);
FirstKeyValueMatchingQualifiersFilter firstKeyValueMatchingQualifiersFilter =
new FirstKeyValueMatchingQualifiersFilter(set);
assertTrue(firstKeyValueMatchingQualifiersFilter.areSerializedFieldsEqual(
ProtobufUtil.toFilter(ProtobufUtil.toFilter(firstKeyValueMatchingQualifiersFilter))));
// non-empty qualifiers set
set.add(Bytes.toBytes("col0"));
set.add(Bytes.toBytes("col1"));
firstKeyValueMatchingQualifiersFilter =
new FirstKeyValueMatchingQualifiersFilter(set);
assertTrue(firstKeyValueMatchingQualifiersFilter.areSerializedFieldsEqual(
ProtobufUtil.toFilter(ProtobufUtil.toFilter(firstKeyValueMatchingQualifiersFilter))));
}
@Test
public void testFirstKeyOnlyFilter() throws Exception {
FirstKeyOnlyFilter firstKeyOnlyFilter = new FirstKeyOnlyFilter();

View File

@ -1,81 +0,0 @@
/**
* 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.util.Set;
import java.util.TreeSet;
import junit.framework.TestCase;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.testclassification.FilterTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.ClassRule;
import org.junit.experimental.categories.Category;
@SuppressWarnings("deprecation")
@Category({FilterTests.class, SmallTests.class})
public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestFirstKeyValueMatchingQualifiersFilter.class);
private static final byte[] ROW = Bytes.toBytes("test");
private static final byte[] COLUMN_FAMILY = Bytes.toBytes("test");
private static final byte[] COLUMN_QUALIFIER_1 = Bytes.toBytes("foo");
private static final byte[] COLUMN_QUALIFIER_2 = Bytes.toBytes("foo_2");
private static final byte[] COLUMN_QUALIFIER_3 = Bytes.toBytes("foo_3");
private static final byte[] VAL_1 = Bytes.toBytes("a");
/**
* Test the functionality of
* {@link FirstKeyValueMatchingQualifiersFilter#filterCell(org.apache.hadoop.hbase.Cell)}
*
* @throws Exception
*/
public void testFirstKeyMatchingQualifierFilter() throws Exception {
Set<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
quals.add(COLUMN_QUALIFIER_1);
quals.add(COLUMN_QUALIFIER_2);
Filter filter = new FirstKeyValueMatchingQualifiersFilter(quals);
// Match in first attempt
KeyValue cell;
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
assertTrue("includeAndSetFlag",
filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("flagIsSetSkipToNextRow",
filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
// A mismatch in first attempt and match in second attempt.
filter.reset();
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_3, VAL_1);
System.out.println(filter.filterCell(cell));
assertTrue("includeFlagIsUnset",
filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_2, VAL_1);
assertTrue("includeAndSetFlag",
filter.filterCell(cell) == Filter.ReturnCode.INCLUDE);
cell = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER_1, VAL_1);
assertTrue("flagIsSetSkipToNextRow",
filter.filterCell(cell) == Filter.ReturnCode.NEXT_ROW);
}
}