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