1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapred;
20
21 import java.io.IOException;
22
23 import org.apache.hadoop.conf.Configuration;
24 import org.apache.hadoop.classification.InterfaceAudience;
25 import org.apache.hadoop.classification.InterfaceStability;
26 import org.apache.hadoop.hbase.HBaseConfiguration;
27 import org.apache.hadoop.hbase.catalog.MetaReader;
28 import org.apache.hadoop.hbase.client.Put;
29 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
30 import org.apache.hadoop.hbase.mapreduce.MutationSerialization;
31 import org.apache.hadoop.hbase.mapreduce.ResultSerialization;
32 import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
33 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
34 import org.apache.hadoop.hbase.security.User;
35 import org.apache.hadoop.hbase.security.UserProvider;
36 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
37 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
38 import org.apache.hadoop.io.Text;
39 import org.apache.hadoop.mapred.FileInputFormat;
40 import org.apache.hadoop.mapred.InputFormat;
41 import org.apache.hadoop.mapred.JobConf;
42 import org.apache.hadoop.mapred.OutputFormat;
43 import org.apache.hadoop.mapred.TextInputFormat;
44 import org.apache.hadoop.mapred.TextOutputFormat;
45 import org.apache.hadoop.security.token.Token;
46 import org.apache.zookeeper.KeeperException;
47
48
49
50
51 @Deprecated
52 @InterfaceAudience.Public
53 @InterfaceStability.Stable
54 @SuppressWarnings({ "rawtypes", "unchecked" })
55 public class TableMapReduceUtil {
56
57
58
59
60
61
62
63
64
65
66
67
68 public static void initTableMapJob(String table, String columns,
69 Class<? extends TableMap> mapper,
70 Class<?> outputKeyClass,
71 Class<?> outputValueClass, JobConf job) {
72 initTableMapJob(table, columns, mapper, outputKeyClass, outputValueClass, job, true);
73 }
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88 public static void initTableMapJob(String table, String columns,
89 Class<? extends TableMap> mapper,
90 Class<?> outputKeyClass,
91 Class<?> outputValueClass, JobConf job, boolean addDependencyJars) {
92
93 job.setInputFormat(TableInputFormat.class);
94 job.setMapOutputValueClass(outputValueClass);
95 job.setMapOutputKeyClass(outputKeyClass);
96 job.setMapperClass(mapper);
97 job.setStrings("io.serializations", job.get("io.serializations"),
98 MutationSerialization.class.getName(), ResultSerialization.class.getName());
99 FileInputFormat.addInputPaths(job, table);
100 job.set(TableInputFormat.COLUMN_LIST, columns);
101 if (addDependencyJars) {
102 try {
103 addDependencyJars(job);
104 } catch (IOException e) {
105 e.printStackTrace();
106 }
107 }
108 try {
109 initCredentials(job);
110 } catch (IOException ioe) {
111
112 ioe.printStackTrace();
113 }
114 }
115
116
117
118
119
120
121
122
123
124
125 public static void initTableReduceJob(String table,
126 Class<? extends TableReduce> reducer, JobConf job)
127 throws IOException {
128 initTableReduceJob(table, reducer, job, null);
129 }
130
131
132
133
134
135
136
137
138
139
140
141
142 public static void initTableReduceJob(String table,
143 Class<? extends TableReduce> reducer, JobConf job, Class partitioner)
144 throws IOException {
145 initTableReduceJob(table, reducer, job, partitioner, true);
146 }
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161 public static void initTableReduceJob(String table,
162 Class<? extends TableReduce> reducer, JobConf job, Class partitioner,
163 boolean addDependencyJars) throws IOException {
164 job.setOutputFormat(TableOutputFormat.class);
165 job.setReducerClass(reducer);
166 job.set(TableOutputFormat.OUTPUT_TABLE, table);
167 job.setOutputKeyClass(ImmutableBytesWritable.class);
168 job.setOutputValueClass(Put.class);
169 job.setStrings("io.serializations", job.get("io.serializations"),
170 MutationSerialization.class.getName(), ResultSerialization.class.getName());
171 if (partitioner == HRegionPartitioner.class) {
172 job.setPartitionerClass(HRegionPartitioner.class);
173 int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table);
174 if (job.getNumReduceTasks() > regions) {
175 job.setNumReduceTasks(regions);
176 }
177 } else if (partitioner != null) {
178 job.setPartitionerClass(partitioner);
179 }
180 if (addDependencyJars) {
181 addDependencyJars(job);
182 }
183 initCredentials(job);
184 }
185
186 public static void initCredentials(JobConf job) throws IOException {
187 UserProvider userProvider = UserProvider.instantiate(job);
188 if (userProvider.isHadoopSecurityEnabled()) {
189
190 if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) {
191 job.set("mapreduce.job.credentials.binary", System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
192 }
193 }
194
195 if (userProvider.isHBaseSecurityEnabled()) {
196 try {
197
198 User user = userProvider.getCurrent();
199 Token<AuthenticationTokenIdentifier> authToken = getAuthToken(job, user);
200 if (authToken == null) {
201 user.obtainAuthTokenForJob(job);
202 } else {
203 job.getCredentials().addToken(authToken.getService(), authToken);
204 }
205 } catch (InterruptedException ie) {
206 ie.printStackTrace();
207 Thread.interrupted();
208 }
209 }
210 }
211
212
213
214
215
216 private static Token<AuthenticationTokenIdentifier> getAuthToken(Configuration conf, User user)
217 throws IOException, InterruptedException {
218 ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "mr-init-credentials", null);
219 try {
220 String clusterId = ZKClusterId.readClusterIdZNode(zkw);
221 return new AuthenticationTokenSelector().selectToken(new Text(clusterId), user.getUGI().getTokens());
222 } catch (KeeperException e) {
223 throw new IOException(e);
224 } finally {
225 zkw.close();
226 }
227 }
228
229
230
231
232
233
234
235
236
237 public static void limitNumReduceTasks(String table, JobConf job)
238 throws IOException {
239 int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table);
240 if (job.getNumReduceTasks() > regions)
241 job.setNumReduceTasks(regions);
242 }
243
244
245
246
247
248
249
250
251
252 public static void limitNumMapTasks(String table, JobConf job)
253 throws IOException {
254 int regions = MetaReader.getRegionCount(HBaseConfiguration.create(job), table);
255 if (job.getNumMapTasks() > regions)
256 job.setNumMapTasks(regions);
257 }
258
259
260
261
262
263
264
265
266
267 public static void setNumReduceTasks(String table, JobConf job)
268 throws IOException {
269 job.setNumReduceTasks(MetaReader.getRegionCount(HBaseConfiguration.create(job), table));
270 }
271
272
273
274
275
276
277
278
279
280 public static void setNumMapTasks(String table, JobConf job)
281 throws IOException {
282 job.setNumMapTasks(MetaReader.getRegionCount(HBaseConfiguration.create(job), table));
283 }
284
285
286
287
288
289
290
291
292
293
294 public static void setScannerCaching(JobConf job, int batchSize) {
295 job.setInt("hbase.client.scanner.caching", batchSize);
296 }
297
298
299
300
301 public static void addDependencyJars(JobConf job) throws IOException {
302 org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addHBaseDependencyJars(job);
303 org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil.addDependencyJars(
304 job,
305
306
307 job.getMapOutputKeyClass(),
308 job.getMapOutputValueClass(),
309 job.getOutputKeyClass(),
310 job.getOutputValueClass(),
311 job.getPartitionerClass(),
312 job.getClass("mapred.input.format.class", TextInputFormat.class, InputFormat.class),
313 job.getClass("mapred.output.format.class", TextOutputFormat.class, OutputFormat.class),
314 job.getCombinerClass());
315 }
316 }