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