HBASE-4783 Improve RowCounter to count rows in a specific key range.
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1205510 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2048fe41a0
commit
10407c9a16
|
@ -81,21 +81,43 @@ public class RowCounter {
|
|||
public static Job createSubmittableJob(Configuration conf, String[] args)
|
||||
throws IOException {
|
||||
String tableName = args[0];
|
||||
Job job = new Job(conf, NAME + "_" + tableName);
|
||||
job.setJarByClass(RowCounter.class);
|
||||
// Columns are space delimited
|
||||
String startKey = null;
|
||||
String endKey = null;
|
||||
StringBuilder sb = new StringBuilder();
|
||||
final int columnoffset = 1;
|
||||
for (int i = columnoffset; i < args.length; i++) {
|
||||
if (i > columnoffset) {
|
||||
|
||||
final String rangeSwitch = "--range=";
|
||||
|
||||
// First argument is table name, starting from second
|
||||
for (int i = 1; i < args.length; i++) {
|
||||
if (args[i].startsWith(rangeSwitch)) {
|
||||
String[] startEnd = args[i].substring(rangeSwitch.length()).split(",", 2);
|
||||
if (startEnd.length != 2 || startEnd[1].contains(",")) {
|
||||
printUsage("Please specify range in such format as \"--range=a,b\" " +
|
||||
"or, with only one boundary, \"--range=,b\" or \"--range=a,\"");
|
||||
return null;
|
||||
}
|
||||
startKey = startEnd[0];
|
||||
endKey = startEnd[1];
|
||||
}
|
||||
else {
|
||||
// if no switch, assume column names
|
||||
sb.append(args[i]);
|
||||
sb.append(" ");
|
||||
}
|
||||
sb.append(args[i]);
|
||||
}
|
||||
|
||||
Job job = new Job(conf, NAME + "_" + tableName);
|
||||
job.setJarByClass(RowCounter.class);
|
||||
Scan scan = new Scan();
|
||||
if (startKey != null && !startKey.equals("")) {
|
||||
scan.setStartRow(Bytes.toBytes(startKey));
|
||||
}
|
||||
if (endKey != null && !endKey.equals("")) {
|
||||
scan.setStopRow(Bytes.toBytes(endKey));
|
||||
}
|
||||
scan.setFilter(new FirstKeyOnlyFilter());
|
||||
if (sb.length() > 0) {
|
||||
for (String columnName :sb.toString().split(" ")) {
|
||||
for (String columnName : sb.toString().trim().split(" ")) {
|
||||
String [] fields = columnName.split(":");
|
||||
if(fields.length == 1) {
|
||||
scan.addFamily(Bytes.toBytes(fields[0]));
|
||||
|
@ -104,7 +126,6 @@ public class RowCounter {
|
|||
}
|
||||
}
|
||||
}
|
||||
// Second argument is the table name.
|
||||
job.setOutputFormatClass(NullOutputFormat.class);
|
||||
TableMapReduceUtil.initTableMapperJob(tableName, scan,
|
||||
RowCounterMapper.class, ImmutableBytesWritable.class, Result.class, job);
|
||||
|
@ -112,6 +133,22 @@ public class RowCounter {
|
|||
return job;
|
||||
}
|
||||
|
||||
/*
|
||||
* @param errorMessage Can attach a message when error occurs.
|
||||
*/
|
||||
private static void printUsage(String errorMessage) {
|
||||
System.err.println("ERROR: " + errorMessage);
|
||||
printUsage();
|
||||
}
|
||||
|
||||
/*
|
||||
* Prints usage without error message
|
||||
*/
|
||||
private static void printUsage() {
|
||||
System.err.println("Usage: RowCounter <tablename> " +
|
||||
"[--range=[startKey],[endKey]] [<column1> <column2>...]");
|
||||
}
|
||||
|
||||
/**
|
||||
* Main entry point.
|
||||
*
|
||||
|
@ -122,11 +159,13 @@ public class RowCounter {
|
|||
Configuration conf = HBaseConfiguration.create();
|
||||
String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
|
||||
if (otherArgs.length < 1) {
|
||||
System.err.println("ERROR: Wrong number of parameters: " + args.length);
|
||||
System.err.println("Usage: RowCounter <tablename> [<column1> <column2>...]");
|
||||
printUsage("Wrong number of parameters: " + args.length);
|
||||
System.exit(-1);
|
||||
}
|
||||
Job job = createSubmittableJob(conf, otherArgs);
|
||||
if (job == null) {
|
||||
System.exit(-1);
|
||||
}
|
||||
System.exit(job.waitForCompletion(true) ? 0 : 1);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -133,7 +133,6 @@ extends InputFormat<ImmutableBytesWritable, Result> {
|
|||
keys.getFirst().length == 0) {
|
||||
throw new IOException("Expecting at least one region.");
|
||||
}
|
||||
int count = 0;
|
||||
List<InputSplit> splits = new ArrayList<InputSplit>(keys.getFirst().length);
|
||||
for (int i = 0; i < keys.getFirst().length; i++) {
|
||||
if ( !includeRegionInSplit(keys.getFirst()[i], keys.getSecond()[i])) {
|
||||
|
@ -158,8 +157,9 @@ extends InputFormat<ImmutableBytesWritable, Result> {
|
|||
InputSplit split = new TableSplit(table.getTableName(),
|
||||
splitStart, splitStop, regionLocation);
|
||||
splits.add(split);
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug("getSplits: split -> " + (count++) + " -> " + split);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("getSplits: split -> " + i + " -> " + split);
|
||||
}
|
||||
}
|
||||
}
|
||||
return splits;
|
||||
|
|
Loading…
Reference in New Issue