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 java.io.IOException;
22 import java.lang.reflect.InvocationTargetException;
23 import java.lang.reflect.Method;
24 import java.util.ArrayList;
25 import java.util.Collections;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.TreeMap;
29 import java.util.UUID;
30
31 import org.apache.commons.logging.Log;
32 import org.apache.commons.logging.LogFactory;
33 import org.apache.hadoop.classification.InterfaceAudience;
34 import org.apache.hadoop.classification.InterfaceStability;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.fs.Path;
37 import org.apache.hadoop.hbase.Cell;
38 import org.apache.hadoop.hbase.CellUtil;
39 import org.apache.hadoop.hbase.HBaseConfiguration;
40 import org.apache.hadoop.hbase.KeyValue;
41 import org.apache.hadoop.hbase.KeyValueUtil;
42 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
43 import org.apache.hadoop.hbase.client.Delete;
44 import org.apache.hadoop.hbase.client.HTable;
45 import org.apache.hadoop.hbase.client.Mutation;
46 import org.apache.hadoop.hbase.client.Put;
47 import org.apache.hadoop.hbase.client.Result;
48 import org.apache.hadoop.hbase.filter.Filter;
49 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
50 import org.apache.hadoop.hbase.util.Bytes;
51 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
52 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
53 import org.apache.hadoop.mapreduce.Job;
54 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
55 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
56 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
57 import org.apache.hadoop.util.GenericOptionsParser;
58 import org.apache.zookeeper.KeeperException;
59
60
61
62
63 @InterfaceAudience.Public
64 @InterfaceStability.Stable
65 public class Import {
66 private static final Log LOG = LogFactory.getLog(Import.class);
67 final static String NAME = "import";
68 final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
69 final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
70 final static String FILTER_CLASS_CONF_KEY = "import.filter.class";
71 final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
72
73
74 private static Filter filter;
75
76
77
78
79 static class KeyValueImporter
80 extends TableMapper<ImmutableBytesWritable, KeyValue> {
81 private Map<byte[], byte[]> cfRenameMap;
82
83
84
85
86
87
88
89
90
91 @Override
92 public void map(ImmutableBytesWritable row, Result value,
93 Context context)
94 throws IOException {
95 try {
96 for (Cell kv : value.rawCells()) {
97 kv = filterKv(kv);
98
99 if (kv == null) continue;
100
101 context.write(row, KeyValueUtil.ensureKeyValue(convertKv(kv, cfRenameMap)));
102 }
103 } catch (InterruptedException e) {
104 e.printStackTrace();
105 }
106 }
107
108 @Override
109 public void setup(Context context) {
110 cfRenameMap = createCfRenameMap(context.getConfiguration());
111 filter = instantiateFilter(context.getConfiguration());
112 }
113 }
114
115
116
117
118 static class Importer
119 extends TableMapper<ImmutableBytesWritable, Mutation> {
120 private Map<byte[], byte[]> cfRenameMap;
121 private List<UUID> clusterIds;
122
123
124
125
126
127
128
129
130
131 @Override
132 public void map(ImmutableBytesWritable row, Result value,
133 Context context)
134 throws IOException {
135 try {
136 writeResult(row, value, context);
137 } catch (InterruptedException e) {
138 e.printStackTrace();
139 }
140 }
141
142 private void writeResult(ImmutableBytesWritable key, Result result, Context context)
143 throws IOException, InterruptedException {
144 Put put = null;
145 Delete delete = null;
146 for (Cell kv : result.rawCells()) {
147 kv = filterKv(kv);
148
149 if (kv == null) continue;
150
151 kv = convertKv(kv, cfRenameMap);
152
153 if (CellUtil.isDelete(kv)) {
154 if (delete == null) {
155 delete = new Delete(key.get());
156 }
157 delete.addDeleteMarker(kv);
158 } else {
159 if (put == null) {
160 put = new Put(key.get());
161 }
162 put.add(kv);
163 }
164 }
165 if (put != null) {
166 put.setClusterIds(clusterIds);
167 context.write(key, put);
168 }
169 if (delete != null) {
170 delete.setClusterIds(clusterIds);
171 context.write(key, delete);
172 }
173 }
174
175 @Override
176 public void setup(Context context) {
177 Configuration conf = context.getConfiguration();
178 cfRenameMap = createCfRenameMap(conf);
179 filter = instantiateFilter(conf);
180
181 ZooKeeperWatcher zkw = null;
182 try {
183 zkw = new ZooKeeperWatcher(conf, context.getTaskAttemptID().toString(), null);
184 clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));
185 } catch (ZooKeeperConnectionException e) {
186 LOG.error("Problem connecting to ZooKeper during task setup", e);
187 } catch (KeeperException e) {
188 LOG.error("Problem reading ZooKeeper data during task setup", e);
189 } catch (IOException e) {
190 LOG.error("Problem setting up task", e);
191 } finally {
192 if (zkw != null) zkw.close();
193 }
194 }
195 }
196
197
198
199
200
201
202
203
204 private static Filter instantiateFilter(Configuration conf) {
205
206 Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
207 if (filterClass == null) {
208 LOG.debug("No configured filter class, accepting all keyvalues.");
209 return null;
210 }
211 LOG.debug("Attempting to create filter:" + filterClass);
212
213 try {
214 Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);
215 return (Filter) m.invoke(null, getFilterArgs(conf));
216 } catch (IllegalAccessException e) {
217 LOG.error("Couldn't instantiate filter!", e);
218 throw new RuntimeException(e);
219 } catch (SecurityException e) {
220 LOG.error("Couldn't instantiate filter!", e);
221 throw new RuntimeException(e);
222 } catch (NoSuchMethodException e) {
223 LOG.error("Couldn't instantiate filter!", e);
224 throw new RuntimeException(e);
225 } catch (IllegalArgumentException e) {
226 LOG.error("Couldn't instantiate filter!", e);
227 throw new RuntimeException(e);
228 } catch (InvocationTargetException e) {
229 LOG.error("Couldn't instantiate filter!", e);
230 throw new RuntimeException(e);
231 }
232 }
233
234 private static ArrayList<byte[]> getFilterArgs(Configuration conf) {
235 ArrayList<byte[]> args = new ArrayList<byte[]>();
236 String[] sargs = conf.getStrings(FILTER_ARGS_CONF_KEY);
237 for (String arg : sargs) {
238
239
240 args.add(Bytes.toBytes("'" + arg + "'"));
241 }
242 return args;
243 }
244
245
246
247
248
249
250
251 private static Cell filterKv(Cell kv) throws IOException {
252
253 if (filter != null) {
254 Filter.ReturnCode code = filter.filterKeyValue(kv);
255 LOG.debug("Filter returned:" + code);
256
257 if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
258 .equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))) {
259 LOG.debug("Skipping key: " + kv + " from filter decision: " + code);
260 return null;
261 }
262 }
263 return kv;
264 }
265
266
267 private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) {
268 if(cfRenameMap != null) {
269
270 byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));
271 if(newCfName != null) {
272 kv = new KeyValue(kv.getRowArray(),
273 kv.getRowOffset(),
274 kv.getRowLength(),
275 newCfName,
276 0,
277 newCfName.length,
278 kv.getQualifierArray(),
279 kv.getQualifierOffset(),
280 kv.getQualifierLength(),
281 kv.getTimestamp(),
282 KeyValue.Type.codeToType(kv.getTypeByte()),
283 kv.getValueArray(),
284 kv.getValueOffset(),
285 kv.getValueLength());
286 }
287 }
288 return kv;
289 }
290
291
292 private static Map<byte[], byte[]> createCfRenameMap(Configuration conf) {
293 Map<byte[], byte[]> cfRenameMap = null;
294 String allMappingsPropVal = conf.get(CF_RENAME_PROP);
295 if(allMappingsPropVal != null) {
296
297 String[] allMappings = allMappingsPropVal.split(",");
298 for (String mapping: allMappings) {
299 if(cfRenameMap == null) {
300 cfRenameMap = new TreeMap<byte[],byte[]>(Bytes.BYTES_COMPARATOR);
301 }
302 String [] srcAndDest = mapping.split(":");
303 if(srcAndDest.length != 2) {
304 continue;
305 }
306 cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes());
307 }
308 }
309 return cfRenameMap;
310 }
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325 static public void configureCfRenaming(Configuration conf,
326 Map<String, String> renameMap) {
327 StringBuilder sb = new StringBuilder();
328 for(Map.Entry<String,String> entry: renameMap.entrySet()) {
329 String sourceCf = entry.getKey();
330 String destCf = entry.getValue();
331
332 if(sourceCf.contains(":") || sourceCf.contains(",") ||
333 destCf.contains(":") || destCf.contains(",")) {
334 throw new IllegalArgumentException("Illegal character in CF names: "
335 + sourceCf + ", " + destCf);
336 }
337
338 if(sb.length() != 0) {
339 sb.append(",");
340 }
341 sb.append(sourceCf + ":" + destCf);
342 }
343 conf.set(CF_RENAME_PROP, sb.toString());
344 }
345
346
347
348
349
350
351
352 public static void addFilterAndArguments(Configuration conf, Class<? extends Filter> clazz,
353 List<String> args) {
354 conf.set(Import.FILTER_CLASS_CONF_KEY, clazz.getName());
355
356
357 StringBuilder builder = new StringBuilder();
358 for (int i = 0; i < args.size(); i++) {
359 String arg = args.get(i);
360 builder.append(arg);
361 if (i != args.size() - 1) {
362 builder.append(",");
363 }
364 }
365 conf.set(Import.FILTER_ARGS_CONF_KEY, builder.toString());
366 }
367
368
369
370
371
372
373
374
375 public static Job createSubmittableJob(Configuration conf, String[] args)
376 throws IOException {
377 String tableName = args[0];
378 Path inputDir = new Path(args[1]);
379 Job job = new Job(conf, NAME + "_" + tableName);
380 job.setJarByClass(Importer.class);
381 FileInputFormat.setInputPaths(job, inputDir);
382 job.setInputFormatClass(SequenceFileInputFormat.class);
383 String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
384
385
386 try {
387 Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
388 if (filter != null) {
389 TableMapReduceUtil.addDependencyJars(conf, filter);
390 }
391 } catch (Exception e) {
392 throw new IOException(e);
393 }
394
395 if (hfileOutPath != null) {
396 job.setMapperClass(KeyValueImporter.class);
397 HTable table = new HTable(conf, tableName);
398 job.setReducerClass(KeyValueSortReducer.class);
399 Path outputDir = new Path(hfileOutPath);
400 FileOutputFormat.setOutputPath(job, outputDir);
401 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
402 job.setMapOutputValueClass(KeyValue.class);
403 HFileOutputFormat.configureIncrementalLoad(job, table);
404 TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
405 com.google.common.base.Preconditions.class);
406 } else {
407
408
409 job.setMapperClass(Importer.class);
410 TableMapReduceUtil.initTableReducerJob(tableName, null, job);
411 job.setNumReduceTasks(0);
412 }
413 return job;
414 }
415
416
417
418
419 private static void usage(final String errorMsg) {
420 if (errorMsg != null && errorMsg.length() > 0) {
421 System.err.println("ERROR: " + errorMsg);
422 }
423 System.err.println("Usage: Import [options] <tablename> <inputdir>");
424 System.err.println("By default Import will load data directly into HBase. To instead generate");
425 System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
426 System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
427 System.err
428 .println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");
429 System.err.println(" -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");
430 System.err.println(" -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter");
431 System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the "
432 + CF_RENAME_PROP + " property. Futher, filters will only use the"
433 + "Filter#filterKeyValue(KeyValue) method to determine if the KeyValue should be added;"
434 + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including "
435 + "the KeyValue.");
436 System.err.println("For performance consider the following options:\n"
437 + " -Dmapred.map.tasks.speculative.execution=false\n"
438 + " -Dmapred.reduce.tasks.speculative.execution=false");
439 }
440
441
442
443
444
445
446
447 public static void main(String[] args) throws Exception {
448 Configuration conf = HBaseConfiguration.create();
449 String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
450 if (otherArgs.length < 2) {
451 usage("Wrong number of arguments: " + otherArgs.length);
452 System.exit(-1);
453 }
454 String inputVersionString = System.getProperty(ResultSerialization.IMPORT_FORMAT_VER);
455 if (inputVersionString != null) {
456 conf.set(ResultSerialization.IMPORT_FORMAT_VER, inputVersionString);
457 }
458 Job job = createSubmittableJob(conf, otherArgs);
459 System.exit(job.waitForCompletion(true) ? 0 : 1);
460 }
461 }