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