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.File;
27  import java.io.IOException;
28  import java.lang.reflect.InvocationTargetException;
29  import java.lang.reflect.Method;
30  import java.net.URL;
31  import java.net.URLDecoder;
32  import java.util.ArrayList;
33  import java.util.Enumeration;
34  import java.util.HashMap;
35  import java.util.HashSet;
36  import java.util.List;
37  import java.util.Map;
38  import java.util.Set;
39  import java.util.zip.ZipEntry;
40  import java.util.zip.ZipFile;
41  
42  import org.apache.commons.logging.Log;
43  import org.apache.commons.logging.LogFactory;
44  import org.apache.hadoop.conf.Configuration;
45  import org.apache.hadoop.fs.FileSystem;
46  import org.apache.hadoop.fs.Path;
47  import org.apache.hadoop.hbase.HBaseConfiguration;
48  import org.apache.hadoop.hbase.HConstants;
49  import org.apache.hadoop.hbase.client.HTable;
50  import org.apache.hadoop.hbase.client.Scan;
51  import org.apache.hadoop.hbase.client.UserProvider;
52  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
53  import org.apache.hadoop.hbase.mapreduce.hadoopbackport.JarFinder;
54  import org.apache.hadoop.hbase.security.User;
55  import org.apache.hadoop.hbase.util.Base64;
56  import org.apache.hadoop.hbase.util.Bytes;
57  import org.apache.hadoop.hbase.zookeeper.ClusterId;
58  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
59  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
60  import org.apache.hadoop.io.Writable;
61  import org.apache.hadoop.io.WritableComparable;
62  import org.apache.hadoop.mapreduce.InputFormat;
63  import org.apache.hadoop.mapreduce.Job;
64  import org.apache.hadoop.util.StringUtils;
65  import org.apache.hadoop.security.token.Token;
66  import org.apache.zookeeper.KeeperException;
67  import org.cliffc.high_scale_lib.Counter;
68  
69  /**
70   * Utility for {@link TableMapper} and {@link TableReducer}
71   */
72  @SuppressWarnings("unchecked")
73  public class TableMapReduceUtil {
74    static Log LOG = LogFactory.getLog(TableMapReduceUtil.class);
75    
76    /**
77     * Use this before submitting a TableMap job. It will appropriately set up
78     * the job.
79     *
80     * @param table  The table name to read from.
81     * @param scan  The scan instance with the columns, time range etc.
82     * @param mapper  The mapper class to use.
83     * @param outputKeyClass  The class of the output key.
84     * @param outputValueClass  The class of the output value.
85     * @param job  The current job to adjust.  Make sure the passed job is
86     * carrying all necessary HBase configuration.
87     * @throws IOException When setting up the details fails.
88     */
89    public static void initTableMapperJob(String table, Scan scan,
90        Class<? extends TableMapper> mapper,
91        Class<? extends WritableComparable> outputKeyClass,
92        Class<? extends Writable> outputValueClass, Job job)
93    throws IOException {
94      initTableMapperJob(table, scan, mapper, outputKeyClass, outputValueClass,
95          job, true);
96    }
97  
98  
99    /**
100    * Use this before submitting a TableMap job. It will appropriately set up
101    * the job.
102    *
103    * @param table Binary representation of the table name to read from.
104    * @param scan  The scan instance with the columns, time range etc.
105    * @param mapper  The mapper class to use.
106    * @param outputKeyClass  The class of the output key.
107    * @param outputValueClass  The class of the output value.
108    * @param job  The current job to adjust.  Make sure the passed job is
109    * carrying all necessary HBase configuration.
110    * @throws IOException When setting up the details fails.
111    */
112    public static void initTableMapperJob(byte[] table, Scan scan,
113       Class<? extends TableMapper> mapper,
114       Class<? extends WritableComparable> outputKeyClass,
115       Class<? extends Writable> outputValueClass, Job job)
116   throws IOException {
117       initTableMapperJob(Bytes.toString(table), scan, mapper, outputKeyClass, outputValueClass,
118               job, true);
119   }
120 
121   /**
122    * Use this before submitting a TableMap job. It will appropriately set up
123    * the job.
124    *
125    * @param table  The table name to read from.
126    * @param scan  The scan instance with the columns, time range etc.
127    * @param mapper  The mapper class to use.
128    * @param outputKeyClass  The class of the output key.
129    * @param outputValueClass  The class of the output value.
130    * @param job  The current job to adjust.  Make sure the passed job is
131    * carrying all necessary HBase configuration.
132    * @param addDependencyJars upload HBase jars and jars for any of the configured
133    *           job classes via the distributed cache (tmpjars).
134    * @throws IOException When setting up the details fails.
135    */
136   public static void initTableMapperJob(String table, Scan scan,
137       Class<? extends TableMapper> mapper,
138       Class<? extends WritableComparable> outputKeyClass,
139       Class<? extends Writable> outputValueClass, Job job,
140       boolean addDependencyJars, Class<? extends InputFormat> inputFormatClass)
141   throws IOException {
142     job.setInputFormatClass(inputFormatClass);
143     if (outputValueClass != null) job.setMapOutputValueClass(outputValueClass);
144     if (outputKeyClass != null) job.setMapOutputKeyClass(outputKeyClass);
145     job.setMapperClass(mapper);
146     Configuration conf = job.getConfiguration();
147     HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
148     conf.set(TableInputFormat.INPUT_TABLE, table);
149     conf.set(TableInputFormat.SCAN, convertScanToString(scan));
150     if (addDependencyJars) {
151       addDependencyJars(job);
152     }
153     initCredentials(job);
154   }
155   
156   /**
157    * Use this before submitting a TableMap job. It will appropriately set up
158    * the job.
159    *
160    * @param table Binary representation of the table name to read from.
161    * @param scan  The scan instance with the columns, time range etc.
162    * @param mapper  The mapper class to use.
163    * @param outputKeyClass  The class of the output key.
164    * @param outputValueClass  The class of the output value.
165    * @param job  The current job to adjust.  Make sure the passed job is
166    * carrying all necessary HBase configuration.
167    * @param addDependencyJars upload HBase jars and jars for any of the configured
168    *           job classes via the distributed cache (tmpjars).
169    * @param inputFormatClass The class of the input format
170    * @throws IOException When setting up the details fails.
171    */
172   public static void initTableMapperJob(byte[] table, Scan scan,
173       Class<? extends TableMapper> mapper,
174       Class<? extends WritableComparable> outputKeyClass,
175       Class<? extends Writable> outputValueClass, Job job,
176       boolean addDependencyJars, Class<? extends InputFormat> inputFormatClass)
177   throws IOException {
178       initTableMapperJob(Bytes.toString(table), scan, mapper, outputKeyClass,
179               outputValueClass, job, addDependencyJars, inputFormatClass);
180   }
181   
182   /**
183    * Use this before submitting a TableMap job. It will appropriately set up
184    * the job.
185    *
186    * @param table Binary representation of the table name to read from.
187    * @param scan  The scan instance with the columns, time range etc.
188    * @param mapper  The mapper class to use.
189    * @param outputKeyClass  The class of the output key.
190    * @param outputValueClass  The class of the output value.
191    * @param job  The current job to adjust.  Make sure the passed job is
192    * carrying all necessary HBase configuration.
193    * @param addDependencyJars upload HBase jars and jars for any of the configured
194    *           job classes via the distributed cache (tmpjars).
195    * @throws IOException When setting up the details fails.
196    */
197   public static void initTableMapperJob(byte[] table, Scan scan,
198       Class<? extends TableMapper> mapper,
199       Class<? extends WritableComparable> outputKeyClass,
200       Class<? extends Writable> outputValueClass, Job job,
201       boolean addDependencyJars)
202   throws IOException {
203       initTableMapperJob(Bytes.toString(table), scan, mapper, outputKeyClass,
204               outputValueClass, job, addDependencyJars, TableInputFormat.class);
205   }
206   
207   /**
208    * Use this before submitting a TableMap job. It will appropriately set up
209    * the job.
210    *
211    * @param table The table name to read from.
212    * @param scan  The scan instance with the columns, time range etc.
213    * @param mapper  The mapper class to use.
214    * @param outputKeyClass  The class of the output key.
215    * @param outputValueClass  The class of the output value.
216    * @param job  The current job to adjust.  Make sure the passed job is
217    * carrying all necessary HBase configuration.
218    * @param addDependencyJars upload HBase jars and jars for any of the configured
219    *           job classes via the distributed cache (tmpjars).
220    * @throws IOException When setting up the details fails.
221    */
222   public static void initTableMapperJob(String table, Scan scan,
223       Class<? extends TableMapper> mapper,
224       Class<? extends WritableComparable> outputKeyClass,
225       Class<? extends Writable> outputValueClass, Job job,
226       boolean addDependencyJars)
227   throws IOException {
228       initTableMapperJob(table, scan, mapper, outputKeyClass,
229               outputValueClass, job, addDependencyJars, TableInputFormat.class);
230   }
231   
232   /**
233    * Use this before submitting a Multi TableMap job. It will appropriately set
234    * up the job.
235    *
236    * @param scans The list of {@link Scan} objects to read from.
237    * @param mapper The mapper class to use.
238    * @param outputKeyClass The class of the output key.
239    * @param outputValueClass The class of the output value.
240    * @param job The current job to adjust. Make sure the passed job is carrying
241    *          all necessary HBase configuration.
242    * @throws IOException When setting up the details fails.
243    */
244   public static void initTableMapperJob(List<Scan> scans,
245       Class<? extends TableMapper> mapper,
246       Class<? extends WritableComparable> outputKeyClass,
247       Class<? extends Writable> outputValueClass, Job job) throws IOException {
248     initTableMapperJob(scans, mapper, outputKeyClass, outputValueClass, job,
249         true);
250   }
251 
252   /**
253    * Use this before submitting a Multi TableMap job. It will appropriately set
254    * up the job.
255    *
256    * @param scans The list of {@link Scan} objects to read from.
257    * @param mapper The mapper class to use.
258    * @param outputKeyClass The class of the output key.
259    * @param outputValueClass The class of the output value.
260    * @param job The current job to adjust. Make sure the passed job is carrying
261    *          all necessary HBase configuration.
262    * @param addDependencyJars upload HBase jars and jars for any of the
263    *          configured job classes via the distributed cache (tmpjars).
264    * @throws IOException When setting up the details fails.
265    */
266   public static void initTableMapperJob(List<Scan> scans,
267       Class<? extends TableMapper> mapper,
268       Class<? extends WritableComparable> outputKeyClass,
269       Class<? extends Writable> outputValueClass, Job job,
270       boolean addDependencyJars) throws IOException {
271     job.setInputFormatClass(MultiTableInputFormat.class);
272     if (outputValueClass != null) {
273       job.setMapOutputValueClass(outputValueClass);
274     }
275     if (outputKeyClass != null) {
276       job.setMapOutputKeyClass(outputKeyClass);
277     }
278     job.setMapperClass(mapper);
279     HBaseConfiguration.addHbaseResources(job.getConfiguration());
280     List<String> scanStrings = new ArrayList<String>();
281 
282     for (Scan scan : scans) {
283       scanStrings.add(convertScanToString(scan));
284     }
285     job.getConfiguration().setStrings(MultiTableInputFormat.SCANS,
286       scanStrings.toArray(new String[scanStrings.size()]));
287 
288     if (addDependencyJars) {
289       addDependencyJars(job);
290     }
291   }
292 
293   /**
294    * Sets up the job for reading from a table snapshot. It bypasses hbase
295    * servers and read directly from snapshot files.
296    * 
297    * @param snapshotName
298    *          The name of the snapshot (of a table) to read from.
299    * @param scan
300    *          The scan instance with the columns, time range etc.
301    * @param mapper
302    *          The mapper class to use.
303    * @param outputKeyClass
304    *          The class of the output key.
305    * @param outputValueClass
306    *          The class of the output value.
307    * @param job
308    *          The current job to adjust. Make sure the passed job is carrying
309    *          all necessary HBase configuration.
310    * @param addDependencyJars
311    *          upload HBase jars and jars for any of the configured job classes
312    *          via the distributed cache (tmpjars).
313    * @param tableRootDir
314    *          The directory where the temp table will be created
315    * @throws IOException
316    *           When setting up the details fails.
317    * @see TableSnapshotInputFormat
318    */
319   public static void initTableSnapshotMapperJob(String snapshotName, Scan scan,
320       Class<? extends TableMapper> mapper, Class<?> outputKeyClass, Class<?> outputValueClass,
321       Job job, boolean addDependencyJars, Path tableRootDir) throws IOException {
322 
323     TableSnapshotInputFormat.setInput(job, snapshotName, tableRootDir);
324 
325     Configuration conf = job.getConfiguration();
326 
327     job.setInputFormatClass(TableSnapshotInputFormat.class);
328     if (outputValueClass != null) {
329       job.setMapOutputValueClass(outputValueClass);
330     }
331     if (outputKeyClass != null) {
332       job.setMapOutputKeyClass(outputKeyClass);
333     }
334     job.setMapperClass(mapper);
335     conf.set(TableInputFormat.SCAN, convertScanToString(scan));
336 
337     /*
338      * Enable a basic on-heap cache for these jobs. Any BlockCache implementation based on
339      * direct memory will likely cause the map tasks to OOM when opening the region. This
340      * is done here instead of in TableSnapshotRegionRecordReader in case an advanced user
341      * wants to override this behavior in their job.
342      */
343     job.getConfiguration().setFloat(
344       HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT);
345     job.getConfiguration().setFloat("hbase.offheapcache.percentage", 0f);
346 
347     if (addDependencyJars) {
348       TableMapReduceUtil.addDependencyJars(job);
349     }
350     // We would need even more libraries that hbase-server depends on
351     TableMapReduceUtil.addDependencyJars(job.getConfiguration(), Counter.class);
352   }
353 
354   public static void initCredentials(Job job) throws IOException {
355     UserProvider provider = UserProvider.instantiate(job.getConfiguration());
356 
357     if (provider.isHadoopSecurityEnabled()) {
358       // propagate delegation related props from launcher job to MR job
359       if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) {
360         job.getConfiguration().set("mapreduce.job.credentials.binary",
361                                    System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
362       }
363     }
364 
365     if (provider.isHBaseSecurityEnabled()) {
366       try {
367         // init credentials for remote cluster
368         String quorumAddress = job.getConfiguration().get(
369             TableOutputFormat.QUORUM_ADDRESS);
370         User user = provider.getCurrent();
371         if (quorumAddress != null) {
372           String[] parts = ZKUtil.transformClusterKey(quorumAddress);
373           Configuration peerConf = HBaseConfiguration.create(job
374               .getConfiguration());
375           peerConf.set(HConstants.ZOOKEEPER_QUORUM, parts[0]);
376           peerConf.set("hbase.zookeeper.client.port", parts[1]);
377           peerConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parts[2]);
378           obtainAuthTokenForJob(job, peerConf, user);
379         }
380 
381         obtainAuthTokenForJob(job, job.getConfiguration(), user);
382       } catch (InterruptedException ie) {
383         LOG.info("Interrupted obtaining user authentication token");
384         Thread.interrupted();
385       }
386     }
387   }
388   
389   /**
390    * Obtain an authentication token, for the specified cluster, on behalf of the current user
391    * and add it to the credentials for the given map reduce job.
392    *
393    * The quorumAddress is the key to the ZK ensemble, which contains:
394    * hbase.zookeeper.quorum, hbase.zookeeper.client.port and zookeeper.znode.parent
395    *
396    * @param job The job that requires the permission.
397    * @param quorumAddress string that contains the 3 required configuratins
398    * @throws IOException When the authentication token cannot be obtained.
399    */
400   public static void initCredentialsForCluster(Job job, String quorumAddress)
401       throws IOException {
402     UserProvider userProvider = UserProvider.instantiate(job.getConfiguration());
403     if (userProvider.isHBaseSecurityEnabled()) {
404       try {
405         Configuration peerConf = HBaseConfiguration.create(job.getConfiguration());
406         ZKUtil.applyClusterKeyToConf(peerConf, quorumAddress);
407         obtainAuthTokenForJob(job, peerConf, userProvider.getCurrent());
408       } catch (InterruptedException e) {
409         LOG.info("Interrupted obtaining user authentication token");
410         Thread.interrupted();
411       }
412     }
413   }
414 
415   private static void obtainAuthTokenForJob(Job job, Configuration conf, User user)
416       throws IOException, InterruptedException {
417     Token<?> authToken = getAuthToken(conf, user);
418     if (authToken == null) {
419       user.obtainAuthTokenForJob(conf, job);
420     } else {
421       job.getCredentials().addToken(authToken.getService(), authToken);
422     }
423   }
424 
425   /**
426    * Get the authentication token of the user for the cluster specified in the configuration
427    * @return null if the user does not have the token, otherwise the auth token for the cluster.
428    */
429   private static Token<?> getAuthToken(Configuration conf, User user)
430       throws IOException, InterruptedException {
431     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "mr-init-credentials", null);
432     try {
433       String clusterId = ClusterId.readClusterIdZNode(zkw);
434       return user.getToken("HBASE_AUTH_TOKEN", clusterId);
435     } catch (KeeperException e) {
436       throw new IOException(e);
437     } finally {
438       zkw.close();
439     }
440   }
441 
442   /**
443    * Writes the given scan into a Base64 encoded string.
444    *
445    * @param scan  The scan to write out.
446    * @return The scan saved in a Base64 encoded string.
447    * @throws IOException When writing the scan fails.
448    */
449   static String convertScanToString(Scan scan) throws IOException {
450     ByteArrayOutputStream out = new ByteArrayOutputStream();
451     DataOutputStream dos = new DataOutputStream(out);
452     scan.write(dos);
453     return Base64.encodeBytes(out.toByteArray());
454   }
455 
456   /**
457    * Converts the given Base64 string back into a Scan instance.
458    *
459    * @param base64  The scan details.
460    * @return The newly created Scan instance.
461    * @throws IOException When reading the scan instance fails.
462    */
463   static Scan convertStringToScan(String base64) throws IOException {
464     ByteArrayInputStream bis = new ByteArrayInputStream(Base64.decode(base64));
465     DataInputStream dis = new DataInputStream(bis);
466     Scan scan = new Scan();
467     scan.readFields(dis);
468     return scan;
469   }
470 
471   /**
472    * Use this before submitting a TableReduce job. It will
473    * appropriately set up the JobConf.
474    *
475    * @param table  The output table.
476    * @param reducer  The reducer class to use.
477    * @param job  The current job to adjust.
478    * @throws IOException When determining the region count fails.
479    */
480   public static void initTableReducerJob(String table,
481     Class<? extends TableReducer> reducer, Job job)
482   throws IOException {
483     initTableReducerJob(table, reducer, job, null);
484   }
485 
486   /**
487    * Use this before submitting a TableReduce job. It will
488    * appropriately set up the JobConf.
489    *
490    * @param table  The output table.
491    * @param reducer  The reducer class to use.
492    * @param job  The current job to adjust.
493    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
494    * default partitioner.
495    * @throws IOException When determining the region count fails.
496    */
497   public static void initTableReducerJob(String table,
498     Class<? extends TableReducer> reducer, Job job,
499     Class partitioner) throws IOException {
500     initTableReducerJob(table, reducer, job, partitioner, null, null, null);
501   }
502 
503   /**
504    * Use this before submitting a TableReduce job. It will
505    * appropriately set up the JobConf.
506    *
507    * @param table  The output table.
508    * @param reducer  The reducer class to use.
509    * @param job  The current job to adjust.  Make sure the passed job is
510    * carrying all necessary HBase configuration.
511    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
512    * default partitioner.
513    * @param quorumAddress Distant cluster to write to; default is null for
514    * output to the cluster that is designated in <code>hbase-site.xml</code>.
515    * Set this String to the zookeeper ensemble of an alternate remote cluster
516    * when you would have the reduce write a cluster that is other than the
517    * default; e.g. copying tables between clusters, the source would be
518    * designated by <code>hbase-site.xml</code> and this param would have the
519    * ensemble address of the remote cluster.  The format to pass is particular.
520    * Pass <code> &lt;hbase.zookeeper.quorum>:&lt;hbase.zookeeper.client.port>:&lt;zookeeper.znode.parent>
521    * </code> such as <code>server,server2,server3:2181:/hbase</code>.
522    * @param serverClass redefined hbase.regionserver.class
523    * @param serverImpl redefined hbase.regionserver.impl
524    * @throws IOException When determining the region count fails.
525    */
526   public static void initTableReducerJob(String table,
527     Class<? extends TableReducer> reducer, Job job,
528     Class partitioner, String quorumAddress, String serverClass,
529     String serverImpl) throws IOException {
530     initTableReducerJob(table, reducer, job, partitioner, quorumAddress,
531         serverClass, serverImpl, true);
532   }
533 
534   /**
535    * Use this before submitting a TableReduce job. It will
536    * appropriately set up the JobConf.
537    *
538    * @param table  The output table.
539    * @param reducer  The reducer class to use.
540    * @param job  The current job to adjust.  Make sure the passed job is
541    * carrying all necessary HBase configuration.
542    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
543    * default partitioner.
544    * @param quorumAddress Distant cluster to write to; default is null for
545    * output to the cluster that is designated in <code>hbase-site.xml</code>.
546    * Set this String to the zookeeper ensemble of an alternate remote cluster
547    * when you would have the reduce write a cluster that is other than the
548    * default; e.g. copying tables between clusters, the source would be
549    * designated by <code>hbase-site.xml</code> and this param would have the
550    * ensemble address of the remote cluster.  The format to pass is particular.
551    * Pass <code> &lt;hbase.zookeeper.quorum>:&lt;hbase.zookeeper.client.port>:&lt;zookeeper.znode.parent>
552    * </code> such as <code>server,server2,server3:2181:/hbase</code>.
553    * @param serverClass redefined hbase.regionserver.class
554    * @param serverImpl redefined hbase.regionserver.impl
555    * @param addDependencyJars upload HBase jars and jars for any of the configured
556    *           job classes via the distributed cache (tmpjars).
557    * @throws IOException When determining the region count fails.
558    */
559   public static void initTableReducerJob(String table,
560     Class<? extends TableReducer> reducer, Job job,
561     Class partitioner, String quorumAddress, String serverClass,
562     String serverImpl, boolean addDependencyJars) throws IOException {
563 
564     Configuration conf = job.getConfiguration();    
565     HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
566     job.setOutputFormatClass(TableOutputFormat.class);
567     if (reducer != null) job.setReducerClass(reducer);
568     conf.set(TableOutputFormat.OUTPUT_TABLE, table);
569     // If passed a quorum/ensemble address, pass it on to TableOutputFormat.
570     if (quorumAddress != null) {
571       // Calling this will validate the format
572       ZKUtil.transformClusterKey(quorumAddress);
573       conf.set(TableOutputFormat.QUORUM_ADDRESS,quorumAddress);
574     }
575     if (serverClass != null && serverImpl != null) {
576       conf.set(TableOutputFormat.REGION_SERVER_CLASS, serverClass);
577       conf.set(TableOutputFormat.REGION_SERVER_IMPL, serverImpl);
578     }
579     job.setOutputKeyClass(ImmutableBytesWritable.class);
580     job.setOutputValueClass(Writable.class);
581     if (partitioner == HRegionPartitioner.class) {
582       job.setPartitionerClass(HRegionPartitioner.class);
583       HTable outputTable = new HTable(conf, table);
584       int regions = outputTable.getRegionsInfo().size();
585       if (job.getNumReduceTasks() > regions) {
586         job.setNumReduceTasks(outputTable.getRegionsInfo().size());
587       }
588     } else if (partitioner != null) {
589       job.setPartitionerClass(partitioner);
590     }
591 
592     if (addDependencyJars) {
593       addDependencyJars(job);
594     }
595 
596     initCredentials(job);
597   }
598 
599   /**
600    * Ensures that the given number of reduce tasks for the given job
601    * configuration does not exceed the number of regions for the given table.
602    *
603    * @param table  The table to get the region count for.
604    * @param job  The current job to adjust.
605    * @throws IOException When retrieving the table details fails.
606    */
607   public static void limitNumReduceTasks(String table, Job job)
608   throws IOException {
609     HTable outputTable = new HTable(job.getConfiguration(), table);
610     int regions = outputTable.getRegionsInfo().size();
611     if (job.getNumReduceTasks() > regions)
612       job.setNumReduceTasks(regions);
613   }
614 
615   /**
616    * Sets the number of reduce tasks for the given job configuration to the
617    * number of regions the given table has.
618    *
619    * @param table  The table to get the region count for.
620    * @param job  The current job to adjust.
621    * @throws IOException When retrieving the table details fails.
622    */
623   public static void setNumReduceTasks(String table, Job job)
624   throws IOException {
625     HTable outputTable = new HTable(job.getConfiguration(), table);
626     int regions = outputTable.getRegionsInfo().size();
627     job.setNumReduceTasks(regions);
628   }
629 
630   /**
631    * Sets the number of rows to return and cache with each scanner iteration.
632    * Higher caching values will enable faster mapreduce jobs at the expense of
633    * requiring more heap to contain the cached rows.
634    *
635    * @param job The current job to adjust.
636    * @param batchSize The number of rows to return in batch with each scanner
637    * iteration.
638    */
639   public static void setScannerCaching(Job job, int batchSize) {
640     job.getConfiguration().setInt("hbase.client.scanner.caching", batchSize);
641   }
642 
643   /**
644    * Add HBase and its dependencies (only) to the job configuration.
645    * <p>
646    * This is intended as a low-level API, facilitating code reuse between this
647    * class and its mapred counterpart. It also of use to extenral tools that
648    * need to build a MapReduce job that interacts with HBase but want
649    * fine-grained control over the jars shipped to the cluster.
650    * </p>
651    * @param conf The Configuration object to extend with dependencies.
652    * @see org.apache.hadoop.hbase.mapred.TableMapReduceUtil
653    * @see <a href="https://issues.apache.org/jira/browse/PIG-3285">PIG-3285</a>
654    */
655   public static void addHBaseDependencyJars(Configuration conf) throws IOException {
656     addDependencyJars(conf,
657       org.apache.zookeeper.ZooKeeper.class,
658       com.google.protobuf.Message.class,
659       com.google.common.base.Function.class,
660       com.google.common.collect.ImmutableSet.class,
661       org.apache.hadoop.hbase.util.Bytes.class); //one class from hbase.jar
662   }
663 
664    /**
665     * Returns a classpath string built from the content of the "tmpjars" value in {@code conf}.
666     * Also exposed to shell scripts via `bin/hbase mapredcp`.
667     */
668   public static String buildDependencyClasspath(Configuration conf) {
669     if (conf == null) {
670       throw new IllegalArgumentException("Must provide a configuration object.");
671     }
672     Set<String> paths = new HashSet<String>(conf.getStringCollection("tmpjars"));
673     if (paths.size() == 0) {
674       throw new IllegalArgumentException("Configuration contains no tmpjars.");
675     }
676     StringBuilder sb = new StringBuilder();
677     for (String s : paths) {
678       // entries can take the form 'file:/path/to/file.jar'.
679       int idx = s.indexOf(":");
680       if (idx != -1) s = s.substring(idx + 1);
681       if (sb.length() > 0) sb.append(File.pathSeparator);
682       sb.append(s);
683     }
684     return sb.toString();
685   }
686 
687   /**
688    * Add the HBase dependency jars as well as jars for any of the configured
689    * job classes to the job configuration, so that JobClient will ship them
690    * to the cluster and add them to the DistributedCache.
691    */
692   public static void addDependencyJars(Job job) throws IOException {
693     addHBaseDependencyJars(job.getConfiguration());
694     try {
695       addDependencyJars(job.getConfiguration(),
696         // when making changes here, consider also mapred.TableMapReduceUtil
697         job.getMapOutputKeyClass(),
698         job.getMapOutputValueClass(),
699         job.getInputFormatClass(),
700         job.getOutputKeyClass(),
701         job.getOutputValueClass(),
702         job.getOutputFormatClass(),
703         job.getPartitionerClass(),
704         job.getCombinerClass());
705     } catch (ClassNotFoundException e) {
706       throw new IOException(e);
707     }    
708   }
709   
710   /**
711    * Add the jars containing the given classes to the job's configuration
712    * such that JobClient will ship them to the cluster and add them to
713    * the DistributedCache.
714    */
715   public static void addDependencyJars(Configuration conf,
716       Class<?>... classes) throws IOException {
717 
718     FileSystem localFs = FileSystem.getLocal(conf);
719     Set<String> jars = new HashSet<String>();
720     // Add jars that are already in the tmpjars variable
721     jars.addAll(conf.getStringCollection("tmpjars"));
722 
723     // add jars as we find them to a map of contents jar name so that we can avoid
724     // creating new jars for classes that have already been packaged.
725     Map<String, String> packagedClasses = new HashMap<String, String>();
726 
727     // Add jars containing the specified classes
728     for (Class<?> clazz : classes) {
729       if (clazz == null) continue;
730 
731       Path path = findOrCreateJar(clazz, localFs, packagedClasses);
732       if (path == null) {
733         LOG.warn("Could not find jar for class " + clazz +
734                  " in order to ship it to the cluster.");
735         continue;
736       }
737       if (!localFs.exists(path)) {
738         LOG.warn("Could not validate jar file " + path + " for class "
739                  + clazz);
740         continue;
741       }
742       jars.add(path.toString());
743     }
744     if (jars.isEmpty()) return;
745 
746     conf.set("tmpjars", StringUtils.arrayToString(jars.toArray(new String[0])));
747   }
748 
749   /**
750    * If org.apache.hadoop.util.JarFinder is available (0.23+ hadoop), finds
751    * the Jar for a class or creates it if it doesn't exist. If the class is in
752    * a directory in the classpath, it creates a Jar on the fly with the
753    * contents of the directory and returns the path to that Jar. If a Jar is
754    * created, it is created in the system temporary directory. Otherwise,
755    * returns an existing jar that contains a class of the same name. Maintains
756    * a mapping from jar contents to the tmp jar created.
757    * @param my_class the class to find.
758    * @param fs the FileSystem with which to qualify the returned path.
759    * @param packagedClasses a map of class name to path.
760    * @return a jar file that contains the class.
761    * @throws IOException
762    */
763   private static Path findOrCreateJar(Class<?> my_class, FileSystem fs,
764       Map<String, String> packagedClasses)
765   throws IOException {
766     // attempt to locate an existing jar for the class.
767     String jar = findContainingJar(my_class, packagedClasses);
768     if (null == jar || jar.isEmpty()) {
769       jar = getJar(my_class);
770       updateMap(jar, packagedClasses);
771     }
772 
773     if (null == jar || jar.isEmpty()) {
774       return null;
775     }
776 
777     LOG.debug(String.format("For class %s, using jar %s", my_class.getName(), jar));
778     return new Path(jar).makeQualified(fs);
779   }
780 
781   /**
782    * Add entries to <code>packagedClasses</code> corresponding to class files
783    * contained in <code>jar</code>.
784    * @param jar The jar who's content to list.
785    * @param packagedClasses map[class -> jar]
786    */
787   private static void updateMap(String jar, Map<String, String> packagedClasses) throws IOException {
788     if (null == jar || jar.isEmpty()) {
789       return;
790     }
791     ZipFile zip = null;
792     try {
793       zip = new ZipFile(jar);
794       for (Enumeration<? extends ZipEntry> iter = zip.entries(); iter.hasMoreElements();) {
795         ZipEntry entry = iter.nextElement();
796         if (entry.getName().endsWith("class")) {
797           packagedClasses.put(entry.getName(), jar);
798         }
799       }
800     } finally {
801       if (null != zip) zip.close();
802     }
803   }
804 
805   /**
806    * Find a jar that contains a class of the same name, if any. It will return
807    * a jar file, even if that is not the first thing on the class path that
808    * has a class with the same name. Looks first on the classpath and then in
809    * the <code>packagedClasses</code> map.
810    * @param my_class the class to find.
811    * @return a jar file that contains the class, or null.
812    * @throws IOException
813    */
814   private static String findContainingJar(Class<?> my_class, Map<String, String> packagedClasses)
815       throws IOException {
816     ClassLoader loader = my_class.getClassLoader();
817     String class_file = my_class.getName().replaceAll("\\.", "/") + ".class";
818 
819     // first search the classpath
820     for (Enumeration<URL> itr = loader.getResources(class_file); itr.hasMoreElements();) {
821       URL url = itr.nextElement();
822       if ("jar".equals(url.getProtocol())) {
823         String toReturn = url.getPath();
824         if (toReturn.startsWith("file:")) {
825           toReturn = toReturn.substring("file:".length());
826         }
827         // URLDecoder is a misnamed class, since it actually decodes
828         // x-www-form-urlencoded MIME type rather than actual
829         // URL encoding (which the file path has). Therefore it would
830         // decode +s to ' 's which is incorrect (spaces are actually
831         // either unencoded or encoded as "%20"). Replace +s first, so
832         // that they are kept sacred during the decoding process.
833         toReturn = toReturn.replaceAll("\\+", "%2B");
834         toReturn = URLDecoder.decode(toReturn, "UTF-8");
835         return toReturn.replaceAll("!.*$", "");
836       }
837     }
838 
839     // now look in any jars we've packaged using JarFinder. Returns null when
840     // no jar is found.
841     return packagedClasses.get(class_file);
842   }
843 
844   /**
845    * Invoke 'getJar' on a JarFinder implementation. Useful for some job
846    * configuration contexts (HBASE-8140) and also for testing on MRv2. First
847    * check if we have HADOOP-9426. Lacking that, fall back to the backport.
848    * @param my_class the class to find.
849    * @return a jar file that contains the class, or null.
850    */
851   private static String getJar(Class<?> my_class) {
852     String ret = null;
853     String hadoopJarFinder = "org.apache.hadoop.util.JarFinder";
854     Class<?> jarFinder = null;
855     try {
856       LOG.debug("Looking for " + hadoopJarFinder + ".");
857       jarFinder = Class.forName(hadoopJarFinder);
858       LOG.debug(hadoopJarFinder + " found.");
859       Method getJar = jarFinder.getMethod("getJar", Class.class);
860       ret = (String) getJar.invoke(null, my_class);
861     } catch (ClassNotFoundException e) {
862       LOG.debug("Using backported JarFinder.");
863       ret = JarFinder.getJar(my_class);
864     } catch (InvocationTargetException e) {
865       // function was properly called, but threw it's own exception. Unwrap it
866       // and pass it on.
867       throw new RuntimeException(e.getCause());
868     } catch (Exception e) {
869       // toss all other exceptions, related to reflection failure
870       throw new RuntimeException("getJar invocation failed.", e);
871     }
872 
873     return ret;
874   }
875 }