001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.mapreduce;
019
020import java.io.IOException;
021import org.apache.commons.lang3.ArrayUtils;
022import org.apache.hadoop.conf.Configuration;
023import org.apache.hadoop.conf.Configured;
024import org.apache.hadoop.fs.Path;
025import org.apache.hadoop.hbase.HBaseConfiguration;
026import org.apache.hadoop.hbase.TableName;
027import org.apache.hadoop.hbase.client.Result;
028import org.apache.hadoop.hbase.client.Scan;
029import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
030import org.apache.hadoop.hbase.util.Triple;
031import org.apache.hadoop.mapreduce.Job;
032import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
033import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
034import org.apache.hadoop.util.Tool;
035import org.apache.hadoop.util.ToolRunner;
036import org.apache.yetus.audience.InterfaceAudience;
037
038/**
039 * Export an HBase table. Writes content to sequence files up in HDFS. Use {@link Import} to read it
040 * back in again.
041 */
042@InterfaceAudience.Public
043public class Export extends Configured implements Tool {
044  static final String NAME = "export";
045  static final String JOB_NAME_CONF_KEY = "mapreduce.job.name";
046
047  /**
048   * Sets up the actual job.
049   * @param conf The current configuration.
050   * @param args The command line parameters.
051   * @return The newly created job.
052   * @throws IOException When setting up the job fails.
053   */
054  public static Job createSubmittableJob(Configuration conf, String[] args) throws IOException {
055    Triple<TableName, Scan, Path> arguments = ExportUtils.getArgumentsFromCommandLine(conf, args);
056    String tableName = arguments.getFirst().getNameAsString();
057    Path outputDir = arguments.getThird();
058    Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
059    job.setJobName(NAME + "_" + tableName);
060    job.setJarByClass(Export.class);
061    // Set optional scan parameters
062    Scan s = arguments.getSecond();
063    IdentityTableMapper.initJob(tableName, s, IdentityTableMapper.class, job);
064    // No reducers. Just write straight to output files.
065    job.setNumReduceTasks(0);
066    job.setOutputFormatClass(SequenceFileOutputFormat.class);
067    job.setOutputKeyClass(ImmutableBytesWritable.class);
068    job.setOutputValueClass(Result.class);
069    FileOutputFormat.setOutputPath(job, outputDir); // job conf doesn't contain the conf so doesn't
070                                                    // have a default fs.
071    return job;
072  }
073
074  @Override
075  public int run(String[] args) throws Exception {
076    if (!ExportUtils.isValidArguements(args)) {
077      ExportUtils.usage("Wrong number of arguments: " + ArrayUtils.getLength(args));
078      System.err.println("   -D " + JOB_NAME_CONF_KEY
079        + "=jobName - use the specified mapreduce job name for the export");
080      System.err.println("For MR performance consider the following properties:");
081      System.err.println("   -D mapreduce.map.speculative=false");
082      System.err.println("   -D mapreduce.reduce.speculative=false");
083      return -1;
084    }
085    Job job = createSubmittableJob(getConf(), args);
086    return (job.waitForCompletion(true) ? 0 : 1);
087  }
088
089  /**
090   * Main entry point.
091   * @param args The command line parameters.
092   * @throws Exception When running the job fails.
093   */
094  public static void main(String[] args) throws Exception {
095    int errCode = ToolRunner.run(HBaseConfiguration.create(), new Export(), args);
096    System.exit(errCode);
097  }
098}