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       tabOffsets.add(length);
134       if (tabOffsets.size() > families.length) {
135         throw new BadTsvLineException("Bad line:\n");
136       }
137 
138       return new ParsedLine(tabOffsets, lineBytes);
139     }
140     
141     class ParsedLine {
142       private final ArrayList<Integer> tabOffsets;
143       private byte[] lineBytes;
144       
145       ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
146         this.tabOffsets = tabOffsets;
147         this.lineBytes = lineBytes;
148       }
149       
150       public int getRowKeyOffset() {
151         return getColumnOffset(rowKeyColumnIndex);
152       }
153       public int getRowKeyLength() {
154         return getColumnLength(rowKeyColumnIndex);
155       }
156       public int getColumnOffset(int idx) {
157         if (idx > 0)
158           return tabOffsets.get(idx - 1) + 1;
159         else
160           return 0;
161       }      
162       public int getColumnLength(int idx) {
163         return tabOffsets.get(idx) - getColumnOffset(idx);
164       }
165       public int getColumnCount() {
166         return tabOffsets.size();
167       }
168       public byte[] getLineBytes() {
169         return lineBytes;
170       }
171     }
172     
173     public static class BadTsvLineException extends Exception {
174       public BadTsvLineException(String err) {
175         super(err);
176       }
177       private static final long serialVersionUID = 1L;
178     }
179   }
180   
181   /**
182    * Write table content out to files in hdfs.
183    */
184   static class TsvImporter
185   extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
186   {
187     
188     /** Timestamp for all inserted rows */
189     private long ts;
190 
191     /** Should skip bad lines */
192     private boolean skipBadLines;
193     private Counter badLineCount;
194 
195     private TsvParser parser;
196 
197     @Override
198     protected void setup(Context context) {
199       Configuration conf = context.getConfiguration();
200       parser = new TsvParser(conf.get(COLUMNS_CONF_KEY),
201                              conf.get(SEPARATOR_CONF_KEY, DEFAULT_SEPARATOR));
202       if (parser.getRowKeyColumnIndex() == -1) {
203         throw new RuntimeException("No row key column specified");
204       }
205       ts = System.currentTimeMillis();
206 
207       skipBadLines = context.getConfiguration().getBoolean(
208         SKIP_LINES_CONF_KEY, true);
209       badLineCount = context.getCounter("ImportTsv", "Bad Lines");
210     }
211 
212     /**
213      * Convert a line of TSV text into an HBase table row.
214      */
215     @Override
216     public void map(LongWritable offset, Text value,
217       Context context)
218     throws IOException {
219       byte[] lineBytes = value.getBytes();
220 
221       try {
222         TsvParser.ParsedLine parsed = parser.parse(
223             lineBytes, value.getLength());
224         ImmutableBytesWritable rowKey =
225           new ImmutableBytesWritable(lineBytes,
226               parsed.getRowKeyOffset(),
227               parsed.getRowKeyLength());
228 
229         Put put = new Put(rowKey.copyBytes());
230         for (int i = 0; i < parsed.getColumnCount(); i++) {
231           if (i == parser.getRowKeyColumnIndex()) continue;
232           KeyValue kv = new KeyValue(
233               lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(),
234               parser.getFamily(i), 0, parser.getFamily(i).length,
235               parser.getQualifier(i), 0, parser.getQualifier(i).length,
236               ts,
237               KeyValue.Type.Put,
238               lineBytes, parsed.getColumnOffset(i), parsed.getColumnLength(i));
239           put.add(kv);
240         }
241         context.write(rowKey, put);
242       } catch (BadTsvLineException badLine) {
243         if (skipBadLines) {
244           System.err.println(
245               "Bad line at offset: " + offset.get() + ":\n" +
246               badLine.getMessage());
247           badLineCount.increment(1);
248           return;
249         } else {
250           throw new IOException(badLine);
251         }
252       } catch (InterruptedException e) {
253         e.printStackTrace();
254       }
255     }
256   }
257 
258   /**
259    * Sets up the actual job.
260    *
261    * @param conf  The current configuration.
262    * @param args  The command line parameters.
263    * @return The newly created job.
264    * @throws IOException When setting up the job fails.
265    */
266   public static Job createSubmittableJob(Configuration conf, String[] args)
267   throws IOException {
268     String tableName = args[0];
269     Path inputDir = new Path(args[1]);
270     Job job = new Job(conf, NAME + "_" + tableName);
271     job.setJarByClass(TsvImporter.class);
272     FileInputFormat.setInputPaths(job, inputDir);
273     job.setInputFormatClass(TextInputFormat.class);
274     job.setMapperClass(TsvImporter.class);
275 
276     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
277     if (hfileOutPath != null) {
278       HTable table = new HTable(conf, tableName);
279       job.setReducerClass(PutSortReducer.class);
280       Path outputDir = new Path(hfileOutPath);
281       FileOutputFormat.setOutputPath(job, outputDir);
282       job.setMapOutputKeyClass(ImmutableBytesWritable.class);
283       job.setMapOutputValueClass(Put.class);
284       HFileOutputFormat.configureIncrementalLoad(job, table);
285     } else {
286       // No reducers.  Just write straight to table.  Call initTableReducerJob
287       // to set up the TableOutputFormat.
288       TableMapReduceUtil.initTableReducerJob(tableName, null, job);
289       job.setNumReduceTasks(0);
290     }
291     
292     TableMapReduceUtil.addDependencyJars(job);
293     return job;
294   }
295 
296   /*
297    * @param errorMsg Error message.  Can be null.
298    */
299   private static void usage(final String errorMsg) {
300     if (errorMsg != null && errorMsg.length() > 0) {
301       System.err.println("ERROR: " + errorMsg);
302     }
303     String usage = 
304       "Usage: " + NAME + " -Dimporttsv.columns=a,b,c <tablename> <inputdir>\n" +
305       "\n" +
306       "Imports the given input directory of TSV data into the specified table.\n" +
307       "\n" +
308       "The column names of the TSV data must be specified using the -Dimporttsv.columns\n" +
309       "option. This option takes the form of comma-separated column names, where each\n" +
310       "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
311       "column name HBASE_ROW_KEY is used to designate that this column should be used\n" +
312       "as the row key for each imported record. You must specify exactly one column\n" +
313       "to be the row key.\n" +
314       "\n" +
315       "In order to prepare data for a bulk data load, pass the option:\n" +
316       "  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
317       "\n" +
318       "Other options that may be specified with -D include:\n" +
319       "  -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
320       "  '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs";
321     System.err.println(usage);
322   }
323 
324   /**
325    * Main entry point.
326    *
327    * @param args  The command line parameters.
328    * @throws Exception When running the job fails.
329    */
330   public static void main(String[] args) throws Exception {
331     Configuration conf = HBaseConfiguration.create();
332     String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
333     if (otherArgs.length < 2) {
334       usage("Wrong number of arguments: " + otherArgs.length);
335       System.exit(-1);
336     }
337 
338     // Make sure columns are specified
339     String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
340     if (columns == null) {
341       usage("No columns specified. Please specify with -D" +
342             COLUMNS_CONF_KEY+"=...");
343       System.exit(-1);
344     }
345 
346     // Make sure they specify exactly one column as the row key
347     int rowkeysFound=0;
348     for (String col : columns) {
349       if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
350     }
351     if (rowkeysFound != 1) {
352       usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
353       System.exit(-1);
354     }
355 
356     Job job = createSubmittableJob(conf, otherArgs);
357     System.exit(job.waitForCompletion(true) ? 0 : 1);
358   }
359 
360 }