1 /**
2 *
3 * Licensed to the Apache Software Foundation (ASF) under one
4 * or more contributor license agreements. See the NOTICE file
5 * distributed with this work for additional information
6 * regarding copyright ownership. The ASF licenses this file
7 * to you under the Apache License, Version 2.0 (the
8 * "License"); you may not use this file except in compliance
9 * with the License. You may obtain a copy of the License at
10 *
11 * http://www.apache.org/licenses/LICENSE-2.0
12 *
13 * Unless required by applicable law or agreed to in writing, software
14 * distributed under the License is distributed on an "AS IS" BASIS,
15 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16 * See the License for the specific language governing permissions and
17 * limitations under the License.
18 */
19 package org.apache.hadoop.hbase.mapreduce;
20
21 import java.io.File;
22 import java.io.IOException;
23 import java.lang.reflect.InvocationTargetException;
24 import java.lang.reflect.Method;
25 import java.net.URL;
26 import java.net.URLDecoder;
27 import java.util.ArrayList;
28 import java.util.Enumeration;
29 import java.util.HashMap;
30 import java.util.HashSet;
31 import java.util.List;
32 import java.util.Map;
33 import java.util.Set;
34 import java.util.zip.ZipEntry;
35 import java.util.zip.ZipFile;
36
37 import org.apache.commons.logging.Log;
38 import org.apache.commons.logging.LogFactory;
39 import org.apache.hadoop.hbase.classification.InterfaceAudience;
40 import org.apache.hadoop.hbase.classification.InterfaceStability;
41 import org.apache.hadoop.conf.Configuration;
42 import org.apache.hadoop.fs.FileSystem;
43 import org.apache.hadoop.fs.Path;
44 import org.apache.hadoop.hbase.HBaseConfiguration;
45 import org.apache.hadoop.hbase.HConstants;
46 import org.apache.hadoop.hbase.catalog.MetaReader;
47 import org.apache.hadoop.hbase.client.Put;
48 import org.apache.hadoop.hbase.client.Scan;
49 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
50 import org.apache.hadoop.hbase.mapreduce.hadoopbackport.JarFinder;
51 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
52 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
53 import org.apache.hadoop.hbase.security.User;
54 import org.apache.hadoop.hbase.security.UserProvider;
55 import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
56 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
57 import org.apache.hadoop.hbase.util.Base64;
58 import org.apache.hadoop.hbase.util.Bytes;
59 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
60 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
61 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
62 import org.apache.hadoop.io.Text;
63 import org.apache.hadoop.io.Writable;
64 import org.apache.hadoop.io.WritableComparable;
65 import org.apache.hadoop.mapreduce.InputFormat;
66 import org.apache.hadoop.mapreduce.Job;
67 import org.apache.hadoop.security.token.Token;
68 import org.apache.hadoop.util.StringUtils;
69 import org.apache.zookeeper.KeeperException;
70 import org.cliffc.high_scale_lib.Counter;
71
72 import com.google.protobuf.InvalidProtocolBufferException;
73
74 /**
75 * Utility for {@link TableMapper} and {@link TableReducer}
76 */
77 @SuppressWarnings({ "rawtypes", "unchecked" })
78 @InterfaceAudience.Public
79 @InterfaceStability.Stable
80 public class TableMapReduceUtil {
81 static Log LOG = LogFactory.getLog(TableMapReduceUtil.class);
82
83 /**
84 * Use this before submitting a TableMap job. It will appropriately set up
85 * the job.
86 *
87 * @param table The table name to read from.
88 * @param scan The scan instance with the columns, time range etc.
89 * @param mapper The mapper class to use.
90 * @param outputKeyClass The class of the output key.
91 * @param outputValueClass The class of the output value.
92 * @param job The current job to adjust. Make sure the passed job is
93 * carrying all necessary HBase configuration.
94 * @throws IOException When setting up the details fails.
95 */
96 public static void initTableMapperJob(String table, Scan scan,
97 Class<? extends TableMapper> mapper,
98 Class<?> outputKeyClass,
99 Class<?> outputValueClass, Job job)
100 throws IOException {
101 initTableMapperJob(table, scan, mapper, outputKeyClass, outputValueClass,
102 job, true);
103 }
104
105 /**
106 * Use this before submitting a TableMap job. It will appropriately set up
107 * the job.
108 *
109 * @param table Binary representation of the table name to read from.
110 * @param scan The scan instance with the columns, time range etc.
111 * @param mapper The mapper class to use.
112 * @param outputKeyClass The class of the output key.
113 * @param outputValueClass The class of the output value.
114 * @param job The current job to adjust. Make sure the passed job is
115 * carrying all necessary HBase configuration.
116 * @throws IOException When setting up the details fails.
117 */
118 public static void initTableMapperJob(byte[] table, Scan scan,
119 Class<? extends TableMapper> mapper,
120 Class<?> outputKeyClass,
121 Class<?> outputValueClass, Job job)
122 throws IOException {
123 initTableMapperJob(Bytes.toString(table), scan, mapper, outputKeyClass, outputValueClass,
124 job, true);
125 }
126
127 /**
128 * Use this before submitting a TableMap job. It will appropriately set up
129 * the job.
130 *
131 * @param table The table name to read from.
132 * @param scan The scan instance with the columns, time range etc.
133 * @param mapper The mapper class to use.
134 * @param outputKeyClass The class of the output key.
135 * @param outputValueClass The class of the output value.
136 * @param job The current job to adjust. Make sure the passed job is
137 * carrying all necessary HBase configuration.
138 * @param addDependencyJars upload HBase jars and jars for any of the configured
139 * job classes via the distributed cache (tmpjars).
140 * @throws IOException When setting up the details fails.
141 */
142 public static void initTableMapperJob(String table, Scan scan,
143 Class<? extends TableMapper> mapper,
144 Class<?> outputKeyClass,
145 Class<?> outputValueClass, Job job,
146 boolean addDependencyJars, Class<? extends InputFormat> inputFormatClass)
147 throws IOException {
148 initTableMapperJob(table, scan, mapper, outputKeyClass, outputValueClass, job,
149 addDependencyJars, true, inputFormatClass);
150 }
151
152
153 /**
154 * Use this before submitting a TableMap job. It will appropriately set up
155 * the job.
156 *
157 * @param table The table name to read from.
158 * @param scan The scan instance with the columns, time range etc.
159 * @param mapper The mapper class to use.
160 * @param outputKeyClass The class of the output key.
161 * @param outputValueClass The class of the output value.
162 * @param job The current job to adjust. Make sure the passed job is
163 * carrying all necessary HBase configuration.
164 * @param addDependencyJars upload HBase jars and jars for any of the configured
165 * job classes via the distributed cache (tmpjars).
166 * @param initCredentials whether to initialize hbase auth credentials for the job
167 * @param inputFormatClass the input format
168 * @throws IOException When setting up the details fails.
169 */
170 public static void initTableMapperJob(String table, Scan scan,
171 Class<? extends TableMapper> mapper,
172 Class<?> outputKeyClass,
173 Class<?> outputValueClass, Job job,
174 boolean addDependencyJars, boolean initCredentials,
175 Class<? extends InputFormat> inputFormatClass)
176 throws IOException {
177 job.setInputFormatClass(inputFormatClass);
178 if (outputValueClass != null) job.setMapOutputValueClass(outputValueClass);
179 if (outputKeyClass != null) job.setMapOutputKeyClass(outputKeyClass);
180 job.setMapperClass(mapper);
181 if (Put.class.equals(outputValueClass)) {
182 job.setCombinerClass(PutCombiner.class);
183 }
184 Configuration conf = job.getConfiguration();
185 HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
186 conf.set(TableInputFormat.INPUT_TABLE, table);
187 conf.set(TableInputFormat.SCAN, convertScanToString(scan));
188 conf.setStrings("io.serializations", conf.get("io.serializations"),
189 MutationSerialization.class.getName(), ResultSerialization.class.getName(),
190 KeyValueSerialization.class.getName());
191 if (addDependencyJars) {
192 addDependencyJars(job);
193 }
194 if (initCredentials) {
195 initCredentials(job);
196 }
197 }
198
199 /**
200 * Use this before submitting a TableMap job. It will appropriately set up
201 * the job.
202 *
203 * @param table Binary representation of the table name to read from.
204 * @param scan The scan instance with the columns, time range etc.
205 * @param mapper The mapper class to use.
206 * @param outputKeyClass The class of the output key.
207 * @param outputValueClass The class of the output value.
208 * @param job The current job to adjust. Make sure the passed job is
209 * carrying all necessary HBase configuration.
210 * @param addDependencyJars upload HBase jars and jars for any of the configured
211 * job classes via the distributed cache (tmpjars).
212 * @param inputFormatClass The class of the input format
213 * @throws IOException When setting up the details fails.
214 */
215 public static void initTableMapperJob(byte[] table, Scan scan,
216 Class<? extends TableMapper> mapper,
217 Class<?> outputKeyClass,
218 Class<?> outputValueClass, Job job,
219 boolean addDependencyJars, Class<? extends InputFormat> inputFormatClass)
220 throws IOException {
221 initTableMapperJob(Bytes.toString(table), scan, mapper, outputKeyClass,
222 outputValueClass, job, addDependencyJars, inputFormatClass);
223 }
224
225 /**
226 * Use this before submitting a TableMap job. It will appropriately set up
227 * the job.
228 *
229 * @param table Binary representation of the table name to read from.
230 * @param scan The scan instance with the columns, time range etc.
231 * @param mapper The mapper class to use.
232 * @param outputKeyClass The class of the output key.
233 * @param outputValueClass The class of the output value.
234 * @param job The current job to adjust. Make sure the passed job is
235 * carrying all necessary HBase configuration.
236 * @param addDependencyJars upload HBase jars and jars for any of the configured
237 * job classes via the distributed cache (tmpjars).
238 * @throws IOException When setting up the details fails.
239 */
240 public static void initTableMapperJob(byte[] table, Scan scan,
241 Class<? extends TableMapper> mapper,
242 Class<?> outputKeyClass,
243 Class<?> outputValueClass, Job job,
244 boolean addDependencyJars)
245 throws IOException {
246 initTableMapperJob(Bytes.toString(table), scan, mapper, outputKeyClass,
247 outputValueClass, job, addDependencyJars, TableInputFormat.class);
248 }
249
250 /**
251 * Use this before submitting a TableMap job. It will appropriately set up
252 * the job.
253 *
254 * @param table The table name to read from.
255 * @param scan The scan instance with the columns, time range etc.
256 * @param mapper The mapper class to use.
257 * @param outputKeyClass The class of the output key.
258 * @param outputValueClass The class of the output value.
259 * @param job The current job to adjust. Make sure the passed job is
260 * carrying all necessary HBase configuration.
261 * @param addDependencyJars upload HBase jars and jars for any of the configured
262 * job classes via the distributed cache (tmpjars).
263 * @throws IOException When setting up the details fails.
264 */
265 public static void initTableMapperJob(String table, Scan scan,
266 Class<? extends TableMapper> mapper,
267 Class<?> outputKeyClass,
268 Class<?> outputValueClass, Job job,
269 boolean addDependencyJars)
270 throws IOException {
271 initTableMapperJob(table, scan, mapper, outputKeyClass,
272 outputValueClass, job, addDependencyJars, TableInputFormat.class);
273 }
274
275 /**
276 * Enable a basic on-heap cache for these jobs. Any BlockCache implementation based on
277 * direct memory will likely cause the map tasks to OOM when opening the region. This
278 * is done here instead of in TableSnapshotRegionRecordReader in case an advanced user
279 * wants to override this behavior in their job.
280 */
281 public static void resetCacheConfig(Configuration conf) {
282 conf.setFloat(
283 HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT);
284 conf.setFloat("hbase.offheapcache.percentage", 0f);
285 conf.setFloat("hbase.bucketcache.size", 0f);
286 }
287
288 /**
289 * Sets up the job for reading from a table snapshot. It bypasses hbase servers
290 * and read directly from snapshot files.
291 *
292 * @param snapshotName The name of the snapshot (of a table) to read from.
293 * @param scan The scan instance with the columns, time range etc.
294 * @param mapper The mapper class to use.
295 * @param outputKeyClass The class of the output key.
296 * @param outputValueClass The class of the output value.
297 * @param job The current job to adjust. Make sure the passed job is
298 * carrying all necessary HBase configuration.
299 * @param addDependencyJars upload HBase jars and jars for any of the configured
300 * job classes via the distributed cache (tmpjars).
301 *
302 * @param tmpRestoreDir a temporary directory to copy the snapshot files into. Current user should
303 * have write permissions to this directory, and this should not be a subdirectory of rootdir.
304 * After the job is finished, restore directory can be deleted.
305 * @throws IOException When setting up the details fails.
306 * @see TableSnapshotInputFormat
307 */
308 public static void initTableSnapshotMapperJob(String snapshotName, Scan scan,
309 Class<? extends TableMapper> mapper,
310 Class<?> outputKeyClass,
311 Class<?> outputValueClass, Job job,
312 boolean addDependencyJars, Path tmpRestoreDir)
313 throws IOException {
314 TableSnapshotInputFormat.setInput(job, snapshotName, tmpRestoreDir);
315 initTableMapperJob(snapshotName, scan, mapper, outputKeyClass,
316 outputValueClass, job, addDependencyJars, false, TableSnapshotInputFormat.class);
317 resetCacheConfig(job.getConfiguration());
318 }
319
320 /**
321 * Use this before submitting a Multi TableMap job. It will appropriately set
322 * up the job.
323 *
324 * @param scans The list of {@link Scan} objects to read from.
325 * @param mapper The mapper class to use.
326 * @param outputKeyClass The class of the output key.
327 * @param outputValueClass The class of the output value.
328 * @param job The current job to adjust. Make sure the passed job is carrying
329 * all necessary HBase configuration.
330 * @throws IOException When setting up the details fails.
331 */
332 public static void initTableMapperJob(List<Scan> scans,
333 Class<? extends TableMapper> mapper,
334 Class<? extends WritableComparable> outputKeyClass,
335 Class<? extends Writable> outputValueClass, Job job) throws IOException {
336 initTableMapperJob(scans, mapper, outputKeyClass, outputValueClass, job,
337 true);
338 }
339
340 /**
341 * Use this before submitting a Multi TableMap job. It will appropriately set
342 * up the job.
343 *
344 * @param scans The list of {@link Scan} objects to read from.
345 * @param mapper The mapper class to use.
346 * @param outputKeyClass The class of the output key.
347 * @param outputValueClass The class of the output value.
348 * @param job The current job to adjust. Make sure the passed job is carrying
349 * all necessary HBase configuration.
350 * @param addDependencyJars upload HBase jars and jars for any of the
351 * configured job classes via the distributed cache (tmpjars).
352 * @throws IOException When setting up the details fails.
353 */
354 public static void initTableMapperJob(List<Scan> scans,
355 Class<? extends TableMapper> mapper,
356 Class<? extends WritableComparable> outputKeyClass,
357 Class<? extends Writable> outputValueClass, Job job,
358 boolean addDependencyJars) throws IOException {
359 initTableMapperJob(scans, mapper, outputKeyClass, outputValueClass, job,
360 addDependencyJars, true);
361 }
362
363 /**
364 * Use this before submitting a Multi TableMap job. It will appropriately set
365 * up the job.
366 *
367 * @param scans The list of {@link Scan} objects to read from.
368 * @param mapper The mapper class to use.
369 * @param outputKeyClass The class of the output key.
370 * @param outputValueClass The class of the output value.
371 * @param job The current job to adjust. Make sure the passed job is carrying
372 * all necessary HBase configuration.
373 * @param addDependencyJars upload HBase jars and jars for any of the
374 * configured job classes via the distributed cache (tmpjars).
375 * @param initCredentials whether to initialize hbase auth credentials for the job
376 * @throws IOException When setting up the details fails.
377 */
378 public static void initTableMapperJob(List<Scan> scans,
379 Class<? extends TableMapper> mapper,
380 Class<? extends WritableComparable> outputKeyClass,
381 Class<? extends Writable> outputValueClass, Job job,
382 boolean addDependencyJars,
383 boolean initCredentials) throws IOException {
384 job.setInputFormatClass(MultiTableInputFormat.class);
385 if (outputValueClass != null) {
386 job.setMapOutputValueClass(outputValueClass);
387 }
388 if (outputKeyClass != null) {
389 job.setMapOutputKeyClass(outputKeyClass);
390 }
391 job.setMapperClass(mapper);
392 Configuration conf = job.getConfiguration();
393 HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
394 List<String> scanStrings = new ArrayList<String>();
395
396 for (Scan scan : scans) {
397 scanStrings.add(convertScanToString(scan));
398 }
399 job.getConfiguration().setStrings(MultiTableInputFormat.SCANS,
400 scanStrings.toArray(new String[scanStrings.size()]));
401
402 if (addDependencyJars) {
403 addDependencyJars(job);
404 }
405
406 if (initCredentials) {
407 initCredentials(job);
408 }
409 }
410
411 public static void initCredentials(Job job) throws IOException {
412 UserProvider userProvider = UserProvider.instantiate(job.getConfiguration());
413 if (userProvider.isHadoopSecurityEnabled()) {
414 // propagate delegation related props from launcher job to MR job
415 if (System.getenv("HADOOP_TOKEN_FILE_LOCATION") != null) {
416 job.getConfiguration().set("mapreduce.job.credentials.binary",
417 System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
418 }
419 }
420
421 if (userProvider.isHBaseSecurityEnabled()) {
422 try {
423 // init credentials for remote cluster
424 String quorumAddress = job.getConfiguration().get(TableOutputFormat.QUORUM_ADDRESS);
425 User user = userProvider.getCurrent();
426 if (quorumAddress != null) {
427 Configuration peerConf = HBaseConfiguration.create(job.getConfiguration());
428 ZKUtil.applyClusterKeyToConf(peerConf, quorumAddress);
429 obtainAuthTokenForJob(job, peerConf, user);
430 }
431
432 obtainAuthTokenForJob(job, job.getConfiguration(), user);
433 } catch (InterruptedException ie) {
434 LOG.info("Interrupted obtaining user authentication token");
435 Thread.currentThread().interrupt();
436 }
437 }
438 }
439
440 /**
441 * Obtain an authentication token, for the specified cluster, on behalf of the current user
442 * and add it to the credentials for the given map reduce job.
443 *
444 * The quorumAddress is the key to the ZK ensemble, which contains:
445 * hbase.zookeeper.quorum, hbase.zookeeper.client.port and zookeeper.znode.parent
446 *
447 * @param job The job that requires the permission.
448 * @param quorumAddress string that contains the 3 required configuratins
449 * @throws IOException When the authentication token cannot be obtained.
450 */
451 public static void initCredentialsForCluster(Job job, String quorumAddress)
452 throws IOException {
453 UserProvider userProvider = UserProvider.instantiate(job.getConfiguration());
454 if (userProvider.isHBaseSecurityEnabled()) {
455 try {
456 Configuration peerConf = HBaseConfiguration.create(job.getConfiguration());
457 ZKUtil.applyClusterKeyToConf(peerConf, quorumAddress);
458 obtainAuthTokenForJob(job, peerConf, userProvider.getCurrent());
459 } catch (InterruptedException e) {
460 LOG.info("Interrupted obtaining user authentication token");
461 Thread.interrupted();
462 }
463 }
464 }
465
466 private static void obtainAuthTokenForJob(Job job, Configuration conf, User user)
467 throws IOException, InterruptedException {
468 Token<AuthenticationTokenIdentifier> authToken = getAuthToken(conf, user);
469 if (authToken == null) {
470 user.obtainAuthTokenForJob(conf, job);
471 } else {
472 job.getCredentials().addToken(authToken.getService(), authToken);
473 }
474 }
475
476 /**
477 * Get the authentication token of the user for the cluster specified in the configuration
478 * @return null if the user does not have the token, otherwise the auth token for the cluster.
479 */
480 private static Token<AuthenticationTokenIdentifier> getAuthToken(Configuration conf, User user)
481 throws IOException, InterruptedException {
482 ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "mr-init-credentials", null);
483 try {
484 String clusterId = ZKClusterId.readClusterIdZNode(zkw);
485 return new AuthenticationTokenSelector().selectToken(new Text(clusterId), user.getUGI().getTokens());
486 } catch (KeeperException e) {
487 throw new IOException(e);
488 } finally {
489 zkw.close();
490 }
491 }
492
493 /**
494 * Writes the given scan into a Base64 encoded string.
495 *
496 * @param scan The scan to write out.
497 * @return The scan saved in a Base64 encoded string.
498 * @throws IOException When writing the scan fails.
499 */
500 static String convertScanToString(Scan scan) throws IOException {
501 ClientProtos.Scan proto = ProtobufUtil.toScan(scan);
502 return Base64.encodeBytes(proto.toByteArray());
503 }
504
505 /**
506 * Converts the given Base64 string back into a Scan instance.
507 *
508 * @param base64 The scan details.
509 * @return The newly created Scan instance.
510 * @throws IOException When reading the scan instance fails.
511 */
512 static Scan convertStringToScan(String base64) throws IOException {
513 byte [] decoded = Base64.decode(base64);
514 ClientProtos.Scan scan;
515 try {
516 scan = ClientProtos.Scan.parseFrom(decoded);
517 } catch (InvalidProtocolBufferException ipbe) {
518 throw new IOException(ipbe);
519 }
520
521 return ProtobufUtil.toScan(scan);
522 }
523
524 /**
525 * Use this before submitting a TableReduce job. It will
526 * appropriately set up the JobConf.
527 *
528 * @param table The output table.
529 * @param reducer The reducer class to use.
530 * @param job The current job to adjust.
531 * @throws IOException When determining the region count fails.
532 */
533 public static void initTableReducerJob(String table,
534 Class<? extends TableReducer> reducer, Job job)
535 throws IOException {
536 initTableReducerJob(table, reducer, job, null);
537 }
538
539 /**
540 * Use this before submitting a TableReduce job. It will
541 * appropriately set up the JobConf.
542 *
543 * @param table The output table.
544 * @param reducer The reducer class to use.
545 * @param job The current job to adjust.
546 * @param partitioner Partitioner to use. Pass <code>null</code> to use
547 * default partitioner.
548 * @throws IOException When determining the region count fails.
549 */
550 public static void initTableReducerJob(String table,
551 Class<? extends TableReducer> reducer, Job job,
552 Class partitioner) throws IOException {
553 initTableReducerJob(table, reducer, job, partitioner, null, null, null);
554 }
555
556 /**
557 * Use this before submitting a TableReduce job. It will
558 * appropriately set up the JobConf.
559 *
560 * @param table The output table.
561 * @param reducer The reducer class to use.
562 * @param job The current job to adjust. Make sure the passed job is
563 * carrying all necessary HBase configuration.
564 * @param partitioner Partitioner to use. Pass <code>null</code> to use
565 * default partitioner.
566 * @param quorumAddress Distant cluster to write to; default is null for
567 * output to the cluster that is designated in <code>hbase-site.xml</code>.
568 * Set this String to the zookeeper ensemble of an alternate remote cluster
569 * when you would have the reduce write a cluster that is other than the
570 * default; e.g. copying tables between clusters, the source would be
571 * designated by <code>hbase-site.xml</code> and this param would have the
572 * ensemble address of the remote cluster. The format to pass is particular.
573 * Pass <code> <hbase.zookeeper.quorum>:<hbase.zookeeper.client.port>:<zookeeper.znode.parent>
574 * </code> such as <code>server,server2,server3:2181:/hbase</code>.
575 * @param serverClass redefined hbase.regionserver.class
576 * @param serverImpl redefined hbase.regionserver.impl
577 * @throws IOException When determining the region count fails.
578 */
579 public static void initTableReducerJob(String table,
580 Class<? extends TableReducer> reducer, Job job,
581 Class partitioner, String quorumAddress, String serverClass,
582 String serverImpl) throws IOException {
583 initTableReducerJob(table, reducer, job, partitioner, quorumAddress,
584 serverClass, serverImpl, true);
585 }
586
587 /**
588 * Use this before submitting a TableReduce job. It will
589 * appropriately set up the JobConf.
590 *
591 * @param table The output table.
592 * @param reducer The reducer class to use.
593 * @param job The current job to adjust. Make sure the passed job is
594 * carrying all necessary HBase configuration.
595 * @param partitioner Partitioner to use. Pass <code>null</code> to use
596 * default partitioner.
597 * @param quorumAddress Distant cluster to write to; default is null for
598 * output to the cluster that is designated in <code>hbase-site.xml</code>.
599 * Set this String to the zookeeper ensemble of an alternate remote cluster
600 * when you would have the reduce write a cluster that is other than the
601 * default; e.g. copying tables between clusters, the source would be
602 * designated by <code>hbase-site.xml</code> and this param would have the
603 * ensemble address of the remote cluster. The format to pass is particular.
604 * Pass <code> <hbase.zookeeper.quorum>:<hbase.zookeeper.client.port>:<zookeeper.znode.parent>
605 * </code> such as <code>server,server2,server3:2181:/hbase</code>.
606 * @param serverClass redefined hbase.regionserver.class
607 * @param serverImpl redefined hbase.regionserver.impl
608 * @param addDependencyJars upload HBase jars and jars for any of the configured
609 * job classes via the distributed cache (tmpjars).
610 * @throws IOException When determining the region count fails.
611 */
612 public static void initTableReducerJob(String table,
613 Class<? extends TableReducer> reducer, Job job,
614 Class partitioner, String quorumAddress, String serverClass,
615 String serverImpl, boolean addDependencyJars) throws IOException {
616
617 Configuration conf = job.getConfiguration();
618 HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
619 job.setOutputFormatClass(TableOutputFormat.class);
620 if (reducer != null) job.setReducerClass(reducer);
621 conf.set(TableOutputFormat.OUTPUT_TABLE, table);
622 conf.setStrings("io.serializations", conf.get("io.serializations"),
623 MutationSerialization.class.getName(), ResultSerialization.class.getName());
624 // If passed a quorum/ensemble address, pass it on to TableOutputFormat.
625 if (quorumAddress != null) {
626 // Calling this will validate the format
627 ZKUtil.transformClusterKey(quorumAddress);
628 conf.set(TableOutputFormat.QUORUM_ADDRESS,quorumAddress);
629 }
630 if (serverClass != null && serverImpl != null) {
631 conf.set(TableOutputFormat.REGION_SERVER_CLASS, serverClass);
632 conf.set(TableOutputFormat.REGION_SERVER_IMPL, serverImpl);
633 }
634 job.setOutputKeyClass(ImmutableBytesWritable.class);
635 job.setOutputValueClass(Writable.class);
636 if (partitioner == HRegionPartitioner.class) {
637 job.setPartitionerClass(HRegionPartitioner.class);
638 int regions = MetaReader.getRegionCount(conf, table);
639 if (job.getNumReduceTasks() > regions) {
640 job.setNumReduceTasks(regions);
641 }
642 } else if (partitioner != null) {
643 job.setPartitionerClass(partitioner);
644 }
645
646 if (addDependencyJars) {
647 addDependencyJars(job);
648 }
649
650 initCredentials(job);
651 }
652
653 /**
654 * Ensures that the given number of reduce tasks for the given job
655 * configuration does not exceed the number of regions for the given table.
656 *
657 * @param table The table to get the region count for.
658 * @param job The current job to adjust.
659 * @throws IOException When retrieving the table details fails.
660 */
661 public static void limitNumReduceTasks(String table, Job job)
662 throws IOException {
663 int regions = MetaReader.getRegionCount(job.getConfiguration(), table);
664 if (job.getNumReduceTasks() > regions)
665 job.setNumReduceTasks(regions);
666 }
667
668 /**
669 * Sets the number of reduce tasks for the given job configuration to the
670 * number of regions the given table has.
671 *
672 * @param table The table to get the region count for.
673 * @param job The current job to adjust.
674 * @throws IOException When retrieving the table details fails.
675 */
676 public static void setNumReduceTasks(String table, Job job)
677 throws IOException {
678 job.setNumReduceTasks(MetaReader.getRegionCount(job.getConfiguration(), table));
679 }
680
681 /**
682 * Sets the number of rows to return and cache with each scanner iteration.
683 * Higher caching values will enable faster mapreduce jobs at the expense of
684 * requiring more heap to contain the cached rows.
685 *
686 * @param job The current job to adjust.
687 * @param batchSize The number of rows to return in batch with each scanner
688 * iteration.
689 */
690 public static void setScannerCaching(Job job, int batchSize) {
691 job.getConfiguration().setInt("hbase.client.scanner.caching", batchSize);
692 }
693
694 /**
695 * Add HBase and its dependencies (only) to the job configuration.
696 * <p>
697 * This is intended as a low-level API, facilitating code reuse between this
698 * class and its mapred counterpart. It also of use to extenral tools that
699 * need to build a MapReduce job that interacts with HBase but want
700 * fine-grained control over the jars shipped to the cluster.
701 * </p>
702 * @param conf The Configuration object to extend with dependencies.
703 * @see org.apache.hadoop.hbase.mapred.TableMapReduceUtil
704 * @see <a href="https://issues.apache.org/jira/browse/PIG-3285">PIG-3285</a>
705 */
706 public static void addHBaseDependencyJars(Configuration conf) throws IOException {
707 addDependencyJars(conf,
708 // explicitly pull a class from each module
709 org.apache.hadoop.hbase.HConstants.class, // hbase-common
710 org.apache.hadoop.hbase.protobuf.generated.ClientProtos.class, // hbase-protocol
711 org.apache.hadoop.hbase.client.Put.class, // hbase-client
712 org.apache.hadoop.hbase.CompatibilityFactory.class, // hbase-hadoop-compat
713 org.apache.hadoop.hbase.mapreduce.TableMapper.class, // hbase-server
714 // pull necessary dependencies
715 org.apache.zookeeper.ZooKeeper.class,
716 org.jboss.netty.channel.ChannelFactory.class,
717 com.google.protobuf.Message.class,
718 com.google.common.collect.Lists.class,
719 org.cloudera.htrace.Trace.class,
720 org.cliffc.high_scale_lib.Counter.class); // needed for mapred over snapshots
721 }
722
723 /**
724 * Returns a classpath string built from the content of the "tmpjars" value in {@code conf}.
725 * Also exposed to shell scripts via `bin/hbase mapredcp`.
726 */
727 public static String buildDependencyClasspath(Configuration conf) {
728 if (conf == null) {
729 throw new IllegalArgumentException("Must provide a configuration object.");
730 }
731 Set<String> paths = new HashSet<String>(conf.getStringCollection("tmpjars"));
732 if (paths.size() == 0) {
733 throw new IllegalArgumentException("Configuration contains no tmpjars.");
734 }
735 StringBuilder sb = new StringBuilder();
736 for (String s : paths) {
737 // entries can take the form 'file:/path/to/file.jar'.
738 int idx = s.indexOf(":");
739 if (idx != -1) s = s.substring(idx + 1);
740 if (sb.length() > 0) sb.append(File.pathSeparator);
741 sb.append(s);
742 }
743 return sb.toString();
744 }
745
746 /**
747 * Add the HBase dependency jars as well as jars for any of the configured
748 * job classes to the job configuration, so that JobClient will ship them
749 * to the cluster and add them to the DistributedCache.
750 */
751 public static void addDependencyJars(Job job) throws IOException {
752 addHBaseDependencyJars(job.getConfiguration());
753 try {
754 addDependencyJars(job.getConfiguration(),
755 // when making changes here, consider also mapred.TableMapReduceUtil
756 // pull job classes
757 job.getMapOutputKeyClass(),
758 job.getMapOutputValueClass(),
759 job.getInputFormatClass(),
760 job.getOutputKeyClass(),
761 job.getOutputValueClass(),
762 job.getOutputFormatClass(),
763 job.getPartitionerClass(),
764 job.getCombinerClass());
765 } catch (ClassNotFoundException e) {
766 throw new IOException(e);
767 }
768 }
769
770 /**
771 * Add the jars containing the given classes to the job's configuration
772 * such that JobClient will ship them to the cluster and add them to
773 * the DistributedCache.
774 */
775 public static void addDependencyJars(Configuration conf,
776 Class<?>... classes) throws IOException {
777
778 FileSystem localFs = FileSystem.getLocal(conf);
779 Set<String> jars = new HashSet<String>();
780 // Add jars that are already in the tmpjars variable
781 jars.addAll(conf.getStringCollection("tmpjars"));
782
783 // add jars as we find them to a map of contents jar name so that we can avoid
784 // creating new jars for classes that have already been packaged.
785 Map<String, String> packagedClasses = new HashMap<String, String>();
786
787 // Add jars containing the specified classes
788 for (Class<?> clazz : classes) {
789 if (clazz == null) continue;
790
791 Path path = findOrCreateJar(clazz, localFs, packagedClasses);
792 if (path == null) {
793 LOG.warn("Could not find jar for class " + clazz +
794 " in order to ship it to the cluster.");
795 continue;
796 }
797 if (!localFs.exists(path)) {
798 LOG.warn("Could not validate jar file " + path + " for class "
799 + clazz);
800 continue;
801 }
802 jars.add(path.toString());
803 }
804 if (jars.isEmpty()) return;
805
806 conf.set("tmpjars", StringUtils.arrayToString(jars.toArray(new String[jars.size()])));
807 }
808
809 /**
810 * If org.apache.hadoop.util.JarFinder is available (0.23+ hadoop), finds
811 * the Jar for a class or creates it if it doesn't exist. If the class is in
812 * a directory in the classpath, it creates a Jar on the fly with the
813 * contents of the directory and returns the path to that Jar. If a Jar is
814 * created, it is created in the system temporary directory. Otherwise,
815 * returns an existing jar that contains a class of the same name. Maintains
816 * a mapping from jar contents to the tmp jar created.
817 * @param my_class the class to find.
818 * @param fs the FileSystem with which to qualify the returned path.
819 * @param packagedClasses a map of class name to path.
820 * @return a jar file that contains the class.
821 * @throws IOException
822 */
823 private static Path findOrCreateJar(Class<?> my_class, FileSystem fs,
824 Map<String, String> packagedClasses)
825 throws IOException {
826 // attempt to locate an existing jar for the class.
827 String jar = findContainingJar(my_class, packagedClasses);
828 if (null == jar || jar.isEmpty()) {
829 jar = getJar(my_class);
830 updateMap(jar, packagedClasses);
831 }
832
833 if (null == jar || jar.isEmpty()) {
834 return null;
835 }
836
837 LOG.debug(String.format("For class %s, using jar %s", my_class.getName(), jar));
838 return new Path(jar).makeQualified(fs);
839 }
840
841 /**
842 * Add entries to <code>packagedClasses</code> corresponding to class files
843 * contained in <code>jar</code>.
844 * @param jar The jar who's content to list.
845 * @param packagedClasses map[class -> jar]
846 */
847 private static void updateMap(String jar, Map<String, String> packagedClasses) throws IOException {
848 if (null == jar || jar.isEmpty()) {
849 return;
850 }
851 ZipFile zip = null;
852 try {
853 zip = new ZipFile(jar);
854 for (Enumeration<? extends ZipEntry> iter = zip.entries(); iter.hasMoreElements();) {
855 ZipEntry entry = iter.nextElement();
856 if (entry.getName().endsWith("class")) {
857 packagedClasses.put(entry.getName(), jar);
858 }
859 }
860 } finally {
861 if (null != zip) zip.close();
862 }
863 }
864
865 /**
866 * Find a jar that contains a class of the same name, if any. It will return
867 * a jar file, even if that is not the first thing on the class path that
868 * has a class with the same name. Looks first on the classpath and then in
869 * the <code>packagedClasses</code> map.
870 * @param my_class the class to find.
871 * @return a jar file that contains the class, or null.
872 * @throws IOException
873 */
874 private static String findContainingJar(Class<?> my_class, Map<String, String> packagedClasses)
875 throws IOException {
876 ClassLoader loader = my_class.getClassLoader();
877 String class_file = my_class.getName().replaceAll("\\.", "/") + ".class";
878
879 // first search the classpath
880 for (Enumeration<URL> itr = loader.getResources(class_file); itr.hasMoreElements();) {
881 URL url = itr.nextElement();
882 if ("jar".equals(url.getProtocol())) {
883 String toReturn = url.getPath();
884 if (toReturn.startsWith("file:")) {
885 toReturn = toReturn.substring("file:".length());
886 }
887 // URLDecoder is a misnamed class, since it actually decodes
888 // x-www-form-urlencoded MIME type rather than actual
889 // URL encoding (which the file path has). Therefore it would
890 // decode +s to ' 's which is incorrect (spaces are actually
891 // either unencoded or encoded as "%20"). Replace +s first, so
892 // that they are kept sacred during the decoding process.
893 toReturn = toReturn.replaceAll("\\+", "%2B");
894 toReturn = URLDecoder.decode(toReturn, "UTF-8");
895 return toReturn.replaceAll("!.*$", "");
896 }
897 }
898
899 // now look in any jars we've packaged using JarFinder. Returns null when
900 // no jar is found.
901 return packagedClasses.get(class_file);
902 }
903
904 /**
905 * Invoke 'getJar' on a JarFinder implementation. Useful for some job
906 * configuration contexts (HBASE-8140) and also for testing on MRv2. First
907 * check if we have HADOOP-9426. Lacking that, fall back to the backport.
908 * @param my_class the class to find.
909 * @return a jar file that contains the class, or null.
910 */
911 private static String getJar(Class<?> my_class) {
912 String ret = null;
913 String hadoopJarFinder = "org.apache.hadoop.util.JarFinder";
914 Class<?> jarFinder = null;
915 try {
916 LOG.debug("Looking for " + hadoopJarFinder + ".");
917 jarFinder = Class.forName(hadoopJarFinder);
918 LOG.debug(hadoopJarFinder + " found.");
919 Method getJar = jarFinder.getMethod("getJar", Class.class);
920 ret = (String) getJar.invoke(null, my_class);
921 } catch (ClassNotFoundException e) {
922 LOG.debug("Using backported JarFinder.");
923 ret = JarFinder.getJar(my_class);
924 } catch (InvocationTargetException e) {
925 // function was properly called, but threw it's own exception. Unwrap it
926 // and pass it on.
927 throw new RuntimeException(e.getCause());
928 } catch (Exception e) {
929 // toss all other exceptions, related to reflection failure
930 throw new RuntimeException("getJar invocation failed.", e);
931 }
932
933 return ret;
934 }
935 }