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