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