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.lang.reflect.InvocationTargetException;
24 import java.lang.reflect.Method;
25 import java.util.ArrayList;
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.conf.Configuration;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.hbase.HBaseConfiguration;
36 import org.apache.hadoop.hbase.KeyValue;
37 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
38 import org.apache.hadoop.hbase.client.Delete;
39 import org.apache.hadoop.hbase.client.HConnection;
40 import org.apache.hadoop.hbase.client.HConnectionManager;
41 import org.apache.hadoop.hbase.client.HTable;
42 import org.apache.hadoop.hbase.client.Mutation;
43 import org.apache.hadoop.hbase.client.Put;
44 import org.apache.hadoop.hbase.client.Result;
45 import org.apache.hadoop.hbase.filter.Filter;
46 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
47 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
48 import org.apache.hadoop.hbase.util.Bytes;
49 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
50 import org.apache.hadoop.mapreduce.Job;
51 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
52 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
53 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
54 import org.apache.hadoop.util.GenericOptionsParser;
55 import org.apache.zookeeper.KeeperException;
56
57
58
59
60 public class Import {
61 private static final Log LOG = LogFactory.getLog(Import.class);
62 final static String NAME = "import";
63 final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
64 final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
65 final static String FILTER_CLASS_CONF_KEY = "import.filter.class";
66 final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
67
68
69 private static Filter filter;
70
71
72
73
74 static class KeyValueImporter
75 extends TableMapper<ImmutableBytesWritable, KeyValue> {
76 private Map<byte[], byte[]> cfRenameMap;
77
78
79
80
81
82
83
84
85
86 @Override
87 public void map(ImmutableBytesWritable row, Result value,
88 Context context)
89 throws IOException {
90 try {
91 if (filter == null || !filter.filterRowKey(row.get(), row.getOffset(), row.getLength())) {
92 for (KeyValue kv : value.raw()) {
93 kv = filterKv(kv);
94
95 if (kv == null) continue;
96
97 context.write(row, convertKv(kv, cfRenameMap));
98 }
99 }
100 } catch (InterruptedException e) {
101 e.printStackTrace();
102 }
103 }
104
105 @Override
106 public void setup(Context context) {
107 cfRenameMap = createCfRenameMap(context.getConfiguration());
108 filter = instantiateFilter(context.getConfiguration());
109 }
110 }
111
112
113
114
115 static class Importer
116 extends TableMapper<ImmutableBytesWritable, Mutation> {
117 private Map<byte[], byte[]> cfRenameMap;
118 private UUID clusterId;
119
120
121
122
123
124
125
126
127
128 @Override
129 public void map(ImmutableBytesWritable row, Result value,
130 Context context)
131 throws IOException {
132 try {
133 writeResult(row, value, context);
134 } catch (InterruptedException e) {
135 e.printStackTrace();
136 }
137 }
138
139 private void writeResult(ImmutableBytesWritable key, Result result, Context context)
140 throws IOException, InterruptedException {
141 Put put = null;
142 Delete delete = null;
143 if (filter == null || !filter.filterRowKey(key.get(), key.getOffset(), key.getLength())) {
144 for (KeyValue kv : result.raw()) {
145 kv = filterKv(kv);
146
147 if (kv == null) continue;
148
149 kv = convertKv(kv, cfRenameMap);
150
151 if (kv.isDelete()) {
152 if (delete == null) {
153 delete = new Delete(key.get());
154 }
155 delete.addDeleteMarker(kv);
156 } else {
157 if (put == null) {
158 put = new Put(key.get());
159 }
160 put.add(kv);
161 }
162 }
163 if (put != null) {
164 put.setClusterId(clusterId);
165 context.write(key, put);
166 }
167 if (delete != null) {
168 delete.setClusterId(clusterId);
169 context.write(key, delete);
170 }
171 }
172 }
173
174 @Override
175 public void setup(Context context) {
176 Configuration conf = context.getConfiguration();
177 cfRenameMap = createCfRenameMap(conf);
178 filter = instantiateFilter(conf);
179
180 try {
181 HConnection connection = HConnectionManager.getConnection(conf);
182 ZooKeeperWatcher zkw = connection.getZooKeeperWatcher();
183 ReplicationZookeeper zkHelper = new ReplicationZookeeper(connection, conf, zkw);
184 clusterId = zkHelper.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 }
192
193 }
194 }
195
196
197
198
199
200
201
202
203 private static Filter instantiateFilter(Configuration conf) {
204
205 Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
206 if (filterClass == null) {
207 LOG.debug("No configured filter class, accepting all keyvalues.");
208 return null;
209 }
210 LOG.debug("Attempting to create filter:" + filterClass);
211
212 try {
213 Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);
214 return (Filter) m.invoke(null, getFilterArgs(conf));
215 } catch (IllegalAccessException e) {
216 LOG.error("Couldn't instantiate filter!", e);
217 throw new RuntimeException(e);
218 } catch (SecurityException e) {
219 LOG.error("Couldn't instantiate filter!", e);
220 throw new RuntimeException(e);
221 } catch (NoSuchMethodException e) {
222 LOG.error("Couldn't instantiate filter!", e);
223 throw new RuntimeException(e);
224 } catch (IllegalArgumentException e) {
225 LOG.error("Couldn't instantiate filter!", e);
226 throw new RuntimeException(e);
227 } catch (InvocationTargetException e) {
228 LOG.error("Couldn't instantiate filter!", e);
229 throw new RuntimeException(e);
230 }
231 }
232
233 private static ArrayList<byte[]> getFilterArgs(Configuration conf) {
234 ArrayList<byte[]> args = new ArrayList<byte[]>();
235 String[] sargs = conf.getStrings(FILTER_ARGS_CONF_KEY);
236 for (String arg : sargs) {
237
238
239 args.add(Bytes.toBytes("'" + arg + "'"));
240 }
241 return args;
242 }
243
244
245
246
247
248
249
250 private static KeyValue filterKv(KeyValue kv) {
251
252 if (filter != null) {
253 Filter.ReturnCode code = filter.filterKeyValue(kv);
254 if (LOG.isTraceEnabled()) {
255 LOG.trace("Filter returned:" + code + " for the key value:" + kv);
256 }
257
258 if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
259 .equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))) {
260 return null;
261 }
262 }
263 return kv;
264 }
265
266
267 private static KeyValue convertKv(KeyValue kv, Map<byte[], byte[]> cfRenameMap) {
268 if(cfRenameMap != null) {
269
270 byte[] newCfName = cfRenameMap.get(kv.getFamily());
271 if(newCfName != null) {
272 kv = new KeyValue(kv.getBuffer(),
273 kv.getRowOffset(),
274 kv.getRowLength(),
275 newCfName,
276 0,
277 newCfName.length,
278 kv.getBuffer(),
279 kv.getQualifierOffset(),
280 kv.getQualifierLength(),
281 kv.getTimestamp(),
282 KeyValue.Type.codeToType(kv.getType()),
283 kv.getBuffer(),
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
376 public static Job createSubmittableJob(Configuration conf, String[] args)
377 throws IOException {
378 String tableName = args[0];
379 Path inputDir = new Path(args[1]);
380 Job job = new Job(conf, NAME + "_" + tableName);
381 job.setJarByClass(Importer.class);
382 FileInputFormat.setInputPaths(job, inputDir);
383 job.setInputFormatClass(SequenceFileInputFormat.class);
384 String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
385
386
387 try {
388 Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
389 if (filter != null) {
390 TableMapReduceUtil.addDependencyJars(conf, filter);
391 }
392 } catch (Exception e) {
393 throw new IOException(e);
394 }
395
396 if (hfileOutPath != null) {
397 job.setMapperClass(KeyValueImporter.class);
398 HTable table = new HTable(conf, tableName);
399 job.setReducerClass(KeyValueSortReducer.class);
400 Path outputDir = new Path(hfileOutPath);
401 FileOutputFormat.setOutputPath(job, outputDir);
402 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
403 job.setMapOutputValueClass(KeyValue.class);
404 HFileOutputFormat.configureIncrementalLoad(job, table);
405 TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
406 com.google.common.base.Preconditions.class);
407 } else {
408
409
410 job.setMapperClass(Importer.class);
411 TableMapReduceUtil.initTableReducerJob(tableName, null, job);
412 job.setNumReduceTasks(0);
413 }
414 return job;
415 }
416
417
418
419
420 private static void usage(final String errorMsg) {
421 if (errorMsg != null && errorMsg.length() > 0) {
422 System.err.println("ERROR: " + errorMsg);
423 }
424 System.err.println("Usage: Import [options] <tablename> <inputdir>");
425 System.err.println("By default Import will load data directly into HBase. To instead generate");
426 System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");
427 System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
428 System.err
429 .println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");
430 System.err.println(" -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");
431 System.err.println(" -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter");
432 System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the "
433 + CF_RENAME_PROP + " property. Futher, filters will only use the"
434 + " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify "
435 + " whether the current row needs to be ignored completely for processing and "
436 + "Filter#filterKeyValue(KeyValue) method to determine if the KeyValue should be added;"
437 + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including "
438 + "the KeyValue.");
439 System.err.println("For performance consider the following options:\n"
440 + " -Dmapred.map.tasks.speculative.execution=false\n"
441 + " -Dmapred.reduce.tasks.speculative.execution=false");
442 }
443
444
445
446
447
448
449
450 public static void main(String[] args) throws Exception {
451 Configuration conf = HBaseConfiguration.create();
452 String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
453 if (otherArgs.length < 2) {
454 usage("Wrong number of arguments: " + otherArgs.length);
455 System.exit(-1);
456 }
457 Job job = createSubmittableJob(conf, otherArgs);
458 System.exit(job.waitForCompletion(true) ? 0 : 1);
459 }
460 }