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.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
40
41 @Deprecated
42 @SuppressWarnings("unchecked")
43 public class TableMapReduceUtil {
44
45
46
47
48
49
50
51
52
53
54
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
65
66
67
68
69
70
71
72
73
74
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
100 ioe.printStackTrace();
101 }
102 }
103
104
105
106
107
108
109
110
111
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
121
122
123
124
125
126
127
128
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
138
139
140
141
142
143
144
145
146
147
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
187
188
189
190
191
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
202
203
204
205
206
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
217
218
219
220
221
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
230
231
232
233
234
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
243
244
245
246
247
248
249
250 public static void setScannerCaching(JobConf job, int batchSize) {
251 job.setInt("hbase.client.scanner.caching", batchSize);
252 }
253
254
255
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 }