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.hadoop.hbase.HConstants;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.classification.InterfaceStability;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.Cell;
31  import org.apache.hadoop.hbase.CellUtil;
32  import org.apache.hadoop.hbase.HBaseConfiguration;
33  import org.apache.hadoop.hbase.client.Result;
34  import org.apache.hadoop.hbase.client.Scan;
35  import org.apache.hadoop.hbase.filter.CompareFilter;
36  import org.apache.hadoop.hbase.filter.Filter;
37  import org.apache.hadoop.hbase.filter.PrefixFilter;
38  import org.apache.hadoop.hbase.filter.RegexStringComparator;
39  import org.apache.hadoop.hbase.filter.RowFilter;
40  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
41  import org.apache.hadoop.hbase.util.Bytes;
42  import org.apache.hadoop.io.IntWritable;
43  import org.apache.hadoop.io.Text;
44  import org.apache.hadoop.mapreduce.Job;
45  import org.apache.hadoop.mapreduce.Reducer;
46  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
47  import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
48  import org.apache.hadoop.util.GenericOptionsParser;
49  
50  import com.google.common.base.Preconditions;
51  
52  
53  /**
54   * A job with a a map and reduce phase to count cells in a table.
55   * The counter lists the following stats for a given table:
56   * <pre>
57   * 1. Total number of rows in the table
58   * 2. Total number of CFs across all rows
59   * 3. Total qualifiers across all rows
60   * 4. Total occurrence of each CF
61   * 5. Total occurrence  of each qualifier
62   * 6. Total number of versions of each qualifier.
63   * </pre>
64   *
65   * The cellcounter takes two optional parameters one to use a user
66   * supplied row/family/qualifier string to use in the report and
67   * second a regex based or prefix based row filter to restrict the
68   * count operation to a limited subset of rows from the table.
69   */
70  @InterfaceAudience.Public
71  @InterfaceStability.Stable
72  public class CellCounter {
73    private static final Log LOG =
74      LogFactory.getLog(CellCounter.class.getName());
75  
76  
77    /**
78     * Name of this 'program'.
79     */
80    static final String NAME = "CellCounter";
81  
82    /**
83     * Mapper that runs the count.
84     */
85    static class CellCounterMapper
86    extends TableMapper<Text, IntWritable> {
87      /**
88       * Counter enumeration to count the actual rows.
89       */
90      public static enum Counters {
91        ROWS,
92        CELLS
93      }
94  
95      private Configuration conf;
96      private String separator;
97  
98      // state of current row, family, column needs to persist across map() invocations
99      // in order to properly handle scanner batching, where a single qualifier may have too
100     // many versions for a single map() call
101     private byte[] lastRow;
102     private String currentRowKey;
103     byte[] currentFamily = null;
104     String currentFamilyName = null;
105     byte[] currentQualifier = null;
106     // family + qualifier
107     String currentQualifierName = null;
108     // rowkey + family + qualifier
109     String currentRowQualifierName = null;
110 
111     @Override
112     protected void setup(Context context) throws IOException, InterruptedException {
113       conf = context.getConfiguration();
114       separator = conf.get("ReportSeparator",":");
115     }
116 
117     /**
118      * Maps the data.
119      *
120      * @param row     The current table row key.
121      * @param values  The columns.
122      * @param context The current context.
123      * @throws IOException When something is broken with the data.
124      * @see org.apache.hadoop.mapreduce.Mapper#map(KEYIN, VALUEIN,
125      *      org.apache.hadoop.mapreduce.Mapper.Context)
126      */
127 
128     @Override
129     @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH",
130       justification="Preconditions checks insure we are not going to dereference a null value")
131     public void map(ImmutableBytesWritable row, Result values,
132                     Context context)
133         throws IOException {
134       Preconditions.checkState(values != null,
135           "values passed to the map is null");
136       try {
137         byte[] currentRow = values.getRow();
138         if (lastRow == null || !Bytes.equals(lastRow, currentRow)) {
139           lastRow = currentRow;
140           currentRowKey = Bytes.toStringBinary(currentRow);
141           currentFamily = null;
142           currentQualifier = null;
143           context.getCounter(Counters.ROWS).increment(1);
144           context.write(new Text("Total ROWS"), new IntWritable(1));
145         }
146         if (!values.isEmpty()) {
147           int cellCount = 0;
148           for (Cell value : values.listCells()) {
149             cellCount++;
150             if (currentFamily == null || !CellUtil.matchingFamily(value, currentFamily)) {
151               currentFamily = CellUtil.cloneFamily(value);
152               currentFamilyName = Bytes.toStringBinary(currentFamily);
153               currentQualifier = null;
154               context.getCounter("CF", currentFamilyName).increment(1);
155               if (1 == context.getCounter("CF", currentFamilyName).getValue()) {
156                 context.write(new Text("Total Families Across all Rows"), new IntWritable(1));
157                 context.write(new Text(currentFamily), new IntWritable(1));
158               }
159             }
160             if (currentQualifier == null || !CellUtil.matchingQualifier(value, currentQualifier)) {
161               currentQualifier = CellUtil.cloneQualifier(value);
162               currentQualifierName = currentFamilyName + separator +
163                   Bytes.toStringBinary(currentQualifier);
164               currentRowQualifierName = currentRowKey + separator + currentQualifierName;
165 
166               context.write(new Text("Total Qualifiers across all Rows"),
167                   new IntWritable(1));
168               context.write(new Text(currentQualifierName), new IntWritable(1));
169             }
170             // Increment versions
171             context.write(new Text(currentRowQualifierName + "_Versions"), new IntWritable(1));
172           }
173           context.getCounter(Counters.CELLS).increment(cellCount);
174         }
175       } catch (InterruptedException e) {
176         e.printStackTrace();
177       }
178     }
179   }
180 
181   static class IntSumReducer<Key> extends Reducer<Key, IntWritable,
182       Key, IntWritable> {
183     private IntWritable result = new IntWritable();
184 
185     @Override
186     public void reduce(Key key, Iterable<IntWritable> values, Context context)
187     throws IOException, InterruptedException {
188       int sum = 0;
189       for (IntWritable val : values) {
190         sum += val.get();
191       }
192       result.set(sum);
193       context.write(key, result);
194     }
195   }
196 
197   /**
198    * Sets up the actual job.
199    *
200    * @param conf The current configuration.
201    * @param args The command line parameters.
202    * @return The newly created job.
203    * @throws IOException When setting up the job fails.
204    */
205   public static Job createSubmittableJob(Configuration conf, String[] args)
206       throws IOException {
207     String tableName = args[0];
208     Path outputDir = new Path(args[1]);
209     String reportSeparatorString = (args.length > 2) ? args[2]: ":";
210     conf.set("ReportSeparator", reportSeparatorString);
211     Job job = new Job(conf, NAME + "_" + tableName);
212     job.setJarByClass(CellCounter.class);
213     Scan scan = getConfiguredScanForJob(conf, args);
214     TableMapReduceUtil.initTableMapperJob(tableName, scan,
215         CellCounterMapper.class, ImmutableBytesWritable.class, Result.class, job);
216     job.setNumReduceTasks(1);
217     job.setMapOutputKeyClass(Text.class);
218     job.setMapOutputValueClass(IntWritable.class);
219     job.setOutputFormatClass(TextOutputFormat.class);
220     job.setOutputKeyClass(Text.class);
221     job.setOutputValueClass(IntWritable.class);
222     FileOutputFormat.setOutputPath(job, outputDir);
223     job.setReducerClass(IntSumReducer.class);
224     return job;
225   }
226 
227   private static Scan getConfiguredScanForJob(Configuration conf, String[] args)
228       throws IOException {
229     // create scan with any properties set from TableInputFormat
230     Scan s = TableInputFormat.createScanFromConfiguration(conf);
231     // Set Scan Versions
232     if (conf.get(TableInputFormat.SCAN_MAXVERSIONS) == null) {
233       // default to all versions unless explicitly set
234       s.setMaxVersions(Integer.MAX_VALUE);
235     }
236     s.setCacheBlocks(false);
237     // Set RowFilter or Prefix Filter if applicable.
238     Filter rowFilter = getRowFilter(args);
239     if (rowFilter!= null) {
240       LOG.info("Setting Row Filter for counter.");
241       s.setFilter(rowFilter);
242     }
243     // Set TimeRange if defined
244     long timeRange[] = getTimeRange(args);
245     if (timeRange != null) {
246       LOG.info("Setting TimeRange for counter.");
247       s.setTimeRange(timeRange[0], timeRange[1]);
248     }
249     return s;
250   }
251 
252 
253   private static Filter getRowFilter(String[] args) {
254     Filter rowFilter = null;
255     String filterCriteria = (args.length > 3) ? args[3]: null;
256     if (filterCriteria == null) return null;
257     if (filterCriteria.startsWith("^")) {
258       String regexPattern = filterCriteria.substring(1, filterCriteria.length());
259       rowFilter = new RowFilter(CompareFilter.CompareOp.EQUAL, new RegexStringComparator(regexPattern));
260     } else {
261       rowFilter = new PrefixFilter(Bytes.toBytesBinary(filterCriteria));
262     }
263     return rowFilter;
264   }
265 
266   private static long[] getTimeRange(String[] args) throws IOException {
267     final String startTimeArgKey = "--starttime=";
268     final String endTimeArgKey = "--endtime=";
269     long startTime = 0L;
270     long endTime = 0L;
271 
272     for (int i = 1; i < args.length; i++) {
273       System.out.println("i:" + i + "arg[i]" + args[i]);
274       if (args[i].startsWith(startTimeArgKey)) {
275         startTime = Long.parseLong(args[i].substring(startTimeArgKey.length()));
276       }
277       if (args[i].startsWith(endTimeArgKey)) {
278         endTime = Long.parseLong(args[i].substring(endTimeArgKey.length()));
279       }
280     }
281 
282     if (startTime == 0 && endTime == 0)
283       return null;
284 
285     endTime = endTime == 0 ? HConstants.LATEST_TIMESTAMP : endTime;
286     return new long [] {startTime, endTime};
287   }
288   /**
289    * Main entry point.
290    *
291    * @param args The command line parameters.
292    * @throws Exception When running the job fails.
293    */
294   public static void main(String[] args) throws Exception {
295     Configuration conf = HBaseConfiguration.create();
296     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
297     if (otherArgs.length < 2) {
298       System.err.println("ERROR: Wrong number of parameters: " + args.length);
299       System.err.println("Usage: CellCounter ");
300       System.err.println("       <tablename> <outputDir> <reportSeparator> [^[regex pattern] or " +
301         "[Prefix] for row filter]] --starttime=[starttime] --endtime=[endtime]");
302       System.err.println("  Note: -D properties will be applied to the conf used. ");
303       System.err.println("  Additionally, all of the SCAN properties from TableInputFormat");
304       System.err.println("  can be specified to get fine grained control on what is counted..");
305       System.err.println("   -D " + TableInputFormat.SCAN_ROW_START + "=<rowkey>");
306       System.err.println("   -D " + TableInputFormat.SCAN_ROW_STOP + "=<rowkey>");
307       System.err.println("   -D " + TableInputFormat.SCAN_COLUMNS + "=\"<col1> <col2>...\"");
308       System.err.println("   -D " + TableInputFormat.SCAN_COLUMN_FAMILY + "=<familyName>");
309       System.err.println("   -D " + TableInputFormat.SCAN_TIMESTAMP + "=<timestamp>");
310       System.err.println("   -D " + TableInputFormat.SCAN_TIMERANGE_START + "=<timestamp>");
311       System.err.println("   -D " + TableInputFormat.SCAN_TIMERANGE_END + "=<timestamp>");
312       System.err.println("   -D " + TableInputFormat.SCAN_MAXVERSIONS + "=<count>");
313       System.err.println("   -D " + TableInputFormat.SCAN_CACHEDROWS + "=<count>");
314       System.err.println("   -D " + TableInputFormat.SCAN_BATCHSIZE + "=<count>");
315       System.err.println(" <reportSeparator> parameter can be used to override the default report separator " +
316           "string : used to separate the rowId/column family name and qualifier name.");
317       System.err.println(" [^[regex pattern] or [Prefix] parameter can be used to limit the cell counter count " +
318           "operation to a limited subset of rows from the table based on regex or prefix pattern.");
319       System.exit(-1);
320     }
321     Job job = createSubmittableJob(conf, otherArgs);
322     System.exit(job.waitForCompletion(true) ? 0 : 1);
323   }
324 }