hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r1174920 - in /hbase/branches/0.92: CHANGES.txt src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
Date Fri, 23 Sep 2011 17:51:52 GMT
Author: stack
Date: Fri Sep 23 17:51:52 2011
New Revision: 1174920

URL: http://svn.apache.org/viewvc?rev=1174920&view=rev
Log:
HBASE-4295 rowcounter does not return the correct number of rows in certain circumstances

Modified:
    hbase/branches/0.92/CHANGES.txt
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java

Modified: hbase/branches/0.92/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/CHANGES.txt?rev=1174920&r1=1174919&r2=1174920&view=diff
==============================================================================
--- hbase/branches/0.92/CHANGES.txt (original)
+++ hbase/branches/0.92/CHANGES.txt Fri Sep 23 17:51:52 2011
@@ -626,6 +626,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)

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java?rev=1174920&r1=1174919&r2=1174920&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapred/RowCounter.java Fri Sep
23 17:51:52 2011
@@ -58,20 +58,8 @@ public class RowCounter extends Configur
         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) {

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java?rev=1174920&r1=1174919&r2=1174920&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java Fri
Sep 23 17:51:52 2011
@@ -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);
     }
   }
 



Mime
View raw message