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