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