View Javadoc

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