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