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