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