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 org.apache.hadoop.hbase.util.Base64;
23
24 import java.io.IOException;
25 import java.util.ArrayList;
26
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.fs.Path;
29 import org.apache.hadoop.hbase.HBaseConfiguration;
30 import org.apache.hadoop.hbase.HConstants;
31 import org.apache.hadoop.hbase.client.HTable;
32 import org.apache.hadoop.hbase.client.Put;
33 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
34 import org.apache.hadoop.hbase.util.Bytes;
35 import org.apache.hadoop.mapreduce.Job;
36 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
37 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
38 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
39 import org.apache.hadoop.util.GenericOptionsParser;
40
41 import com.google.common.base.Preconditions;
42 import com.google.common.base.Splitter;
43 import com.google.common.collect.Lists;
44
45
46
47
48
49
50
51
52
53 public class ImportTsv {
54 final static String NAME = "importtsv";
55
56 final static String MAPPER_CONF_KEY = "importtsv.mapper.class";
57 final static String SKIP_LINES_CONF_KEY = "importtsv.skip.bad.lines";
58 final static String BULK_OUTPUT_CONF_KEY = "importtsv.bulk.output";
59 final static String COLUMNS_CONF_KEY = "importtsv.columns";
60 final static String SEPARATOR_CONF_KEY = "importtsv.separator";
61 final static String TIMESTAMP_CONF_KEY = "importtsv.timestamp";
62 final static String DEFAULT_SEPARATOR = "\t";
63 final static Class DEFAULT_MAPPER = TsvImporterMapper.class;
64
65 static class TsvParser {
66
67
68
69 private final byte[][] families;
70 private final byte[][] qualifiers;
71
72 private final byte separatorByte;
73
74 private int rowKeyColumnIndex;
75
76 public static String ROWKEY_COLUMN_SPEC="HBASE_ROW_KEY";
77
78
79
80
81
82 public TsvParser(String columnsSpecification, String separatorStr) {
83
84 byte[] separator = Bytes.toBytes(separatorStr);
85 Preconditions.checkArgument(separator.length == 1,
86 "TsvParser only supports single-byte separators");
87 separatorByte = separator[0];
88
89
90 ArrayList<String> columnStrings = Lists.newArrayList(
91 Splitter.on(',').trimResults().split(columnsSpecification));
92
93 families = new byte[columnStrings.size()][];
94 qualifiers = new byte[columnStrings.size()][];
95
96 for (int i = 0; i < columnStrings.size(); i++) {
97 String str = columnStrings.get(i);
98 if (ROWKEY_COLUMN_SPEC.equals(str)) {
99 rowKeyColumnIndex = i;
100 continue;
101 }
102 String[] parts = str.split(":", 2);
103 if (parts.length == 1) {
104 families[i] = str.getBytes();
105 qualifiers[i] = HConstants.EMPTY_BYTE_ARRAY;
106 } else {
107 families[i] = parts[0].getBytes();
108 qualifiers[i] = parts[1].getBytes();
109 }
110 }
111 }
112
113 public int getRowKeyColumnIndex() {
114 return rowKeyColumnIndex;
115 }
116 public byte[] getFamily(int idx) {
117 return families[idx];
118 }
119 public byte[] getQualifier(int idx) {
120 return qualifiers[idx];
121 }
122
123 public ParsedLine parse(byte[] lineBytes, int length)
124 throws BadTsvLineException {
125
126 ArrayList<Integer> tabOffsets = new ArrayList<Integer>(families.length);
127 for (int i = 0; i < length; i++) {
128 if (lineBytes[i] == separatorByte) {
129 tabOffsets.add(i);
130 }
131 }
132 if (tabOffsets.isEmpty()) {
133 throw new BadTsvLineException("No delimiter");
134 }
135
136 tabOffsets.add(length);
137
138 if (tabOffsets.size() > families.length) {
139 throw new BadTsvLineException("Excessive columns");
140 } else if (tabOffsets.size() <= getRowKeyColumnIndex()) {
141 throw new BadTsvLineException("No row key");
142 }
143 return new ParsedLine(tabOffsets, lineBytes);
144 }
145
146 class ParsedLine {
147 private final ArrayList<Integer> tabOffsets;
148 private byte[] lineBytes;
149
150 ParsedLine(ArrayList<Integer> tabOffsets, byte[] lineBytes) {
151 this.tabOffsets = tabOffsets;
152 this.lineBytes = lineBytes;
153 }
154
155 public int getRowKeyOffset() {
156 return getColumnOffset(rowKeyColumnIndex);
157 }
158 public int getRowKeyLength() {
159 return getColumnLength(rowKeyColumnIndex);
160 }
161 public int getColumnOffset(int idx) {
162 if (idx > 0)
163 return tabOffsets.get(idx - 1) + 1;
164 else
165 return 0;
166 }
167 public int getColumnLength(int idx) {
168 return tabOffsets.get(idx) - getColumnOffset(idx);
169 }
170 public int getColumnCount() {
171 return tabOffsets.size();
172 }
173 public byte[] getLineBytes() {
174 return lineBytes;
175 }
176 }
177
178 public static class BadTsvLineException extends Exception {
179 public BadTsvLineException(String err) {
180 super(err);
181 }
182 private static final long serialVersionUID = 1L;
183 }
184 }
185
186
187
188
189
190
191
192
193
194 public static Job createSubmittableJob(Configuration conf, String[] args)
195 throws IOException, ClassNotFoundException {
196
197
198
199 String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
200 if (actualSeparator != null) {
201 conf.set(SEPARATOR_CONF_KEY, new String(
202 Base64.encodeBytes(actualSeparator.getBytes())));
203 }
204
205
206 String mapperClassName = conf.get(MAPPER_CONF_KEY);
207 Class mapperClass = mapperClassName != null ?
208 Class.forName(mapperClassName) : DEFAULT_MAPPER;
209
210 String tableName = args[0];
211 Path inputDir = new Path(args[1]);
212 Job job = new Job(conf, NAME + "_" + tableName);
213 job.setJarByClass(mapperClass);
214 FileInputFormat.setInputPaths(job, inputDir);
215 job.setInputFormatClass(TextInputFormat.class);
216 job.setMapperClass(mapperClass);
217
218 String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
219 if (hfileOutPath != null) {
220 HTable table = new HTable(conf, tableName);
221 job.setReducerClass(PutSortReducer.class);
222 Path outputDir = new Path(hfileOutPath);
223 FileOutputFormat.setOutputPath(job, outputDir);
224 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
225 job.setMapOutputValueClass(Put.class);
226 HFileOutputFormat.configureIncrementalLoad(job, table);
227 } else {
228
229
230 TableMapReduceUtil.initTableReducerJob(tableName, null, job);
231 job.setNumReduceTasks(0);
232 }
233
234 TableMapReduceUtil.addDependencyJars(job);
235 TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
236 com.google.common.base.Function.class
237 return job;
238 }
239
240
241
242
243 private static void usage(final String errorMsg) {
244 if (errorMsg != null && errorMsg.length() > 0) {
245 System.err.println("ERROR: " + errorMsg);
246 }
247 String usage =
248 "Usage: " + NAME + " -Dimporttsv.columns=a,b,c <tablename> <inputdir>\n" +
249 "\n" +
250 "Imports the given input directory of TSV data into the specified table.\n" +
251 "\n" +
252 "The column names of the TSV data must be specified using the -Dimporttsv.columns\n" +
253 "option. This option takes the form of comma-separated column names, where each\n" +
254 "column name is either a simple column family, or a columnfamily:qualifier. The special\n" +
255 "column name HBASE_ROW_KEY is used to designate that this column should be used\n" +
256 "as the row key for each imported record. You must specify exactly one column\n" +
257 "to be the row key, and you must specify a column name for every column that exists in the\n" +
258 "input data.\n" +
259 "\n" +
260 "By default importtsv will load data directly into HBase. To instead generate\n" +
261 "HFiles of data to prepare for a bulk data load, pass the option:\n" +
262 " -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output\n" +
263 " Note: if you do not use this option, then the target table must already exist in HBase\n" +
264 "\n" +
265 "Other options that may be specified with -D include:\n" +
266 " -D" + SKIP_LINES_CONF_KEY + "=false - fail if encountering an invalid line\n" +
267 " '-D" + SEPARATOR_CONF_KEY + "=|' - eg separate on pipes instead of tabs\n" +
268 " -D" + TIMESTAMP_CONF_KEY + "=currentTimeAsLong - use the specified timestamp for the import\n" +
269 " -D" + MAPPER_CONF_KEY + "=my.Mapper - A user-defined Mapper to use instead of " + DEFAULT_MAPPER.getName() + "\n";
270
271 System.err.println(usage);
272 }
273
274
275
276
277
278
279
280 public static void main(String[] args) throws Exception {
281 Configuration conf = HBaseConfiguration.create();
282 String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
283 if (otherArgs.length < 2) {
284 usage("Wrong number of arguments: " + otherArgs.length);
285 System.exit(-1);
286 }
287
288
289 String columns[] = conf.getStrings(COLUMNS_CONF_KEY);
290 if (columns == null) {
291 usage("No columns specified. Please specify with -D" +
292 COLUMNS_CONF_KEY+"=...");
293 System.exit(-1);
294 }
295
296
297 int rowkeysFound=0;
298 for (String col : columns) {
299 if (col.equals(TsvParser.ROWKEY_COLUMN_SPEC)) rowkeysFound++;
300 }
301 if (rowkeysFound != 1) {
302 usage("Must specify exactly one column as " + TsvParser.ROWKEY_COLUMN_SPEC);
303 System.exit(-1);
304 }
305
306
307 if (columns.length < 2) {
308 usage("One or more columns in addition to the row key are required");
309 System.exit(-1);
310 }
311
312 Job job = createSubmittableJob(conf, otherArgs);
313 System.exit(job.waitForCompletion(true) ? 0 : 1);
314 }
315
316 }