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 static java.lang.String.format;
22  
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.HashSet;
26  import java.util.Set;
27  
28  import org.apache.commons.logging.Log;
29  import org.apache.commons.logging.LogFactory;
30  import org.apache.hadoop.classification.InterfaceAudience;
31  import org.apache.hadoop.classification.InterfaceStability;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.conf.Configured;
34  import org.apache.hadoop.fs.Path;
35  import org.apache.hadoop.hbase.HBaseConfiguration;
36  import org.apache.hadoop.hbase.HColumnDescriptor;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.HTableDescriptor;
39  import org.apache.hadoop.hbase.TableName;
40  import org.apache.hadoop.hbase.client.HBaseAdmin;
41  import org.apache.hadoop.hbase.client.HTable;
42  import org.apache.hadoop.hbase.client.Put;
43  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
44  import org.apache.hadoop.hbase.util.Base64;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.apache.hadoop.hbase.util.Pair;
47  import org.apache.hadoop.io.Text;
48  import org.apache.hadoop.mapreduce.Job;
49  import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
50  import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
51  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
52  import org.apache.hadoop.util.GenericOptionsParser;
53  import org.apache.hadoop.util.Tool;
54  import org.apache.hadoop.util.ToolRunner;
55  
56  import com.google.common.base.Preconditions;
57  import com.google.common.base.Splitter;
58  import com.google.common.collect.Lists;
59  
60  /**
61   * Tool to import data from a TSV file.
62   *
63   * This tool is rather simplistic - it doesn't do any quoting or
64   * escaping, but is useful for many data loads.
65   *
66   * @see ImportTsv#usage(String)
67   */
68  @InterfaceAudience.Public
69  @InterfaceStability.Stable
70  public class ImportTsv extends Configured implements Tool {
71  
72    protected static final Log LOG = LogFactory.getLog(ImportTsv.class);
73  
74    final static String NAME = "importtsv";
75  
76    public final static String MAPPER_CONF_KEY = "importtsv.mapper.class";
77    public final static String BULK_OUTPUT_CONF_KEY = "importtsv.bulk.output";
78    public final static String TIMESTAMP_CONF_KEY = "importtsv.timestamp";
79    public final static String JOB_NAME_CONF_KEY = "mapred.job.name";
80    // TODO: the rest of these configs are used exclusively by TsvImporterMapper.
81    // Move them out of the tool and let the mapper handle its own validation.
82    public final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
83    public final static String COLUMNS_CONF_KEY = "importtsv.columns";
84    public final static String SEPARATOR_CONF_KEY = "importtsv.separator";
85  
86    final static String DEFAULT_SEPARATOR = "\t";
87    final static Class DEFAULT_MAPPER = TsvImporterMapper.class;
88  
89    public static class TsvParser {
90      /**
91       * Column families and qualifiers mapped to the TSV columns
92       */
93      private final byte[][] families;
94      private final byte[][] qualifiers;
95  
96      private final byte separatorByte;
97  
98      private int rowKeyColumnIndex;
99  
100     private int maxColumnCount;
101 
102     // Default value must be negative
103     public static final int DEFAULT_TIMESTAMP_COLUMN_INDEX = -1;
104 
105     private int timestampKeyColumnIndex = DEFAULT_TIMESTAMP_COLUMN_INDEX;
106 
107     public static final String ROWKEY_COLUMN_SPEC = "HBASE_ROW_KEY";
108 
109     public static final String TIMESTAMPKEY_COLUMN_SPEC = "HBASE_TS_KEY";
110 
111     /**
112      * @param columnsSpecification the list of columns to parser out, comma separated.
113      * The row key should be the special token TsvParser.ROWKEY_COLUMN_SPEC
114      */
115     public TsvParser(String columnsSpecification, String separatorStr) {
116       // Configure separator
117       byte[] separator = Bytes.toBytes(separatorStr);
118       Preconditions.checkArgument(separator.length == 1,
119         "TsvParser only supports single-byte separators");
120       separatorByte = separator[0];
121 
122       // Configure columns
123       ArrayList<String> columnStrings = Lists.newArrayList(
124         Splitter.on(',').trimResults().split(columnsSpecification));
125 
126       maxColumnCount = columnStrings.size();
127       families = new byte[maxColumnCount][];
128       qualifiers = new byte[maxColumnCount][];
129 
130       for (int i = 0; i < columnStrings.size(); i++) {
131         String str = columnStrings.get(i);
132         if (ROWKEY_COLUMN_SPEC.equals(str)) {
133           rowKeyColumnIndex = i;
134           continue;
135         }
136         
137         if (TIMESTAMPKEY_COLUMN_SPEC.equals(str)) {
138           timestampKeyColumnIndex = i;
139           continue;
140         }
141         
142         String[] parts = str.split(":", 2);
143         if (parts.length == 1) {
144           families[i] = str.getBytes();
145           qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
146         } else {
147           families[i] = parts[0].getBytes();
148           qualifiers[i] = parts[1].getBytes();
149         }
150       }
151     }
152 
153     public boolean hasTimestamp() {
154       return timestampKeyColumnIndex != DEFAULT_TIMESTAMP_COLUMN_INDEX;
155     }
156 
157     public int getTimestampKeyColumnIndex() {
158       return timestampKeyColumnIndex;
159     }
160 
161     public int getRowKeyColumnIndex() {
162       return rowKeyColumnIndex;
163     }
164     public byte[] getFamily(int idx) {
165       return families[idx];
166     }
167     public byte[] getQualifier(int idx) {
168       return qualifiers[idx];
169     }
170 
171     public ParsedLine parse(byte[] lineBytes, int length)
172     throws BadTsvLineException {
173       // Enumerate separator offsets
174       ArrayList<Integer> tabOffsets = new ArrayList<Integer>(maxColumnCount);
175       for (int i = 0; i < length; i++) {
176         if (lineBytes[i] == separatorByte) {
177           tabOffsets.add(i);
178         }
179       }
180       if (tabOffsets.isEmpty()) {
181         throw new BadTsvLineException("No delimiter");
182       }
183 
184       tabOffsets.add(length);
185 
186       if (tabOffsets.size() > maxColumnCount) {
187         throw new BadTsvLineException("Excessive columns");
188       } else if (tabOffsets.size() <= getRowKeyColumnIndex()) {
189         throw new BadTsvLineException("No row key");
190       } else if (hasTimestamp()
191           && tabOffsets.size() <= getTimestampKeyColumnIndex()) {
192         throw new BadTsvLineException("No timestamp");
193       }
194       return new ParsedLine(tabOffsets, lineBytes);
195     }
196 
197     class ParsedLine {
198       private final ArrayList<Integer> tabOffsets;
199       private byte[] lineBytes;
200 
201       ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
202         this.tabOffsets = tabOffsets;
203         this.lineBytes = lineBytes;
204       }
205 
206       public int getRowKeyOffset() {
207         return getColumnOffset(rowKeyColumnIndex);
208       }
209       public int getRowKeyLength() {
210         return getColumnLength(rowKeyColumnIndex);
211       }
212       
213       public long getTimestamp(long ts) throws BadTsvLineException {
214         // Return ts if HBASE_TS_KEY is not configured in column spec
215         if (!hasTimestamp()) {
216           return ts;
217         }
218 
219         String timeStampStr = Bytes.toString(lineBytes,
220             getColumnOffset(timestampKeyColumnIndex),
221             getColumnLength(timestampKeyColumnIndex));
222         try {
223           return Long.parseLong(timeStampStr);
224         } catch (NumberFormatException nfe) {
225           // treat this record as bad record
226           throw new BadTsvLineException("Invalid timestamp " + timeStampStr);
227         }
228       }
229       
230       public int getColumnOffset(int idx) {
231         if (idx > 0)
232           return tabOffsets.get(idx - 1) + 1;
233         else
234           return 0;
235       }
236       public int getColumnLength(int idx) {
237         return tabOffsets.get(idx) - getColumnOffset(idx);
238       }
239       public int getColumnCount() {
240         return tabOffsets.size();
241       }
242       public byte[] getLineBytes() {
243         return lineBytes;
244       }
245     }
246 
247     public static class BadTsvLineException extends Exception {
248       public BadTsvLineException(String err) {
249         super(err);
250       }
251       private static final long serialVersionUID = 1L;
252     }
253 
254     public Pair<Integer, Integer> parseRowKey(byte[] lineBytes, int length)
255         throws BadTsvLineException {
256       int rkColumnIndex = 0;
257       int startPos = 0, endPos = 0;
258       for (int i = 0; i <= length; i++) {
259         if (i == length || lineBytes[i] == separatorByte) {
260           endPos = i - 1;
261           if (rkColumnIndex++ == getRowKeyColumnIndex()) {
262             if ((endPos + 1) == startPos) {
263               throw new BadTsvLineException("Empty value for ROW KEY.");
264             }
265             break;
266           } else {
267             startPos = endPos + 2;
268           }
269         }
270         if (i == length) {
271           throw new BadTsvLineException(
272               "Row key does not exist as number of columns in the line"
273                   + " are less than row key position.");
274         }
275       }
276       return new Pair<Integer, Integer>(startPos, endPos);
277     }
278   }
279 
280   /**
281    * Sets up the actual job.
282    *
283    * @param conf  The current configuration.
284    * @param args  The command line parameters.
285    * @return The newly created job.
286    * @throws IOException When setting up the job fails.
287    */
288   public static Job createSubmittableJob(Configuration conf, String[] args)
289       throws IOException, ClassNotFoundException {
290 
291     HBaseAdmin admin = new HBaseAdmin(conf);
292 
293     // Support non-XML supported characters
294     // by re-encoding the passed separator as a Base64 string.
295     String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
296     if (actualSeparator != null) {
297       conf.set(SEPARATOR_CONF_KEY,
298                Base64.encodeBytes(actualSeparator.getBytes()));
299     }
300 
301     // See if a non-default Mapper was set
302     String mapperClassName = conf.get(MAPPER_CONF_KEY);
303     Class mapperClass = mapperClassName != null ?
304         Class.forName(mapperClassName) : DEFAULT_MAPPER;
305 
306     conf.setStrings("io.serializations", conf.get("io.serializations"),
307         MutationSerialization.class.getName(), ResultSerialization.class.getName());
308 
309     String tableName = args[0];
310     Path inputDir = new Path(args[1]);
311     String jobName = conf.get(JOB_NAME_CONF_KEY,NAME + "_" + tableName);
312     Job job = new Job(conf, jobName);
313     job.setJarByClass(mapperClass);
314     FileInputFormat.setInputPaths(job, inputDir);
315     job.setInputFormatClass(TextInputFormat.class);
316     job.setMapperClass(mapperClass);
317 
318     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
319     String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
320     if (hfileOutPath != null) {
321       if (!admin.tableExists(tableName)) {
322         LOG.warn(format("Table '%s' does not exist.", tableName));
323         // TODO: this is backwards. Instead of depending on the existence of a table,
324         // create a sane splits file for HFileOutputFormat based on data sampling.
325         createTable(admin, tableName, columns);
326       }
327       HTable table = new HTable(conf, tableName);
328       job.setReducerClass(PutSortReducer.class);
329       Path outputDir = new Path(hfileOutPath);
330       FileOutputFormat.setOutputPath(job, outputDir);
331       job.setMapOutputKeyClass(ImmutableBytesWritable.class);
332       if (mapperClass.equals(TsvImporterTextMapper.class)) {
333         job.setMapOutputValueClass(Text.class);
334         job.setReducerClass(TextSortReducer.class);
335       } else {
336         job.setMapOutputValueClass(Put.class);
337         job.setCombinerClass(PutCombiner.class);
338       }
339       HFileOutputFormat.configureIncrementalLoad(job, table);
340     } else {
341       if (mapperClass.equals(TsvImporterTextMapper.class)) {
342         usage(TsvImporterTextMapper.class.toString()
343             + " should not be used for non bulkloading case. use "
344             + TsvImporterMapper.class.toString()
345             + " or custom mapper whose value type is Put.");
346         System.exit(-1);
347       }
348       // No reducers. Just write straight to table. Call initTableReducerJob
349       // to set up the TableOutputFormat.
350       TableMapReduceUtil.initTableReducerJob(tableName, null, job);
351       job.setNumReduceTasks(0);
352     }
353 
354     TableMapReduceUtil.addDependencyJars(job);
355     TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
356         com.google.common.base.Function.class /* Guava used by TsvParser */);
357     return job;
358   }
359 
360   private static void createTable(HBaseAdmin admin, String tableName, String[] columns)
361       throws IOException {
362     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
363     Set<String> cfSet = new HashSet<String>();
364     for (String aColumn : columns) {
365       if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)) continue;
366       // we are only concerned with the first one (in case this is a cf:cq)
367       cfSet.add(aColumn.split(":", 2)[0]);
368     }
369     for (String cf : cfSet) {
370       HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(cf));
371       htd.addFamily(hcd);
372     }
373     LOG.warn(format("Creating table '%s' with '%s' columns and default descriptors.",
374       tableName, cfSet));
375     admin.createTable(htd);
376   }
377 
378   /*
379    * @param errorMsg Error message.  Can be null.
380    */
381   private static void usage(final String errorMsg) {
382     if (errorMsg != null && errorMsg.length() > 0) {
383       System.err.println("ERROR: " + errorMsg);
384     }
385     String usage = 
386       "Usage: " + NAME + " -D"+ COLUMNS_CONF_KEY + "=a,b,c <tablename> <inputdir>\n" +
387       "\n" +
388       "Imports the given input directory of TSV data into the specified table.\n" +
389       "\n" +
390       "The column names of the TSV data must be specified using the -D" + COLUMNS_CONF_KEY + "\n" +
391       "option. This option takes the form of comma-separated column names, where each\n" +
392       "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
393       "column name " + TsvParser.ROWKEY_COLUMN_SPEC + " is used to designate that this column should be used\n" +
394       "as the row key for each imported record. You must specify exactly one column\n" +
395       "to be the row key, and you must specify a column name for every column that exists in the\n" +
396       "input data. Another special column" + TsvParser.TIMESTAMPKEY_COLUMN_SPEC +
397       " designates that this column should be\n" +
398       "used as timestamp for each record. Unlike " + TsvParser.ROWKEY_COLUMN_SPEC + ", " +
399       TsvParser.TIMESTAMPKEY_COLUMN_SPEC + " is optional.\n" +
400       "You must specify at most one column as timestamp key for each imported record.\n" +
401       "Record with invalid timestamps (blank, non-numeric) will be treated as bad record.\n" +
402       "Note: if you use this option, then '" + TIMESTAMP_CONF_KEY + "' option will be ignored.\n" +
403       "\n" +
404       "By default importtsv will load data directly into HBase. To instead generate\n" +
405       "HFiles of data to prepare for a bulk data load, pass the option:\n" +
406       "  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
407       "  Note: if you do not use this option, then the target table must already exist in HBase\n" +
408       "\n" +
409       "Other options that may be specified with -D include:\n" +
410       "  -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
411       "  '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs\n" +
412       "  -D" + TIMESTAMP_CONF_KEY + "=currentTimeAsLong - use the specified timestamp for the import\n" +
413       "  -D" + MAPPER_CONF_KEY + "=my.Mapper - A user-defined Mapper to use instead of " +
414       DEFAULT_MAPPER.getName() + "\n" +
415       "  -D" + JOB_NAME_CONF_KEY + "=jobName - use the specified mapreduce job name for the import\n" +
416       "For performance consider the following options:\n" +
417       "  -Dmapred.map.tasks.speculative.execution=false\n" +
418       "  -Dmapred.reduce.tasks.speculative.execution=false";
419 
420     System.err.println(usage);
421   }
422 
423   @Override
424   public int run(String[] args) throws Exception {
425     setConf(HBaseConfiguration.create(getConf()));
426     String[] otherArgs = new GenericOptionsParser(getConf(), args).getRemainingArgs();
427     if (otherArgs.length < 2) {
428       usage("Wrong number of arguments: " + otherArgs.length);
429       return -1;
430     }
431 
432     // When MAPPER_CONF_KEY is null, the user wants to use the provided TsvImporterMapper, so
433     // perform validation on these additional args. When it's not null, user has provided their
434     // own mapper, thus these validation are not relevant.
435     // TODO: validation for TsvImporterMapper, not this tool. Move elsewhere.
436     if (null == getConf().get(MAPPER_CONF_KEY)) {
437       // Make sure columns are specified
438       String columns[] = getConf().getStrings(COLUMNS_CONF_KEY);
439       if (columns == null) {
440         usage("No columns specified. Please specify with -D" +
441             COLUMNS_CONF_KEY+"=...");
442         return -1;
443       }
444 
445       // Make sure they specify exactly one column as the row key
446       int rowkeysFound = 0;
447       for (String col : columns) {
448         if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
449       }
450       if (rowkeysFound != 1) {
451         usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
452         return -1;
453       }
454 
455       // Make sure we have at most one column as the timestamp key
456       int tskeysFound = 0;
457       for (String col : columns) {
458         if (col.equals(TsvParser.TIMESTAMPKEY_COLUMN_SPEC))
459           tskeysFound++;
460       }
461       if (tskeysFound > 1) {
462         usage("Must specify at most one column as "
463             + TsvParser.TIMESTAMPKEY_COLUMN_SPEC);
464         return -1;
465       }
466     
467       // Make sure one or more columns are specified excluding rowkey and
468       // timestamp key
469       if (columns.length - (rowkeysFound + tskeysFound) < 1) {
470         usage("One or more columns in addition to the row key and timestamp(optional) are required");
471         return -1;
472       }
473     }
474 
475     // If timestamp option is not specified, use current system time.
476     long timstamp = getConf().getLong(TIMESTAMP_CONF_KEY, System.currentTimeMillis());
477 
478     // Set it back to replace invalid timestamp (non-numeric) with current
479     // system time
480     getConf().setLong(TIMESTAMP_CONF_KEY, timstamp);
481     
482     Job job = createSubmittableJob(getConf(), otherArgs);
483     return job.waitForCompletion(true) ? 0 : 1;
484   }
485 
486   public static void main(String[] args) throws Exception {
487     int status = ToolRunner.run(new ImportTsv(), args);
488     System.exit(status);
489   }
490 }