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