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  
23  import org.apache.hadoop.hbase.HBaseConfiguration;
24  import org.apache.hadoop.hbase.catalog.MetaReader;
25  import org.apache.hadoop.hbase.client.Put;
26  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
27  import org.apache.hadoop.hbase.mapreduce.MutationSerialization;
28  import org.apache.hadoop.hbase.mapreduce.ResultSerialization;
29  import org.apache.hadoop.hbase.security.User;
30  import org.apache.hadoop.mapred.FileInputFormat;
31  import org.apache.hadoop.mapred.JobConf;
32  import org.apache.hadoop.mapred.InputFormat;
33  import org.apache.hadoop.mapred.OutputFormat;
34  import org.apache.hadoop.mapred.TextInputFormat;
35  import org.apache.hadoop.mapred.TextOutputFormat;
36  import org.apache.hadoop.mapred.jobcontrol.Job;
37  
38  /**
39   * Utility for {@link TableMap} and {@link TableReduce}
40   */
41  @Deprecated
42  @SuppressWarnings("unchecked")
43  public class TableMapReduceUtil {
44  
45    /**
46     * Use this before submitting a TableMap job. It will
47     * appropriately set up the JobConf.
48     *
49     * @param table  The table name to read from.
50     * @param columns  The columns to scan.
51     * @param mapper  The mapper class to use.
52     * @param outputKeyClass  The class of the output key.
53     * @param outputValueClass  The class of the output value.
54     * @param job  The current job configuration to adjust.
55     */
56    public static void initTableMapJob(String table, String columns,
57      Class<? extends TableMap> mapper,
58      Class<?> outputKeyClass,
59      Class<?> outputValueClass, JobConf job) {
60      initTableMapJob(table, columns, mapper, outputKeyClass, outputValueClass, job, true);
61    }
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     * @param addDependencyJars upload HBase jars and jars for any of the configured
74     *           job classes via the distributed cache (tmpjars).
75     */
76    public static void initTableMapJob(String table, String columns,
77      Class<? extends TableMap> mapper,
78      Class<?> outputKeyClass,
79      Class<?> outputValueClass, JobConf job, boolean addDependencyJars) {
80  
81      job.setInputFormat(TableInputFormat.class);
82      job.setMapOutputValueClass(outputValueClass);
83      job.setMapOutputKeyClass(outputKeyClass);
84      job.setMapperClass(mapper);
85      job.setStrings("io.serializations", job.get("io.serializations"),
86          MutationSerialization.class.getName(), ResultSerialization.class.getName());
87      FileInputFormat.addInputPaths(job, table);
88      job.set(TableInputFormat.COLUMN_LIST, columns);
89      if (addDependencyJars) {
90        try {
91          addDependencyJars(job);
92        } catch (IOException e) {
93          e.printStackTrace();
94        }
95      }
96      try {
97        initCredentials(job);
98      } catch (IOException ioe) {
99        // just spit out the stack trace?  really?
100       ioe.printStackTrace();
101     }
102   }
103 
104   /**
105    * Use this before submitting a TableReduce job. It will
106    * appropriately set up the JobConf.
107    *
108    * @param table  The output table.
109    * @param reducer  The reducer class to use.
110    * @param job  The current job configuration to adjust.
111    * @throws IOException When determining the region count fails.
112    */
113   public static void initTableReduceJob(String table,
114     Class<? extends TableReduce> reducer, JobConf job)
115   throws IOException {
116     initTableReduceJob(table, reducer, job, null);
117   }
118 
119   /**
120    * Use this before submitting a TableReduce job. It will
121    * appropriately set up the JobConf.
122    *
123    * @param table  The output table.
124    * @param reducer  The reducer class to use.
125    * @param job  The current job configuration to adjust.
126    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
127    * default partitioner.
128    * @throws IOException When determining the region count fails.
129    */
130   public static void initTableReduceJob(String table,
131     Class<? extends TableReduce> reducer, JobConf job, Class partitioner)
132   throws IOException {
133     initTableReduceJob(table, reducer, job, partitioner, true);
134   }
135 
136   /**
137    * Use this before submitting a TableReduce job. It will
138    * appropriately set up the JobConf.
139    *
140    * @param table  The output table.
141    * @param reducer  The reducer class to use.
142    * @param job  The current job configuration to adjust.
143    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
144    * default partitioner.
145    * @param addDependencyJars upload HBase jars and jars for any of the configured
146    *           job classes via the distributed cache (tmpjars).
147    * @throws IOException When determining the region count fails.
148    */
149   public static void initTableReduceJob(String table,
150     Class<? extends TableReduce> reducer, JobConf job, Class partitioner,
151     boolean addDependencyJars) throws IOException {
152     job.setOutputFormat(TableOutputFormat.class);
153     job.setReducerClass(reducer);
154     job.set(TableOutputFormat.OUTPUT_TABLE, table);
155     job.setOutputKeyClass(ImmutableBytesWritable.class);
156     job.setOutputValueClass(Put.class);
157     job.setStrings("io.serializations", job.get("io.serializations"),
158         MutationSerialization.class.getName(), ResultSerialization.class.getName());
159     if (partitioner == HRegionPartitioner.class) {
160       job.setPartitionerClass(HRegionPartitioner.class);
161       int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table);
162       if (job.getNumReduceTasks() > regions) {
163         job.setNumReduceTasks(regions);
164       }
165     } else if (partitioner != null) {
166       job.setPartitionerClass(partitioner);
167     }
168     if (addDependencyJars) {
169       addDependencyJars(job);
170     }
171     initCredentials(job);
172   }
173 
174   public static void initCredentials(JobConf job) throws IOException {
175     if (User.isHBaseSecurityEnabled(job)) {
176       try {
177         User.getCurrent().obtainAuthTokenForJob(job);
178       } catch (InterruptedException ie) {
179         ie.printStackTrace();
180         Thread.interrupted();
181       }
182     }
183   }
184 
185   /**
186    * Ensures that the given number of reduce tasks for the given job
187    * configuration does not exceed the number of regions for the given table.
188    *
189    * @param table  The table to get the region count for.
190    * @param job  The current job configuration to adjust.
191    * @throws IOException When retrieving the table details fails.
192    */
193   public static void limitNumReduceTasks(String table, JobConf job)
194   throws IOException {
195     int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table);
196     if (job.getNumReduceTasks() > regions)
197       job.setNumReduceTasks(regions);
198   }
199 
200   /**
201    * Ensures that the given number of map tasks for the given job
202    * configuration does not exceed the number of regions for the given table.
203    *
204    * @param table  The table to get the region count for.
205    * @param job  The current job configuration to adjust.
206    * @throws IOException When retrieving the table details fails.
207    */
208   public static void limitNumMapTasks(String table, JobConf job)
209   throws IOException {
210     int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table);
211     if (job.getNumMapTasks() > regions)
212       job.setNumMapTasks(regions);
213   }
214 
215   /**
216    * Sets the number of reduce tasks for the given job configuration to the
217    * number of regions the given table has.
218    *
219    * @param table  The table to get the region count for.
220    * @param job  The current job configuration to adjust.
221    * @throws IOException When retrieving the table details fails.
222    */
223   public static void setNumReduceTasks(String table, JobConf job)
224   throws IOException {
225     job.setNumReduceTasks(MetaReader.getRegionCount(HBaseConfiguration.create(job), table));
226   }
227 
228   /**
229    * Sets the number of map tasks for the given job configuration to the
230    * number of regions the given table has.
231    *
232    * @param table  The table to get the region count for.
233    * @param job  The current job configuration to adjust.
234    * @throws IOException When retrieving the table details fails.
235    */
236   public static void setNumMapTasks(String table, JobConf job)
237   throws IOException {
238     job.setNumMapTasks(MetaReader.getRegionCount(HBaseConfiguration.create(job), table));
239   }
240 
241   /**
242    * Sets the number of rows to return and cache with each scanner iteration.
243    * Higher caching values will enable faster mapreduce jobs at the expense of
244    * requiring more heap to contain the cached rows.
245    *
246    * @param job The current job configuration to adjust.
247    * @param batchSize The number of rows to return in batch with each scanner
248    * iteration.
249    */
250   public static void setScannerCaching(JobConf job, int batchSize) {
251     job.setInt("hbase.client.scanner.caching", batchSize);
252   }
253 
254   /**
255    * @see org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil#addDependencyJars(Job)
256    */
257   public static void addDependencyJars(JobConf job) throws IOException {
258     org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addDependencyJars(
259       job,
260       org.apache.zookeeper.ZooKeeper.class,
261       com.google.common.base.Function.class,
262       com.google.protobuf.Message.class,
263       job.getMapOutputKeyClass(),
264       job.getMapOutputValueClass(),
265       job.getOutputKeyClass(),
266       job.getOutputValueClass(),
267       job.getPartitionerClass(),
268       job.getClass("mapred.input.format.class", TextInputFormat.class, InputFormat.class),
269       job.getClass("mapred.output.format.class", TextOutputFormat.class, OutputFormat.class),
270       job.getCombinerClass());
271   }
272 }