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