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