1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.mapreduce;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.fs.Path;
27 import org.apache.hadoop.hbase.HBaseConfiguration;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.KeyValue;
30 import org.apache.hadoop.hbase.client.HTable;
31 import org.apache.hadoop.hbase.client.Put;
32 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33 import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException;
34 import org.apache.hadoop.hbase.util.Bytes;
35 import org.apache.hadoop.io.LongWritable;
36 import org.apache.hadoop.io.Text;
37 import org.apache.hadoop.mapreduce.Counter;
38 import org.apache.hadoop.mapreduce.Job;
39 import org.apache.hadoop.mapreduce.Mapper;
40 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
41 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
42 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
43 import org.apache.hadoop.util.GenericOptionsParser;
44
45 import com.google.common.base.Preconditions;
46 import com.google.common.base.Splitter;
47 import com.google.common.collect.Lists;
48
49
50
51
52
53
54
55
56
57 public class ImportTsv {
58 final static String NAME = "importtsv";
59
60 final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
61 final static String BULK_OUTPUT_CONF_KEY = "importtsv.bulk.output";
62 final static String COLUMNS_CONF_KEY = "importtsv.columns";
63 final static String SEPARATOR_CONF_KEY = "importtsv.separator";
64 final static String DEFAULT_SEPARATOR = "\t";
65
66 static class TsvParser {
67
68
69
70 private final byte[][] families;
71 private final byte[][] qualifiers;
72
73 private final byte separatorByte;
74
75 private int rowKeyColumnIndex;
76
77 public static String ROWKEY_COLUMN_SPEC="HBASE_ROW_KEY";
78
79
80
81
82
83 public TsvParser(String columnsSpecification, String separatorStr) {
84
85 byte[] separator = Bytes.toBytes(separatorStr);
86 Preconditions.checkArgument(separator.length == 1,
87 "TsvParser only supports single-byte separators");
88 separatorByte = separator[0];
89
90
91 ArrayList<String> columnStrings = Lists.newArrayList(
92 Splitter.on(',').trimResults().split(columnsSpecification));
93
94 families = new byte[columnStrings.size()][];
95 qualifiers = new byte[columnStrings.size()][];
96
97 for (int i = 0; i < columnStrings.size(); i++) {
98 String str = columnStrings.get(i);
99 if (ROWKEY_COLUMN_SPEC.equals(str)) {
100 rowKeyColumnIndex = i;
101 continue;
102 }
103 String[] parts = str.split(":", 2);
104 if (parts.length == 1) {
105 families[i] = str.getBytes();
106 qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
107 } else {
108 families[i] = parts[0].getBytes();
109 qualifiers[i] = parts[1].getBytes();
110 }
111 }
112 }
113
114 public int getRowKeyColumnIndex() {
115 return rowKeyColumnIndex;
116 }
117 public byte[] getFamily(int idx) {
118 return families[idx];
119 }
120 public byte[] getQualifier(int idx) {
121 return qualifiers[idx];
122 }
123
124 public ParsedLine parse(byte[] lineBytes, int length)
125 throws BadTsvLineException {
126
127 ArrayList<Integer> tabOffsets = new ArrayList<Integer>(families.length);
128 for (int i = 0; i < length; i++) {
129 if (lineBytes[i] == separatorByte) {
130 tabOffsets.add(i);
131 }
132 }
133 if (tabOffsets.isEmpty()) {
134 throw new BadTsvLineException("No delimiter");
135 }
136
137 tabOffsets.add(length);
138
139 if (tabOffsets.size() > families.length) {
140 throw new BadTsvLineException("Excessive columns");
141 } else if (tabOffsets.size() <= getRowKeyColumnIndex()) {
142 throw new BadTsvLineException("No row key");
143 }
144 return new ParsedLine(tabOffsets, lineBytes);
145 }
146
147 class ParsedLine {
148 private final ArrayList<Integer> tabOffsets;
149 private byte[] lineBytes;
150
151 ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
152 this.tabOffsets = tabOffsets;
153 this.lineBytes = lineBytes;
154 }
155
156 public int getRowKeyOffset() {
157 return getColumnOffset(rowKeyColumnIndex);
158 }
159 public int getRowKeyLength() {
160 return getColumnLength(rowKeyColumnIndex);
161 }
162 public int getColumnOffset(int idx) {
163 if (idx > 0)
164 return tabOffsets.get(idx - 1) + 1;
165 else
166 return 0;
167 }
168 public int getColumnLength(int idx) {
169 return tabOffsets.get(idx) - getColumnOffset(idx);
170 }
171 public int getColumnCount() {
172 return tabOffsets.size();
173 }
174 public byte[] getLineBytes() {
175 return lineBytes;
176 }
177 }
178
179 public static class BadTsvLineException extends Exception {
180 public BadTsvLineException(String err) {
181 super(err);
182 }
183 private static final long serialVersionUID = 1L;
184 }
185 }
186
187
188
189
190 static class TsvImporter
191 extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
192 {
193
194
195 private long ts;
196
197
198 private boolean skipBadLines;
199 private Counter badLineCount;
200
201 private TsvParser parser;
202
203 @Override
204 protected void setup(Context context) {
205 Configuration conf = context.getConfiguration();
206 parser = new TsvParser(conf.get(COLUMNS_CONF_KEY),
207 conf.get(SEPARATOR_CONF_KEY, DEFAULT_SEPARATOR));
208 if (parser.getRowKeyColumnIndex() == -1) {
209 throw new RuntimeException("No row key column specified");
210 }
211 ts = System.currentTimeMillis();
212
213 skipBadLines = context.getConfiguration().getBoolean(
214 SKIP_LINES_CONF_KEY, true);
215 badLineCount = context.getCounter("ImportTsv", "Bad Lines");
216 }
217
218
219
220
221 @Override
222 public void map(LongWritable offset, Text value,
223 Context context)
224 throws IOException {
225 byte[] lineBytes = value.getBytes();
226
227 try {
228 TsvParser.ParsedLine parsed = parser.parse(
229 lineBytes, value.getLength());
230 ImmutableBytesWritable rowKey =
231 new ImmutableBytesWritable(lineBytes,
232 parsed.getRowKeyOffset(),
233 parsed.getRowKeyLength());
234
235 Put put = new Put(rowKey.copyBytes());
236 for (int i = 0; i < parsed.getColumnCount(); i++) {
237 if (i == parser.getRowKeyColumnIndex()) continue;
238 KeyValue kv = new KeyValue(
239 lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(),
240 parser.getFamily(i), 0, parser.getFamily(i).length,
241 parser.getQualifier(i), 0, parser.getQualifier(i).length,
242 ts,
243 KeyValue.Type.Put,
244 lineBytes, parsed.getColumnOffset(i), parsed.getColumnLength(i));
245 put.add(kv);
246 }
247 context.write(rowKey, put);
248 } catch (BadTsvLineException badLine) {
249 if (skipBadLines) {
250 System.err.println(
251 "Bad line at offset: " + offset.get() + ":\n" +
252 badLine.getMessage());
253 badLineCount.increment(1);
254 return;
255 } else {
256 throw new IOException(badLine);
257 }
258 } catch (InterruptedException e) {
259 e.printStackTrace();
260 }
261 }
262 }
263
264
265
266
267
268
269
270
271
272 public static Job createSubmittableJob(Configuration conf, String[] args)
273 throws IOException {
274 String tableName = args[0];
275 Path inputDir = new Path(args[1]);
276 Job job = new Job(conf, NAME + "_" + tableName);
277 job.setJarByClass(TsvImporter.class);
278 FileInputFormat.setInputPaths(job, inputDir);
279 job.setInputFormatClass(TextInputFormat.class);
280 job.setMapperClass(TsvImporter.class);
281
282 String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
283 if (hfileOutPath != null) {
284 HTable table = new HTable(conf, tableName);
285 job.setReducerClass(PutSortReducer.class);
286 Path outputDir = new Path(hfileOutPath);
287 FileOutputFormat.setOutputPath(job, outputDir);
288 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
289 job.setMapOutputValueClass(Put.class);
290 HFileOutputFormat.configureIncrementalLoad(job, table);
291 } else {
292
293
294 TableMapReduceUtil.initTableReducerJob(tableName, null, job);
295 job.setNumReduceTasks(0);
296 }
297
298 TableMapReduceUtil.addDependencyJars(job);
299 TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
300 com.google.common.base.Function.class
301 return job;
302 }
303
304
305
306
307 private static void usage(final String errorMsg) {
308 if (errorMsg != null && errorMsg.length() > 0) {
309 System.err.println("ERROR: " + errorMsg);
310 }
311 String usage =
312 "Usage: " + NAME + " -Dimporttsv.columns=a,b,c <tablename> <inputdir>\n" +
313 "\n" +
314 "Imports the given input directory of TSV data into the specified table.\n" +
315 "\n" +
316 "The column names of the TSV data must be specified using the -Dimporttsv.columns\n" +
317 "option. This option takes the form of comma-separated column names, where each\n" +
318 "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
319 "column name HBASE_ROW_KEY is used to designate that this column should be used\n" +
320 "as the row key for each imported record. You must specify exactly one column\n" +
321 "to be the row key.\n" +
322 "\n" +
323 "In order to prepare data for a bulk data load, pass the option:\n" +
324 " -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
325 "\n" +
326 "Other options that may be specified with -D include:\n" +
327 " -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
328 " '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs";
329 System.err.println(usage);
330 }
331
332
333
334
335
336
337
338 public static void main(String[] args) throws Exception {
339 Configuration conf = HBaseConfiguration.create();
340 String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
341 if (otherArgs.length < 2) {
342 usage("Wrong number of arguments: " + otherArgs.length);
343 System.exit(-1);
344 }
345
346
347 String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
348 if (columns == null) {
349 usage("No columns specified. Please specify with -D" +
350 COLUMNS_CONF_KEY+"=...");
351 System.exit(-1);
352 }
353
354
355 int rowkeysFound=0;
356 for (String col : columns) {
357 if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
358 }
359 if (rowkeysFound != 1) {
360 usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
361 System.exit(-1);
362 }
363
364
365 if (columns.length < 2) {
366 usage("One or more columns in addition to the row key are required");
367 System.exit(-1);
368 }
369
370 Job job = createSubmittableJob(conf, otherArgs);
371 System.exit(job.waitForCompletion(true) ? 0 : 1);
372 }
373
374 }