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.conf.Configuration;
25  import org.apache.hadoop.hbase.HBaseConfiguration;
26  import org.apache.hadoop.hbase.client.HTable;
27  import org.apache.hadoop.hbase.client.Put;
28  import org.apache.hadoop.hbase.client.UserProvider;
29  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
30  import org.apache.hadoop.hbase.security.User;
31  import org.apache.hadoop.hbase.zookeeper.ClusterId;
32  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
33  import org.apache.hadoop.io.Writable;
34  import org.apache.hadoop.io.WritableComparable;
35  import org.apache.hadoop.mapred.FileInputFormat;
36  import org.apache.hadoop.mapred.JobConf;
37  import org.apache.hadoop.mapred.InputFormat;
38  import org.apache.hadoop.mapred.OutputFormat;
39  import org.apache.hadoop.mapred.TextInputFormat;
40  import org.apache.hadoop.mapred.TextOutputFormat;
41  import org.apache.hadoop.security.token.Token;
42  import org.apache.zookeeper.KeeperException;
43  
44  /**
45   * Utility for {@link TableMap} and {@link TableReduce}
46   */
47  @Deprecated
48  @SuppressWarnings("unchecked")
49  public class TableMapReduceUtil {
50  
51    /**
52     * Use this before submitting a TableMap job. It will
53     * appropriately set up the JobConf.
54     *
55     * @param table  The table name to read from.
56     * @param columns  The columns to scan.
57     * @param mapper  The mapper class to use.
58     * @param outputKeyClass  The class of the output key.
59     * @param outputValueClass  The class of the output value.
60     * @param job  The current job configuration to adjust.
61     */
62    public static void initTableMapJob(String table, String columns,
63      Class<? extends TableMap> mapper,
64      Class<? extends WritableComparable> outputKeyClass,
65      Class<? extends Writable> outputValueClass, JobConf job) {
66      initTableMapJob(table, columns, mapper, outputKeyClass, outputValueClass, job, true);
67    }
68  
69    /**
70     * Use this before submitting a TableMap job. It will
71     * appropriately set up the JobConf.
72     *
73     * @param table  The table name to read from.
74     * @param columns  The columns to scan.
75     * @param mapper  The mapper class to use.
76     * @param outputKeyClass  The class of the output key.
77     * @param outputValueClass  The class of the output value.
78     * @param job  The current job configuration to adjust.
79     * @param addDependencyJars upload HBase jars and jars for any of the configured
80     *           job classes via the distributed cache (tmpjars).
81     */
82    public static void initTableMapJob(String table, String columns,
83      Class<? extends TableMap> mapper,
84      Class<? extends WritableComparable> outputKeyClass,
85      Class<? extends Writable> outputValueClass, JobConf job, boolean addDependencyJars) {
86  
87      job.setInputFormat(TableInputFormat.class);
88      job.setMapOutputValueClass(outputValueClass);
89      job.setMapOutputKeyClass(outputKeyClass);
90      job.setMapperClass(mapper);
91      FileInputFormat.addInputPaths(job, table);
92      job.set(TableInputFormat.COLUMN_LIST, columns);
93      if (addDependencyJars) {
94        try {
95          addDependencyJars(job);
96        } catch (IOException e) {
97          e.printStackTrace();
98        }
99      }
100     try {
101       initCredentials(job);
102     } catch (IOException ioe) {
103       // just spit out the stack trace?  really?
104       ioe.printStackTrace();
105     }
106   }
107 
108   /**
109    * Use this before submitting a TableReduce job. It will
110    * appropriately set up the JobConf.
111    *
112    * @param table  The output table.
113    * @param reducer  The reducer class to use.
114    * @param job  The current job configuration to adjust.
115    * @throws IOException When determining the region count fails.
116    */
117   public static void initTableReduceJob(String table,
118     Class<? extends TableReduce> reducer, JobConf job)
119   throws IOException {
120     initTableReduceJob(table, reducer, job, null);
121   }
122 
123   /**
124    * Use this before submitting a TableReduce job. It will
125    * appropriately set up the JobConf.
126    *
127    * @param table  The output table.
128    * @param reducer  The reducer class to use.
129    * @param job  The current job configuration to adjust.
130    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
131    * default partitioner.
132    * @throws IOException When determining the region count fails.
133    */
134   public static void initTableReduceJob(String table,
135     Class<? extends TableReduce> reducer, JobConf job, Class partitioner)
136   throws IOException {
137     initTableReduceJob(table, reducer, job, partitioner, true);
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 configuration to adjust.
147    * @param partitioner  Partitioner to use. Pass <code>null</code> to use
148    * default partitioner.
149    * @param addDependencyJars upload HBase jars and jars for any of the configured
150    *           job classes via the distributed cache (tmpjars).
151    * @throws IOException When determining the region count fails.
152    */
153   public static void initTableReduceJob(String table,
154     Class<? extends TableReduce> reducer, JobConf job, Class partitioner,
155     boolean addDependencyJars) throws IOException {
156     job.setOutputFormat(TableOutputFormat.class);
157     job.setReducerClass(reducer);
158     job.set(TableOutputFormat.OUTPUT_TABLE, table);
159     job.setOutputKeyClass(ImmutableBytesWritable.class);
160     job.setOutputValueClass(Put.class);
161     if (partitioner == HRegionPartitioner.class) {
162       job.setPartitionerClass(HRegionPartitioner.class);
163       HTable outputTable = new HTable(HBaseConfiguration.create(job), table);
164       int regions = outputTable.getRegionsInfo().size();
165       if (job.getNumReduceTasks() > regions) {
166         job.setNumReduceTasks(outputTable.getRegionsInfo().size());
167       }
168     } else if (partitioner != null) {
169       job.setPartitionerClass(partitioner);
170     }
171     if (addDependencyJars) {
172       addDependencyJars(job);
173     }
174     initCredentials(job);
175   }
176 
177   public static void initCredentials(JobConf job) throws IOException {
178     UserProvider provider = UserProvider.instantiate(job);
179 
180     if (provider.isHadoopSecurityEnabled()) {
181       // propagate delegation related props from launcher job to MR job
182       if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) {
183         job.set("mapreduce.job.credentials.binary",
184                 System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
185       }
186     }
187 
188     if (provider.isHBaseSecurityEnabled()) {
189       try {
190         User user = provider.getCurrent();
191         Token<?> authToken = getAuthToken(job, user);
192         if (authToken == null) {
193           user.obtainAuthTokenForJob(job);
194         } else {
195           job.getCredentials().addToken(authToken.getService(), authToken);
196         }
197       } catch (InterruptedException ie) {
198         ie.printStackTrace();
199         Thread.interrupted();
200       }
201     }
202   }
203 
204   /**
205    * Get the authentication token of the user for the cluster specified in the configuration
206    * @return null if the user does not have the token, otherwise the auth token for the cluster.
207    */
208   private static Token<?> getAuthToken(Configuration conf, User user)
209       throws IOException, InterruptedException {
210     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "mr-init-credentials", null);
211     try {
212       String clusterId = ClusterId.readClusterIdZNode(zkw);
213       return user.getToken("HBASE_AUTH_TOKEN", clusterId);
214     } catch (KeeperException e) {
215       throw new IOException(e);
216     } finally {
217       zkw.close();
218     }
219   }
220 
221   /**
222    * Ensures that the given number of reduce tasks for the given job
223    * configuration does not exceed the number of regions for the given table.
224    *
225    * @param table  The table to get the region count for.
226    * @param job  The current job configuration to adjust.
227    * @throws IOException When retrieving the table details fails.
228    */
229   public static void limitNumReduceTasks(String table, JobConf job)
230   throws IOException {
231     HTable outputTable = new HTable(HBaseConfiguration.create(job), table);
232     int regions = outputTable.getRegionsInfo().size();
233     if (job.getNumReduceTasks() > regions)
234       job.setNumReduceTasks(regions);
235   }
236 
237   /**
238    * Ensures that the given number of map tasks for the given job
239    * configuration does not exceed the number of regions for the given table.
240    *
241    * @param table  The table to get the region count for.
242    * @param job  The current job configuration to adjust.
243    * @throws IOException When retrieving the table details fails.
244    */
245   public static void limitNumMapTasks(String table, JobConf job)
246   throws IOException {
247     HTable outputTable = new HTable(HBaseConfiguration.create(job), table);
248     int regions = outputTable.getRegionsInfo().size();
249     if (job.getNumMapTasks() > regions)
250       job.setNumMapTasks(regions);
251   }
252 
253   /**
254    * Sets the number of reduce tasks for the given job configuration to the
255    * number of regions the given table has.
256    *
257    * @param table  The table to get the region count for.
258    * @param job  The current job configuration to adjust.
259    * @throws IOException When retrieving the table details fails.
260    */
261   public static void setNumReduceTasks(String table, JobConf job)
262   throws IOException {
263     HTable outputTable = new HTable(HBaseConfiguration.create(job), table);
264     int regions = outputTable.getRegionsInfo().size();
265     job.setNumReduceTasks(regions);
266   }
267 
268   /**
269    * Sets the number of map tasks for the given job configuration to the
270    * number of regions the given table has.
271    *
272    * @param table  The table to get the region count for.
273    * @param job  The current job configuration to adjust.
274    * @throws IOException When retrieving the table details fails.
275    */
276   public static void setNumMapTasks(String table, JobConf job)
277   throws IOException {
278     HTable outputTable = new HTable(HBaseConfiguration.create(job), table);
279     int regions = outputTable.getRegionsInfo().size();
280     job.setNumMapTasks(regions);
281   }
282 
283   /**
284    * Sets the number of rows to return and cache with each scanner iteration.
285    * Higher caching values will enable faster mapreduce jobs at the expense of
286    * requiring more heap to contain the cached rows.
287    *
288    * @param job The current job configuration to adjust.
289    * @param batchSize The number of rows to return in batch with each scanner
290    * iteration.
291    */
292   public static void setScannerCaching(JobConf job, int batchSize) {
293     job.setInt("hbase.client.scanner.caching", batchSize);
294   }
295 
296   /**
297    * @see org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil#addDependencyJars(org.apache.hadoop.mapreduce.Job)
298    */
299   public static void addDependencyJars(JobConf job) throws IOException {
300     org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addHBaseDependencyJars(job);
301     org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addDependencyJars(
302       job,
303       // when making changes here, consider also mapreduce.TableMapReduceUtil
304       job.getMapOutputKeyClass(),
305       job.getMapOutputValueClass(),
306       job.getOutputKeyClass(),
307       job.getOutputValueClass(),
308       job.getPartitionerClass(),
309       job.getClass("mapred.input.format.class", TextInputFormat.class, InputFormat.class),
310       job.getClass("mapred.output.format.class", TextOutputFormat.class, OutputFormat.class),
311       job.getCombinerClass());
312   }
313 }