001 /** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018 019 package org.apache.hadoop.mapred; 020 021 022 import java.io.IOException; 023 import java.net.URL; 024 import java.net.URLDecoder; 025 import java.util.Enumeration; 026 import java.util.regex.Pattern; 027 028 import org.apache.commons.logging.Log; 029 import org.apache.commons.logging.LogFactory; 030 import org.apache.hadoop.classification.InterfaceAudience; 031 import org.apache.hadoop.classification.InterfaceAudience.Private; 032 import org.apache.hadoop.classification.InterfaceStability; 033 import org.apache.hadoop.conf.Configuration; 034 import org.apache.hadoop.fs.FileStatus; 035 import org.apache.hadoop.fs.FileSystem; 036 import org.apache.hadoop.fs.Path; 037 import org.apache.hadoop.io.LongWritable; 038 import org.apache.hadoop.io.RawComparator; 039 import org.apache.hadoop.io.Text; 040 import org.apache.hadoop.io.WritableComparable; 041 import org.apache.hadoop.io.WritableComparator; 042 import org.apache.hadoop.io.compress.CompressionCodec; 043 import org.apache.hadoop.mapred.lib.HashPartitioner; 044 import org.apache.hadoop.mapred.lib.IdentityMapper; 045 import org.apache.hadoop.mapred.lib.IdentityReducer; 046 import org.apache.hadoop.mapred.lib.KeyFieldBasedComparator; 047 import org.apache.hadoop.mapred.lib.KeyFieldBasedPartitioner; 048 import org.apache.hadoop.mapreduce.MRConfig; 049 import org.apache.hadoop.mapreduce.MRJobConfig; 050 import org.apache.hadoop.mapreduce.filecache.DistributedCache; 051 import org.apache.hadoop.mapreduce.util.ConfigUtil; 052 import org.apache.hadoop.security.Credentials; 053 import org.apache.hadoop.util.ClassUtil; 054 import org.apache.hadoop.util.ReflectionUtils; 055 import org.apache.hadoop.util.Tool; 056 import org.apache.log4j.Level; 057 058 /** 059 * A map/reduce job configuration. 060 * 061 * <p><code>JobConf</code> is the primary interface for a user to describe a 062 * map-reduce job to the Hadoop framework for execution. The framework tries to 063 * faithfully execute the job as-is described by <code>JobConf</code>, however: 064 * <ol> 065 * <li> 066 * Some configuration parameters might have been marked as 067 * <a href="{@docRoot}/org/apache/hadoop/conf/Configuration.html#FinalParams"> 068 * final</a> by administrators and hence cannot be altered. 069 * </li> 070 * <li> 071 * While some job parameters are straight-forward to set 072 * (e.g. {@link #setNumReduceTasks(int)}), some parameters interact subtly 073 * rest of the framework and/or job-configuration and is relatively more 074 * complex for the user to control finely (e.g. {@link #setNumMapTasks(int)}). 075 * </li> 076 * </ol></p> 077 * 078 * <p><code>JobConf</code> typically specifies the {@link Mapper}, combiner 079 * (if any), {@link Partitioner}, {@link Reducer}, {@link InputFormat} and 080 * {@link OutputFormat} implementations to be used etc. 081 * 082 * <p>Optionally <code>JobConf</code> is used to specify other advanced facets 083 * of the job such as <code>Comparator</code>s to be used, files to be put in 084 * the {@link DistributedCache}, whether or not intermediate and/or job outputs 085 * are to be compressed (and how), debugability via user-provided scripts 086 * ( {@link #setMapDebugScript(String)}/{@link #setReduceDebugScript(String)}), 087 * for doing post-processing on task logs, task's stdout, stderr, syslog. 088 * and etc.</p> 089 * 090 * <p>Here is an example on how to configure a job via <code>JobConf</code>:</p> 091 * <p><blockquote><pre> 092 * // Create a new JobConf 093 * JobConf job = new JobConf(new Configuration(), MyJob.class); 094 * 095 * // Specify various job-specific parameters 096 * job.setJobName("myjob"); 097 * 098 * FileInputFormat.setInputPaths(job, new Path("in")); 099 * FileOutputFormat.setOutputPath(job, new Path("out")); 100 * 101 * job.setMapperClass(MyJob.MyMapper.class); 102 * job.setCombinerClass(MyJob.MyReducer.class); 103 * job.setReducerClass(MyJob.MyReducer.class); 104 * 105 * job.setInputFormat(SequenceFileInputFormat.class); 106 * job.setOutputFormat(SequenceFileOutputFormat.class); 107 * </pre></blockquote></p> 108 * 109 * @see JobClient 110 * @see ClusterStatus 111 * @see Tool 112 * @see DistributedCache 113 */ 114 @InterfaceAudience.Public 115 @InterfaceStability.Stable 116 public class JobConf extends Configuration { 117 118 private static final Log LOG = LogFactory.getLog(JobConf.class); 119 120 static{ 121 ConfigUtil.loadResources(); 122 } 123 124 /** 125 * @deprecated Use {@link #MAPRED_JOB_MAP_MEMORY_MB_PROPERTY} and 126 * {@link #MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY} 127 */ 128 @Deprecated 129 public static final String MAPRED_TASK_MAXVMEM_PROPERTY = 130 "mapred.task.maxvmem"; 131 132 /** 133 * @deprecated 134 */ 135 @Deprecated 136 public static final String UPPER_LIMIT_ON_TASK_VMEM_PROPERTY = 137 "mapred.task.limit.maxvmem"; 138 139 /** 140 * @deprecated 141 */ 142 @Deprecated 143 public static final String MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY = 144 "mapred.task.default.maxvmem"; 145 146 /** 147 * @deprecated 148 */ 149 @Deprecated 150 public static final String MAPRED_TASK_MAXPMEM_PROPERTY = 151 "mapred.task.maxpmem"; 152 153 /** 154 * A value which if set for memory related configuration options, 155 * indicates that the options are turned off. 156 */ 157 public static final long DISABLED_MEMORY_LIMIT = -1L; 158 159 /** 160 * Property name for the configuration property mapreduce.cluster.local.dir 161 */ 162 public static final String MAPRED_LOCAL_DIR_PROPERTY = MRConfig.LOCAL_DIR; 163 164 /** 165 * Name of the queue to which jobs will be submitted, if no queue 166 * name is mentioned. 167 */ 168 public static final String DEFAULT_QUEUE_NAME = "default"; 169 170 static final String MAPRED_JOB_MAP_MEMORY_MB_PROPERTY = 171 JobContext.MAP_MEMORY_MB; 172 173 static final String MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY = 174 JobContext.REDUCE_MEMORY_MB; 175 176 /** Pattern for the default unpacking behavior for job jars */ 177 public static final Pattern UNPACK_JAR_PATTERN_DEFAULT = 178 Pattern.compile("(?:classes/|lib/).*"); 179 180 /** 181 * Configuration key to set the java command line options for the child 182 * map and reduce tasks. 183 * 184 * Java opts for the task tracker child processes. 185 * The following symbol, if present, will be interpolated: @taskid@. 186 * It is replaced by current TaskID. Any other occurrences of '@' will go 187 * unchanged. 188 * For example, to enable verbose gc logging to a file named for the taskid in 189 * /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of: 190 * -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc 191 * 192 * The configuration variable {@link #MAPRED_TASK_ENV} can be used to pass 193 * other environment variables to the child processes. 194 * 195 * @deprecated Use {@link #MAPRED_MAP_TASK_JAVA_OPTS} or 196 * {@link #MAPRED_REDUCE_TASK_JAVA_OPTS} 197 */ 198 @Deprecated 199 public static final String MAPRED_TASK_JAVA_OPTS = "mapred.child.java.opts"; 200 201 /** 202 * Configuration key to set the java command line options for the map tasks. 203 * 204 * Java opts for the task tracker child map processes. 205 * The following symbol, if present, will be interpolated: @taskid@. 206 * It is replaced by current TaskID. Any other occurrences of '@' will go 207 * unchanged. 208 * For example, to enable verbose gc logging to a file named for the taskid in 209 * /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of: 210 * -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc 211 * 212 * The configuration variable {@link #MAPRED_MAP_TASK_ENV} can be used to pass 213 * other environment variables to the map processes. 214 */ 215 public static final String MAPRED_MAP_TASK_JAVA_OPTS = 216 JobContext.MAP_JAVA_OPTS; 217 218 /** 219 * Configuration key to set the java command line options for the reduce tasks. 220 * 221 * Java opts for the task tracker child reduce processes. 222 * The following symbol, if present, will be interpolated: @taskid@. 223 * It is replaced by current TaskID. Any other occurrences of '@' will go 224 * unchanged. 225 * For example, to enable verbose gc logging to a file named for the taskid in 226 * /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of: 227 * -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc 228 * 229 * The configuration variable {@link #MAPRED_REDUCE_TASK_ENV} can be used to 230 * pass process environment variables to the reduce processes. 231 */ 232 public static final String MAPRED_REDUCE_TASK_JAVA_OPTS = 233 JobContext.REDUCE_JAVA_OPTS; 234 235 public static final String DEFAULT_MAPRED_TASK_JAVA_OPTS = "-Xmx200m"; 236 237 /** 238 * @deprecated 239 * Configuration key to set the maximum virtual memory available to the child 240 * map and reduce tasks (in kilo-bytes). This has been deprecated and will no 241 * longer have any effect. 242 */ 243 @Deprecated 244 public static final String MAPRED_TASK_ULIMIT = "mapred.child.ulimit"; 245 246 /** 247 * @deprecated 248 * Configuration key to set the maximum virtual memory available to the 249 * map tasks (in kilo-bytes). This has been deprecated and will no 250 * longer have any effect. 251 */ 252 @Deprecated 253 public static final String MAPRED_MAP_TASK_ULIMIT = "mapreduce.map.ulimit"; 254 255 /** 256 * @deprecated 257 * Configuration key to set the maximum virtual memory available to the 258 * reduce tasks (in kilo-bytes). This has been deprecated and will no 259 * longer have any effect. 260 */ 261 @Deprecated 262 public static final String MAPRED_REDUCE_TASK_ULIMIT = 263 "mapreduce.reduce.ulimit"; 264 265 266 /** 267 * Configuration key to set the environment of the child map/reduce tasks. 268 * 269 * The format of the value is <code>k1=v1,k2=v2</code>. Further it can 270 * reference existing environment variables via <code>$key</code>. 271 * 272 * Example: 273 * <ul> 274 * <li> A=foo - This will set the env variable A to foo. </li> 275 * <li> B=$X:c This is inherit tasktracker's X env variable. </li> 276 * </ul> 277 * 278 * @deprecated Use {@link #MAPRED_MAP_TASK_ENV} or 279 * {@link #MAPRED_REDUCE_TASK_ENV} 280 */ 281 @Deprecated 282 public static final String MAPRED_TASK_ENV = "mapred.child.env"; 283 284 /** 285 * Configuration key to set the maximum virutal memory available to the 286 * map tasks. 287 * 288 * The format of the value is <code>k1=v1,k2=v2</code>. Further it can 289 * reference existing environment variables via <code>$key</code>. 290 * 291 * Example: 292 * <ul> 293 * <li> A=foo - This will set the env variable A to foo. </li> 294 * <li> B=$X:c This is inherit tasktracker's X env variable. </li> 295 * </ul> 296 */ 297 public static final String MAPRED_MAP_TASK_ENV = JobContext.MAP_ENV; 298 299 /** 300 * Configuration key to set the maximum virutal memory available to the 301 * reduce tasks. 302 * 303 * The format of the value is <code>k1=v1,k2=v2</code>. Further it can 304 * reference existing environment variables via <code>$key</code>. 305 * 306 * Example: 307 * <ul> 308 * <li> A=foo - This will set the env variable A to foo. </li> 309 * <li> B=$X:c This is inherit tasktracker's X env variable. </li> 310 * </ul> 311 */ 312 public static final String MAPRED_REDUCE_TASK_ENV = JobContext.REDUCE_ENV; 313 314 private Credentials credentials = new Credentials(); 315 316 /** 317 * Configuration key to set the logging {@link Level} for the map task. 318 * 319 * The allowed logging levels are: 320 * OFF, FATAL, ERROR, WARN, INFO, DEBUG, TRACE and ALL. 321 */ 322 public static final String MAPRED_MAP_TASK_LOG_LEVEL = 323 JobContext.MAP_LOG_LEVEL; 324 325 /** 326 * Configuration key to set the logging {@link Level} for the reduce task. 327 * 328 * The allowed logging levels are: 329 * OFF, FATAL, ERROR, WARN, INFO, DEBUG, TRACE and ALL. 330 */ 331 public static final String MAPRED_REDUCE_TASK_LOG_LEVEL = 332 JobContext.REDUCE_LOG_LEVEL; 333 334 /** 335 * Default logging level for map/reduce tasks. 336 */ 337 public static final Level DEFAULT_LOG_LEVEL = Level.INFO; 338 339 340 /** 341 * Construct a map/reduce job configuration. 342 */ 343 public JobConf() { 344 checkAndWarnDeprecation(); 345 } 346 347 /** 348 * Construct a map/reduce job configuration. 349 * 350 * @param exampleClass a class whose containing jar is used as the job's jar. 351 */ 352 public JobConf(Class exampleClass) { 353 setJarByClass(exampleClass); 354 checkAndWarnDeprecation(); 355 } 356 357 /** 358 * Construct a map/reduce job configuration. 359 * 360 * @param conf a Configuration whose settings will be inherited. 361 */ 362 public JobConf(Configuration conf) { 363 super(conf); 364 365 if (conf instanceof JobConf) { 366 JobConf that = (JobConf)conf; 367 credentials = that.credentials; 368 } 369 370 checkAndWarnDeprecation(); 371 } 372 373 374 /** Construct a map/reduce job configuration. 375 * 376 * @param conf a Configuration whose settings will be inherited. 377 * @param exampleClass a class whose containing jar is used as the job's jar. 378 */ 379 public JobConf(Configuration conf, Class exampleClass) { 380 this(conf); 381 setJarByClass(exampleClass); 382 } 383 384 385 /** Construct a map/reduce configuration. 386 * 387 * @param config a Configuration-format XML job description file. 388 */ 389 public JobConf(String config) { 390 this(new Path(config)); 391 } 392 393 /** Construct a map/reduce configuration. 394 * 395 * @param config a Configuration-format XML job description file. 396 */ 397 public JobConf(Path config) { 398 super(); 399 addResource(config); 400 checkAndWarnDeprecation(); 401 } 402 403 /** A new map/reduce configuration where the behavior of reading from the 404 * default resources can be turned off. 405 * <p/> 406 * If the parameter {@code loadDefaults} is false, the new instance 407 * will not load resources from the default files. 408 * 409 * @param loadDefaults specifies whether to load from the default files 410 */ 411 public JobConf(boolean loadDefaults) { 412 super(loadDefaults); 413 checkAndWarnDeprecation(); 414 } 415 416 /** 417 * Get credentials for the job. 418 * @return credentials for the job 419 */ 420 public Credentials getCredentials() { 421 return credentials; 422 } 423 424 @Private 425 public void setCredentials(Credentials credentials) { 426 this.credentials = credentials; 427 } 428 429 /** 430 * Get the user jar for the map-reduce job. 431 * 432 * @return the user jar for the map-reduce job. 433 */ 434 public String getJar() { return get(JobContext.JAR); } 435 436 /** 437 * Set the user jar for the map-reduce job. 438 * 439 * @param jar the user jar for the map-reduce job. 440 */ 441 public void setJar(String jar) { set(JobContext.JAR, jar); } 442 443 /** 444 * Get the pattern for jar contents to unpack on the tasktracker 445 */ 446 public Pattern getJarUnpackPattern() { 447 return getPattern(JobContext.JAR_UNPACK_PATTERN, UNPACK_JAR_PATTERN_DEFAULT); 448 } 449 450 451 /** 452 * Set the job's jar file by finding an example class location. 453 * 454 * @param cls the example class. 455 */ 456 public void setJarByClass(Class cls) { 457 String jar = ClassUtil.findContainingJar(cls); 458 if (jar != null) { 459 setJar(jar); 460 } 461 } 462 463 public String[] getLocalDirs() throws IOException { 464 return getTrimmedStrings(MRConfig.LOCAL_DIR); 465 } 466 467 /** 468 * Use MRAsyncDiskService.moveAndDeleteAllVolumes instead. 469 */ 470 @Deprecated 471 public void deleteLocalFiles() throws IOException { 472 String[] localDirs = getLocalDirs(); 473 for (int i = 0; i < localDirs.length; i++) { 474 FileSystem.getLocal(this).delete(new Path(localDirs[i]), true); 475 } 476 } 477 478 public void deleteLocalFiles(String subdir) throws IOException { 479 String[] localDirs = getLocalDirs(); 480 for (int i = 0; i < localDirs.length; i++) { 481 FileSystem.getLocal(this).delete(new Path(localDirs[i], subdir), true); 482 } 483 } 484 485 /** 486 * Constructs a local file name. Files are distributed among configured 487 * local directories. 488 */ 489 public Path getLocalPath(String pathString) throws IOException { 490 return getLocalPath(MRConfig.LOCAL_DIR, pathString); 491 } 492 493 /** 494 * Get the reported username for this job. 495 * 496 * @return the username 497 */ 498 public String getUser() { 499 return get(JobContext.USER_NAME); 500 } 501 502 /** 503 * Set the reported username for this job. 504 * 505 * @param user the username for this job. 506 */ 507 public void setUser(String user) { 508 set(JobContext.USER_NAME, user); 509 } 510 511 512 513 /** 514 * Set whether the framework should keep the intermediate files for 515 * failed tasks. 516 * 517 * @param keep <code>true</code> if framework should keep the intermediate files 518 * for failed tasks, <code>false</code> otherwise. 519 * 520 */ 521 public void setKeepFailedTaskFiles(boolean keep) { 522 setBoolean(JobContext.PRESERVE_FAILED_TASK_FILES, keep); 523 } 524 525 /** 526 * Should the temporary files for failed tasks be kept? 527 * 528 * @return should the files be kept? 529 */ 530 public boolean getKeepFailedTaskFiles() { 531 return getBoolean(JobContext.PRESERVE_FAILED_TASK_FILES, false); 532 } 533 534 /** 535 * Set a regular expression for task names that should be kept. 536 * The regular expression ".*_m_000123_0" would keep the files 537 * for the first instance of map 123 that ran. 538 * 539 * @param pattern the java.util.regex.Pattern to match against the 540 * task names. 541 */ 542 public void setKeepTaskFilesPattern(String pattern) { 543 set(JobContext.PRESERVE_FILES_PATTERN, pattern); 544 } 545 546 /** 547 * Get the regular expression that is matched against the task names 548 * to see if we need to keep the files. 549 * 550 * @return the pattern as a string, if it was set, othewise null. 551 */ 552 public String getKeepTaskFilesPattern() { 553 return get(JobContext.PRESERVE_FILES_PATTERN); 554 } 555 556 /** 557 * Set the current working directory for the default file system. 558 * 559 * @param dir the new current working directory. 560 */ 561 public void setWorkingDirectory(Path dir) { 562 dir = new Path(getWorkingDirectory(), dir); 563 set(JobContext.WORKING_DIR, dir.toString()); 564 } 565 566 /** 567 * Get the current working directory for the default file system. 568 * 569 * @return the directory name. 570 */ 571 public Path getWorkingDirectory() { 572 String name = get(JobContext.WORKING_DIR); 573 if (name != null) { 574 return new Path(name); 575 } else { 576 try { 577 Path dir = FileSystem.get(this).getWorkingDirectory(); 578 set(JobContext.WORKING_DIR, dir.toString()); 579 return dir; 580 } catch (IOException e) { 581 throw new RuntimeException(e); 582 } 583 } 584 } 585 586 /** 587 * Sets the number of tasks that a spawned task JVM should run 588 * before it exits 589 * @param numTasks the number of tasks to execute; defaults to 1; 590 * -1 signifies no limit 591 */ 592 public void setNumTasksToExecutePerJvm(int numTasks) { 593 setInt(JobContext.JVM_NUMTASKS_TORUN, numTasks); 594 } 595 596 /** 597 * Get the number of tasks that a spawned JVM should execute 598 */ 599 public int getNumTasksToExecutePerJvm() { 600 return getInt(JobContext.JVM_NUMTASKS_TORUN, 1); 601 } 602 603 /** 604 * Get the {@link InputFormat} implementation for the map-reduce job, 605 * defaults to {@link TextInputFormat} if not specified explicity. 606 * 607 * @return the {@link InputFormat} implementation for the map-reduce job. 608 */ 609 public InputFormat getInputFormat() { 610 return ReflectionUtils.newInstance(getClass("mapred.input.format.class", 611 TextInputFormat.class, 612 InputFormat.class), 613 this); 614 } 615 616 /** 617 * Set the {@link InputFormat} implementation for the map-reduce job. 618 * 619 * @param theClass the {@link InputFormat} implementation for the map-reduce 620 * job. 621 */ 622 public void setInputFormat(Class<? extends InputFormat> theClass) { 623 setClass("mapred.input.format.class", theClass, InputFormat.class); 624 } 625 626 /** 627 * Get the {@link OutputFormat} implementation for the map-reduce job, 628 * defaults to {@link TextOutputFormat} if not specified explicity. 629 * 630 * @return the {@link OutputFormat} implementation for the map-reduce job. 631 */ 632 public OutputFormat getOutputFormat() { 633 return ReflectionUtils.newInstance(getClass("mapred.output.format.class", 634 TextOutputFormat.class, 635 OutputFormat.class), 636 this); 637 } 638 639 /** 640 * Get the {@link OutputCommitter} implementation for the map-reduce job, 641 * defaults to {@link FileOutputCommitter} if not specified explicitly. 642 * 643 * @return the {@link OutputCommitter} implementation for the map-reduce job. 644 */ 645 public OutputCommitter getOutputCommitter() { 646 return (OutputCommitter)ReflectionUtils.newInstance( 647 getClass("mapred.output.committer.class", FileOutputCommitter.class, 648 OutputCommitter.class), this); 649 } 650 651 /** 652 * Set the {@link OutputCommitter} implementation for the map-reduce job. 653 * 654 * @param theClass the {@link OutputCommitter} implementation for the map-reduce 655 * job. 656 */ 657 public void setOutputCommitter(Class<? extends OutputCommitter> theClass) { 658 setClass("mapred.output.committer.class", theClass, OutputCommitter.class); 659 } 660 661 /** 662 * Set the {@link OutputFormat} implementation for the map-reduce job. 663 * 664 * @param theClass the {@link OutputFormat} implementation for the map-reduce 665 * job. 666 */ 667 public void setOutputFormat(Class<? extends OutputFormat> theClass) { 668 setClass("mapred.output.format.class", theClass, OutputFormat.class); 669 } 670 671 /** 672 * Should the map outputs be compressed before transfer? 673 * Uses the SequenceFile compression. 674 * 675 * @param compress should the map outputs be compressed? 676 */ 677 public void setCompressMapOutput(boolean compress) { 678 setBoolean(JobContext.MAP_OUTPUT_COMPRESS, compress); 679 } 680 681 /** 682 * Are the outputs of the maps be compressed? 683 * 684 * @return <code>true</code> if the outputs of the maps are to be compressed, 685 * <code>false</code> otherwise. 686 */ 687 public boolean getCompressMapOutput() { 688 return getBoolean(JobContext.MAP_OUTPUT_COMPRESS, false); 689 } 690 691 /** 692 * Set the given class as the {@link CompressionCodec} for the map outputs. 693 * 694 * @param codecClass the {@link CompressionCodec} class that will compress 695 * the map outputs. 696 */ 697 public void 698 setMapOutputCompressorClass(Class<? extends CompressionCodec> codecClass) { 699 setCompressMapOutput(true); 700 setClass(JobContext.MAP_OUTPUT_COMPRESS_CODEC, codecClass, 701 CompressionCodec.class); 702 } 703 704 /** 705 * Get the {@link CompressionCodec} for compressing the map outputs. 706 * 707 * @param defaultValue the {@link CompressionCodec} to return if not set 708 * @return the {@link CompressionCodec} class that should be used to compress the 709 * map outputs. 710 * @throws IllegalArgumentException if the class was specified, but not found 711 */ 712 public Class<? extends CompressionCodec> 713 getMapOutputCompressorClass(Class<? extends CompressionCodec> defaultValue) { 714 Class<? extends CompressionCodec> codecClass = defaultValue; 715 String name = get(JobContext.MAP_OUTPUT_COMPRESS_CODEC); 716 if (name != null) { 717 try { 718 codecClass = getClassByName(name).asSubclass(CompressionCodec.class); 719 } catch (ClassNotFoundException e) { 720 throw new IllegalArgumentException("Compression codec " + name + 721 " was not found.", e); 722 } 723 } 724 return codecClass; 725 } 726 727 /** 728 * Get the key class for the map output data. If it is not set, use the 729 * (final) output key class. This allows the map output key class to be 730 * different than the final output key class. 731 * 732 * @return the map output key class. 733 */ 734 public Class<?> getMapOutputKeyClass() { 735 Class<?> retv = getClass(JobContext.MAP_OUTPUT_KEY_CLASS, null, Object.class); 736 if (retv == null) { 737 retv = getOutputKeyClass(); 738 } 739 return retv; 740 } 741 742 /** 743 * Set the key class for the map output data. This allows the user to 744 * specify the map output key class to be different than the final output 745 * value class. 746 * 747 * @param theClass the map output key class. 748 */ 749 public void setMapOutputKeyClass(Class<?> theClass) { 750 setClass(JobContext.MAP_OUTPUT_KEY_CLASS, theClass, Object.class); 751 } 752 753 /** 754 * Get the value class for the map output data. If it is not set, use the 755 * (final) output value class This allows the map output value class to be 756 * different than the final output value class. 757 * 758 * @return the map output value class. 759 */ 760 public Class<?> getMapOutputValueClass() { 761 Class<?> retv = getClass(JobContext.MAP_OUTPUT_VALUE_CLASS, null, 762 Object.class); 763 if (retv == null) { 764 retv = getOutputValueClass(); 765 } 766 return retv; 767 } 768 769 /** 770 * Set the value class for the map output data. This allows the user to 771 * specify the map output value class to be different than the final output 772 * value class. 773 * 774 * @param theClass the map output value class. 775 */ 776 public void setMapOutputValueClass(Class<?> theClass) { 777 setClass(JobContext.MAP_OUTPUT_VALUE_CLASS, theClass, Object.class); 778 } 779 780 /** 781 * Get the key class for the job output data. 782 * 783 * @return the key class for the job output data. 784 */ 785 public Class<?> getOutputKeyClass() { 786 return getClass(JobContext.OUTPUT_KEY_CLASS, 787 LongWritable.class, Object.class); 788 } 789 790 /** 791 * Set the key class for the job output data. 792 * 793 * @param theClass the key class for the job output data. 794 */ 795 public void setOutputKeyClass(Class<?> theClass) { 796 setClass(JobContext.OUTPUT_KEY_CLASS, theClass, Object.class); 797 } 798 799 /** 800 * Get the {@link RawComparator} comparator used to compare keys. 801 * 802 * @return the {@link RawComparator} comparator used to compare keys. 803 */ 804 public RawComparator getOutputKeyComparator() { 805 Class<? extends RawComparator> theClass = getClass( 806 JobContext.KEY_COMPARATOR, null, RawComparator.class); 807 if (theClass != null) 808 return ReflectionUtils.newInstance(theClass, this); 809 return WritableComparator.get(getMapOutputKeyClass().asSubclass(WritableComparable.class)); 810 } 811 812 /** 813 * Set the {@link RawComparator} comparator used to compare keys. 814 * 815 * @param theClass the {@link RawComparator} comparator used to 816 * compare keys. 817 * @see #setOutputValueGroupingComparator(Class) 818 */ 819 public void setOutputKeyComparatorClass(Class<? extends RawComparator> theClass) { 820 setClass(JobContext.KEY_COMPARATOR, 821 theClass, RawComparator.class); 822 } 823 824 /** 825 * Set the {@link KeyFieldBasedComparator} options used to compare keys. 826 * 827 * @param keySpec the key specification of the form -k pos1[,pos2], where, 828 * pos is of the form f[.c][opts], where f is the number 829 * of the key field to use, and c is the number of the first character from 830 * the beginning of the field. Fields and character posns are numbered 831 * starting with 1; a character position of zero in pos2 indicates the 832 * field's last character. If '.c' is omitted from pos1, it defaults to 1 833 * (the beginning of the field); if omitted from pos2, it defaults to 0 834 * (the end of the field). opts are ordering options. The supported options 835 * are: 836 * -n, (Sort numerically) 837 * -r, (Reverse the result of comparison) 838 */ 839 public void setKeyFieldComparatorOptions(String keySpec) { 840 setOutputKeyComparatorClass(KeyFieldBasedComparator.class); 841 set(KeyFieldBasedComparator.COMPARATOR_OPTIONS, keySpec); 842 } 843 844 /** 845 * Get the {@link KeyFieldBasedComparator} options 846 */ 847 public String getKeyFieldComparatorOption() { 848 return get(KeyFieldBasedComparator.COMPARATOR_OPTIONS); 849 } 850 851 /** 852 * Set the {@link KeyFieldBasedPartitioner} options used for 853 * {@link Partitioner} 854 * 855 * @param keySpec the key specification of the form -k pos1[,pos2], where, 856 * pos is of the form f[.c][opts], where f is the number 857 * of the key field to use, and c is the number of the first character from 858 * the beginning of the field. Fields and character posns are numbered 859 * starting with 1; a character position of zero in pos2 indicates the 860 * field's last character. If '.c' is omitted from pos1, it defaults to 1 861 * (the beginning of the field); if omitted from pos2, it defaults to 0 862 * (the end of the field). 863 */ 864 public void setKeyFieldPartitionerOptions(String keySpec) { 865 setPartitionerClass(KeyFieldBasedPartitioner.class); 866 set(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS, keySpec); 867 } 868 869 /** 870 * Get the {@link KeyFieldBasedPartitioner} options 871 */ 872 public String getKeyFieldPartitionerOption() { 873 return get(KeyFieldBasedPartitioner.PARTITIONER_OPTIONS); 874 } 875 876 /** 877 * Get the user defined {@link WritableComparable} comparator for 878 * grouping keys of inputs to the reduce. 879 * 880 * @return comparator set by the user for grouping values. 881 * @see #setOutputValueGroupingComparator(Class) for details. 882 */ 883 public RawComparator getOutputValueGroupingComparator() { 884 Class<? extends RawComparator> theClass = getClass( 885 JobContext.GROUP_COMPARATOR_CLASS, null, RawComparator.class); 886 if (theClass == null) { 887 return getOutputKeyComparator(); 888 } 889 890 return ReflectionUtils.newInstance(theClass, this); 891 } 892 893 /** 894 * Set the user defined {@link RawComparator} comparator for 895 * grouping keys in the input to the reduce. 896 * 897 * <p>This comparator should be provided if the equivalence rules for keys 898 * for sorting the intermediates are different from those for grouping keys 899 * before each call to 900 * {@link Reducer#reduce(Object, java.util.Iterator, OutputCollector, Reporter)}.</p> 901 * 902 * <p>For key-value pairs (K1,V1) and (K2,V2), the values (V1, V2) are passed 903 * in a single call to the reduce function if K1 and K2 compare as equal.</p> 904 * 905 * <p>Since {@link #setOutputKeyComparatorClass(Class)} can be used to control 906 * how keys are sorted, this can be used in conjunction to simulate 907 * <i>secondary sort on values</i>.</p> 908 * 909 * <p><i>Note</i>: This is not a guarantee of the reduce sort being 910 * <i>stable</i> in any sense. (In any case, with the order of available 911 * map-outputs to the reduce being non-deterministic, it wouldn't make 912 * that much sense.)</p> 913 * 914 * @param theClass the comparator class to be used for grouping keys. 915 * It should implement <code>RawComparator</code>. 916 * @see #setOutputKeyComparatorClass(Class) 917 */ 918 public void setOutputValueGroupingComparator( 919 Class<? extends RawComparator> theClass) { 920 setClass(JobContext.GROUP_COMPARATOR_CLASS, 921 theClass, RawComparator.class); 922 } 923 924 /** 925 * Should the framework use the new context-object code for running 926 * the mapper? 927 * @return true, if the new api should be used 928 */ 929 public boolean getUseNewMapper() { 930 return getBoolean("mapred.mapper.new-api", false); 931 } 932 /** 933 * Set whether the framework should use the new api for the mapper. 934 * This is the default for jobs submitted with the new Job api. 935 * @param flag true, if the new api should be used 936 */ 937 public void setUseNewMapper(boolean flag) { 938 setBoolean("mapred.mapper.new-api", flag); 939 } 940 941 /** 942 * Should the framework use the new context-object code for running 943 * the reducer? 944 * @return true, if the new api should be used 945 */ 946 public boolean getUseNewReducer() { 947 return getBoolean("mapred.reducer.new-api", false); 948 } 949 /** 950 * Set whether the framework should use the new api for the reducer. 951 * This is the default for jobs submitted with the new Job api. 952 * @param flag true, if the new api should be used 953 */ 954 public void setUseNewReducer(boolean flag) { 955 setBoolean("mapred.reducer.new-api", flag); 956 } 957 958 /** 959 * Get the value class for job outputs. 960 * 961 * @return the value class for job outputs. 962 */ 963 public Class<?> getOutputValueClass() { 964 return getClass(JobContext.OUTPUT_VALUE_CLASS, Text.class, Object.class); 965 } 966 967 /** 968 * Set the value class for job outputs. 969 * 970 * @param theClass the value class for job outputs. 971 */ 972 public void setOutputValueClass(Class<?> theClass) { 973 setClass(JobContext.OUTPUT_VALUE_CLASS, theClass, Object.class); 974 } 975 976 /** 977 * Get the {@link Mapper} class for the job. 978 * 979 * @return the {@link Mapper} class for the job. 980 */ 981 public Class<? extends Mapper> getMapperClass() { 982 return getClass("mapred.mapper.class", IdentityMapper.class, Mapper.class); 983 } 984 985 /** 986 * Set the {@link Mapper} class for the job. 987 * 988 * @param theClass the {@link Mapper} class for the job. 989 */ 990 public void setMapperClass(Class<? extends Mapper> theClass) { 991 setClass("mapred.mapper.class", theClass, Mapper.class); 992 } 993 994 /** 995 * Get the {@link MapRunnable} class for the job. 996 * 997 * @return the {@link MapRunnable} class for the job. 998 */ 999 public Class<? extends MapRunnable> getMapRunnerClass() { 1000 return getClass("mapred.map.runner.class", 1001 MapRunner.class, MapRunnable.class); 1002 } 1003 1004 /** 1005 * Expert: Set the {@link MapRunnable} class for the job. 1006 * 1007 * Typically used to exert greater control on {@link Mapper}s. 1008 * 1009 * @param theClass the {@link MapRunnable} class for the job. 1010 */ 1011 public void setMapRunnerClass(Class<? extends MapRunnable> theClass) { 1012 setClass("mapred.map.runner.class", theClass, MapRunnable.class); 1013 } 1014 1015 /** 1016 * Get the {@link Partitioner} used to partition {@link Mapper}-outputs 1017 * to be sent to the {@link Reducer}s. 1018 * 1019 * @return the {@link Partitioner} used to partition map-outputs. 1020 */ 1021 public Class<? extends Partitioner> getPartitionerClass() { 1022 return getClass("mapred.partitioner.class", 1023 HashPartitioner.class, Partitioner.class); 1024 } 1025 1026 /** 1027 * Set the {@link Partitioner} class used to partition 1028 * {@link Mapper}-outputs to be sent to the {@link Reducer}s. 1029 * 1030 * @param theClass the {@link Partitioner} used to partition map-outputs. 1031 */ 1032 public void setPartitionerClass(Class<? extends Partitioner> theClass) { 1033 setClass("mapred.partitioner.class", theClass, Partitioner.class); 1034 } 1035 1036 /** 1037 * Get the {@link Reducer} class for the job. 1038 * 1039 * @return the {@link Reducer} class for the job. 1040 */ 1041 public Class<? extends Reducer> getReducerClass() { 1042 return getClass("mapred.reducer.class", 1043 IdentityReducer.class, Reducer.class); 1044 } 1045 1046 /** 1047 * Set the {@link Reducer} class for the job. 1048 * 1049 * @param theClass the {@link Reducer} class for the job. 1050 */ 1051 public void setReducerClass(Class<? extends Reducer> theClass) { 1052 setClass("mapred.reducer.class", theClass, Reducer.class); 1053 } 1054 1055 /** 1056 * Get the user-defined <i>combiner</i> class used to combine map-outputs 1057 * before being sent to the reducers. Typically the combiner is same as the 1058 * the {@link Reducer} for the job i.e. {@link #getReducerClass()}. 1059 * 1060 * @return the user-defined combiner class used to combine map-outputs. 1061 */ 1062 public Class<? extends Reducer> getCombinerClass() { 1063 return getClass("mapred.combiner.class", null, Reducer.class); 1064 } 1065 1066 /** 1067 * Set the user-defined <i>combiner</i> class used to combine map-outputs 1068 * before being sent to the reducers. 1069 * 1070 * <p>The combiner is an application-specified aggregation operation, which 1071 * can help cut down the amount of data transferred between the 1072 * {@link Mapper} and the {@link Reducer}, leading to better performance.</p> 1073 * 1074 * <p>The framework may invoke the combiner 0, 1, or multiple times, in both 1075 * the mapper and reducer tasks. In general, the combiner is called as the 1076 * sort/merge result is written to disk. The combiner must: 1077 * <ul> 1078 * <li> be side-effect free</li> 1079 * <li> have the same input and output key types and the same input and 1080 * output value types</li> 1081 * </ul></p> 1082 * 1083 * <p>Typically the combiner is same as the <code>Reducer</code> for the 1084 * job i.e. {@link #setReducerClass(Class)}.</p> 1085 * 1086 * @param theClass the user-defined combiner class used to combine 1087 * map-outputs. 1088 */ 1089 public void setCombinerClass(Class<? extends Reducer> theClass) { 1090 setClass("mapred.combiner.class", theClass, Reducer.class); 1091 } 1092 1093 /** 1094 * Should speculative execution be used for this job? 1095 * Defaults to <code>true</code>. 1096 * 1097 * @return <code>true</code> if speculative execution be used for this job, 1098 * <code>false</code> otherwise. 1099 */ 1100 public boolean getSpeculativeExecution() { 1101 return (getMapSpeculativeExecution() || getReduceSpeculativeExecution()); 1102 } 1103 1104 /** 1105 * Turn speculative execution on or off for this job. 1106 * 1107 * @param speculativeExecution <code>true</code> if speculative execution 1108 * should be turned on, else <code>false</code>. 1109 */ 1110 public void setSpeculativeExecution(boolean speculativeExecution) { 1111 setMapSpeculativeExecution(speculativeExecution); 1112 setReduceSpeculativeExecution(speculativeExecution); 1113 } 1114 1115 /** 1116 * Should speculative execution be used for this job for map tasks? 1117 * Defaults to <code>true</code>. 1118 * 1119 * @return <code>true</code> if speculative execution be 1120 * used for this job for map tasks, 1121 * <code>false</code> otherwise. 1122 */ 1123 public boolean getMapSpeculativeExecution() { 1124 return getBoolean(JobContext.MAP_SPECULATIVE, true); 1125 } 1126 1127 /** 1128 * Turn speculative execution on or off for this job for map tasks. 1129 * 1130 * @param speculativeExecution <code>true</code> if speculative execution 1131 * should be turned on for map tasks, 1132 * else <code>false</code>. 1133 */ 1134 public void setMapSpeculativeExecution(boolean speculativeExecution) { 1135 setBoolean(JobContext.MAP_SPECULATIVE, speculativeExecution); 1136 } 1137 1138 /** 1139 * Should speculative execution be used for this job for reduce tasks? 1140 * Defaults to <code>true</code>. 1141 * 1142 * @return <code>true</code> if speculative execution be used 1143 * for reduce tasks for this job, 1144 * <code>false</code> otherwise. 1145 */ 1146 public boolean getReduceSpeculativeExecution() { 1147 return getBoolean(JobContext.REDUCE_SPECULATIVE, true); 1148 } 1149 1150 /** 1151 * Turn speculative execution on or off for this job for reduce tasks. 1152 * 1153 * @param speculativeExecution <code>true</code> if speculative execution 1154 * should be turned on for reduce tasks, 1155 * else <code>false</code>. 1156 */ 1157 public void setReduceSpeculativeExecution(boolean speculativeExecution) { 1158 setBoolean(JobContext.REDUCE_SPECULATIVE, 1159 speculativeExecution); 1160 } 1161 1162 /** 1163 * Get configured the number of reduce tasks for this job. 1164 * Defaults to <code>1</code>. 1165 * 1166 * @return the number of reduce tasks for this job. 1167 */ 1168 public int getNumMapTasks() { return getInt(JobContext.NUM_MAPS, 1); } 1169 1170 /** 1171 * Set the number of map tasks for this job. 1172 * 1173 * <p><i>Note</i>: This is only a <i>hint</i> to the framework. The actual 1174 * number of spawned map tasks depends on the number of {@link InputSplit}s 1175 * generated by the job's {@link InputFormat#getSplits(JobConf, int)}. 1176 * 1177 * A custom {@link InputFormat} is typically used to accurately control 1178 * the number of map tasks for the job.</p> 1179 * 1180 * <h4 id="NoOfMaps">How many maps?</h4> 1181 * 1182 * <p>The number of maps is usually driven by the total size of the inputs 1183 * i.e. total number of blocks of the input files.</p> 1184 * 1185 * <p>The right level of parallelism for maps seems to be around 10-100 maps 1186 * per-node, although it has been set up to 300 or so for very cpu-light map 1187 * tasks. Task setup takes awhile, so it is best if the maps take at least a 1188 * minute to execute.</p> 1189 * 1190 * <p>The default behavior of file-based {@link InputFormat}s is to split the 1191 * input into <i>logical</i> {@link InputSplit}s based on the total size, in 1192 * bytes, of input files. However, the {@link FileSystem} blocksize of the 1193 * input files is treated as an upper bound for input splits. A lower bound 1194 * on the split size can be set via 1195 * <a href="{@docRoot}/../mapred-default.html#mapreduce.input.fileinputformat.split.minsize"> 1196 * mapreduce.input.fileinputformat.split.minsize</a>.</p> 1197 * 1198 * <p>Thus, if you expect 10TB of input data and have a blocksize of 128MB, 1199 * you'll end up with 82,000 maps, unless {@link #setNumMapTasks(int)} is 1200 * used to set it even higher.</p> 1201 * 1202 * @param n the number of map tasks for this job. 1203 * @see InputFormat#getSplits(JobConf, int) 1204 * @see FileInputFormat 1205 * @see FileSystem#getDefaultBlockSize() 1206 * @see FileStatus#getBlockSize() 1207 */ 1208 public void setNumMapTasks(int n) { setInt(JobContext.NUM_MAPS, n); } 1209 1210 /** 1211 * Get configured the number of reduce tasks for this job. Defaults to 1212 * <code>1</code>. 1213 * 1214 * @return the number of reduce tasks for this job. 1215 */ 1216 public int getNumReduceTasks() { return getInt(JobContext.NUM_REDUCES, 1); } 1217 1218 /** 1219 * Set the requisite number of reduce tasks for this job. 1220 * 1221 * <h4 id="NoOfReduces">How many reduces?</h4> 1222 * 1223 * <p>The right number of reduces seems to be <code>0.95</code> or 1224 * <code>1.75</code> multiplied by (<<i>no. of nodes</i>> * 1225 * <a href="{@docRoot}/../mapred-default.html#mapreduce.tasktracker.reduce.tasks.maximum"> 1226 * mapreduce.tasktracker.reduce.tasks.maximum</a>). 1227 * </p> 1228 * 1229 * <p>With <code>0.95</code> all of the reduces can launch immediately and 1230 * start transfering map outputs as the maps finish. With <code>1.75</code> 1231 * the faster nodes will finish their first round of reduces and launch a 1232 * second wave of reduces doing a much better job of load balancing.</p> 1233 * 1234 * <p>Increasing the number of reduces increases the framework overhead, but 1235 * increases load balancing and lowers the cost of failures.</p> 1236 * 1237 * <p>The scaling factors above are slightly less than whole numbers to 1238 * reserve a few reduce slots in the framework for speculative-tasks, failures 1239 * etc.</p> 1240 * 1241 * <h4 id="ReducerNone">Reducer NONE</h4> 1242 * 1243 * <p>It is legal to set the number of reduce-tasks to <code>zero</code>.</p> 1244 * 1245 * <p>In this case the output of the map-tasks directly go to distributed 1246 * file-system, to the path set by 1247 * {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Also, the 1248 * framework doesn't sort the map-outputs before writing it out to HDFS.</p> 1249 * 1250 * @param n the number of reduce tasks for this job. 1251 */ 1252 public void setNumReduceTasks(int n) { setInt(JobContext.NUM_REDUCES, n); } 1253 1254 /** 1255 * Get the configured number of maximum attempts that will be made to run a 1256 * map task, as specified by the <code>mapreduce.map.maxattempts</code> 1257 * property. If this property is not already set, the default is 4 attempts. 1258 * 1259 * @return the max number of attempts per map task. 1260 */ 1261 public int getMaxMapAttempts() { 1262 return getInt(JobContext.MAP_MAX_ATTEMPTS, 4); 1263 } 1264 1265 /** 1266 * Expert: Set the number of maximum attempts that will be made to run a 1267 * map task. 1268 * 1269 * @param n the number of attempts per map task. 1270 */ 1271 public void setMaxMapAttempts(int n) { 1272 setInt(JobContext.MAP_MAX_ATTEMPTS, n); 1273 } 1274 1275 /** 1276 * Get the configured number of maximum attempts that will be made to run a 1277 * reduce task, as specified by the <code>mapreduce.reduce.maxattempts</code> 1278 * property. If this property is not already set, the default is 4 attempts. 1279 * 1280 * @return the max number of attempts per reduce task. 1281 */ 1282 public int getMaxReduceAttempts() { 1283 return getInt(JobContext.REDUCE_MAX_ATTEMPTS, 4); 1284 } 1285 /** 1286 * Expert: Set the number of maximum attempts that will be made to run a 1287 * reduce task. 1288 * 1289 * @param n the number of attempts per reduce task. 1290 */ 1291 public void setMaxReduceAttempts(int n) { 1292 setInt(JobContext.REDUCE_MAX_ATTEMPTS, n); 1293 } 1294 1295 /** 1296 * Get the user-specified job name. This is only used to identify the 1297 * job to the user. 1298 * 1299 * @return the job's name, defaulting to "". 1300 */ 1301 public String getJobName() { 1302 return get(JobContext.JOB_NAME, ""); 1303 } 1304 1305 /** 1306 * Set the user-specified job name. 1307 * 1308 * @param name the job's new name. 1309 */ 1310 public void setJobName(String name) { 1311 set(JobContext.JOB_NAME, name); 1312 } 1313 1314 /** 1315 * Get the user-specified session identifier. The default is the empty string. 1316 * 1317 * The session identifier is used to tag metric data that is reported to some 1318 * performance metrics system via the org.apache.hadoop.metrics API. The 1319 * session identifier is intended, in particular, for use by Hadoop-On-Demand 1320 * (HOD) which allocates a virtual Hadoop cluster dynamically and transiently. 1321 * HOD will set the session identifier by modifying the mapred-site.xml file 1322 * before starting the cluster. 1323 * 1324 * When not running under HOD, this identifer is expected to remain set to 1325 * the empty string. 1326 * 1327 * @return the session identifier, defaulting to "". 1328 */ 1329 @Deprecated 1330 public String getSessionId() { 1331 return get("session.id", ""); 1332 } 1333 1334 /** 1335 * Set the user-specified session identifier. 1336 * 1337 * @param sessionId the new session id. 1338 */ 1339 @Deprecated 1340 public void setSessionId(String sessionId) { 1341 set("session.id", sessionId); 1342 } 1343 1344 /** 1345 * Set the maximum no. of failures of a given job per tasktracker. 1346 * If the no. of task failures exceeds <code>noFailures</code>, the 1347 * tasktracker is <i>blacklisted</i> for this job. 1348 * 1349 * @param noFailures maximum no. of failures of a given job per tasktracker. 1350 */ 1351 public void setMaxTaskFailuresPerTracker(int noFailures) { 1352 setInt(JobContext.MAX_TASK_FAILURES_PER_TRACKER, noFailures); 1353 } 1354 1355 /** 1356 * Expert: Get the maximum no. of failures of a given job per tasktracker. 1357 * If the no. of task failures exceeds this, the tasktracker is 1358 * <i>blacklisted</i> for this job. 1359 * 1360 * @return the maximum no. of failures of a given job per tasktracker. 1361 */ 1362 public int getMaxTaskFailuresPerTracker() { 1363 return getInt(JobContext.MAX_TASK_FAILURES_PER_TRACKER, 3); 1364 } 1365 1366 /** 1367 * Get the maximum percentage of map tasks that can fail without 1368 * the job being aborted. 1369 * 1370 * Each map task is executed a minimum of {@link #getMaxMapAttempts()} 1371 * attempts before being declared as <i>failed</i>. 1372 * 1373 * Defaults to <code>zero</code>, i.e. <i>any</i> failed map-task results in 1374 * the job being declared as {@link JobStatus#FAILED}. 1375 * 1376 * @return the maximum percentage of map tasks that can fail without 1377 * the job being aborted. 1378 */ 1379 public int getMaxMapTaskFailuresPercent() { 1380 return getInt(JobContext.MAP_FAILURES_MAX_PERCENT, 0); 1381 } 1382 1383 /** 1384 * Expert: Set the maximum percentage of map tasks that can fail without the 1385 * job being aborted. 1386 * 1387 * Each map task is executed a minimum of {@link #getMaxMapAttempts} attempts 1388 * before being declared as <i>failed</i>. 1389 * 1390 * @param percent the maximum percentage of map tasks that can fail without 1391 * the job being aborted. 1392 */ 1393 public void setMaxMapTaskFailuresPercent(int percent) { 1394 setInt(JobContext.MAP_FAILURES_MAX_PERCENT, percent); 1395 } 1396 1397 /** 1398 * Get the maximum percentage of reduce tasks that can fail without 1399 * the job being aborted. 1400 * 1401 * Each reduce task is executed a minimum of {@link #getMaxReduceAttempts()} 1402 * attempts before being declared as <i>failed</i>. 1403 * 1404 * Defaults to <code>zero</code>, i.e. <i>any</i> failed reduce-task results 1405 * in the job being declared as {@link JobStatus#FAILED}. 1406 * 1407 * @return the maximum percentage of reduce tasks that can fail without 1408 * the job being aborted. 1409 */ 1410 public int getMaxReduceTaskFailuresPercent() { 1411 return getInt(JobContext.REDUCE_FAILURES_MAXPERCENT, 0); 1412 } 1413 1414 /** 1415 * Set the maximum percentage of reduce tasks that can fail without the job 1416 * being aborted. 1417 * 1418 * Each reduce task is executed a minimum of {@link #getMaxReduceAttempts()} 1419 * attempts before being declared as <i>failed</i>. 1420 * 1421 * @param percent the maximum percentage of reduce tasks that can fail without 1422 * the job being aborted. 1423 */ 1424 public void setMaxReduceTaskFailuresPercent(int percent) { 1425 setInt(JobContext.REDUCE_FAILURES_MAXPERCENT, percent); 1426 } 1427 1428 /** 1429 * Set {@link JobPriority} for this job. 1430 * 1431 * @param prio the {@link JobPriority} for this job. 1432 */ 1433 public void setJobPriority(JobPriority prio) { 1434 set(JobContext.PRIORITY, prio.toString()); 1435 } 1436 1437 /** 1438 * Get the {@link JobPriority} for this job. 1439 * 1440 * @return the {@link JobPriority} for this job. 1441 */ 1442 public JobPriority getJobPriority() { 1443 String prio = get(JobContext.PRIORITY); 1444 if(prio == null) { 1445 return JobPriority.NORMAL; 1446 } 1447 1448 return JobPriority.valueOf(prio); 1449 } 1450 1451 /** 1452 * Set JobSubmitHostName for this job. 1453 * 1454 * @param hostname the JobSubmitHostName for this job. 1455 */ 1456 void setJobSubmitHostName(String hostname) { 1457 set(MRJobConfig.JOB_SUBMITHOST, hostname); 1458 } 1459 1460 /** 1461 * Get the JobSubmitHostName for this job. 1462 * 1463 * @return the JobSubmitHostName for this job. 1464 */ 1465 String getJobSubmitHostName() { 1466 String hostname = get(MRJobConfig.JOB_SUBMITHOST); 1467 1468 return hostname; 1469 } 1470 1471 /** 1472 * Set JobSubmitHostAddress for this job. 1473 * 1474 * @param hostadd the JobSubmitHostAddress for this job. 1475 */ 1476 void setJobSubmitHostAddress(String hostadd) { 1477 set(MRJobConfig.JOB_SUBMITHOSTADDR, hostadd); 1478 } 1479 1480 /** 1481 * Get JobSubmitHostAddress for this job. 1482 * 1483 * @return JobSubmitHostAddress for this job. 1484 */ 1485 String getJobSubmitHostAddress() { 1486 String hostadd = get(MRJobConfig.JOB_SUBMITHOSTADDR); 1487 1488 return hostadd; 1489 } 1490 1491 /** 1492 * Get whether the task profiling is enabled. 1493 * @return true if some tasks will be profiled 1494 */ 1495 public boolean getProfileEnabled() { 1496 return getBoolean(JobContext.TASK_PROFILE, false); 1497 } 1498 1499 /** 1500 * Set whether the system should collect profiler information for some of 1501 * the tasks in this job? The information is stored in the user log 1502 * directory. 1503 * @param newValue true means it should be gathered 1504 */ 1505 public void setProfileEnabled(boolean newValue) { 1506 setBoolean(JobContext.TASK_PROFILE, newValue); 1507 } 1508 1509 /** 1510 * Get the profiler configuration arguments. 1511 * 1512 * The default value for this property is 1513 * "-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s" 1514 * 1515 * @return the parameters to pass to the task child to configure profiling 1516 */ 1517 public String getProfileParams() { 1518 return get(JobContext.TASK_PROFILE_PARAMS, 1519 "-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y," + 1520 "verbose=n,file=%s"); 1521 } 1522 1523 /** 1524 * Set the profiler configuration arguments. If the string contains a '%s' it 1525 * will be replaced with the name of the profiling output file when the task 1526 * runs. 1527 * 1528 * This value is passed to the task child JVM on the command line. 1529 * 1530 * @param value the configuration string 1531 */ 1532 public void setProfileParams(String value) { 1533 set(JobContext.TASK_PROFILE_PARAMS, value); 1534 } 1535 1536 /** 1537 * Get the range of maps or reduces to profile. 1538 * @param isMap is the task a map? 1539 * @return the task ranges 1540 */ 1541 public IntegerRanges getProfileTaskRange(boolean isMap) { 1542 return getRange((isMap ? JobContext.NUM_MAP_PROFILES : 1543 JobContext.NUM_REDUCE_PROFILES), "0-2"); 1544 } 1545 1546 /** 1547 * Set the ranges of maps or reduces to profile. setProfileEnabled(true) 1548 * must also be called. 1549 * @param newValue a set of integer ranges of the map ids 1550 */ 1551 public void setProfileTaskRange(boolean isMap, String newValue) { 1552 // parse the value to make sure it is legal 1553 new Configuration.IntegerRanges(newValue); 1554 set((isMap ? JobContext.NUM_MAP_PROFILES : JobContext.NUM_REDUCE_PROFILES), 1555 newValue); 1556 } 1557 1558 /** 1559 * Set the debug script to run when the map tasks fail. 1560 * 1561 * <p>The debug script can aid debugging of failed map tasks. The script is 1562 * given task's stdout, stderr, syslog, jobconf files as arguments.</p> 1563 * 1564 * <p>The debug command, run on the node where the map failed, is:</p> 1565 * <p><pre><blockquote> 1566 * $script $stdout $stderr $syslog $jobconf. 1567 * </blockquote></pre></p> 1568 * 1569 * <p> The script file is distributed through {@link DistributedCache} 1570 * APIs. The script needs to be symlinked. </p> 1571 * 1572 * <p>Here is an example on how to submit a script 1573 * <p><blockquote><pre> 1574 * job.setMapDebugScript("./myscript"); 1575 * DistributedCache.createSymlink(job); 1576 * DistributedCache.addCacheFile("/debug/scripts/myscript#myscript"); 1577 * </pre></blockquote></p> 1578 * 1579 * @param mDbgScript the script name 1580 */ 1581 public void setMapDebugScript(String mDbgScript) { 1582 set(JobContext.MAP_DEBUG_SCRIPT, mDbgScript); 1583 } 1584 1585 /** 1586 * Get the map task's debug script. 1587 * 1588 * @return the debug Script for the mapred job for failed map tasks. 1589 * @see #setMapDebugScript(String) 1590 */ 1591 public String getMapDebugScript() { 1592 return get(JobContext.MAP_DEBUG_SCRIPT); 1593 } 1594 1595 /** 1596 * Set the debug script to run when the reduce tasks fail. 1597 * 1598 * <p>The debug script can aid debugging of failed reduce tasks. The script 1599 * is given task's stdout, stderr, syslog, jobconf files as arguments.</p> 1600 * 1601 * <p>The debug command, run on the node where the map failed, is:</p> 1602 * <p><pre><blockquote> 1603 * $script $stdout $stderr $syslog $jobconf. 1604 * </blockquote></pre></p> 1605 * 1606 * <p> The script file is distributed through {@link DistributedCache} 1607 * APIs. The script file needs to be symlinked </p> 1608 * 1609 * <p>Here is an example on how to submit a script 1610 * <p><blockquote><pre> 1611 * job.setReduceDebugScript("./myscript"); 1612 * DistributedCache.createSymlink(job); 1613 * DistributedCache.addCacheFile("/debug/scripts/myscript#myscript"); 1614 * </pre></blockquote></p> 1615 * 1616 * @param rDbgScript the script name 1617 */ 1618 public void setReduceDebugScript(String rDbgScript) { 1619 set(JobContext.REDUCE_DEBUG_SCRIPT, rDbgScript); 1620 } 1621 1622 /** 1623 * Get the reduce task's debug Script 1624 * 1625 * @return the debug script for the mapred job for failed reduce tasks. 1626 * @see #setReduceDebugScript(String) 1627 */ 1628 public String getReduceDebugScript() { 1629 return get(JobContext.REDUCE_DEBUG_SCRIPT); 1630 } 1631 1632 /** 1633 * Get the uri to be invoked in-order to send a notification after the job 1634 * has completed (success/failure). 1635 * 1636 * @return the job end notification uri, <code>null</code> if it hasn't 1637 * been set. 1638 * @see #setJobEndNotificationURI(String) 1639 */ 1640 public String getJobEndNotificationURI() { 1641 return get(JobContext.MR_JOB_END_NOTIFICATION_URL); 1642 } 1643 1644 /** 1645 * Set the uri to be invoked in-order to send a notification after the job 1646 * has completed (success/failure). 1647 * 1648 * <p>The uri can contain 2 special parameters: <tt>$jobId</tt> and 1649 * <tt>$jobStatus</tt>. Those, if present, are replaced by the job's 1650 * identifier and completion-status respectively.</p> 1651 * 1652 * <p>This is typically used by application-writers to implement chaining of 1653 * Map-Reduce jobs in an <i>asynchronous manner</i>.</p> 1654 * 1655 * @param uri the job end notification uri 1656 * @see JobStatus 1657 * @see <a href="{@docRoot}/org/apache/hadoop/mapred/JobClient.html# 1658 * JobCompletionAndChaining">Job Completion and Chaining</a> 1659 */ 1660 public void setJobEndNotificationURI(String uri) { 1661 set(JobContext.MR_JOB_END_NOTIFICATION_URL, uri); 1662 } 1663 1664 /** 1665 * Get job-specific shared directory for use as scratch space 1666 * 1667 * <p> 1668 * When a job starts, a shared directory is created at location 1669 * <code> 1670 * ${mapreduce.cluster.local.dir}/taskTracker/$user/jobcache/$jobid/work/ </code>. 1671 * This directory is exposed to the users through 1672 * <code>mapreduce.job.local.dir </code>. 1673 * So, the tasks can use this space 1674 * as scratch space and share files among them. </p> 1675 * This value is available as System property also. 1676 * 1677 * @return The localized job specific shared directory 1678 */ 1679 public String getJobLocalDir() { 1680 return get(JobContext.JOB_LOCAL_DIR); 1681 } 1682 1683 /** 1684 * Get memory required to run a map task of the job, in MB. 1685 * 1686 * If a value is specified in the configuration, it is returned. 1687 * Else, it returns {@link #DISABLED_MEMORY_LIMIT}. 1688 * <p/> 1689 * For backward compatibility, if the job configuration sets the 1690 * key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different 1691 * from {@link #DISABLED_MEMORY_LIMIT}, that value will be used 1692 * after converting it from bytes to MB. 1693 * @return memory required to run a map task of the job, in MB, 1694 * or {@link #DISABLED_MEMORY_LIMIT} if unset. 1695 */ 1696 public long getMemoryForMapTask() { 1697 long value = getDeprecatedMemoryValue(); 1698 if (value == DISABLED_MEMORY_LIMIT) { 1699 value = normalizeMemoryConfigValue( 1700 getLong(JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY, 1701 DISABLED_MEMORY_LIMIT)); 1702 } 1703 return value; 1704 } 1705 1706 public void setMemoryForMapTask(long mem) { 1707 setLong(JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY, mem); 1708 } 1709 1710 /** 1711 * Get memory required to run a reduce task of the job, in MB. 1712 * 1713 * If a value is specified in the configuration, it is returned. 1714 * Else, it returns {@link #DISABLED_MEMORY_LIMIT}. 1715 * <p/> 1716 * For backward compatibility, if the job configuration sets the 1717 * key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different 1718 * from {@link #DISABLED_MEMORY_LIMIT}, that value will be used 1719 * after converting it from bytes to MB. 1720 * @return memory required to run a reduce task of the job, in MB, 1721 * or {@link #DISABLED_MEMORY_LIMIT} if unset. 1722 */ 1723 public long getMemoryForReduceTask() { 1724 long value = getDeprecatedMemoryValue(); 1725 if (value == DISABLED_MEMORY_LIMIT) { 1726 value = normalizeMemoryConfigValue( 1727 getLong(JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY, 1728 DISABLED_MEMORY_LIMIT)); 1729 } 1730 return value; 1731 } 1732 1733 // Return the value set to the key MAPRED_TASK_MAXVMEM_PROPERTY, 1734 // converted into MBs. 1735 // Returns DISABLED_MEMORY_LIMIT if unset, or set to a negative 1736 // value. 1737 private long getDeprecatedMemoryValue() { 1738 long oldValue = getLong(MAPRED_TASK_MAXVMEM_PROPERTY, 1739 DISABLED_MEMORY_LIMIT); 1740 oldValue = normalizeMemoryConfigValue(oldValue); 1741 if (oldValue != DISABLED_MEMORY_LIMIT) { 1742 oldValue /= (1024*1024); 1743 } 1744 return oldValue; 1745 } 1746 1747 public void setMemoryForReduceTask(long mem) { 1748 setLong(JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY, mem); 1749 } 1750 1751 /** 1752 * Return the name of the queue to which this job is submitted. 1753 * Defaults to 'default'. 1754 * 1755 * @return name of the queue 1756 */ 1757 public String getQueueName() { 1758 return get(JobContext.QUEUE_NAME, DEFAULT_QUEUE_NAME); 1759 } 1760 1761 /** 1762 * Set the name of the queue to which this job should be submitted. 1763 * 1764 * @param queueName Name of the queue 1765 */ 1766 public void setQueueName(String queueName) { 1767 set(JobContext.QUEUE_NAME, queueName); 1768 } 1769 1770 /** 1771 * Normalize the negative values in configuration 1772 * 1773 * @param val 1774 * @return normalized value 1775 */ 1776 public static long normalizeMemoryConfigValue(long val) { 1777 if (val < 0) { 1778 val = DISABLED_MEMORY_LIMIT; 1779 } 1780 return val; 1781 } 1782 1783 /** 1784 * Compute the number of slots required to run a single map task-attempt 1785 * of this job. 1786 * @param slotSizePerMap cluster-wide value of the amount of memory required 1787 * to run a map-task 1788 * @return the number of slots required to run a single map task-attempt 1789 * 1 if memory parameters are disabled. 1790 */ 1791 int computeNumSlotsPerMap(long slotSizePerMap) { 1792 if ((slotSizePerMap==DISABLED_MEMORY_LIMIT) || 1793 (getMemoryForMapTask()==DISABLED_MEMORY_LIMIT)) { 1794 return 1; 1795 } 1796 return (int)(Math.ceil((float)getMemoryForMapTask() / (float)slotSizePerMap)); 1797 } 1798 1799 /** 1800 * Compute the number of slots required to run a single reduce task-attempt 1801 * of this job. 1802 * @param slotSizePerReduce cluster-wide value of the amount of memory 1803 * required to run a reduce-task 1804 * @return the number of slots required to run a single reduce task-attempt 1805 * 1 if memory parameters are disabled 1806 */ 1807 int computeNumSlotsPerReduce(long slotSizePerReduce) { 1808 if ((slotSizePerReduce==DISABLED_MEMORY_LIMIT) || 1809 (getMemoryForReduceTask()==DISABLED_MEMORY_LIMIT)) { 1810 return 1; 1811 } 1812 return 1813 (int)(Math.ceil((float)getMemoryForReduceTask() / (float)slotSizePerReduce)); 1814 } 1815 1816 /** 1817 * Find a jar that contains a class of the same name, if any. 1818 * It will return a jar file, even if that is not the first thing 1819 * on the class path that has a class with the same name. 1820 * 1821 * @param my_class the class to find. 1822 * @return a jar file that contains the class, or null. 1823 * @throws IOException 1824 */ 1825 public static String findContainingJar(Class my_class) { 1826 return ClassUtil.findContainingJar(my_class); 1827 } 1828 1829 /** 1830 * Get the memory required to run a task of this job, in bytes. See 1831 * {@link #MAPRED_TASK_MAXVMEM_PROPERTY} 1832 * <p/> 1833 * This method is deprecated. Now, different memory limits can be 1834 * set for map and reduce tasks of a job, in MB. 1835 * <p/> 1836 * For backward compatibility, if the job configuration sets the 1837 * key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different 1838 * from {@link #DISABLED_MEMORY_LIMIT}, that value is returned. 1839 * Otherwise, this method will return the larger of the values returned by 1840 * {@link #getMemoryForMapTask()} and {@link #getMemoryForReduceTask()} 1841 * after converting them into bytes. 1842 * 1843 * @return Memory required to run a task of this job, in bytes, 1844 * or {@link #DISABLED_MEMORY_LIMIT}, if unset. 1845 * @see #setMaxVirtualMemoryForTask(long) 1846 * @deprecated Use {@link #getMemoryForMapTask()} and 1847 * {@link #getMemoryForReduceTask()} 1848 */ 1849 @Deprecated 1850 public long getMaxVirtualMemoryForTask() { 1851 LOG.warn( 1852 "getMaxVirtualMemoryForTask() is deprecated. " + 1853 "Instead use getMemoryForMapTask() and getMemoryForReduceTask()"); 1854 1855 long value = getLong(MAPRED_TASK_MAXVMEM_PROPERTY, DISABLED_MEMORY_LIMIT); 1856 value = normalizeMemoryConfigValue(value); 1857 if (value == DISABLED_MEMORY_LIMIT) { 1858 value = Math.max(getMemoryForMapTask(), getMemoryForReduceTask()); 1859 value = normalizeMemoryConfigValue(value); 1860 if (value != DISABLED_MEMORY_LIMIT) { 1861 value *= 1024*1024; 1862 } 1863 } 1864 return value; 1865 } 1866 1867 /** 1868 * Set the maximum amount of memory any task of this job can use. See 1869 * {@link #MAPRED_TASK_MAXVMEM_PROPERTY} 1870 * <p/> 1871 * mapred.task.maxvmem is split into 1872 * mapreduce.map.memory.mb 1873 * and mapreduce.map.memory.mb,mapred 1874 * each of the new key are set 1875 * as mapred.task.maxvmem / 1024 1876 * as new values are in MB 1877 * 1878 * @param vmem Maximum amount of virtual memory in bytes any task of this job 1879 * can use. 1880 * @see #getMaxVirtualMemoryForTask() 1881 * @deprecated 1882 * Use {@link #setMemoryForMapTask(long mem)} and 1883 * Use {@link #setMemoryForReduceTask(long mem)} 1884 */ 1885 @Deprecated 1886 public void setMaxVirtualMemoryForTask(long vmem) { 1887 LOG.warn("setMaxVirtualMemoryForTask() is deprecated."+ 1888 "Instead use setMemoryForMapTask() and setMemoryForReduceTask()"); 1889 if(vmem != DISABLED_MEMORY_LIMIT && vmem < 0) { 1890 setMemoryForMapTask(DISABLED_MEMORY_LIMIT); 1891 setMemoryForReduceTask(DISABLED_MEMORY_LIMIT); 1892 } 1893 1894 if(get(JobConf.MAPRED_TASK_MAXVMEM_PROPERTY) == null) { 1895 setMemoryForMapTask(vmem / (1024 * 1024)); //Changing bytes to mb 1896 setMemoryForReduceTask(vmem / (1024 * 1024));//Changing bytes to mb 1897 }else{ 1898 this.setLong(JobConf.MAPRED_TASK_MAXVMEM_PROPERTY,vmem); 1899 } 1900 } 1901 1902 /** 1903 * @deprecated this variable is deprecated and nolonger in use. 1904 */ 1905 @Deprecated 1906 public long getMaxPhysicalMemoryForTask() { 1907 LOG.warn("The API getMaxPhysicalMemoryForTask() is deprecated." 1908 + " Refer to the APIs getMemoryForMapTask() and" 1909 + " getMemoryForReduceTask() for details."); 1910 return -1; 1911 } 1912 1913 /* 1914 * @deprecated this 1915 */ 1916 @Deprecated 1917 public void setMaxPhysicalMemoryForTask(long mem) { 1918 LOG.warn("The API setMaxPhysicalMemoryForTask() is deprecated." 1919 + " The value set is ignored. Refer to " 1920 + " setMemoryForMapTask() and setMemoryForReduceTask() for details."); 1921 } 1922 1923 static String deprecatedString(String key) { 1924 return "The variable " + key + " is no longer used."; 1925 } 1926 1927 private void checkAndWarnDeprecation() { 1928 if(get(JobConf.MAPRED_TASK_MAXVMEM_PROPERTY) != null) { 1929 LOG.warn(JobConf.deprecatedString(JobConf.MAPRED_TASK_MAXVMEM_PROPERTY) 1930 + " Instead use " + JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY 1931 + " and " + JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY); 1932 } 1933 if(get(JobConf.MAPRED_TASK_ULIMIT) != null ) { 1934 LOG.warn(JobConf.deprecatedString(JobConf.MAPRED_TASK_ULIMIT)); 1935 } 1936 if(get(JobConf.MAPRED_MAP_TASK_ULIMIT) != null ) { 1937 LOG.warn(JobConf.deprecatedString(JobConf.MAPRED_MAP_TASK_ULIMIT)); 1938 } 1939 if(get(JobConf.MAPRED_REDUCE_TASK_ULIMIT) != null ) { 1940 LOG.warn(JobConf.deprecatedString(JobConf.MAPRED_REDUCE_TASK_ULIMIT)); 1941 } 1942 } 1943 1944 1945 } 1946