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