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