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.classification.InterfaceAudience;
26  import org.apache.hadoop.classification.InterfaceStability;
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.fs.FileSystem;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.HBaseConfiguration;
31  import org.apache.hadoop.hbase.client.Result;
32  import org.apache.hadoop.hbase.client.Scan;
33  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
34  import org.apache.hadoop.hbase.filter.Filter;
35  import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
36  import org.apache.hadoop.hbase.filter.PrefixFilter;
37  import org.apache.hadoop.hbase.filter.RegexStringComparator;
38  import org.apache.hadoop.hbase.filter.RowFilter;
39  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
40  import org.apache.hadoop.hbase.util.Bytes;
41  import org.apache.hadoop.mapreduce.Job;
42  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
43  import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
44  import org.apache.hadoop.util.GenericOptionsParser;
45  
46  /**
47  * Export an HBase table.
48  * Writes content to sequence files up in HDFS.  Use {@link Import} to read it
49  * back in again.
50  */
51  @InterfaceAudience.Public
52  @InterfaceStability.Stable
53  public class Export {
54    private static final Log LOG = LogFactory.getLog(Export.class);
55    final static String NAME = "export";
56    final static String RAW_SCAN = "hbase.mapreduce.include.deleted.rows";
57    final static String EXPORT_BATCHING = "hbase.export.scanner.batch";
58  
59    /**
60     * Sets up the actual job.
61     *
62     * @param conf  The current configuration.
63     * @param args  The command line parameters.
64     * @return The newly created job.
65     * @throws IOException When setting up the job fails.
66     */
67    public static Job createSubmittableJob(Configuration conf, String[] args)
68    throws IOException {
69      String tableName = args[0];
70      Path outputDir = new Path(args[1]);
71      Job job = new Job(conf, NAME + "_" + tableName);
72      job.setJobName(NAME + "_" + tableName);
73      job.setJarByClass(Export.class);
74      // Set optional scan parameters
75      Scan s = getConfiguredScanForJob(conf, args);
76      IdentityTableMapper.initJob(tableName, s, IdentityTableMapper.class, job);
77      // No reducers.  Just write straight to output files.
78      job.setNumReduceTasks(0);
79      job.setOutputFormatClass(SequenceFileOutputFormat.class);
80      job.setOutputKeyClass(ImmutableBytesWritable.class);
81      job.setOutputValueClass(Result.class);
82      FileOutputFormat.setOutputPath(job, outputDir); // job conf doesn't contain the conf so doesn't have a default fs.
83      return job;
84    }
85  
86    private static Scan getConfiguredScanForJob(Configuration conf, String[] args) throws IOException {
87      Scan s = new Scan();
88      // Optional arguments.
89      // Set Scan Versions
90      int versions = args.length > 2? Integer.parseInt(args[2]): 1;
91      s.setMaxVersions(versions);
92      // Set Scan Range
93      long startTime = args.length > 3? Long.parseLong(args[3]): 0L;
94      long endTime = args.length > 4? Long.parseLong(args[4]): Long.MAX_VALUE;
95      s.setTimeRange(startTime, endTime);
96      // Set cache blocks
97      s.setCacheBlocks(false);
98      // Set Scan Column Family
99      boolean raw = Boolean.parseBoolean(conf.get(RAW_SCAN));
100     if (raw) {
101       s.setRaw(raw);
102     }
103     
104     if (conf.get(TableInputFormat.SCAN_COLUMN_FAMILY) != null) {
105       s.addFamily(Bytes.toBytes(conf.get(TableInputFormat.SCAN_COLUMN_FAMILY)));
106     }
107     // Set RowFilter or Prefix Filter if applicable.
108     Filter exportFilter = getExportFilter(args);
109     if (exportFilter!= null) {
110         LOG.info("Setting Scan Filter for Export.");
111       s.setFilter(exportFilter);
112     }
113 
114     int batching = conf.getInt(EXPORT_BATCHING, -1);
115     if (batching !=  -1){
116       try {
117         s.setBatch(batching);
118       } catch (IncompatibleFilterException e) {
119         LOG.error("Batching could not be set", e);
120       }
121     }
122     LOG.info("versions=" + versions + ", starttime=" + startTime +
123       ", endtime=" + endTime + ", keepDeletedCells=" + raw);
124     return s;
125   }
126 
127   private static Filter getExportFilter(String[] args) {
128     Filter exportFilter = null;
129     String filterCriteria = (args.length > 5) ? args[5]: null;
130     if (filterCriteria == null) return null;
131     if (filterCriteria.startsWith("^")) {
132       String regexPattern = filterCriteria.substring(1, filterCriteria.length());
133       exportFilter = new RowFilter(CompareOp.EQUAL, new RegexStringComparator(regexPattern));
134     } else {
135       exportFilter = new PrefixFilter(Bytes.toBytes(filterCriteria));
136     }
137     return exportFilter;
138   }
139 
140   /*
141    * @param errorMsg Error message.  Can be null.
142    */
143   private static void usage(final String errorMsg) {
144     if (errorMsg != null && errorMsg.length() > 0) {
145       System.err.println("ERROR: " + errorMsg);
146     }
147     System.err.println("Usage: Export [-D <property=value>]* <tablename> <outputdir> [<versions> " +
148       "[<starttime> [<endtime>]] [^[regex pattern] or [Prefix] to filter]]\n");
149     System.err.println("  Note: -D properties will be applied to the conf used. ");
150     System.err.println("  For example: ");
151     System.err.println("   -D mapred.output.compress=true");
152     System.err.println("   -D mapred.output.compression.codec=org.apache.hadoop.io.compress.GzipCodec");
153     System.err.println("   -D mapred.output.compression.type=BLOCK");
154     System.err.println("  Additionally, the following SCAN properties can be specified");
155     System.err.println("  to control/limit what is exported..");
156     System.err.println("   -D " + TableInputFormat.SCAN_COLUMN_FAMILY + "=<familyName>");
157     System.err.println("   -D " + RAW_SCAN + "=true");
158     System.err.println("For performance consider the following properties:\n"
159         + "   -Dhbase.client.scanner.caching=100\n"
160         + "   -Dmapred.map.tasks.speculative.execution=false\n"
161         + "   -Dmapred.reduce.tasks.speculative.execution=false");
162     System.err.println("For tables with very wide rows consider setting the batch size as below:\n"
163         + "   -D" + EXPORT_BATCHING + "=10");
164   }
165 
166   /**
167    * Main entry point.
168    *
169    * @param args  The command line parameters.
170    * @throws Exception When running the job fails.
171    */
172   public static void main(String[] args) throws Exception {
173     Configuration conf = HBaseConfiguration.create();
174     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
175     if (otherArgs.length < 2) {
176       usage("Wrong number of arguments: " + otherArgs.length);
177       System.exit(-1);
178     }
179     Job job = createSubmittableJob(conf, otherArgs);
180     System.exit(job.waitForCompletion(true)? 0 : 1);
181   }
182 }