1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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.net.URL;
25  import java.net.URLDecoder;
26  import java.util.ArrayList;
27  import java.util.Enumeration;
28  import java.util.HashMap;
29  import java.util.HashSet;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.Set;
33  import java.util.zip.ZipEntry;
34  import java.util.zip.ZipFile;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.classification.InterfaceAudience;
39  import org.apache.hadoop.classification.InterfaceStability;
40  import org.apache.hadoop.conf.Configuration;
41  import org.apache.hadoop.fs.FileSystem;
42  import org.apache.hadoop.fs.Path;
43  import org.apache.hadoop.hbase.HBaseConfiguration;
44  import org.apache.hadoop.hbase.catalog.MetaReader;
45  import org.apache.hadoop.hbase.client.Put;
46  import org.apache.hadoop.hbase.client.Scan;
47  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
48  import org.apache.hadoop.hbase.mapreduce.hadoopbackport.JarFinder;
49  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
50  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
51  import org.apache.hadoop.hbase.security.User;
52  import org.apache.hadoop.hbase.util.Base64;
53  import org.apache.hadoop.hbase.util.Bytes;
54  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
55  import org.apache.hadoop.io.Writable;
56  import org.apache.hadoop.io.WritableComparable;
57  import org.apache.hadoop.mapreduce.InputFormat;
58  import org.apache.hadoop.mapreduce.Job;
59  import org.apache.hadoop.util.StringUtils;
60  
61  import com.google.protobuf.InvalidProtocolBufferException;
62  
63  /**
64   * Utility for {@link TableMapper} and {@link TableReducer}
65   */
66  @SuppressWarnings("unchecked")
67  @InterfaceAudience.Public
68  @InterfaceStability.Stable
69  public class TableMapReduceUtil {
70    static Log LOG = LogFactory.getLog(TableMapReduceUtil.class);
71  
72    /**
73     * Use this before submitting a TableMap job. It will appropriately set up
74     * the job.
75     *
76     * @param table  The table name to read from.
77     * @param scan  The scan instance with the columns, time range etc.
78     * @param mapper  The mapper class to use.
79     * @param outputKeyClass  The class of the output key.
80     * @param outputValueClass  The class of the output value.
81     * @param job  The current job to adjust.  Make sure the passed job is
82     * carrying all necessary HBase configuration.
83     * @throws IOException When setting up the details fails.
84     */
85    public static void initTableMapperJob(String table, Scan scan,
86        Class<? extends TableMapper> mapper,
87        Class<?> outputKeyClass,
88        Class<?> outputValueClass, Job job)
89    throws IOException {
90      initTableMapperJob(table, scan, mapper, outputKeyClass, outputValueClass,
91          job, true);
92    }
93  
94  
95    /**
96     * Use this before submitting a TableMap job. It will appropriately set up
97     * the job.
98     *
99     * @param table Binary representation of the table name to read from.
100    * @param scan  The scan instance with the columns, time range etc.
101    * @param mapper  The mapper class to use.
102    * @param outputKeyClass  The class of the output key.
103    * @param outputValueClass  The class of the output value.
104    * @param job  The current job to adjust.  Make sure the passed job is
105    * carrying all necessary HBase configuration.
106    * @throws IOException When setting up the details fails.
107    */
108    public static void initTableMapperJob(byte[] table, Scan scan,
109       Class<? extends TableMapper> mapper,
110       Class<?> outputKeyClass,
111       Class<?> outputValueClass, Job job)
112   throws IOException {
113       initTableMapperJob(Bytes.toString(table), scan, mapper, outputKeyClass, outputValueClass,
114               job, true);
115   }
116 
117   /**
118    * Use this before submitting a TableMap job. It will appropriately set up
119    * the job.
120    *
121    * @param table  The table name to read from.
122    * @param scan  The scan instance with the columns, time range etc.
123    * @param mapper  The mapper class to use.
124    * @param outputKeyClass  The class of the output key.
125    * @param outputValueClass  The class of the output value.
126    * @param job  The current job to adjust.  Make sure the passed job is
127    * carrying all necessary HBase configuration.
128    * @param addDependencyJars upload HBase jars and jars for any of the configured
129    *           job classes via the distributed cache (tmpjars).
130    * @throws IOException When setting up the details fails.
131    */
132   public static void initTableMapperJob(String table, Scan scan,
133       Class<? extends TableMapper> mapper,
134       Class<?> outputKeyClass,
135       Class<?> outputValueClass, Job job,
136       boolean addDependencyJars, Class<? extends InputFormat> inputFormatClass)
137   throws IOException {
138     job.setInputFormatClass(inputFormatClass);
139     if (outputValueClass != null) job.setMapOutputValueClass(outputValueClass);
140     if (outputKeyClass != null) job.setMapOutputKeyClass(outputKeyClass);
141     job.setMapperClass(mapper);
142     if (Put.class.equals(outputValueClass)) {
143       job.setCombinerClass(PutCombiner.class);
144     }
145     Configuration conf = job.getConfiguration();
146     HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
147     conf.set(TableInputFormat.INPUT_TABLE, table);
148     conf.set(TableInputFormat.SCAN, convertScanToString(scan));
149     conf.setStrings("io.serializations", conf.get("io.serializations"),
150         MutationSerialization.class.getName(), ResultSerialization.class.getName(),
151         KeyValueSerialization.class.getName());
152     if (addDependencyJars) {
153       addDependencyJars(job);
154     }
155     initCredentials(job);
156   }
157 
158   /**
159    * Use this before submitting a TableMap job. It will appropriately set up
160    * the job.
161    *
162    * @param table Binary representation of the table name to read from.
163    * @param scan  The scan instance with the columns, time range etc.
164    * @param mapper  The mapper class to use.
165    * @param outputKeyClass  The class of the output key.
166    * @param outputValueClass  The class of the output value.
167    * @param job  The current job to adjust.  Make sure the passed job is
168    * carrying all necessary HBase configuration.
169    * @param addDependencyJars upload HBase jars and jars for any of the configured
170    *           job classes via the distributed cache (tmpjars).
171    * @param inputFormatClass The class of the input format
172    * @throws IOException When setting up the details fails.
173    */
174   public static void initTableMapperJob(byte[] table, Scan scan,
175       Class<? extends TableMapper> mapper,
176       Class<?> outputKeyClass,
177       Class<?> outputValueClass, Job job,
178       boolean addDependencyJars, Class<? extends InputFormat> inputFormatClass)
179   throws IOException {
180       initTableMapperJob(Bytes.toString(table), scan, mapper, outputKeyClass,
181               outputValueClass, job, addDependencyJars, inputFormatClass);
182   }
183 
184   /**
185    * Use this before submitting a TableMap job. It will appropriately set up
186    * the job.
187    *
188    * @param table Binary representation of the table name to read from.
189    * @param scan  The scan instance with the columns, time range etc.
190    * @param mapper  The mapper class to use.
191    * @param outputKeyClass  The class of the output key.
192    * @param outputValueClass  The class of the output value.
193    * @param job  The current job to adjust.  Make sure the passed job is
194    * carrying all necessary HBase configuration.
195    * @param addDependencyJars upload HBase jars and jars for any of the configured
196    *           job classes via the distributed cache (tmpjars).
197    * @throws IOException When setting up the details fails.
198    */
199   public static void initTableMapperJob(byte[] table, Scan scan,
200       Class<? extends TableMapper> mapper,
201       Class<?> outputKeyClass,
202       Class<?> outputValueClass, Job job,
203       boolean addDependencyJars)
204   throws IOException {
205       initTableMapperJob(Bytes.toString(table), scan, mapper, outputKeyClass,
206               outputValueClass, job, addDependencyJars, TableInputFormat.class);
207   }
208 
209   /**
210    * Use this before submitting a TableMap job. It will appropriately set up
211    * the job.
212    *
213    * @param table The table name to read from.
214    * @param scan  The scan instance with the columns, time range etc.
215    * @param mapper  The mapper class to use.
216    * @param outputKeyClass  The class of the output key.
217    * @param outputValueClass  The class of the output value.
218    * @param job  The current job to adjust.  Make sure the passed job is
219    * carrying all necessary HBase configuration.
220    * @param addDependencyJars upload HBase jars and jars for any of the configured
221    *           job classes via the distributed cache (tmpjars).
222    * @throws IOException When setting up the details fails.
223    */
224   public static void initTableMapperJob(String table, Scan scan,
225       Class<? extends TableMapper> mapper,
226       Class<?> outputKeyClass,
227       Class<?> outputValueClass, Job job,
228       boolean addDependencyJars)
229   throws IOException {
230       initTableMapperJob(table, scan, mapper, outputKeyClass,
231               outputValueClass, job, addDependencyJars, TableInputFormat.class);
232   }
233 
234   /**
235    * Use this before submitting a Multi TableMap job. It will appropriately set
236    * up the job.
237    *
238    * @param scans The list of {@link Scan} objects to read from.
239    * @param mapper The mapper class to use.
240    * @param outputKeyClass The class of the output key.
241    * @param outputValueClass The class of the output value.
242    * @param job The current job to adjust. Make sure the passed job is carrying
243    *          all necessary HBase configuration.
244    * @throws IOException When setting up the details fails.
245    */
246   public static void initTableMapperJob(List<Scan> scans,
247       Class<? extends TableMapper> mapper,
248       Class<? extends WritableComparable> outputKeyClass,
249       Class<? extends Writable> outputValueClass, Job job) throws IOException {
250     initTableMapperJob(scans, mapper, outputKeyClass, outputValueClass, job,
251         true);
252   }
253 
254   /**
255    * Use this before submitting a Multi TableMap job. It will appropriately set
256    * up the job.
257    *
258    * @param scans The list of {@link Scan} objects to read from.
259    * @param mapper The mapper class to use.
260    * @param outputKeyClass The class of the output key.
261    * @param outputValueClass The class of the output value.
262    * @param job The current job to adjust. Make sure the passed job is carrying
263    *          all necessary HBase configuration.
264    * @param addDependencyJars upload HBase jars and jars for any of the
265    *          configured job classes via the distributed cache (tmpjars).
266    * @throws IOException When setting up the details fails.
267    */
268   public static void initTableMapperJob(List<Scan> scans,
269       Class<? extends TableMapper> mapper,
270       Class<? extends WritableComparable> outputKeyClass,
271       Class<? extends Writable> outputValueClass, Job job,
272       boolean addDependencyJars) throws IOException {
273     job.setInputFormatClass(MultiTableInputFormat.class);
274     if (outputValueClass != null) {
275       job.setMapOutputValueClass(outputValueClass);
276     }
277     if (outputKeyClass != null) {
278       job.setMapOutputKeyClass(outputKeyClass);
279     }
280     job.setMapperClass(mapper);
281     HBaseConfiguration.addHbaseResources(job.getConfiguration());
282     List<String> scanStrings = new ArrayList<String>();
283 
284     for (Scan scan : scans) {
285       scanStrings.add(convertScanToString(scan));
286     }
287     job.getConfiguration().setStrings(MultiTableInputFormat.SCANS,
288       scanStrings.toArray(new String[scanStrings.size()]));
289 
290     if (addDependencyJars) {
291       addDependencyJars(job);
292     }
293   }
294 
295   public static void initCredentials(Job job) throws IOException {
296     if (User.isHBaseSecurityEnabled(job.getConfiguration())) {
297       try {
298         // init credentials for remote cluster
299         String quorumAddress = job.getConfiguration().get(TableOutputFormat.QUORUM_ADDRESS);
300         if (quorumAddress != null) {
301           Configuration peerConf = HBaseConfiguration.create(job.getConfiguration());
302           ZKUtil.applyClusterKeyToConf(peerConf, quorumAddress);
303           User.getCurrent().obtainAuthTokenForJob(peerConf, job);
304         }
305         User.getCurrent().obtainAuthTokenForJob(job.getConfiguration(), job);
306       } catch (InterruptedException ie) {
307         LOG.info("Interrupted obtaining user authentication token");
308         Thread.interrupted();
309       }
310     }
311   }
312 
313   /**
314    * Writes the given scan into a Base64 encoded string.
315    *
316    * @param scan  The scan to write out.
317    * @return The scan saved in a Base64 encoded string.
318    * @throws IOException When writing the scan fails.
319    */
320   static String convertScanToString(Scan scan) throws IOException {
321     ClientProtos.Scan proto = ProtobufUtil.toScan(scan);
322     return Base64.encodeBytes(proto.toByteArray());
323   }
324 
325   /**
326    * Converts the given Base64 string back into a Scan instance.
327    *
328    * @param base64  The scan details.
329    * @return The newly created Scan instance.
330    * @throws IOException When reading the scan instance fails.
331    */
332   static Scan convertStringToScan(String base64) throws IOException {
333     byte [] decoded = Base64.decode(base64);
334     ClientProtos.Scan scan;
335     try {
336       scan = ClientProtos.Scan.parseFrom(decoded);
337     } catch (InvalidProtocolBufferException ipbe) {
338       throw new IOException(ipbe);
339     }
340 
341     return ProtobufUtil.toScan(scan);
342   }
343 
344   /**
345    * Use this before submitting a TableReduce job. It will
346    * appropriately set up the JobConf.
347    *
348    * @param table  The output table.
349    * @param reducer  The reducer class to use.
350    * @param job  The current job to adjust.
351    * @throws IOException When determining the region count fails.
352    */
353   public static void initTableReducerJob(String table,
354     Class<? extends TableReducer> reducer, Job job)
355   throws IOException {
356     initTableReducerJob(table, reducer, job, null);
357   }
358 
359   /**
360    * Use this before submitting a TableReduce job. It will
361    * appropriately set up the JobConf.
362    *
363    * @param table  The output table.
364    * @param reducer  The reducer class to use.
365    * @param job  The current job to adjust.
366    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
367    * default partitioner.
368    * @throws IOException When determining the region count fails.
369    */
370   public static void initTableReducerJob(String table,
371     Class<? extends TableReducer> reducer, Job job,
372     Class partitioner) throws IOException {
373     initTableReducerJob(table, reducer, job, partitioner, null, null, null);
374   }
375 
376   /**
377    * Use this before submitting a TableReduce job. It will
378    * appropriately set up the JobConf.
379    *
380    * @param table  The output table.
381    * @param reducer  The reducer class to use.
382    * @param job  The current job to adjust.  Make sure the passed job is
383    * carrying all necessary HBase configuration.
384    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
385    * default partitioner.
386    * @param quorumAddress Distant cluster to write to; default is null for
387    * output to the cluster that is designated in <code>hbase-site.xml</code>.
388    * Set this String to the zookeeper ensemble of an alternate remote cluster
389    * when you would have the reduce write a cluster that is other than the
390    * default; e.g. copying tables between clusters, the source would be
391    * designated by <code>hbase-site.xml</code> and this param would have the
392    * ensemble address of the remote cluster.  The format to pass is particular.
393    * Pass <code> <hbase.zookeeper.quorum>:<hbase.zookeeper.client.port>:<zookeeper.znode.parent>
394    * </code> such as <code>server,server2,server3:2181:/hbase</code>.
395    * @param serverClass redefined hbase.regionserver.class
396    * @param serverImpl redefined hbase.regionserver.impl
397    * @throws IOException When determining the region count fails.
398    */
399   public static void initTableReducerJob(String table,
400     Class<? extends TableReducer> reducer, Job job,
401     Class partitioner, String quorumAddress, String serverClass,
402     String serverImpl) throws IOException {
403     initTableReducerJob(table, reducer, job, partitioner, quorumAddress,
404         serverClass, serverImpl, true);
405   }
406 
407   /**
408    * Use this before submitting a TableReduce job. It will
409    * appropriately set up the JobConf.
410    *
411    * @param table  The output table.
412    * @param reducer  The reducer class to use.
413    * @param job  The current job to adjust.  Make sure the passed job is
414    * carrying all necessary HBase configuration.
415    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
416    * default partitioner.
417    * @param quorumAddress Distant cluster to write to; default is null for
418    * output to the cluster that is designated in <code>hbase-site.xml</code>.
419    * Set this String to the zookeeper ensemble of an alternate remote cluster
420    * when you would have the reduce write a cluster that is other than the
421    * default; e.g. copying tables between clusters, the source would be
422    * designated by <code>hbase-site.xml</code> and this param would have the
423    * ensemble address of the remote cluster.  The format to pass is particular.
424    * Pass <code> <hbase.zookeeper.quorum>:<hbase.zookeeper.client.port>:<zookeeper.znode.parent>
425    * </code> such as <code>server,server2,server3:2181:/hbase</code>.
426    * @param serverClass redefined hbase.regionserver.class
427    * @param serverImpl redefined hbase.regionserver.impl
428    * @param addDependencyJars upload HBase jars and jars for any of the configured
429    *           job classes via the distributed cache (tmpjars).
430    * @throws IOException When determining the region count fails.
431    */
432   public static void initTableReducerJob(String table,
433     Class<? extends TableReducer> reducer, Job job,
434     Class partitioner, String quorumAddress, String serverClass,
435     String serverImpl, boolean addDependencyJars) throws IOException {
436 
437     Configuration conf = job.getConfiguration();
438     HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
439     job.setOutputFormatClass(TableOutputFormat.class);
440     if (reducer != null) job.setReducerClass(reducer);
441     conf.set(TableOutputFormat.OUTPUT_TABLE, table);
442     conf.setStrings("io.serializations", conf.get("io.serializations"),
443         MutationSerialization.class.getName(), ResultSerialization.class.getName());
444     // If passed a quorum/ensemble address, pass it on to TableOutputFormat.
445     if (quorumAddress != null) {
446       // Calling this will validate the format
447       ZKUtil.transformClusterKey(quorumAddress);
448       conf.set(TableOutputFormat.QUORUM_ADDRESS,quorumAddress);
449     }
450     if (serverClass != null && serverImpl != null) {
451       conf.set(TableOutputFormat.REGION_SERVER_CLASS, serverClass);
452       conf.set(TableOutputFormat.REGION_SERVER_IMPL, serverImpl);
453     }
454     job.setOutputKeyClass(ImmutableBytesWritable.class);
455     job.setOutputValueClass(Writable.class);
456     if (partitioner == HRegionPartitioner.class) {
457       job.setPartitionerClass(HRegionPartitioner.class);
458       int regions = MetaReader.getRegionCount(conf, table);
459       if (job.getNumReduceTasks() > regions) {
460         job.setNumReduceTasks(regions);
461       }
462     } else if (partitioner != null) {
463       job.setPartitionerClass(partitioner);
464     }
465 
466     if (addDependencyJars) {
467       addDependencyJars(job);
468     }
469 
470     initCredentials(job);
471   }
472 
473   /**
474    * Ensures that the given number of reduce tasks for the given job
475    * configuration does not exceed the number of regions for the given table.
476    *
477    * @param table  The table to get the region count for.
478    * @param job  The current job to adjust.
479    * @throws IOException When retrieving the table details fails.
480    */
481   public static void limitNumReduceTasks(String table, Job job)
482   throws IOException {
483     int regions = MetaReader.getRegionCount(job.getConfiguration(), table);
484     if (job.getNumReduceTasks() > regions)
485       job.setNumReduceTasks(regions);
486   }
487 
488   /**
489    * Sets the number of reduce tasks for the given job configuration to the
490    * number of regions the given table has.
491    *
492    * @param table  The table to get the region count for.
493    * @param job  The current job to adjust.
494    * @throws IOException When retrieving the table details fails.
495    */
496   public static void setNumReduceTasks(String table, Job job)
497   throws IOException {
498     job.setNumReduceTasks(MetaReader.getRegionCount(job.getConfiguration(), table));
499   }
500 
501   /**
502    * Sets the number of rows to return and cache with each scanner iteration.
503    * Higher caching values will enable faster mapreduce jobs at the expense of
504    * requiring more heap to contain the cached rows.
505    *
506    * @param job The current job to adjust.
507    * @param batchSize The number of rows to return in batch with each scanner
508    * iteration.
509    */
510   public static void setScannerCaching(Job job, int batchSize) {
511     job.getConfiguration().setInt("hbase.client.scanner.caching", batchSize);
512   }
513 
514   /**
515    * Add the HBase dependency jars as well as jars for any of the configured
516    * job classes to the job configuration, so that JobClient will ship them
517    * to the cluster and add them to the DistributedCache.
518    */
519   public static void addDependencyJars(Job job) throws IOException {
520     try {
521       addDependencyJars(job.getConfiguration(),
522           // explicitly pull a class from each module
523           org.apache.hadoop.hbase.HConstants.class,                      // hbase-common
524           org.apache.hadoop.hbase.protobuf.generated.ClientProtos.class, // hbase-protocol
525           org.apache.hadoop.hbase.client.Put.class,                      // hbase-client
526           org.apache.hadoop.hbase.CompatibilityFactory.class,            // hbase-hadoop-compat
527           // pull necessary dependencies
528           org.apache.zookeeper.ZooKeeper.class,
529           com.google.protobuf.Message.class,
530           com.google.common.collect.Lists.class,
531           org.cloudera.htrace.Trace.class,
532           // pull job classes
533           job.getMapOutputKeyClass(),
534           job.getMapOutputValueClass(),
535           job.getInputFormatClass(),
536           job.getOutputKeyClass(),
537           job.getOutputValueClass(),
538           job.getOutputFormatClass(),
539           job.getPartitionerClass(),
540           job.getCombinerClass());
541     } catch (ClassNotFoundException e) {
542       throw new IOException(e);
543     }
544   }
545 
546   /**
547    * Add the jars containing the given classes to the job's configuration
548    * such that JobClient will ship them to the cluster and add them to
549    * the DistributedCache.
550    */
551   public static void addDependencyJars(Configuration conf,
552       Class<?>... classes) throws IOException {
553 
554     FileSystem localFs = FileSystem.getLocal(conf);
555     Set<String> jars = new HashSet<String>();
556     // Add jars that are already in the tmpjars variable
557     jars.addAll(conf.getStringCollection("tmpjars"));
558 
559     // add jars as we find them to a map of contents jar name so that we can avoid
560     // creating new jars for classes that have already been packaged.
561     Map<String, String> packagedClasses = new HashMap<String, String>();
562 
563     // Add jars containing the specified classes
564     for (Class<?> clazz : classes) {
565       if (clazz == null) continue;
566 
567       Path path = findOrCreateJar(clazz, localFs, packagedClasses);
568       if (path == null) {
569         LOG.warn("Could not find jar for class " + clazz +
570                  " in order to ship it to the cluster.");
571         continue;
572       }
573       if (!localFs.exists(path)) {
574         LOG.warn("Could not validate jar file " + path + " for class "
575                  + clazz);
576         continue;
577       }
578       jars.add(path.toString());
579     }
580     if (jars.isEmpty()) return;
581 
582     conf.set("tmpjars",
583              StringUtils.arrayToString(jars.toArray(new String[0])));
584   }
585 
586   /**
587    * If org.apache.hadoop.util.JarFinder is available (0.23+ hadoop), finds
588    * the Jar for a class or creates it if it doesn't exist. If the class is in
589    * a directory in the classpath, it creates a Jar on the fly with the
590    * contents of the directory and returns the path to that Jar. If a Jar is
591    * created, it is created in the system temporary directory. Otherwise,
592    * returns an existing jar that contains a class of the same name. Maintains
593    * a mapping from jar contents to the tmp jar created.
594    * @param my_class the class to find.
595    * @param fs the FileSystem with which to qualify the returned path.
596    * @param packagedClasses a map of class name to path.
597    * @return a jar file that contains the class.
598    * @throws IOException
599    */
600   private static Path findOrCreateJar(Class<?> my_class, FileSystem fs,
601       Map<String, String> packagedClasses)
602   throws IOException {
603     // attempt to locate an existing jar for the class.
604     String jar = findContainingJar(my_class, packagedClasses);
605     if (null == jar || jar.isEmpty()) {
606       jar = getJar(my_class);
607       updateMap(jar, packagedClasses);
608     }
609 
610     if (null == jar || jar.isEmpty()) {
611       throw new IOException("Cannot locate resource for class " + my_class.getName());
612     }
613 
614     LOG.debug(String.format("For class %s, using jar %s", my_class.getName(), jar));
615     return new Path(jar).makeQualified(fs);
616   }
617 
618   /**
619    * Add entries to <code>packagedClasses</code> corresponding to class files
620    * contained in <code>jar</code>.
621    * @param jar The jar who's content to list.
622    * @param packagedClasses map[class -> jar]
623    */
624   private static void updateMap(String jar, Map<String, String> packagedClasses) throws IOException {
625     ZipFile zip = null;
626     try {
627       zip = new ZipFile(jar);
628       for (Enumeration<? extends ZipEntry> iter = zip.entries(); iter.hasMoreElements();) {
629         ZipEntry entry = iter.nextElement();
630         if (entry.getName().endsWith("class")) {
631           packagedClasses.put(entry.getName(), jar);
632         }
633       }
634     } finally {
635       if (null != zip) zip.close();
636     }
637   }
638 
639   /**
640    * Find a jar that contains a class of the same name, if any. It will return
641    * a jar file, even if that is not the first thing on the class path that
642    * has a class with the same name. Looks first on the classpath and then in
643    * the <code>packagedClasses</code> map.
644    * @param my_class the class to find.
645    * @return a jar file that contains the class, or null.
646    * @throws IOException
647    */
648   private static String findContainingJar(Class<?> my_class, Map<String, String> packagedClasses)
649       throws IOException {
650     ClassLoader loader = my_class.getClassLoader();
651     String class_file = my_class.getName().replaceAll("\\.", "/") + ".class";
652 
653     // first search the classpath
654     for (Enumeration<URL> itr = loader.getResources(class_file); itr.hasMoreElements();) {
655       URL url = itr.nextElement();
656       if ("jar".equals(url.getProtocol())) {
657         String toReturn = url.getPath();
658         if (toReturn.startsWith("file:")) {
659           toReturn = toReturn.substring("file:".length());
660         }
661         // URLDecoder is a misnamed class, since it actually decodes
662         // x-www-form-urlencoded MIME type rather than actual
663         // URL encoding (which the file path has). Therefore it would
664         // decode +s to ' 's which is incorrect (spaces are actually
665         // either unencoded or encoded as "%20"). Replace +s first, so
666         // that they are kept sacred during the decoding process.
667         toReturn = toReturn.replaceAll("\\+", "%2B");
668         toReturn = URLDecoder.decode(toReturn, "UTF-8");
669         return toReturn.replaceAll("!.*$", "");
670       }
671     }
672 
673     // now look in any jars we've packaged using JarFinder. Returns null when
674     // no jar is found.
675     return packagedClasses.get(class_file);
676   }
677 
678   /**
679    * Invoke 'getJar' on a JarFinder implementation. Useful for some job
680    * configuration contexts (HBASE-8140) and also for testing on MRv2. First
681    * check if we have HADOOP-9426. Lacking that, fall back to the backport.
682    * @param my_class the class to find.
683    * @return a jar file that contains the class, or null.
684    */
685   private static String getJar(Class<?> my_class) {
686     String ret = null;
687     String hadoopJarFinder = "org.apache.hadoop.util.JarFinder";
688     Class<?> jarFinder = null;
689     try {
690       LOG.debug("Looking for " + hadoopJarFinder + ".");
691       jarFinder = Class.forName(hadoopJarFinder);
692       LOG.debug(hadoopJarFinder + " found.");
693       Method getJar = jarFinder.getMethod("getJar", Class.class);
694       ret = (String) getJar.invoke(null, my_class);
695     } catch (ClassNotFoundException e) {
696       LOG.debug("Using backported JarFinder.");
697       ret = JarFinder.getJar(my_class);
698     } catch (InvocationTargetException e) {
699       // function was properly called, but threw it's own exception. Unwrap it
700       // and pass it on.
701       throw new RuntimeException(e.getCause());
702     } catch (Exception e) {
703       // toss all other exceptions, related to reflection failure
704       throw new RuntimeException("getJar invocation failed.", e);
705     }
706 
707     return ret;
708   }
709 }