HBASE-13347 Deprecate FirstKeyValueMatchingQualifiersFilter. (Abhishek)

This commit is contained in:
anoopsjohn 2015-11-24 09:50:56 +05:30
parent 9a91f5ac81
commit daba867734
4 changed files with 5 additions and 12 deletions

View File

@ -43,9 +43,11 @@ import com.google.protobuf.InvalidProtocolBufferException;
* caveat, this filter is only useful for special cases
* like {@link org.apache.hadoop.hbase.mapreduce.RowCounter}.
* <p>
* @deprecated Deprecated in 2.0. See HBASE-13347
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@Deprecated
public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
private Set<byte []> qualifiers;

View File

@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException;
import java.util.Set;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -34,7 +32,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.filter.FirstKeyValueMatchingQualifiersFilter;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapreduce.Counter;
@ -150,7 +147,6 @@ public class RowCounter extends Configured implements Tool {
job.setJarByClass(RowCounter.class);
Scan scan = new Scan();
scan.setCacheBlocks(false);
Set<byte []> qualifiers = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
if (startKey != null && !startKey.equals("")) {
scan.setStartRow(Bytes.toBytes(startKey));
}
@ -170,14 +166,7 @@ public class RowCounter extends Configured implements Tool {
}
}
}
// specified column may or may not be part of first key value for the row.
// Hence do not use FirstKeyOnlyFilter if scan has columns, instead use
// FirstKeyValueMatchingQualifiersFilter.
if (qualifiers.size() == 0) {
scan.setFilter(new FirstKeyOnlyFilter());
} else {
scan.setFilter(new FirstKeyValueMatchingQualifiersFilter(qualifiers));
}
scan.setFilter(new FirstKeyOnlyFilter());
scan.setTimeRange(startTime, endTime == 0 ? HConstants.LATEST_TIMESTAMP : endTime);
job.setOutputFormatClass(NullOutputFormat.class);
TableMapReduceUtil.initTableMapperJob(tableName, scan,

View File

@ -127,6 +127,7 @@ public class TestFilterSerialization {
ProtobufUtil.toFilter(ProtobufUtil.toFilter(filterWrapper))));
}
@SuppressWarnings("deprecation")
@Test
public void testFirstKeyValueMatchingQualifiersFilter() throws Exception {
// empty qualifiers set

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.experimental.categories.Category;
@SuppressWarnings("deprecation")
@Category({FilterTests.class, SmallTests.class})
public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase {
private static final byte[] ROW = Bytes.toBytes("test");