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