View Javadoc

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