HBASE-13347 Deprecate FirstKeyValueMatchingQualifiersFilter. (Abhishek)

This commit is contained in:
anoopsjohn 2015-11-24 09:52:37 +05:30
parent 2ce27951b0
commit 81e7eb2805
1 changed files with 1 additions and 12 deletions

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.lang.StringUtils;
import org.apache.hadoop.hbase.HConstants;
@ -31,7 +29,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.Job;
@ -134,7 +131,6 @@ public class RowCounter {
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));
}
@ -154,14 +150,7 @@ public class RowCounter {
}
}
}
// 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,