001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.mapreduce;
020
021import java.io.IOException;
022import java.util.List;
023import java.util.ArrayList;
024
025import org.apache.commons.lang3.StringUtils;
026import org.apache.hadoop.hbase.HConstants;
027import org.apache.hadoop.hbase.util.AbstractHBaseTool;
028import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
029import org.apache.hbase.thirdparty.org.apache.commons.cli.BasicParser;
030import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
031import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
032import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
033import org.apache.hbase.thirdparty.org.apache.commons.cli.MissingOptionException;
034import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
035import org.apache.yetus.audience.InterfaceAudience;
036import org.slf4j.Logger;
037import org.slf4j.LoggerFactory;
038import org.apache.hadoop.conf.Configuration;
039import org.apache.hadoop.hbase.client.Result;
040import org.apache.hadoop.hbase.client.Scan;
041import org.apache.hadoop.hbase.filter.FilterBase;
042import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
043import org.apache.hadoop.hbase.filter.MultiRowRangeFilter;
044import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
045import org.apache.hadoop.hbase.util.Bytes;
046import org.apache.hadoop.mapreduce.Counter;
047import org.apache.hadoop.mapreduce.Job;
048import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
049
050/**
051 * A job with a just a map phase to count rows. Map outputs table rows IF the
052 * input row has columns that have content.
053 */
054@InterfaceAudience.Public
055public class RowCounter extends AbstractHBaseTool {
056
057  private static final Logger LOG = LoggerFactory.getLogger(RowCounter.class);
058
059  /** Name of this 'program'. */
060  static final String NAME = "rowcounter";
061
062  private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
063  private final static String EXPECTED_COUNT_KEY = RowCounter.class.getName() + ".expected_count";
064
065  private final static String OPT_START_TIME = "starttime";
066  private final static String OPT_END_TIME = "endtime";
067  private final static String OPT_RANGE = "range";
068  private final static String OPT_EXPECTED_COUNT = "expectedCount";
069
070  private String tableName;
071  private List<MultiRowRangeFilter.RowRange> rowRangeList;
072  private long startTime;
073  private long endTime;
074  private long expectedCount;
075  private List<String> columns = new ArrayList<>();
076
077  /**
078   * Mapper that runs the count.
079   */
080  static class RowCounterMapper
081  extends TableMapper<ImmutableBytesWritable, Result> {
082
083    /** Counter enumeration to count the actual rows. */
084    public static enum Counters {ROWS}
085
086    /**
087     * Maps the data.
088     *
089     * @param row  The current table row key.
090     * @param values  The columns.
091     * @param context  The current context.
092     * @throws IOException When something is broken with the data.
093     * @see org.apache.hadoop.mapreduce.Mapper#map(Object, Object, Context)
094     */
095    @Override
096    public void map(ImmutableBytesWritable row, Result values,
097      Context context)
098    throws IOException {
099      // Count every row containing data, whether it's in qualifiers or values
100      context.getCounter(Counters.ROWS).increment(1);
101    }
102  }
103
104  /**
105   * Sets up the actual job.
106   *
107   * @param conf  The current configuration.
108   * @return The newly created job.
109   * @throws IOException When setting up the job fails.
110   */
111  public Job createSubmittableJob(Configuration conf) throws IOException {
112    Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
113    job.setJarByClass(RowCounter.class);
114    Scan scan = new Scan();
115    scan.setCacheBlocks(false);
116    setScanFilter(scan, rowRangeList);
117
118    for (String columnName : this.columns) {
119      String family = StringUtils.substringBefore(columnName, ":");
120      String qualifier = StringUtils.substringAfter(columnName, ":");
121      if (StringUtils.isBlank(qualifier)) {
122        scan.addFamily(Bytes.toBytes(family));
123      } else {
124        scan.addColumn(Bytes.toBytes(family), Bytes.toBytes(qualifier));
125      }
126    }
127
128    if(this.expectedCount >= 0) {
129      conf.setLong(EXPECTED_COUNT_KEY, this.expectedCount);
130    }
131
132    scan.setTimeRange(startTime, endTime);
133    job.setOutputFormatClass(NullOutputFormat.class);
134    TableMapReduceUtil.initTableMapperJob(tableName, scan,
135      RowCounterMapper.class, ImmutableBytesWritable.class, Result.class, job);
136    job.setNumReduceTasks(0);
137    return job;
138  }
139
140  private static List<MultiRowRangeFilter.RowRange> parseRowRangeParameter(String arg) {
141    final List<String> rangesSplit = Splitter.on(";").splitToList(arg);
142    final List<MultiRowRangeFilter.RowRange> rangeList = new ArrayList<>();
143    for (String range : rangesSplit) {
144      if(range!=null && !range.isEmpty()) {
145        List<String> startEnd = Splitter.on(",").splitToList(range);
146        if (startEnd.size() != 2 || startEnd.get(1).contains(",")) {
147          throw new IllegalArgumentException("Wrong range specification: " + range);
148        }
149        String startKey = startEnd.get(0);
150        String endKey = startEnd.get(1);
151        rangeList.add(new MultiRowRangeFilter.RowRange(Bytes.toBytesBinary(startKey),
152            true, Bytes.toBytesBinary(endKey), false));
153      }
154    }
155    return rangeList;
156  }
157
158  /**
159   * Sets filter {@link FilterBase} to the {@link Scan} instance.
160   * If provided rowRangeList contains more than one element,
161   * method sets filter which is instance of {@link MultiRowRangeFilter}.
162   * Otherwise, method sets filter which is instance of {@link FirstKeyOnlyFilter}.
163   * If rowRangeList contains exactly one element, startRow and stopRow are set to the scan.
164   * @param scan
165   * @param rowRangeList
166   */
167  private static void setScanFilter(Scan scan, List<MultiRowRangeFilter.RowRange> rowRangeList) {
168    final int size = rowRangeList == null ? 0 : rowRangeList.size();
169    if (size <= 1) {
170      scan.setFilter(new FirstKeyOnlyFilter());
171    }
172    if (size == 1) {
173      MultiRowRangeFilter.RowRange range = rowRangeList.get(0);
174      scan.setStartRow(range.getStartRow()); //inclusive
175      scan.setStopRow(range.getStopRow());   //exclusive
176    } else if (size > 1) {
177      scan.setFilter(new MultiRowRangeFilter(rowRangeList));
178    }
179  }
180
181  @Override
182  protected void printUsage() {
183    StringBuilder footerBuilder = new StringBuilder();
184    footerBuilder.append("For performance, consider the following configuration properties:\n");
185    footerBuilder.append("-Dhbase.client.scanner.caching=100\n");
186    footerBuilder.append("-Dmapreduce.map.speculative=false\n");
187    printUsage("hbase rowcounter <tablename> [options] [<column1> <column2>...]",
188        "Options:", footerBuilder.toString());
189  }
190
191  @Override
192  protected void printUsage(final String usageStr, final String usageHeader,
193      final String usageFooter) {
194    HelpFormatter helpFormatter = new HelpFormatter();
195    helpFormatter.setWidth(120);
196    helpFormatter.setOptionComparator(new AbstractHBaseTool.OptionsOrderComparator());
197    helpFormatter.setLongOptSeparator("=");
198    helpFormatter.printHelp(usageStr, usageHeader, options, usageFooter);
199  }
200
201  @Override
202  protected void addOptions() {
203    Option startTimeOption = Option.builder(null).valueSeparator('=').hasArg(true).
204        desc("starting time filter to start counting rows from.").longOpt(OPT_START_TIME).build();
205    Option endTimeOption = Option.builder(null).valueSeparator('=').hasArg(true).
206        desc("end time filter limit, to only count rows up to this timestamp.").
207        longOpt(OPT_END_TIME).build();
208    Option rangeOption = Option.builder(null).valueSeparator('=').hasArg(true).
209        desc("[startKey],[endKey][;[startKey],[endKey]...]]").longOpt(OPT_RANGE).build();
210    Option expectedOption = Option.builder(null).valueSeparator('=').hasArg(true).
211        desc("expected number of rows to be count.").longOpt(OPT_EXPECTED_COUNT).build();
212    addOption(startTimeOption);
213    addOption(endTimeOption);
214    addOption(rangeOption);
215    addOption(expectedOption);
216  }
217
218  @Override
219  protected void processOptions(CommandLine cmd) throws IllegalArgumentException{
220    this.tableName = cmd.getArgList().get(0);
221    if(cmd.getOptionValue(OPT_RANGE)!=null) {
222      this.rowRangeList = parseRowRangeParameter(cmd.getOptionValue(OPT_RANGE));
223    }
224    this.endTime = cmd.getOptionValue(OPT_END_TIME) == null ? HConstants.LATEST_TIMESTAMP :
225        Long.parseLong(cmd.getOptionValue(OPT_END_TIME));
226    this.expectedCount = cmd.getOptionValue(OPT_EXPECTED_COUNT) == null ? Long.MIN_VALUE :
227        Long.parseLong(cmd.getOptionValue(OPT_EXPECTED_COUNT));
228    this.startTime = cmd.getOptionValue(OPT_START_TIME) == null ? 0 :
229        Long.parseLong(cmd.getOptionValue(OPT_START_TIME));
230
231    for(int i=1; i<cmd.getArgList().size(); i++){
232      String argument = cmd.getArgList().get(i);
233      if(!argument.startsWith("-")){
234        this.columns.add(argument);
235      }
236    }
237
238    if (endTime < startTime) {
239      throw new IllegalArgumentException("--endtime=" + endTime +
240          " needs to be greater than --starttime=" + startTime);
241    }
242  }
243
244  @Override
245  protected void processOldArgs(List<String> args) {
246    List<String> copiedArgs = new ArrayList<>(args);
247    args.removeAll(copiedArgs);
248    for(String arg : copiedArgs){
249      if(arg.startsWith("-") && arg.contains("=")){
250        String[] kv = arg.split("=");
251        args.add(kv[0]);
252        args.add(kv[1]);
253      } else {
254        args.add(arg);
255      }
256    }
257  }
258
259  @Override
260  protected int doWork() throws Exception {
261    Job job = createSubmittableJob(getConf());
262    if (job == null) {
263      return -1;
264    }
265    boolean success = job.waitForCompletion(true);
266    final long expectedCount = getConf().getLong(EXPECTED_COUNT_KEY, -1);
267    if (success && expectedCount != -1) {
268      final Counter counter = job.getCounters().findCounter(RowCounterMapper.Counters.ROWS);
269      success = expectedCount == counter.getValue();
270      if (!success) {
271        LOG.error("Failing job because count of '" + counter.getValue() +
272            "' does not match expected count of '" + expectedCount + "'");
273      }
274    }
275    return (success ? 0 : 1);
276  }
277
278  /**
279   * Main entry point.
280   * @param args The command line parameters.
281   * @throws Exception When running the job fails.
282   */
283  public static void main(String[] args) throws Exception {
284    new RowCounter().doStaticMain(args);
285  }
286
287  static class RowCounterCommandLineParser extends BasicParser {
288
289    @Override
290    protected void checkRequiredOptions() throws MissingOptionException {
291      if(this.cmd.getArgList().size()<1 || this.cmd.getArgList().get(0).startsWith("-")){
292        throw new MissingOptionException("First argument must be a valid table name.");
293      }
294    }
295  }
296
297  @Override
298  protected CommandLineParser newParser() {
299    return new RowCounterCommandLineParser();
300  }
301
302}