HBASE-4295 rowcounter does not return the correct number of rows in certain circumstances
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1174919 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d4b9970aae
commit
9c41ffc734
|
@ -632,6 +632,8 @@ Release 0.90.5 - Unreleased
|
|||
running zk with new format root servername
|
||||
HBASE-4387 Error while syncing: DFSOutputStream is closed
|
||||
(Lars Hofhansl)
|
||||
HBASE-4295 rowcounter does not return the correct number of rows in
|
||||
certain circumstances (David Revell)
|
||||
|
||||
IMPROVEMENT
|
||||
HBASE-4205 Enhance HTable javadoc (Eric Charles)
|
||||
|
|
|
@ -58,20 +58,8 @@ public class RowCounter extends Configured implements Tool {
|
|||
OutputCollector<ImmutableBytesWritable, Result> output,
|
||||
Reporter reporter)
|
||||
throws IOException {
|
||||
boolean content = false;
|
||||
|
||||
for (KeyValue value: values.list()) {
|
||||
if (value.getValue().length > 0) {
|
||||
content = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!content) {
|
||||
// Don't count rows that are all empty values.
|
||||
return;
|
||||
}
|
||||
// Give out same value every time. We're only interested in the row/key
|
||||
reporter.incrCounter(Counters.ROWS, 1);
|
||||
// Count every row containing data, whether it's in qualifiers or values
|
||||
reporter.incrCounter(Counters.ROWS, 1);
|
||||
}
|
||||
|
||||
public void configure(JobConf jc) {
|
||||
|
|
|
@ -65,12 +65,8 @@ public class RowCounter {
|
|||
public void map(ImmutableBytesWritable row, Result values,
|
||||
Context context)
|
||||
throws IOException {
|
||||
for (KeyValue value: values.list()) {
|
||||
if (value.getValue().length > 0) {
|
||||
context.getCounter(Counters.ROWS).increment(1);
|
||||
break;
|
||||
}
|
||||
}
|
||||
// Count every row containing data, whether it's in qualifiers or values
|
||||
context.getCounter(Counters.ROWS).increment(1);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue