View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.mapreduce;
21  
22  import org.apache.hadoop.hbase.util.Base64;
23  
24  import java.io.IOException;
25  import java.util.ArrayList;
26  import java.util.HashSet;
27  import java.util.Set;
28  
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.Path;
31  import org.apache.hadoop.hbase.HBaseConfiguration;
32  import org.apache.hadoop.hbase.HColumnDescriptor;
33  import org.apache.hadoop.hbase.HConstants;
34  import org.apache.hadoop.hbase.HTableDescriptor;
35  import org.apache.hadoop.hbase.client.HBaseAdmin;
36  import org.apache.hadoop.hbase.client.HTable;
37  import org.apache.hadoop.hbase.client.Put;
38  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
39  import org.apache.hadoop.hbase.util.Bytes;
40  import org.apache.hadoop.mapreduce.Job;
41  import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
42  import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
43  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
44  import org.apache.hadoop.util.GenericOptionsParser;
45  
46  import com.google.common.base.Preconditions;
47  import com.google.common.base.Splitter;
48  import com.google.common.collect.Lists;
49  
50  /**
51   * Tool to import data from a TSV file.
52   *
53   * This tool is rather simplistic - it doesn't do any quoting or
54   * escaping, but is useful for many data loads.
55   *
56   * @see ImportTsv#usage(String)
57   */
58  public class ImportTsv {
59    final static String NAME = "importtsv";
60  
61    final static String MAPPER_CONF_KEY = "importtsv.mapper.class";
62    final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
63    final static String BULK_OUTPUT_CONF_KEY = "importtsv.bulk.output";
64    final static String COLUMNS_CONF_KEY = "importtsv.columns";
65    final static String SEPARATOR_CONF_KEY = "importtsv.separator";
66    final static String TIMESTAMP_CONF_KEY = "importtsv.timestamp";
67    final static String DEFAULT_SEPARATOR = "\t";
68    final static Class DEFAULT_MAPPER = TsvImporterMapper.class;
69    private static HBaseAdmin hbaseAdmin;
70  
71    static class TsvParser {
72      /**
73       * Column families and qualifiers mapped to the TSV columns
74       */
75      private final byte[][] families;
76      private final byte[][] qualifiers;
77  
78      private final byte separatorByte;
79  
80      private int rowKeyColumnIndex;
81  
82      private int maxColumnCount;
83  
84      // Default value must be negative
85      public static final int DEFAULT_TIMESTAMP_COLUMN_INDEX = -1;
86  
87      private int timestampKeyColumnIndex = DEFAULT_TIMESTAMP_COLUMN_INDEX;
88  
89      public static String ROWKEY_COLUMN_SPEC = "HBASE_ROW_KEY";
90  
91      public static String TIMESTAMPKEY_COLUMN_SPEC = "HBASE_TS_KEY";
92  
93      /**
94       * @param columnsSpecification the list of columns to parser out, comma separated.
95       * The row key should be the special token TsvParser.ROWKEY_COLUMN_SPEC
96       */
97      public TsvParser(String columnsSpecification, String separatorStr) {
98        // Configure separator
99        byte[] separator = Bytes.toBytes(separatorStr);
100       Preconditions.checkArgument(separator.length == 1,
101         "TsvParser only supports single-byte separators");
102       separatorByte = separator[0];
103 
104       // Configure columns
105       ArrayList<String> columnStrings = Lists.newArrayList(
106         Splitter.on(',').trimResults().split(columnsSpecification));
107 
108       maxColumnCount = columnStrings.size();
109       families = new byte[maxColumnCount][];
110       qualifiers = new byte[maxColumnCount][];
111 
112       for (int i = 0; i < columnStrings.size(); i++) {
113         String str = columnStrings.get(i);
114         if (ROWKEY_COLUMN_SPEC.equals(str)) {
115           rowKeyColumnIndex = i;
116           continue;
117         }
118         
119         if (TIMESTAMPKEY_COLUMN_SPEC.equals(str)) {
120           timestampKeyColumnIndex = i;
121           continue;
122         }
123         
124         String[] parts = str.split(":", 2);
125         if (parts.length == 1) {
126           families[i] = str.getBytes();
127           qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
128         } else {
129           families[i] = parts[0].getBytes();
130           qualifiers[i] = parts[1].getBytes();
131         }
132       }
133     }
134 
135     public boolean hasTimestamp() {
136       return timestampKeyColumnIndex != DEFAULT_TIMESTAMP_COLUMN_INDEX;
137     }
138 
139     public int getTimestampKeyColumnIndex() {
140       return timestampKeyColumnIndex;
141     }
142 
143     public int getRowKeyColumnIndex() {
144       return rowKeyColumnIndex;
145     }
146     public byte[] getFamily(int idx) {
147       return families[idx];
148     }
149     public byte[] getQualifier(int idx) {
150       return qualifiers[idx];
151     }
152 
153     public ParsedLine parse(byte[] lineBytes, int length)
154     throws BadTsvLineException {
155       // Enumerate separator offsets
156       ArrayList<Integer> tabOffsets = new ArrayList<Integer>(maxColumnCount);
157       for (int i = 0; i < length; i++) {
158         if (lineBytes[i] == separatorByte) {
159           tabOffsets.add(i);
160         }
161       }
162       if (tabOffsets.isEmpty()) {
163         throw new BadTsvLineException("No delimiter");
164       }
165 
166       tabOffsets.add(length);
167 
168       if (tabOffsets.size() > maxColumnCount) {
169         throw new BadTsvLineException("Excessive columns");
170       } else if (tabOffsets.size() <= getRowKeyColumnIndex()) {
171         throw new BadTsvLineException("No row key");
172       } else if (hasTimestamp()
173           && tabOffsets.size() <= getTimestampKeyColumnIndex()) {
174         throw new BadTsvLineException("No timestamp");
175       }
176       return new ParsedLine(tabOffsets, lineBytes);
177     }
178 
179     class ParsedLine {
180       private final ArrayList<Integer> tabOffsets;
181       private byte[] lineBytes;
182 
183       ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
184         this.tabOffsets = tabOffsets;
185         this.lineBytes = lineBytes;
186       }
187 
188       public int getRowKeyOffset() {
189         return getColumnOffset(rowKeyColumnIndex);
190       }
191       public int getRowKeyLength() {
192         return getColumnLength(rowKeyColumnIndex);
193       }
194       
195       public long getTimestamp(long ts) throws BadTsvLineException {
196         // Return ts if HBASE_TS_KEY is not configured in column spec
197         if (!hasTimestamp()) {
198           return ts;
199         }
200 
201         String timeStampStr = Bytes.toString(lineBytes,
202             getColumnOffset(timestampKeyColumnIndex),
203             getColumnLength(timestampKeyColumnIndex));
204         try {
205           return Long.parseLong(timeStampStr);
206         } catch (NumberFormatException nfe) {
207           // treat this record as bad record
208           throw new BadTsvLineException("Invalid timestamp " + timeStampStr);
209         }
210       }
211       
212       public int getColumnOffset(int idx) {
213         if (idx > 0)
214           return tabOffsets.get(idx - 1) + 1;
215         else
216           return 0;
217       }
218       public int getColumnLength(int idx) {
219         return tabOffsets.get(idx) - getColumnOffset(idx);
220       }
221       public int getColumnCount() {
222         return tabOffsets.size();
223       }
224       public byte[] getLineBytes() {
225         return lineBytes;
226       }
227     }
228 
229     public static class BadTsvLineException extends Exception {
230       public BadTsvLineException(String err) {
231         super(err);
232       }
233       private static final long serialVersionUID = 1L;
234     }
235   }
236 
237   /**
238    * Sets up the actual job.
239    *
240    * @param conf  The current configuration.
241    * @param args  The command line parameters.
242    * @return The newly created job.
243    * @throws IOException When setting up the job fails.
244    */
245   public static Job createSubmittableJob(Configuration conf, String[] args)
246   throws IOException, ClassNotFoundException {
247 
248     // Support non-XML supported characters
249     // by re-encoding the passed separator as a Base64 string.
250     String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
251     if (actualSeparator != null) {
252       conf.set(SEPARATOR_CONF_KEY,
253                Base64.encodeBytes(actualSeparator.getBytes()));
254     }
255 
256     // See if a non-default Mapper was set
257     String mapperClassName = conf.get(MAPPER_CONF_KEY);
258     Class mapperClass = mapperClassName != null ?
259         Class.forName(mapperClassName) : DEFAULT_MAPPER;
260 
261     String tableName = args[0];
262     Path inputDir = new Path(args[1]);
263     Job job = new Job(conf, NAME + "_" + tableName);
264     job.setJarByClass(mapperClass);
265     FileInputFormat.setInputPaths(job, inputDir);
266     job.setInputFormatClass(TextInputFormat.class);
267     job.setMapperClass(mapperClass);
268 
269     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
270     if (hfileOutPath != null) {
271       if (!doesTableExist(tableName)) {
272         createTable(conf, tableName);
273       }
274       HTable table = new HTable(conf, tableName);
275       job.setReducerClass(PutSortReducer.class);
276       Path outputDir = new Path(hfileOutPath);
277       FileOutputFormat.setOutputPath(job, outputDir);
278       job.setMapOutputKeyClass(ImmutableBytesWritable.class);
279       job.setMapOutputValueClass(Put.class);
280       HFileOutputFormat.configureIncrementalLoad(job, table);
281     } else {
282       // No reducers.  Just write straight to table.  Call initTableReducerJob
283       // to set up the TableOutputFormat.
284       TableMapReduceUtil.initTableReducerJob(tableName, null, job);
285       job.setNumReduceTasks(0);
286     }
287 
288     TableMapReduceUtil.addDependencyJars(job);
289     TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
290         com.google.common.base.Function.class /* Guava used by TsvParser */);
291     return job;
292   }
293 
294   private static boolean doesTableExist(String tableName) throws IOException {
295     return hbaseAdmin.tableExists(tableName.getBytes());
296   }
297 
298   private static void createTable(Configuration conf, String tableName)
299       throws IOException {
300     HTableDescriptor htd = new HTableDescriptor(tableName.getBytes());
301     String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
302     Set<String> cfSet = new HashSet<String>();
303     for (String aColumn : columns) {
304       if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)
305           || TsvParser.TIMESTAMPKEY_COLUMN_SPEC.equals(aColumn)) continue;
306       // we are only concerned with the first one (in case this is a cf:cq)
307       cfSet.add(aColumn.split(":", 2)[0]);
308     }
309     for (String cf : cfSet) {
310       HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(cf));
311       htd.addFamily(hcd);
312     }
313     hbaseAdmin.createTable(htd);
314   }
315 
316   /*
317    * @param errorMsg Error message.  Can be null.
318    */
319   private static void usage(final String errorMsg) {
320     if (errorMsg != null && errorMsg.length() > 0) {
321       System.err.println("ERROR: " + errorMsg);
322     }
323     String usage = 
324       "Usage: " + NAME + " -Dimporttsv.columns=a,b,c <tablename> <inputdir>\n" +
325       "\n" +
326       "Imports the given input directory of TSV data into the specified table.\n" +
327       "\n" +
328       "The column names of the TSV data must be specified using the -Dimporttsv.columns\n" +
329       "option. This option takes the form of comma-separated column names, where each\n" +
330       "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
331       "column name HBASE_ROW_KEY is used to designate that this column should be used\n" +
332       "as the row key for each imported record. You must specify exactly one column\n" +
333       "to be the row key, and you must specify a column name for every column that exists in the\n" +
334       "input data. Another special column HBASE_TS_KEY designates that this column should be\n" +
335       "used as timestamp for each record. Unlike HBASE_ROW_KEY, HBASE_TS_KEY is optional.\n" +
336       "You must specify atmost one column as timestamp key for each imported record.\n" +
337       "Record with invalid timestamps (blank, non-numeric) will be treated as bad record.\n" +
338       "Note: if you use this option, then 'importtsv.timestamp' option will be ignored.\n" +
339       "\n" +
340       "By default importtsv will load data directly into HBase. To instead generate\n" +
341       "HFiles of data to prepare for a bulk data load, pass the option:\n" +
342       "  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
343       "  Note: if you do not use this option, then the target table must already exist in HBase\n" +
344       "\n" +
345       "Other options that may be specified with -D include:\n" +
346       "  -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
347       "  '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs\n" +
348       "  -D" + TIMESTAMP_CONF_KEY + "=currentTimeAsLong - use the specified timestamp for the import\n" +
349       "  -D" + MAPPER_CONF_KEY + "=my.Mapper - A user-defined Mapper to use instead of " + DEFAULT_MAPPER.getName() + "\n" +
350       "For performance consider the following options:\n" +
351       "  -Dmapred.map.tasks.speculative.execution=false\n" +
352       "  -Dmapred.reduce.tasks.speculative.execution=false";
353 
354     System.err.println(usage);
355   }
356 
357   /**
358    * Used only by test method
359    * @param conf
360    */
361   static void createHbaseAdmin(Configuration conf) throws IOException {
362     hbaseAdmin = new HBaseAdmin(conf);
363   }
364 
365   /**
366    * Main entry point.
367    *
368    * @param args  The command line parameters.
369    * @throws Exception When running the job fails.
370    */
371   public static void main(String[] args) throws Exception {
372     Configuration conf = HBaseConfiguration.create();
373     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
374     if (otherArgs.length < 2) {
375       usage("Wrong number of arguments: " + otherArgs.length);
376       System.exit(-1);
377     }
378 
379     // Make sure columns are specified
380     String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
381     if (columns == null) {
382       usage("No columns specified. Please specify with -D" +
383             COLUMNS_CONF_KEY+"=...");
384       System.exit(-1);
385     }
386 
387     // Make sure they specify exactly one column as the row key
388     int rowkeysFound=0;
389     for (String col : columns) {
390       if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
391     }
392     if (rowkeysFound != 1) {
393       usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
394       System.exit(-1);
395     }
396 
397     // Make sure we have at most one column as the timestamp key
398     int tskeysFound = 0;
399     for (String col : columns) {
400       if (col.equals(TsvParser.TIMESTAMPKEY_COLUMN_SPEC))
401         tskeysFound++;
402     }
403     if (tskeysFound > 1) {
404       usage("Must specify at most one column as "
405           + TsvParser.TIMESTAMPKEY_COLUMN_SPEC);
406       System.exit(-1);
407     }
408     
409     // Make sure one or more columns are specified excluding rowkey and
410     // timestamp key
411     if (columns.length - (rowkeysFound + tskeysFound) < 1) {
412       usage("One or more columns in addition to the row key and timestamp(optional) are required");
413       System.exit(-1);
414     }
415 
416     // If timestamp option is not specified, use current system time.
417     long timstamp = conf
418         .getLong(TIMESTAMP_CONF_KEY, System.currentTimeMillis());
419 
420     // Set it back to replace invalid timestamp (non-numeric) with current
421     // system time
422     conf.setLong(TIMESTAMP_CONF_KEY, timstamp); 
423     
424     hbaseAdmin = new HBaseAdmin(conf);
425     Job job = createSubmittableJob(conf, otherArgs);
426     System.exit(job.waitForCompletion(true) ? 0 : 1);
427   }
428 
429 }