View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.mapred;
20  
21  import java.io.IOException;
22  import java.util.Collection;
23  import java.util.Map;
24  
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.classification.InterfaceStability;
28  import org.apache.hadoop.fs.Path;
29  import org.apache.hadoop.hbase.HBaseConfiguration;
30  import org.apache.hadoop.hbase.MetaTableAccessor;
31  import org.apache.hadoop.hbase.TableName;
32  import org.apache.hadoop.hbase.client.Connection;
33  import org.apache.hadoop.hbase.client.ConnectionFactory;
34  import org.apache.hadoop.hbase.client.Put;
35  import org.apache.hadoop.hbase.client.Scan;
36  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
37  import org.apache.hadoop.hbase.mapreduce.MutationSerialization;
38  import org.apache.hadoop.hbase.mapreduce.ResultSerialization;
39  import org.apache.hadoop.hbase.security.User;
40  import org.apache.hadoop.hbase.security.UserProvider;
41  import org.apache.hadoop.hbase.security.token.TokenUtil;
42  import org.apache.hadoop.hbase.util.RegionSplitter;
43  import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
44  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
45  import org.apache.hadoop.io.Text;
46  import org.apache.hadoop.mapred.FileInputFormat;
47  import org.apache.hadoop.mapred.InputFormat;
48  import org.apache.hadoop.mapred.JobConf;
49  import org.apache.hadoop.mapred.OutputFormat;
50  import org.apache.hadoop.mapred.TextInputFormat;
51  import org.apache.hadoop.mapred.TextOutputFormat;
52  import org.apache.hadoop.security.token.Token;
53  import org.apache.zookeeper.KeeperException;
54  
55  /**
56   * Utility for {@link TableMap} and {@link TableReduce}
57   */
58  @InterfaceAudience.Public
59  @InterfaceStability.Stable
60  @SuppressWarnings({ "rawtypes", "unchecked" })
61  public class TableMapReduceUtil {
62  
63    /**
64     * Use this before submitting a TableMap job. It will
65     * appropriately set up the JobConf.
66     *
67     * @param table  The table name to read from.
68     * @param columns  The columns to scan.
69     * @param mapper  The mapper class to use.
70     * @param outputKeyClass  The class of the output key.
71     * @param outputValueClass  The class of the output value.
72     * @param job  The current job configuration to adjust.
73     */
74    public static void initTableMapJob(String table, String columns,
75      Class<? extends TableMap> mapper,
76      Class<?> outputKeyClass,
77      Class<?> outputValueClass, JobConf job) {
78      initTableMapJob(table, columns, mapper, outputKeyClass, outputValueClass, job,
79        true, TableInputFormat.class);
80    }
81  
82    public static void initTableMapJob(String table, String columns,
83      Class<? extends TableMap> mapper,
84      Class<?> outputKeyClass,
85      Class<?> outputValueClass, JobConf job, boolean addDependencyJars) {
86      initTableMapJob(table, columns, mapper, outputKeyClass, outputValueClass, job,
87        addDependencyJars, TableInputFormat.class);
88    }
89  
90    /**
91     * Use this before submitting a TableMap job. It will
92     * appropriately set up the JobConf.
93     *
94     * @param table  The table name to read from.
95     * @param columns  The columns to scan.
96     * @param mapper  The mapper class to use.
97     * @param outputKeyClass  The class of the output key.
98     * @param outputValueClass  The class of the output value.
99     * @param job  The current job configuration to adjust.
100    * @param addDependencyJars upload HBase jars and jars for any of the configured
101    *           job classes via the distributed cache (tmpjars).
102    */
103   public static void initTableMapJob(String table, String columns,
104     Class<? extends TableMap> mapper,
105     Class<?> outputKeyClass,
106     Class<?> outputValueClass, JobConf job, boolean addDependencyJars,
107     Class<? extends InputFormat> inputFormat) {
108 
109     job.setInputFormat(inputFormat);
110     job.setMapOutputValueClass(outputValueClass);
111     job.setMapOutputKeyClass(outputKeyClass);
112     job.setMapperClass(mapper);
113     job.setStrings("io.serializations", job.get("io.serializations"),
114         MutationSerialization.class.getName(), ResultSerialization.class.getName());
115     FileInputFormat.addInputPaths(job, table);
116     job.set(TableInputFormat.COLUMN_LIST, columns);
117     if (addDependencyJars) {
118       try {
119         addDependencyJars(job);
120       } catch (IOException e) {
121         e.printStackTrace();
122       }
123     }
124     try {
125       initCredentials(job);
126     } catch (IOException ioe) {
127       // just spit out the stack trace?  really?
128       ioe.printStackTrace();
129     }
130   }
131 
132   /**
133    * Sets up the job for reading from one or more multiple table snapshots, with one or more scans
134    * per snapshot.
135    * It bypasses hbase servers and read directly from snapshot files.
136    *
137    * @param snapshotScans     map of snapshot name to scans on that snapshot.
138    * @param mapper            The mapper class to use.
139    * @param outputKeyClass    The class of the output key.
140    * @param outputValueClass  The class of the output value.
141    * @param job               The current job to adjust.  Make sure the passed job is
142    *                          carrying all necessary HBase configuration.
143    * @param addDependencyJars upload HBase jars and jars for any of the configured
144    *                          job classes via the distributed cache (tmpjars).
145    */
146   public static void initMultiTableSnapshotMapperJob(Map<String, Collection<Scan>> snapshotScans,
147       Class<? extends TableMap> mapper, Class<?> outputKeyClass, Class<?> outputValueClass,
148       JobConf job, boolean addDependencyJars, Path tmpRestoreDir) throws IOException {
149     MultiTableSnapshotInputFormat.setInput(job, snapshotScans, tmpRestoreDir);
150 
151     job.setInputFormat(MultiTableSnapshotInputFormat.class);
152     if (outputValueClass != null) {
153       job.setMapOutputValueClass(outputValueClass);
154     }
155     if (outputKeyClass != null) {
156       job.setMapOutputKeyClass(outputKeyClass);
157     }
158     job.setMapperClass(mapper);
159     if (addDependencyJars) {
160       addDependencyJars(job);
161     }
162 
163     org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.resetCacheConfig(job);
164   }
165 
166   /**
167    * Sets up the job for reading from a table snapshot. It bypasses hbase servers
168    * and read directly from snapshot files.
169    *
170    * @param snapshotName The name of the snapshot (of a table) to read from.
171    * @param columns  The columns to scan.
172    * @param mapper  The mapper class to use.
173    * @param outputKeyClass  The class of the output key.
174    * @param outputValueClass  The class of the output value.
175    * @param job  The current job to adjust.  Make sure the passed job is
176    * carrying all necessary HBase configuration.
177    * @param addDependencyJars upload HBase jars and jars for any of the configured
178    *           job classes via the distributed cache (tmpjars).
179    * @param tmpRestoreDir a temporary directory to copy the snapshot files into. Current user should
180    * have write permissions to this directory, and this should not be a subdirectory of rootdir.
181    * After the job is finished, restore directory can be deleted.
182    * @throws IOException When setting up the details fails.
183    * @see TableSnapshotInputFormat
184    */
185   public static void initTableSnapshotMapJob(String snapshotName, String columns,
186       Class<? extends TableMap> mapper,
187       Class<?> outputKeyClass,
188       Class<?> outputValueClass, JobConf job,
189       boolean addDependencyJars, Path tmpRestoreDir)
190   throws IOException {
191     TableSnapshotInputFormat.setInput(job, snapshotName, tmpRestoreDir);
192     initTableMapJob(snapshotName, columns, mapper, outputKeyClass, outputValueClass, job,
193       addDependencyJars, TableSnapshotInputFormat.class);
194     org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.resetCacheConfig(job);
195   }
196 
197   /**
198    * Sets up the job for reading from a table snapshot. It bypasses hbase servers
199    * and read directly from snapshot files.
200    *
201    * @param snapshotName The name of the snapshot (of a table) to read from.
202    * @param columns  The columns to scan.
203    * @param mapper  The mapper class to use.
204    * @param outputKeyClass  The class of the output key.
205    * @param outputValueClass  The class of the output value.
206    * @param jobConf  The current job to adjust.  Make sure the passed job is
207    * carrying all necessary HBase configuration.
208    * @param addDependencyJars upload HBase jars and jars for any of the configured
209    *           job classes via the distributed cache (tmpjars).
210    * @param tmpRestoreDir a temporary directory to copy the snapshot files into. Current user should
211    * have write permissions to this directory, and this should not be a subdirectory of rootdir.
212    * After the job is finished, restore directory can be deleted.
213    * @param splitAlgo algorithm to split
214    * @param numSplitsPerRegion how many input splits to generate per one region
215    * @throws IOException When setting up the details fails.
216    * @see TableSnapshotInputFormat
217    */
218   public static void initTableSnapshotMapJob(String snapshotName, String columns,
219                                              Class<? extends TableMap> mapper,
220                                              Class<?> outputKeyClass,
221                                              Class<?> outputValueClass, JobConf jobConf,
222                                              boolean addDependencyJars, Path tmpRestoreDir,
223                                              RegionSplitter.SplitAlgorithm splitAlgo,
224                                              int numSplitsPerRegion)
225           throws IOException {
226     TableSnapshotInputFormat.setInput(jobConf, snapshotName, tmpRestoreDir, splitAlgo,
227             numSplitsPerRegion);
228     initTableMapJob(snapshotName, columns, mapper, outputKeyClass, outputValueClass, jobConf,
229             addDependencyJars, TableSnapshotInputFormat.class);
230     org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.resetCacheConfig(jobConf);
231   }
232 
233 
234   /**
235    * Use this before submitting a TableReduce job. It will
236    * appropriately set up the JobConf.
237    *
238    * @param table  The output table.
239    * @param reducer  The reducer class to use.
240    * @param job  The current job configuration to adjust.
241    * @throws IOException When determining the region count fails.
242    */
243   public static void initTableReduceJob(String table,
244     Class<? extends TableReduce> reducer, JobConf job)
245   throws IOException {
246     initTableReduceJob(table, reducer, job, null);
247   }
248 
249   /**
250    * Use this before submitting a TableReduce job. It will
251    * appropriately set up the JobConf.
252    *
253    * @param table  The output table.
254    * @param reducer  The reducer class to use.
255    * @param job  The current job configuration to adjust.
256    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
257    * default partitioner.
258    * @throws IOException When determining the region count fails.
259    */
260   public static void initTableReduceJob(String table,
261     Class<? extends TableReduce> reducer, JobConf job, Class partitioner)
262   throws IOException {
263     initTableReduceJob(table, reducer, job, partitioner, true);
264   }
265 
266   /**
267    * Use this before submitting a TableReduce job. It will
268    * appropriately set up the JobConf.
269    *
270    * @param table  The output table.
271    * @param reducer  The reducer class to use.
272    * @param job  The current job configuration to adjust.
273    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
274    * default partitioner.
275    * @param addDependencyJars upload HBase jars and jars for any of the configured
276    *           job classes via the distributed cache (tmpjars).
277    * @throws IOException When determining the region count fails.
278    */
279   public static void initTableReduceJob(String table,
280     Class<? extends TableReduce> reducer, JobConf job, Class partitioner,
281     boolean addDependencyJars) throws IOException {
282     job.setOutputFormat(TableOutputFormat.class);
283     job.setReducerClass(reducer);
284     job.set(TableOutputFormat.OUTPUT_TABLE, table);
285     job.setOutputKeyClass(ImmutableBytesWritable.class);
286     job.setOutputValueClass(Put.class);
287     job.setStrings("io.serializations", job.get("io.serializations"),
288         MutationSerialization.class.getName(), ResultSerialization.class.getName());
289     if (partitioner == HRegionPartitioner.class) {
290       job.setPartitionerClass(HRegionPartitioner.class);
291       int regions =
292         MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
293       if (job.getNumReduceTasks() > regions) {
294         job.setNumReduceTasks(regions);
295       }
296     } else if (partitioner != null) {
297       job.setPartitionerClass(partitioner);
298     }
299     if (addDependencyJars) {
300       addDependencyJars(job);
301     }
302     initCredentials(job);
303   }
304 
305   public static void initCredentials(JobConf job) throws IOException {
306     UserProvider userProvider = UserProvider.instantiate(job);
307     if (userProvider.isHadoopSecurityEnabled()) {
308       // propagate delegation related props from launcher job to MR job
309       if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) {
310         job.set("mapreduce.job.credentials.binary", System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
311       }
312     }
313 
314     if (userProvider.isHBaseSecurityEnabled()) {
315       Connection conn = ConnectionFactory.createConnection(job);
316       try {
317         // login the server principal (if using secure Hadoop)
318         User user = userProvider.getCurrent();
319         TokenUtil.addTokenForJob(conn, job, user);
320       } catch (InterruptedException ie) {
321         ie.printStackTrace();
322         Thread.currentThread().interrupt();
323       } finally {
324         conn.close();
325       }
326     }
327   }
328 
329   /**
330    * Ensures that the given number of reduce tasks for the given job
331    * configuration does not exceed the number of regions for the given table.
332    *
333    * @param table  The table to get the region count for.
334    * @param job  The current job configuration to adjust.
335    * @throws IOException When retrieving the table details fails.
336    */
337   // Used by tests.
338   public static void limitNumReduceTasks(String table, JobConf job)
339   throws IOException {
340     int regions =
341       MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
342     if (job.getNumReduceTasks() > regions)
343       job.setNumReduceTasks(regions);
344   }
345 
346   /**
347    * Ensures that the given number of map tasks for the given job
348    * configuration does not exceed the number of regions for the given table.
349    *
350    * @param table  The table to get the region count for.
351    * @param job  The current job configuration to adjust.
352    * @throws IOException When retrieving the table details fails.
353    */
354   // Used by tests.
355   public static void limitNumMapTasks(String table, JobConf job)
356   throws IOException {
357     int regions =
358       MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
359     if (job.getNumMapTasks() > regions)
360       job.setNumMapTasks(regions);
361   }
362 
363   /**
364    * Sets the number of reduce tasks for the given job configuration to the
365    * number of regions the given table has.
366    *
367    * @param table  The table to get the region count for.
368    * @param job  The current job configuration to adjust.
369    * @throws IOException When retrieving the table details fails.
370    */
371   public static void setNumReduceTasks(String table, JobConf job)
372   throws IOException {
373     job.setNumReduceTasks(MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job),
374       TableName.valueOf(table)));
375   }
376 
377   /**
378    * Sets the number of map tasks for the given job configuration to the
379    * number of regions the given table has.
380    *
381    * @param table  The table to get the region count for.
382    * @param job  The current job configuration to adjust.
383    * @throws IOException When retrieving the table details fails.
384    */
385   public static void setNumMapTasks(String table, JobConf job)
386   throws IOException {
387     job.setNumMapTasks(MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job),
388       TableName.valueOf(table)));
389   }
390 
391   /**
392    * Sets the number of rows to return and cache with each scanner iteration.
393    * Higher caching values will enable faster mapreduce jobs at the expense of
394    * requiring more heap to contain the cached rows.
395    *
396    * @param job The current job configuration to adjust.
397    * @param batchSize The number of rows to return in batch with each scanner
398    * iteration.
399    */
400   public static void setScannerCaching(JobConf job, int batchSize) {
401     job.setInt("hbase.client.scanner.caching", batchSize);
402   }
403 
404   /**
405    * @see org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil#addDependencyJars(org.apache.hadoop.mapreduce.Job)
406    */
407   public static void addDependencyJars(JobConf job) throws IOException {
408     org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addHBaseDependencyJars(job);
409     org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addDependencyJarsForClasses(
410       job,
411       // when making changes here, consider also mapreduce.TableMapReduceUtil
412       // pull job classes
413       job.getMapOutputKeyClass(),
414       job.getMapOutputValueClass(),
415       job.getOutputKeyClass(),
416       job.getOutputValueClass(),
417       job.getPartitionerClass(),
418       job.getClass("mapred.input.format.class", TextInputFormat.class, InputFormat.class),
419       job.getClass("mapred.output.format.class", TextOutputFormat.class, OutputFormat.class),
420       job.getCombinerClass());
421   }
422 }