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