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