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.List;
26 import java.util.Map;
27 import java.util.TreeMap;
28 import java.util.UUID;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.classification.InterfaceAudience;
33 import org.apache.hadoop.classification.InterfaceStability;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.hbase.HBaseConfiguration;
37 import org.apache.hadoop.hbase.KeyValue;
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.exceptions.ZooKeeperConnectionException;
46 import org.apache.hadoop.hbase.filter.Filter;
47 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
48 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
49 import org.apache.hadoop.hbase.util.Bytes;
50 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
51 import org.apache.hadoop.mapreduce.Job;
52 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
53 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
54 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
55 import org.apache.hadoop.util.GenericOptionsParser;
56 import org.apache.zookeeper.KeeperException;
57
58
59
60
61 @InterfaceAudience.Public
62 @InterfaceStability.Stable
63 public class Import {
64 private static final Log LOG = LogFactory.getLog(Import.class);
65 final static String NAME = "import";
66 final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
67 final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";
68 final static String FILTER_CLASS_CONF_KEY = "import.filter.class";
69 final static String FILTER_ARGS_CONF_KEY = "import.filter.args";
70
71
72 private static Filter filter;
73
74
75
76
77 static class KeyValueImporter
78 extends TableMapper<ImmutableBytesWritable, KeyValue> {
79 private Map<byte[], byte[]> cfRenameMap;
80
81
82
83
84
85
86
87
88
89 @Override
90 public void map(ImmutableBytesWritable row, Result value,
91 Context context)
92 throws IOException {
93 try {
94 for (KeyValue kv : value.raw()) {
95 kv = filterKv(kv);
96
97 if (kv == null) continue;
98
99 context.write(row, convertKv(kv, cfRenameMap));
100 }
101 } catch (InterruptedException e) {
102 e.printStackTrace();
103 }
104 }
105
106 @Override
107 public void setup(Context context) {
108 cfRenameMap = createCfRenameMap(context.getConfiguration());
109 filter = instantiateFilter(context.getConfiguration());
110 }
111 }
112
113
114
115
116 static class Importer
117 extends TableMapper<ImmutableBytesWritable, Mutation> {
118 private Map<byte[], byte[]> cfRenameMap;
119 private UUID clusterId;
120
121
122
123
124
125
126
127
128
129 @Override
130 public void map(ImmutableBytesWritable row, Result value,
131 Context context)
132 throws IOException {
133 try {
134 writeResult(row, value, context);
135 } catch (InterruptedException e) {
136 e.printStackTrace();
137 }
138 }
139
140 private void writeResult(ImmutableBytesWritable key, Result result, Context context)
141 throws IOException, InterruptedException {
142 Put put = null;
143 Delete delete = null;
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 @Override
174 public void setup(Context context) {
175 Configuration conf = context.getConfiguration();
176 cfRenameMap = createCfRenameMap(conf);
177 filter = instantiateFilter(conf);
178
179 try {
180 HConnection connection = HConnectionManager.getConnection(conf);
181 ZooKeeperWatcher zkw = connection.getZooKeeperWatcher();
182 ReplicationZookeeper zkHelper = new ReplicationZookeeper(connection, conf, zkw);
183 clusterId = zkHelper.getUUIDForCluster(zkw);
184 } catch (ZooKeeperConnectionException e) {
185 LOG.error("Problem connecting to ZooKeper during task setup", e);
186 } catch (KeeperException e) {
187 LOG.error("Problem reading ZooKeeper data during task setup", e);
188 } catch (IOException e) {
189 LOG.error("Problem setting up task", e);
190 }
191
192 }
193 }
194
195
196
197
198
199
200
201
202 private static Filter instantiateFilter(Configuration conf) {
203
204 Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);
205 if (filterClass == null) {
206 LOG.debug("No configured filter class, accepting all keyvalues.");
207 return null;
208 }
209 LOG.debug("Attempting to create filter:" + filterClass);
210
211 try {
212 Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);
213 return (Filter) m.invoke(null, getFilterArgs(conf));
214 } catch (IllegalAccessException e) {
215 LOG.error("Couldn't instantiate filter!", e);
216 throw new RuntimeException(e);
217 } catch (SecurityException e) {
218 LOG.error("Couldn't instantiate filter!", e);
219 throw new RuntimeException(e);
220 } catch (NoSuchMethodException e) {
221 LOG.error("Couldn't instantiate filter!", e);
222 throw new RuntimeException(e);
223 } catch (IllegalArgumentException e) {
224 LOG.error("Couldn't instantiate filter!", e);
225 throw new RuntimeException(e);
226 } catch (InvocationTargetException e) {
227 LOG.error("Couldn't instantiate filter!", e);
228 throw new RuntimeException(e);
229 }
230 }
231
232 private static ArrayList<byte[]> getFilterArgs(Configuration conf) {
233 ArrayList<byte[]> args = new ArrayList<byte[]>();
234 String[] sargs = conf.getStrings(FILTER_ARGS_CONF_KEY);
235 for (String arg : sargs) {
236
237
238 args.add(Bytes.toBytes("'" + arg + "'"));
239 }
240 return args;
241 }
242
243
244
245
246
247
248
249 private static KeyValue filterKv(KeyValue kv) throws IOException {
250
251 if (filter != null) {
252 Filter.ReturnCode code = filter.filterKeyValue(kv);
253 System.out.println("Filter returned:" + code);
254
255 if (!(code.equals(Filter.ReturnCode.INCLUDE) || code
256 .equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))) {
257 if (LOG.isDebugEnabled()) {
258 System.out.println("Skipping key: " + kv + " from filter decision: " + code);
259 }
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 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 Job job = createSubmittableJob(conf, otherArgs);
455 System.exit(job.waitForCompletion(true) ? 0 : 1);
456 }
457 }