Author: stack
Date: Tue Apr 15 06:18:05 2014
New Revision: 1587445
URL: http://svn.apache.org/r1587445
Log:
HBASE-10966 RowCounter misinterprets column names that have colons in their
qualifier
Modified:
hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
Modified:
hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
URL:
http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java?rev=1587445&r1=1587444&r2=1587445&view=diff
==============================================================================
---
hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
(original)
+++
hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/mapreduce/RowCounter.java
Tue Apr 15 06:18:05 2014
@@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException;
+import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -33,6 +34,7 @@ import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.util.GenericOptionsParser;
+
/**
* A job with a just a map phase to count rows. Map outputs table rows IF the
* input row has columns that have content.
@@ -119,11 +121,12 @@ public class RowCounter {
scan.setFilter(new FirstKeyOnlyFilter());
if (sb.length() > 0) {
for (String columnName : sb.toString().trim().split(" ")) {
- String [] fields = columnName.split(":");
- if(fields.length == 1) {
- scan.addFamily(Bytes.toBytes(fields[0]));
+ String family = StringUtils.substringBefore(columnName, ":");
+ String qualifier = StringUtils.substringAfter(columnName, ":");
+ if (StringUtils.isBlank(qualifier)) {
+ scan.addFamily(Bytes.toBytes(family));
} else {
- scan.addColumn(Bytes.toBytes(fields[0]), Bytes.toBytes(fields[1]));
+ scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(qualifier));
}
}
}