1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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
62
63
64
65
66
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
81
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 public final static String ATTRIBUTE_SEPERATOR_CONF_KEY = "attributes.seperator";
86 final static String DEFAULT_SEPARATOR = "\t";
87 final static String DEFAULT_ATTRIBUTES_SEPERATOR = "=>";
88 final static String DEFAULT_MULTIPLE_ATTRIBUTES_SEPERATOR = ",";
89 final static Class DEFAULT_MAPPER = TsvImporterMapper.class;
90
91 public static class TsvParser {
92
93
94
95 private final byte[][] families;
96 private final byte[][] qualifiers;
97
98 private final byte separatorByte;
99
100 private int rowKeyColumnIndex;
101
102 private int maxColumnCount;
103
104
105 public static final int DEFAULT_TIMESTAMP_COLUMN_INDEX = -1;
106
107 private int timestampKeyColumnIndex = DEFAULT_TIMESTAMP_COLUMN_INDEX;
108
109 public static final String ROWKEY_COLUMN_SPEC = "HBASE_ROW_KEY";
110
111 public static final String TIMESTAMPKEY_COLUMN_SPEC = "HBASE_TS_KEY";
112
113 public static final String ATTRIBUTES_COLUMN_SPEC = "HBASE_ATTRIBUTES_KEY";
114
115 public static final String CELL_VISIBILITY_COLUMN_SPEC = "HBASE_CELL_VISIBILITY";
116
117 private int attrKeyColumnIndex = DEFAULT_ATTRIBUTES_COLUMN_INDEX;
118
119 public static final int DEFAULT_ATTRIBUTES_COLUMN_INDEX = -1;
120
121 public static final int DEFAULT_CELL_VISIBILITY_COLUMN_INDEX = -1;
122
123 private int cellVisibilityColumnIndex = DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
124
125
126
127
128
129 public TsvParser(String columnsSpecification, String separatorStr) {
130
131 byte[] separator = Bytes.toBytes(separatorStr);
132 Preconditions.checkArgument(separator.length == 1,
133 "TsvParser only supports single-byte separators");
134 separatorByte = separator[0];
135
136
137 ArrayList<String> columnStrings = Lists.newArrayList(
138 Splitter.on(',').trimResults().split(columnsSpecification));
139
140 maxColumnCount = columnStrings.size();
141 families = new byte[maxColumnCount][];
142 qualifiers = new byte[maxColumnCount][];
143
144 for (int i = 0; i < columnStrings.size(); i++) {
145 String str = columnStrings.get(i);
146 if (ROWKEY_COLUMN_SPEC.equals(str)) {
147 rowKeyColumnIndex = i;
148 continue;
149 }
150 if (TIMESTAMPKEY_COLUMN_SPEC.equals(str)) {
151 timestampKeyColumnIndex = i;
152 continue;
153 }
154 if(ATTRIBUTES_COLUMN_SPEC.equals(str)) {
155 attrKeyColumnIndex = i;
156 continue;
157 }
158 if(CELL_VISIBILITY_COLUMN_SPEC.equals(str)) {
159 cellVisibilityColumnIndex = i;
160 continue;
161 }
162 String[] parts = str.split(":", 2);
163 if (parts.length == 1) {
164 families[i] = str.getBytes();
165 qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
166 } else {
167 families[i] = parts[0].getBytes();
168 qualifiers[i] = parts[1].getBytes();
169 }
170 }
171 }
172
173 public boolean hasTimestamp() {
174 return timestampKeyColumnIndex != DEFAULT_TIMESTAMP_COLUMN_INDEX;
175 }
176
177 public int getTimestampKeyColumnIndex() {
178 return timestampKeyColumnIndex;
179 }
180
181 public boolean hasAttributes() {
182 return attrKeyColumnIndex != DEFAULT_ATTRIBUTES_COLUMN_INDEX;
183 }
184
185 public boolean hasCellVisibility() {
186 return cellVisibilityColumnIndex != DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
187 }
188
189 public int getAttributesKeyColumnIndex() {
190 return attrKeyColumnIndex;
191 }
192
193 public int getCellVisibilityColumnIndex() {
194 return cellVisibilityColumnIndex;
195 }
196 public int getRowKeyColumnIndex() {
197 return rowKeyColumnIndex;
198 }
199 public byte[] getFamily(int idx) {
200 return families[idx];
201 }
202 public byte[] getQualifier(int idx) {
203 return qualifiers[idx];
204 }
205
206 public ParsedLine parse(byte[] lineBytes, int length)
207 throws BadTsvLineException {
208
209 ArrayList<Integer> tabOffsets = new ArrayList<Integer>(maxColumnCount);
210 for (int i = 0; i < length; i++) {
211 if (lineBytes[i] == separatorByte) {
212 tabOffsets.add(i);
213 }
214 }
215 if (tabOffsets.isEmpty()) {
216 throw new BadTsvLineException("No delimiter");
217 }
218
219 tabOffsets.add(length);
220
221 if (tabOffsets.size() > maxColumnCount) {
222 throw new BadTsvLineException("Excessive columns");
223 } else if (tabOffsets.size() <= getRowKeyColumnIndex()) {
224 throw new BadTsvLineException("No row key");
225 } else if (hasTimestamp()
226 && tabOffsets.size() <= getTimestampKeyColumnIndex()) {
227 throw new BadTsvLineException("No timestamp");
228 } else if (hasAttributes() && tabOffsets.size() <= getAttributesKeyColumnIndex()) {
229 throw new BadTsvLineException("No attributes specified");
230 } else if(hasCellVisibility() && tabOffsets.size() <= getCellVisibilityColumnIndex()) {
231 throw new BadTsvLineException("No cell visibility specified");
232 }
233 return new ParsedLine(tabOffsets, lineBytes);
234 }
235
236 class ParsedLine {
237 private final ArrayList<Integer> tabOffsets;
238 private byte[] lineBytes;
239
240 ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
241 this.tabOffsets = tabOffsets;
242 this.lineBytes = lineBytes;
243 }
244
245 public int getRowKeyOffset() {
246 return getColumnOffset(rowKeyColumnIndex);
247 }
248 public int getRowKeyLength() {
249 return getColumnLength(rowKeyColumnIndex);
250 }
251
252 public long getTimestamp(long ts) throws BadTsvLineException {
253
254 if (!hasTimestamp()) {
255 return ts;
256 }
257
258 String timeStampStr = Bytes.toString(lineBytes,
259 getColumnOffset(timestampKeyColumnIndex),
260 getColumnLength(timestampKeyColumnIndex));
261 try {
262 return Long.parseLong(timeStampStr);
263 } catch (NumberFormatException nfe) {
264
265 throw new BadTsvLineException("Invalid timestamp " + timeStampStr);
266 }
267 }
268
269 private String getAttributes() {
270 if (!hasAttributes()) {
271 return null;
272 } else {
273 return Bytes.toString(lineBytes, getColumnOffset(attrKeyColumnIndex),
274 getColumnLength(attrKeyColumnIndex));
275 }
276 }
277
278 public String[] getIndividualAttributes() {
279 String attributes = getAttributes();
280 if (attributes != null) {
281 return attributes.split(DEFAULT_MULTIPLE_ATTRIBUTES_SEPERATOR);
282 } else {
283 return null;
284 }
285 }
286
287 public int getAttributeKeyOffset() {
288 if (hasAttributes()) {
289 return getColumnOffset(attrKeyColumnIndex);
290 } else {
291 return DEFAULT_ATTRIBUTES_COLUMN_INDEX;
292 }
293 }
294
295 public int getAttributeKeyLength() {
296 if (hasAttributes()) {
297 return getColumnLength(attrKeyColumnIndex);
298 } else {
299 return DEFAULT_ATTRIBUTES_COLUMN_INDEX;
300 }
301 }
302
303 public int getCellVisibilityColumnOffset() {
304 if (hasCellVisibility()) {
305 return getColumnOffset(cellVisibilityColumnIndex);
306 } else {
307 return DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
308 }
309 }
310
311 public int getCellVisibilityColumnLength() {
312 if (hasCellVisibility()) {
313 return getColumnLength(cellVisibilityColumnIndex);
314 } else {
315 return DEFAULT_CELL_VISIBILITY_COLUMN_INDEX;
316 }
317 }
318
319 public String getCellVisibility() {
320 if (!hasCellVisibility()) {
321 return null;
322 } else {
323 return Bytes.toString(lineBytes, getColumnOffset(cellVisibilityColumnIndex),
324 getColumnLength(cellVisibilityColumnIndex));
325 }
326 }
327
328 public int getColumnOffset(int idx) {
329 if (idx > 0)
330 return tabOffsets.get(idx - 1) + 1;
331 else
332 return 0;
333 }
334 public int getColumnLength(int idx) {
335 return tabOffsets.get(idx) - getColumnOffset(idx);
336 }
337 public int getColumnCount() {
338 return tabOffsets.size();
339 }
340 public byte[] getLineBytes() {
341 return lineBytes;
342 }
343 }
344
345 public static class BadTsvLineException extends Exception {
346 public BadTsvLineException(String err) {
347 super(err);
348 }
349 private static final long serialVersionUID = 1L;
350 }
351
352 public Pair<Integer, Integer> parseRowKey(byte[] lineBytes, int length)
353 throws BadTsvLineException {
354 int rkColumnIndex = 0;
355 int startPos = 0, endPos = 0;
356 for (int i = 0; i <= length; i++) {
357 if (i == length || lineBytes[i] == separatorByte) {
358 endPos = i - 1;
359 if (rkColumnIndex++ == getRowKeyColumnIndex()) {
360 if ((endPos + 1) == startPos) {
361 throw new BadTsvLineException("Empty value for ROW KEY.");
362 }
363 break;
364 } else {
365 startPos = endPos + 2;
366 }
367 }
368 if (i == length) {
369 throw new BadTsvLineException(
370 "Row key does not exist as number of columns in the line"
371 + " are less than row key position.");
372 }
373 }
374 return new Pair<Integer, Integer>(startPos, endPos);
375 }
376 }
377
378
379
380
381
382
383
384
385
386 public static Job createSubmittableJob(Configuration conf, String[] args)
387 throws IOException, ClassNotFoundException {
388
389 HBaseAdmin admin = new HBaseAdmin(conf);
390
391
392
393 String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
394 if (actualSeparator != null) {
395 conf.set(SEPARATOR_CONF_KEY,
396 Base64.encodeBytes(actualSeparator.getBytes()));
397 }
398
399
400 String mapperClassName = conf.get(MAPPER_CONF_KEY);
401 Class mapperClass = mapperClassName != null ?
402 Class.forName(mapperClassName) : DEFAULT_MAPPER;
403
404 String tableName = args[0];
405 Path inputDir = new Path(args[1]);
406 String jobName = conf.get(JOB_NAME_CONF_KEY,NAME + "_" + tableName);
407 Job job = new Job(conf, jobName);
408 job.setJarByClass(mapperClass);
409 FileInputFormat.setInputPaths(job, inputDir);
410 job.setInputFormatClass(TextInputFormat.class);
411 job.setMapperClass(mapperClass);
412
413 String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
414 String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
415 if (hfileOutPath != null) {
416 if (!admin.tableExists(tableName)) {
417 LOG.warn(format("Table '%s' does not exist.", tableName));
418
419
420 createTable(admin, tableName, columns);
421 }
422 HTable table = new HTable(conf, tableName);
423 job.setReducerClass(PutSortReducer.class);
424 Path outputDir = new Path(hfileOutPath);
425 FileOutputFormat.setOutputPath(job, outputDir);
426 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
427 if (mapperClass.equals(TsvImporterTextMapper.class)) {
428 job.setMapOutputValueClass(Text.class);
429 job.setReducerClass(TextSortReducer.class);
430 } else {
431 job.setMapOutputValueClass(Put.class);
432 job.setCombinerClass(PutCombiner.class);
433 }
434 HFileOutputFormat.configureIncrementalLoad(job, table);
435 } else {
436 if (mapperClass.equals(TsvImporterTextMapper.class)) {
437 usage(TsvImporterTextMapper.class.toString()
438 + " should not be used for non bulkloading case. use "
439 + TsvImporterMapper.class.toString()
440 + " or custom mapper whose value type is Put.");
441 System.exit(-1);
442 }
443
444
445 TableMapReduceUtil.initTableReducerJob(tableName, null, job);
446 job.setNumReduceTasks(0);
447 }
448
449 TableMapReduceUtil.addDependencyJars(job);
450 TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
451 com.google.common.base.Function.class
452 return job;
453 }
454
455 private static void createTable(HBaseAdmin admin, String tableName, String[] columns)
456 throws IOException {
457 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
458 Set<String> cfSet = new HashSet<String>();
459 for (String aColumn : columns) {
460 if (TsvParser.ROWKEY_COLUMN_SPEC.equals(aColumn)
461 || TsvParser.TIMESTAMPKEY_COLUMN_SPEC.equals(aColumn)
462 || TsvParser.CELL_VISIBILITY_COLUMN_SPEC.equals(aColumn)
463 || TsvParser.ATTRIBUTES_COLUMN_SPEC.equals(aColumn))
464 continue;
465
466 cfSet.add(aColumn.split(":", 2)[0]);
467 }
468 for (String cf : cfSet) {
469 HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(cf));
470 htd.addFamily(hcd);
471 }
472 LOG.warn(format("Creating table '%s' with '%s' columns and default descriptors.",
473 tableName, cfSet));
474 admin.createTable(htd);
475 }
476
477
478
479
480 private static void usage(final String errorMsg) {
481 if (errorMsg != null && errorMsg.length() > 0) {
482 System.err.println("ERROR: " + errorMsg);
483 }
484 String usage =
485 "Usage: " + NAME + " -D"+ COLUMNS_CONF_KEY + "=a,b,c <tablename> <inputdir>\n" +
486 "\n" +
487 "Imports the given input directory of TSV data into the specified table.\n" +
488 "\n" +
489 "The column names of the TSV data must be specified using the -D" + COLUMNS_CONF_KEY + "\n" +
490 "option. This option takes the form of comma-separated column names, where each\n" +
491 "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
492 "column name " + TsvParser.ROWKEY_COLUMN_SPEC + " is used to designate that this column should be used\n" +
493 "as the row key for each imported record. You must specify exactly one column\n" +
494 "to be the row key, and you must specify a column name for every column that exists in the\n" +
495 "input data. Another special column" + TsvParser.TIMESTAMPKEY_COLUMN_SPEC +
496 " designates that this column should be\n" +
497 "used as timestamp for each record. Unlike " + TsvParser.ROWKEY_COLUMN_SPEC + ", " +
498 TsvParser.TIMESTAMPKEY_COLUMN_SPEC + " is optional.\n" +
499 "You must specify at most one column as timestamp key for each imported record.\n" +
500 "Record with invalid timestamps (blank, non-numeric) will be treated as bad record.\n" +
501 "Note: if you use this option, then '" + TIMESTAMP_CONF_KEY + "' option will be ignored.\n" +
502 "\n" +
503 TsvParser.ATTRIBUTES_COLUMN_SPEC+" can be used to specify Operation Attributes per record.\n"+
504 " Should be specified as key=>value where "+TsvParser.DEFAULT_ATTRIBUTES_COLUMN_INDEX+ " is used \n"+
505 " as the seperator. Note that more than one OperationAttributes can be specified.\n"+
506 "By default importtsv will load data directly into HBase. To instead generate\n" +
507 "HFiles of data to prepare for a bulk data load, pass the option:\n" +
508 " -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
509 " Note: if you do not use this option, then the target table must already exist in HBase\n" +
510 "\n" +
511 "Other options that may be specified with -D include:\n" +
512 " -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
513 " '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs\n" +
514 " -D" + TIMESTAMP_CONF_KEY + "=currentTimeAsLong - use the specified timestamp for the import\n" +
515 " -D" + MAPPER_CONF_KEY + "=my.Mapper - A user-defined Mapper to use instead of " +
516 DEFAULT_MAPPER.getName() + "\n" +
517 " -D" + JOB_NAME_CONF_KEY + "=jobName - use the specified mapreduce job name for the import\n" +
518 "For performance consider the following options:\n" +
519 " -Dmapred.map.tasks.speculative.execution=false\n" +
520 " -Dmapred.reduce.tasks.speculative.execution=false";
521
522 System.err.println(usage);
523 }
524
525 @Override
526 public int run(String[] args) throws Exception {
527 setConf(HBaseConfiguration.create(getConf()));
528 String[] otherArgs = new GenericOptionsParser(getConf(), args).getRemainingArgs();
529 if (otherArgs.length < 2) {
530 usage("Wrong number of arguments: " + otherArgs.length);
531 return -1;
532 }
533
534
535
536
537
538 if (null == getConf().get(MAPPER_CONF_KEY)) {
539
540 String columns[] = getConf().getStrings(COLUMNS_CONF_KEY);
541 if (columns == null) {
542 usage("No columns specified. Please specify with -D" +
543 COLUMNS_CONF_KEY+"=...");
544 return -1;
545 }
546
547
548 int rowkeysFound = 0;
549 for (String col : columns) {
550 if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
551 }
552 if (rowkeysFound != 1) {
553 usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
554 return -1;
555 }
556
557
558 int tskeysFound = 0;
559 for (String col : columns) {
560 if (col.equals(TsvParser.TIMESTAMPKEY_COLUMN_SPEC))
561 tskeysFound++;
562 }
563 if (tskeysFound > 1) {
564 usage("Must specify at most one column as "
565 + TsvParser.TIMESTAMPKEY_COLUMN_SPEC);
566 return -1;
567 }
568
569 int attrKeysFound = 0;
570 for (String col : columns) {
571 if (col.equals(TsvParser.ATTRIBUTES_COLUMN_SPEC))
572 attrKeysFound++;
573 }
574 if (attrKeysFound > 1) {
575 usage("Must specify at most one column as "
576 + TsvParser.ATTRIBUTES_COLUMN_SPEC);
577 return -1;
578 }
579
580
581
582 if (columns.length - (rowkeysFound + tskeysFound + attrKeysFound) < 1) {
583 usage("One or more columns in addition to the row key and timestamp(optional) are required");
584 return -1;
585 }
586 }
587
588
589 long timstamp = getConf().getLong(TIMESTAMP_CONF_KEY, System.currentTimeMillis());
590
591
592
593 getConf().setLong(TIMESTAMP_CONF_KEY, timstamp);
594
595 Job job = createSubmittableJob(getConf(), otherArgs);
596 return job.waitForCompletion(true) ? 0 : 1;
597 }
598
599 public static void main(String[] args) throws Exception {
600 int status = ToolRunner.run(new ImportTsv(), args);
601 System.exit(status);
602 }
603 }