1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.mapreduce;
21
22 import java.io.IOException;
23 import org.apache.commons.lang.StringUtils;
24
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.HBaseConfiguration;
27 import org.apache.hadoop.hbase.KeyValue;
28 import org.apache.hadoop.hbase.client.Result;
29 import org.apache.hadoop.hbase.client.Scan;
30 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
31 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
32 import org.apache.hadoop.hbase.util.Bytes;
33 import org.apache.hadoop.mapreduce.Job;
34 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
35 import org.apache.hadoop.util.GenericOptionsParser;
36
37
38
39
40
41
42 public class RowCounter {
43
44
45 static final String NAME = "rowcounter";
46
47
48
49
50 static class RowCounterMapper
51 extends TableMapper<ImmutableBytesWritable, Result> {
52
53
54 public static enum Counters {ROWS}
55
56
57
58
59
60
61
62
63
64
65
66 @Override
67 public void map(ImmutableBytesWritable row, Result values,
68 Context context)
69 throws IOException {
70
71 context.getCounter(Counters.ROWS).increment(1);
72 }
73 }
74
75
76
77
78
79
80
81
82
83 public static Job createSubmittableJob(Configuration conf, String[] args)
84 throws IOException {
85 String tableName = args[0];
86 String startKey = null;
87 String endKey = null;
88 StringBuilder sb = new StringBuilder();
89
90 final String rangeSwitch = "--range=";
91
92
93 for (int i = 1; i < args.length; i++) {
94 if (args[i].startsWith(rangeSwitch)) {
95 String[] startEnd = args[i].substring(rangeSwitch.length()).split(",", 2);
96 if (startEnd.length != 2 || startEnd[1].contains(",")) {
97 printUsage("Please specify range in such format as \"--range=a,b\" " +
98 "or, with only one boundary, \"--range=,b\" or \"--range=a,\"");
99 return null;
100 }
101 startKey = startEnd[0];
102 endKey = startEnd[1];
103 }
104 else {
105
106 sb.append(args[i]);
107 sb.append(" ");
108 }
109 }
110
111 Job job = new Job(conf, NAME + "_" + tableName);
112 job.setJarByClass(RowCounter.class);
113 Scan scan = new Scan();
114 scan.setCacheBlocks(false);
115 if (startKey != null && !startKey.equals("")) {
116 scan.setStartRow(Bytes.toBytes(startKey));
117 }
118 if (endKey != null && !endKey.equals("")) {
119 scan.setStopRow(Bytes.toBytes(endKey));
120 }
121 scan.setFilter(new FirstKeyOnlyFilter());
122 if (sb.length() > 0) {
123 for (String columnName : sb.toString().trim().split(" ")) {
124 String family = StringUtils.substringBefore(columnName, ":");
125 String qualifier = StringUtils.substringAfter(columnName, ":");
126 if (StringUtils.isBlank(qualifier)) {
127 scan.addFamily(Bytes.toBytes(family));
128 } else {
129 scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(qualifier));
130 }
131 }
132 }
133 job.setOutputFormatClass(NullOutputFormat.class);
134 TableMapReduceUtil.initTableMapperJob(tableName, scan,
135 RowCounterMapper.class, ImmutableBytesWritable.class, Result.class, job);
136 job.setNumReduceTasks(0);
137 return job;
138 }
139
140
141
142
143 private static void printUsage(String errorMessage) {
144 System.err.println("ERROR: " + errorMessage);
145 printUsage();
146 }
147
148
149
150
151 private static void printUsage() {
152 System.err.println("Usage: RowCounter [options] <tablename> " +
153 "[--range=[startKey],[endKey]] [<column1> <column2>...]");
154 System.err.println("For performance consider the following options:\n"
155 + "-Dhbase.client.scanner.caching=100\n"
156 + "-Dmapred.map.tasks.speculative.execution=false");
157 }
158
159
160
161
162
163
164
165 public static void main(String[] args) throws Exception {
166 Configuration conf = HBaseConfiguration.create();
167 String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
168 if (otherArgs.length < 1) {
169 printUsage("Wrong number of parameters: " + args.length);
170 System.exit(-1);
171 }
172 Job job = createSubmittableJob(conf, otherArgs);
173 if (job == null) {
174 System.exit(-1);
175 }
176 System.exit(job.waitForCompletion(true) ? 0 : 1);
177 }
178 }