View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.mapreduce;
20  
21  import java.io.IOException;
22  
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.commons.lang.StringUtils;
26  import org.apache.hadoop.hbase.HConstants;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.classification.InterfaceStability;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.conf.Configured;
31  import org.apache.hadoop.hbase.HBaseConfiguration;
32  import org.apache.hadoop.hbase.client.Result;
33  import org.apache.hadoop.hbase.client.Scan;
34  import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
35  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
36  import org.apache.hadoop.hbase.util.Bytes;
37  import org.apache.hadoop.mapreduce.Counter;
38  import org.apache.hadoop.mapreduce.Job;
39  import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
40  import org.apache.hadoop.util.Tool;
41  import org.apache.hadoop.util.ToolRunner;
42  
43  /**
44   * A job with a just a map phase to count rows. Map outputs table rows IF the
45   * input row has columns that have content.
46   */
47  @InterfaceAudience.Public
48  @InterfaceStability.Stable
49  public class RowCounter extends Configured implements Tool {
50  
51    private static final Log LOG = LogFactory.getLog(RowCounter.class);
52  
53    /** Name of this 'program'. */
54    static final String NAME = "rowcounter";
55  
56    private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
57    private final static String EXPECTED_COUNT_KEY = RowCounter.class.getName() + ".expected_count";
58  
59    /**
60     * Mapper that runs the count.
61     */
62    static class RowCounterMapper
63    extends TableMapper<ImmutableBytesWritable, Result> {
64  
65      /** Counter enumeration to count the actual rows. */
66      public static enum Counters {ROWS}
67  
68      /**
69       * Maps the data.
70       *
71       * @param row  The current table row key.
72       * @param values  The columns.
73       * @param context  The current context.
74       * @throws IOException When something is broken with the data.
75       * @see org.apache.hadoop.mapreduce.Mapper#map(KEYIN, VALUEIN,
76       *   org.apache.hadoop.mapreduce.Mapper.Context)
77       */
78      @Override
79      public void map(ImmutableBytesWritable row, Result values,
80        Context context)
81      throws IOException {
82        // Count every row containing data, whether it's in qualifiers or values
83        context.getCounter(Counters.ROWS).increment(1);
84      }
85    }
86  
87    /**
88     * Sets up the actual job.
89     *
90     * @param conf  The current configuration.
91     * @param args  The command line parameters.
92     * @return The newly created job.
93     * @throws IOException When setting up the job fails.
94     */
95    public static Job createSubmittableJob(Configuration conf, String[] args)
96    throws IOException {
97      String tableName = args[0];
98      String startKey = null;
99      String endKey = null;
100     long startTime = 0;
101     long endTime = 0;
102 
103     StringBuilder sb = new StringBuilder();
104 
105     final String rangeSwitch = "--range=";
106     final String startTimeArgKey = "--starttime=";
107     final String endTimeArgKey = "--endtime=";
108     final String expectedCountArg = "--expected-count=";
109 
110     // First argument is table name, starting from second
111     for (int i = 1; i < args.length; i++) {
112       if (args[i].startsWith(rangeSwitch)) {
113         String[] startEnd = args[i].substring(rangeSwitch.length()).split(",", 2);
114         if (startEnd.length != 2 || startEnd[1].contains(",")) {
115           printUsage("Please specify range in such format as \"--range=a,b\" " +
116               "or, with only one boundary, \"--range=,b\" or \"--range=a,\"");
117           return null;
118         }
119         startKey = startEnd[0];
120         endKey = startEnd[1];
121       }
122       if (startTime < endTime) {
123         printUsage("--endtime=" + endTime + " needs to be greater than --starttime=" + startTime);
124         return null;
125       }
126       if (args[i].startsWith(startTimeArgKey)) {
127         startTime = Long.parseLong(args[i].substring(startTimeArgKey.length()));
128         continue;
129       }
130       if (args[i].startsWith(endTimeArgKey)) {
131         endTime = Long.parseLong(args[i].substring(endTimeArgKey.length()));
132         continue;
133       }
134       if (args[i].startsWith(expectedCountArg)) {
135         conf.setLong(EXPECTED_COUNT_KEY,
136             Long.parseLong(args[i].substring(expectedCountArg.length())));
137         continue;
138       }
139       else {
140         // if no switch, assume column names
141         sb.append(args[i]);
142         sb.append(" ");
143       }
144     }
145 
146     Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
147     job.setJarByClass(RowCounter.class);
148     Scan scan = new Scan();
149     scan.setCacheBlocks(false);
150     if (startKey != null && !startKey.equals("")) {
151       scan.setStartRow(Bytes.toBytes(startKey));
152     }
153     if (endKey != null && !endKey.equals("")) {
154       scan.setStopRow(Bytes.toBytes(endKey));
155     }
156     if (sb.length() > 0) {
157       for (String columnName : sb.toString().trim().split(" ")) {
158         String family = StringUtils.substringBefore(columnName, ":");
159         String qualifier = StringUtils.substringAfter(columnName, ":");
160 
161         if (StringUtils.isBlank(qualifier)) {
162           scan.addFamily(Bytes.toBytes(family));
163         }
164         else {
165           scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(qualifier));
166         }
167       }
168     }
169     scan.setFilter(new FirstKeyOnlyFilter());
170     scan.setTimeRange(startTime, endTime == 0 ? HConstants.LATEST_TIMESTAMP : endTime);
171     job.setOutputFormatClass(NullOutputFormat.class);
172     TableMapReduceUtil.initTableMapperJob(tableName, scan,
173       RowCounterMapper.class, ImmutableBytesWritable.class, Result.class, job);
174     job.setNumReduceTasks(0);
175     return job;
176   }
177 
178   /*
179    * @param errorMessage Can attach a message when error occurs.
180    */
181   private static void printUsage(String errorMessage) {
182     System.err.println("ERROR: " + errorMessage);
183     printUsage();
184   }
185 
186   /**
187    * Prints usage without error message.
188    * Note that we don't document --expected-count, because it's intended for test.
189    */
190   private static void printUsage() {
191     System.err.println("Usage: RowCounter [options] <tablename> " +
192         "[--starttime=[start] --endtime=[end] " +
193         "[--range=[startKey],[endKey]] [<column1> <column2>...]");
194     System.err.println("For performance consider the following options:\n"
195         + "-Dhbase.client.scanner.caching=100\n"
196         + "-Dmapreduce.map.speculative=false");
197   }
198 
199   @Override
200   public int run(String[] args) throws Exception {
201     if (args.length < 1) {
202       printUsage("Wrong number of parameters: " + args.length);
203       return -1;
204     }
205     Job job = createSubmittableJob(getConf(), args);
206     if (job == null) {
207       return -1;
208     }
209     boolean success = job.waitForCompletion(true);
210     final long expectedCount = getConf().getLong(EXPECTED_COUNT_KEY, -1);
211     if (success && expectedCount != -1) {
212       final Counter counter = job.getCounters().findCounter(RowCounterMapper.Counters.ROWS);
213       success = expectedCount == counter.getValue();
214       if (!success) {
215         LOG.error("Failing job because count of '" + counter.getValue() +
216             "' does not match expected count of '" + expectedCount + "'");
217       }
218     }
219     return (success ? 0 : 1);
220   }
221 
222   /**
223    * Main entry point.
224    * @param args The command line parameters.
225    * @throws Exception When running the job fails.
226    */
227   public static void main(String[] args) throws Exception {
228     int errCode = ToolRunner.run(HBaseConfiguration.create(), new RowCounter(), args);
229     System.exit(errCode);
230   }
231 
232 }