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;
022
023import org.apache.commons.lang3.ArrayUtils;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.conf.Configured;
026import org.apache.hadoop.fs.Path;
027import org.apache.hadoop.hbase.TableName;
028import org.apache.hadoop.hbase.client.Result;
029import org.apache.hadoop.hbase.client.Scan;
030import org.apache.hadoop.hbase.HBaseConfiguration;
031import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
032import org.apache.hadoop.hbase.util.Triple;
033import org.apache.hadoop.mapreduce.Job;
034import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
035import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
036import org.apache.hadoop.util.Tool;
037import org.apache.hadoop.util.ToolRunner;
038import org.apache.yetus.audience.InterfaceAudience;
039
040/**
041 * Export an HBase table.
042 * Writes content to sequence files up in HDFS.  Use {@link Import} to read it
043 * back in again.
044 */
045@InterfaceAudience.Public
046public class Export extends Configured implements Tool {
047  static final String NAME = "export";
048  static final String JOB_NAME_CONF_KEY = "mapreduce.job.name";
049
050  /**
051   * Sets up the actual job.
052   *
053   * @param conf  The current configuration.
054   * @param args  The command line parameters.
055   * @return The newly created job.
056   * @throws IOException When setting up the job fails.
057   */
058  public static Job createSubmittableJob(Configuration conf, String[] args)
059  throws IOException {
060    Triple<TableName, Scan, Path> arguments = ExportUtils.getArgumentsFromCommandLine(conf, args);
061    String tableName = arguments.getFirst().getNameAsString();
062    Path outputDir = arguments.getThird();
063    Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
064    job.setJobName(NAME + "_" + tableName);
065    job.setJarByClass(Export.class);
066    // Set optional scan parameters
067    Scan s = arguments.getSecond();
068    IdentityTableMapper.initJob(tableName, s, IdentityTableMapper.class, job);
069    // No reducers.  Just write straight to output files.
070    job.setNumReduceTasks(0);
071    job.setOutputFormatClass(SequenceFileOutputFormat.class);
072    job.setOutputKeyClass(ImmutableBytesWritable.class);
073    job.setOutputValueClass(Result.class);
074    FileOutputFormat.setOutputPath(job, outputDir); // job conf doesn't contain the conf so doesn't have a default fs.
075    return job;
076  }
077
078  @Override
079  public int run(String[] args) throws Exception {
080    if (!ExportUtils.isValidArguements(args)) {
081      ExportUtils.usage("Wrong number of arguments: " + ArrayUtils.getLength(args));
082      System.err.println("   -D " + JOB_NAME_CONF_KEY
083              + "=jobName - use the specified mapreduce job name for the export");
084      System.err.println("For MR performance consider the following properties:");
085      System.err.println("   -D mapreduce.map.speculative=false");
086      System.err.println("   -D mapreduce.reduce.speculative=false");
087      return -1;
088    }
089    Job job = createSubmittableJob(getConf(), args);
090    return (job.waitForCompletion(true) ? 0 : 1);
091  }
092
093  /**
094   * Main entry point.
095   * @param args The command line parameters.
096   * @throws Exception When running the job fails.
097   */
098  public static void main(String[] args) throws Exception {
099    int errCode = ToolRunner.run(HBaseConfiguration.create(), new Export(), args);
100    System.exit(errCode);
101  }
102}