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 java.io.IOException;
23  import java.util.ArrayList;
24  
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.fs.Path;
27  import org.apache.hadoop.hbase.HBaseConfiguration;
28  import org.apache.hadoop.hbase.HConstants;
29  import org.apache.hadoop.hbase.KeyValue;
30  import org.apache.hadoop.hbase.client.HTable;
31  import org.apache.hadoop.hbase.client.Put;
32  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33  import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException;
34  import org.apache.hadoop.hbase.util.Bytes;
35  import org.apache.hadoop.io.LongWritable;
36  import org.apache.hadoop.io.Text;
37  import org.apache.hadoop.mapreduce.Counter;
38  import org.apache.hadoop.mapreduce.Job;
39  import org.apache.hadoop.mapreduce.Mapper;
40  import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
41  import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
42  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
43  import org.apache.hadoop.util.GenericOptionsParser;
44  
45  import com.google.common.base.Preconditions;
46  import com.google.common.base.Splitter;
47  import com.google.common.collect.Lists;
48  
49  /**
50   * Tool to import data from a TSV file.
51   *
52   * This tool is rather simplistic - it doesn't do any quoting or
53   * escaping, but is useful for many data loads.
54   *
55   * @see ImportTsv#usage(String)
56   */
57  public class ImportTsv {
58    final static String NAME = "importtsv";
59  
60    final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
61    final static String BULK_OUTPUT_CONF_KEY = "importtsv.bulk.output";
62    final static String COLUMNS_CONF_KEY = "importtsv.columns";
63    final static String SEPARATOR_CONF_KEY = "importtsv.separator";
64    final static String DEFAULT_SEPARATOR = "\t";
65  
66    static class TsvParser {
67      /**
68       * Column families and qualifiers mapped to the TSV columns
69       */
70      private final byte[][] families;
71      private final byte[][] qualifiers;
72  
73      private final byte separatorByte;
74  
75      private int rowKeyColumnIndex;
76      
77      public static String ROWKEY_COLUMN_SPEC="HBASE_ROW_KEY";
78  
79      /**
80       * @param columnsSpecification the list of columns to parser out, comma separated.
81       * The row key should be the special token TsvParser.ROWKEY_COLUMN_SPEC
82       */
83      public TsvParser(String columnsSpecification, String separatorStr) {
84        // Configure separator
85        byte[] separator = Bytes.toBytes(separatorStr);
86        Preconditions.checkArgument(separator.length == 1,
87          "TsvParser only supports single-byte separators");
88        separatorByte = separator[0];
89  
90        // Configure columns
91        ArrayList<String> columnStrings = Lists.newArrayList(
92          Splitter.on(',').trimResults().split(columnsSpecification));
93        
94        families = new byte[columnStrings.size()][];
95        qualifiers = new byte[columnStrings.size()][];
96  
97        for (int i = 0; i < columnStrings.size(); i++) {
98          String str = columnStrings.get(i);
99          if (ROWKEY_COLUMN_SPEC.equals(str)) {
100           rowKeyColumnIndex = i;
101           continue;
102         }
103         String[] parts = str.split(":", 2);
104         if (parts.length == 1) {
105           families[i] = str.getBytes();
106           qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
107         } else {
108           families[i] = parts[0].getBytes();
109           qualifiers[i] = parts[1].getBytes();
110         }
111       }
112     }
113     
114     public int getRowKeyColumnIndex() {
115       return rowKeyColumnIndex;
116     }
117     public byte[] getFamily(int idx) {
118       return families[idx];
119     }
120     public byte[] getQualifier(int idx) {
121       return qualifiers[idx];
122     }
123     
124     public ParsedLine parse(byte[] lineBytes, int length)
125     throws BadTsvLineException {
126       // Enumerate separator offsets
127       ArrayList<Integer> tabOffsets = new ArrayList<Integer>(families.length);
128       for (int i = 0; i < length; i++) {
129         if (lineBytes[i] == separatorByte) {
130           tabOffsets.add(i);
131         }
132       }
133       if (tabOffsets.isEmpty()) {
134         throw new BadTsvLineException("No delimiter");
135       }
136 
137       tabOffsets.add(length);
138 
139       if (tabOffsets.size() > families.length) {
140         throw new BadTsvLineException("Excessive columns");
141       } else if (tabOffsets.size() <= getRowKeyColumnIndex()) {
142         throw new BadTsvLineException("No row key");
143       }
144       return new ParsedLine(tabOffsets, lineBytes);
145     }
146     
147     class ParsedLine {
148       private final ArrayList<Integer> tabOffsets;
149       private byte[] lineBytes;
150       
151       ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
152         this.tabOffsets = tabOffsets;
153         this.lineBytes = lineBytes;
154       }
155       
156       public int getRowKeyOffset() {
157         return getColumnOffset(rowKeyColumnIndex);
158       }
159       public int getRowKeyLength() {
160         return getColumnLength(rowKeyColumnIndex);
161       }
162       public int getColumnOffset(int idx) {
163         if (idx > 0)
164           return tabOffsets.get(idx - 1) + 1;
165         else
166           return 0;
167       }      
168       public int getColumnLength(int idx) {
169         return tabOffsets.get(idx) - getColumnOffset(idx);
170       }
171       public int getColumnCount() {
172         return tabOffsets.size();
173       }
174       public byte[] getLineBytes() {
175         return lineBytes;
176       }
177     }
178     
179     public static class BadTsvLineException extends Exception {
180       public BadTsvLineException(String err) {
181         super(err);
182       }
183       private static final long serialVersionUID = 1L;
184     }
185   }
186   
187   /**
188    * Write table content out to files in hdfs.
189    */
190   static class TsvImporter
191   extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
192   {
193     
194     /** Timestamp for all inserted rows */
195     private long ts;
196 
197     /** Should skip bad lines */
198     private boolean skipBadLines;
199     private Counter badLineCount;
200 
201     private TsvParser parser;
202 
203     @Override
204     protected void setup(Context context) {
205       Configuration conf = context.getConfiguration();
206       parser = new TsvParser(conf.get(COLUMNS_CONF_KEY),
207                              conf.get(SEPARATOR_CONF_KEY, DEFAULT_SEPARATOR));
208       if (parser.getRowKeyColumnIndex() == -1) {
209         throw new RuntimeException("No row key column specified");
210       }
211       ts = System.currentTimeMillis();
212 
213       skipBadLines = context.getConfiguration().getBoolean(
214         SKIP_LINES_CONF_KEY, true);
215       badLineCount = context.getCounter("ImportTsv", "Bad Lines");
216     }
217 
218     /**
219      * Convert a line of TSV text into an HBase table row.
220      */
221     @Override
222     public void map(LongWritable offset, Text value,
223       Context context)
224     throws IOException {
225       byte[] lineBytes = value.getBytes();
226 
227       try {
228         TsvParser.ParsedLine parsed = parser.parse(
229             lineBytes, value.getLength());
230         ImmutableBytesWritable rowKey =
231           new ImmutableBytesWritable(lineBytes,
232               parsed.getRowKeyOffset(),
233               parsed.getRowKeyLength());
234 
235         Put put = new Put(rowKey.copyBytes());
236         for (int i = 0; i < parsed.getColumnCount(); i++) {
237           if (i == parser.getRowKeyColumnIndex()) continue;
238           KeyValue kv = new KeyValue(
239               lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(),
240               parser.getFamily(i), 0, parser.getFamily(i).length,
241               parser.getQualifier(i), 0, parser.getQualifier(i).length,
242               ts,
243               KeyValue.Type.Put,
244               lineBytes, parsed.getColumnOffset(i), parsed.getColumnLength(i));
245           put.add(kv);
246         }
247         context.write(rowKey, put);
248       } catch (BadTsvLineException badLine) {
249         if (skipBadLines) {
250           System.err.println(
251               "Bad line at offset: " + offset.get() + ":\n" +
252               badLine.getMessage());
253           badLineCount.increment(1);
254           return;
255         } else {
256           throw new IOException(badLine);
257         }
258       } catch (InterruptedException e) {
259         e.printStackTrace();
260       }
261     }
262   }
263 
264   /**
265    * Sets up the actual job.
266    *
267    * @param conf  The current configuration.
268    * @param args  The command line parameters.
269    * @return The newly created job.
270    * @throws IOException When setting up the job fails.
271    */
272   public static Job createSubmittableJob(Configuration conf, String[] args)
273   throws IOException {
274     String tableName = args[0];
275     Path inputDir = new Path(args[1]);
276     Job job = new Job(conf, NAME + "_" + tableName);
277     job.setJarByClass(TsvImporter.class);
278     FileInputFormat.setInputPaths(job, inputDir);
279     job.setInputFormatClass(TextInputFormat.class);
280     job.setMapperClass(TsvImporter.class);
281 
282     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
283     if (hfileOutPath != null) {
284       HTable table = new HTable(conf, tableName);
285       job.setReducerClass(PutSortReducer.class);
286       Path outputDir = new Path(hfileOutPath);
287       FileOutputFormat.setOutputPath(job, outputDir);
288       job.setMapOutputKeyClass(ImmutableBytesWritable.class);
289       job.setMapOutputValueClass(Put.class);
290       HFileOutputFormat.configureIncrementalLoad(job, table);
291     } else {
292       // No reducers.  Just write straight to table.  Call initTableReducerJob
293       // to set up the TableOutputFormat.
294       TableMapReduceUtil.initTableReducerJob(tableName, null, job);
295       job.setNumReduceTasks(0);
296     }
297     
298     TableMapReduceUtil.addDependencyJars(job);
299     TableMapReduceUtil.addDependencyJars(job.getConfiguration(), 
300         com.google.common.base.Function.class /* Guava used by TsvParser */);
301     return job;
302   }
303 
304   /*
305    * @param errorMsg Error message.  Can be null.
306    */
307   private static void usage(final String errorMsg) {
308     if (errorMsg != null && errorMsg.length() > 0) {
309       System.err.println("ERROR: " + errorMsg);
310     }
311     String usage = 
312       "Usage: " + NAME + " -Dimporttsv.columns=a,b,c <tablename> <inputdir>\n" +
313       "\n" +
314       "Imports the given input directory of TSV data into the specified table.\n" +
315       "\n" +
316       "The column names of the TSV data must be specified using the -Dimporttsv.columns\n" +
317       "option. This option takes the form of comma-separated column names, where each\n" +
318       "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
319       "column name HBASE_ROW_KEY is used to designate that this column should be used\n" +
320       "as the row key for each imported record. You must specify exactly one column\n" +
321       "to be the row key.\n" +
322       "\n" +
323       "In order to prepare data for a bulk data load, pass the option:\n" +
324       "  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
325       "\n" +
326       "Other options that may be specified with -D include:\n" +
327       "  -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
328       "  '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs";
329     System.err.println(usage);
330   }
331 
332   /**
333    * Main entry point.
334    *
335    * @param args  The command line parameters.
336    * @throws Exception When running the job fails.
337    */
338   public static void main(String[] args) throws Exception {
339     Configuration conf = HBaseConfiguration.create();
340     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
341     if (otherArgs.length < 2) {
342       usage("Wrong number of arguments: " + otherArgs.length);
343       System.exit(-1);
344     }
345 
346     // Make sure columns are specified
347     String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
348     if (columns == null) {
349       usage("No columns specified. Please specify with -D" +
350             COLUMNS_CONF_KEY+"=...");
351       System.exit(-1);
352     }
353 
354     // Make sure they specify exactly one column as the row key
355     int rowkeysFound=0;
356     for (String col : columns) {
357       if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
358     }
359     if (rowkeysFound != 1) {
360       usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
361       System.exit(-1);
362     }
363 
364     // Make sure one or more columns are specified
365     if (columns.length < 2) {
366       usage("One or more columns in addition to the row key are required");
367       System.exit(-1);
368     }
369 
370     Job job = createSubmittableJob(conf, otherArgs);
371     System.exit(job.waitForCompletion(true) ? 0 : 1);
372   }
373 
374 }