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