org.apache.hadoop.mapred
Class JobConf

java.lang.Object
  extended by org.apache.hadoop.conf.Configuration
      extended by org.apache.hadoop.mapred.JobConf
All Implemented Interfaces:
Iterable<Map.Entry<String,String>>, Writable

public class JobConf
extends Configuration

A map/reduce job configuration.

JobConf is the primary interface for a user to describe a map-reduce job to the Hadoop framework for execution. The framework tries to faithfully execute the job as-is described by JobConf, however:

  1. Some configuration parameters might have been marked as final by administrators and hence cannot be altered.
  2. While some job parameters are straight-forward to set (e.g. setNumReduceTasks(int)), some parameters interact subtly rest of the framework and/or job-configuration and is relatively more complex for the user to control finely (e.g. setNumMapTasks(int)).

JobConf typically specifies the Mapper, combiner (if any), Partitioner, Reducer, InputFormat and OutputFormat implementations to be used etc.

Optionally JobConf is used to specify other advanced facets of the job such as Comparators to be used, files to be put in the DistributedCache, whether or not intermediate and/or job outputs are to be compressed (and how), debugability via user-provided scripts ( setMapDebugScript(String)/setReduceDebugScript(String)), for doing post-processing on task logs, task's stdout, stderr, syslog. and etc.

Here is an example on how to configure a job via JobConf:

     // Create a new JobConf
     JobConf job = new JobConf(new Configuration(), MyJob.class);
     
     // Specify various job-specific parameters     
     job.setJobName("myjob");
     
     FileInputFormat.setInputPaths(job, new Path("in"));
     FileOutputFormat.setOutputPath(job, new Path("out"));
     
     job.setMapperClass(MyJob.MyMapper.class);
     job.setCombinerClass(MyJob.MyReducer.class);
     job.setReducerClass(MyJob.MyReducer.class);
     
     job.setInputFormat(SequenceFileInputFormat.class);
     job.setOutputFormat(SequenceFileOutputFormat.class);
 

See Also:
JobClient, ClusterStatus, Tool, DistributedCache

Nested Class Summary
 
Nested classes/interfaces inherited from class org.apache.hadoop.conf.Configuration
Configuration.IntegerRanges
 
Field Summary
static String DEFAULT_MAPRED_TASK_JAVA_OPTS
           
static boolean DEFAULT_MAPREDUCE_RECOVER_JOB
           
static String DEFAULT_QUEUE_NAME
          Name of the queue to which jobs will be submitted, if no queue name is mentioned.
static long DISABLED_MEMORY_LIMIT
          A value which if set for memory related configuration options, indicates that the options are turned off.
static String MAPRED_JOB_MAP_MEMORY_MB_PROPERTY
           
static String MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY
           
static String MAPRED_LOCAL_DIR_PROPERTY
          Property name for the configuration property mapred.local.dir
static String MAPRED_MAP_TASK_ENV
          Configuration key to set the maximum virutal memory available to the map tasks.
static String MAPRED_MAP_TASK_JAVA_OPTS
          Configuration key to set the java command line options for the map tasks.
static String MAPRED_MAP_TASK_ULIMIT
          Configuration key to set the maximum virutal memory available to the map tasks (in kilo-bytes).
static String MAPRED_REDUCE_TASK_ENV
          Configuration key to set the maximum virutal memory available to the reduce tasks.
static String MAPRED_REDUCE_TASK_JAVA_OPTS
          Configuration key to set the java command line options for the reduce tasks.
static String MAPRED_REDUCE_TASK_ULIMIT
          Configuration key to set the maximum virutal memory available to the reduce tasks (in kilo-bytes).
static String MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY
          Deprecated.  
static String MAPRED_TASK_ENV
          Deprecated. Use MAPRED_MAP_TASK_ENV or MAPRED_REDUCE_TASK_ENV
static String MAPRED_TASK_JAVA_OPTS
          Deprecated. Use MAPRED_MAP_TASK_JAVA_OPTS or MAPRED_REDUCE_TASK_JAVA_OPTS
static String MAPRED_TASK_MAXPMEM_PROPERTY
          Deprecated.  
static String MAPRED_TASK_MAXVMEM_PROPERTY
          Deprecated. Use MAPRED_JOB_MAP_MEMORY_MB_PROPERTY and MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY
static String MAPRED_TASK_ULIMIT
          Deprecated. Use MAPRED_MAP_TASK_ULIMIT or MAPRED_REDUCE_TASK_ULIMIT
static String MAPREDUCE_RECOVER_JOB
           
static String UPPER_LIMIT_ON_TASK_VMEM_PROPERTY
          Deprecated.  
static String WORKFLOW_ADJACENCY_PREFIX_PATTERN
           
static String WORKFLOW_ADJACENCY_PREFIX_STRING
           
static String WORKFLOW_ID
           
static String WORKFLOW_NAME
           
static String WORKFLOW_NODE_NAME
           
static String WORKFLOW_TAGS
           
 
Constructor Summary
JobConf()
          Construct a map/reduce job configuration.
JobConf(boolean loadDefaults)
          A new map/reduce configuration where the behavior of reading from the default resources can be turned off.
JobConf(Class exampleClass)
          Construct a map/reduce job configuration.
JobConf(Configuration conf)
          Construct a map/reduce job configuration.
JobConf(Configuration conf, Class exampleClass)
          Construct a map/reduce job configuration.
JobConf(Path config)
          Construct a map/reduce configuration.
JobConf(String config)
          Construct a map/reduce configuration.
 
Method Summary
 void deleteLocalFiles()
           
 void deleteLocalFiles(String subdir)
           
 Class<? extends Reducer> getCombinerClass()
          Get the user-defined combiner class used to combine map-outputs before being sent to the reducers.
 boolean getCompressMapOutput()
          Are the outputs of the maps be compressed?
 Credentials getCredentials()
          Get credentials for the job.
 InputFormat getInputFormat()
          Get the InputFormat implementation for the map-reduce job, defaults to TextInputFormat if not specified explicity.
 String getJar()
          Get the user jar for the map-reduce job.
 String getJobEndNotificationURI()
          Get the uri to be invoked in-order to send a notification after the job has completed (success/failure).
 String getJobLocalDir()
          Get job-specific shared directory for use as scratch space
 String getJobName()
          Get the user-specified job name.
 JobPriority getJobPriority()
          Get the JobPriority for this job.
 boolean getKeepFailedTaskFiles()
          Should the temporary files for failed tasks be kept?
 String getKeepTaskFilesPattern()
          Get the regular expression that is matched against the task names to see if we need to keep the files.
 String getKeyFieldComparatorOption()
          Get the KeyFieldBasedComparator options
 String getKeyFieldPartitionerOption()
          Get the KeyFieldBasedPartitioner options
 String[] getLocalDirs()
           
 Path getLocalPath(String pathString)
          Constructs a local file name.
 String getMapDebugScript()
          Get the map task's debug script.
 Class<? extends CompressionCodec> getMapOutputCompressorClass(Class<? extends CompressionCodec> defaultValue)
          Get the CompressionCodec for compressing the map outputs.
 Class<?> getMapOutputKeyClass()
          Get the key class for the map output data.
 Class<?> getMapOutputValueClass()
          Get the value class for the map output data.
 Class<? extends Mapper> getMapperClass()
          Get the Mapper class for the job.
 Class<? extends MapRunnable> getMapRunnerClass()
          Get the MapRunnable class for the job.
 boolean getMapSpeculativeExecution()
          Should speculative execution be used for this job for map tasks? Defaults to true.
 int getMaxMapAttempts()
          Get the configured number of maximum attempts that will be made to run a map task, as specified by the mapred.map.max.attempts property.
 int getMaxMapTaskFailuresPercent()
          Get the maximum percentage of map tasks that can fail without the job being aborted.
 long getMaxPhysicalMemoryForTask()
          Deprecated. this variable is deprecated and nolonger in use.
 int getMaxReduceAttempts()
          Get the configured number of maximum attempts that will be made to run a reduce task, as specified by the mapred.reduce.max.attempts property.
 int getMaxReduceTaskFailuresPercent()
          Get the maximum percentage of reduce tasks that can fail without the job being aborted.
 int getMaxTaskFailuresPerTracker()
          Expert: Get the maximum no.
 long getMaxVirtualMemoryForTask()
          Deprecated. Use getMemoryForMapTask() and getMemoryForReduceTask()
 long getMemoryForMapTask()
          Get memory required to run a map task of the job, in MB.
 long getMemoryForReduceTask()
          Get memory required to run a reduce task of the job, in MB.
 int getNumMapTasks()
          Get configured the number of reduce tasks for this job.
 int getNumReduceTasks()
          Get configured the number of reduce tasks for this job.
 int getNumTasksToExecutePerJvm()
          Get the number of tasks that a spawned JVM should execute
 OutputCommitter getOutputCommitter()
          Get the OutputCommitter implementation for the map-reduce job, defaults to FileOutputCommitter if not specified explicitly.
 OutputFormat getOutputFormat()
          Get the OutputFormat implementation for the map-reduce job, defaults to TextOutputFormat if not specified explicity.
 Class<?> getOutputKeyClass()
          Get the key class for the job output data.
 RawComparator getOutputKeyComparator()
          Get the RawComparator comparator used to compare keys.
 Class<?> getOutputValueClass()
          Get the value class for job outputs.
 RawComparator getOutputValueGroupingComparator()
          Get the user defined WritableComparable comparator for grouping keys of inputs to the reduce.
 Class<? extends Partitioner> getPartitionerClass()
          Get the Partitioner used to partition Mapper-outputs to be sent to the Reducers.
 boolean getProfileEnabled()
          Get whether the task profiling is enabled.
 String getProfileParams()
          Get the profiler configuration arguments.
 Configuration.IntegerRanges getProfileTaskRange(boolean isMap)
          Get the range of maps or reduces to profile.
 String getQueueName()
          Return the name of the queue to which this job is submitted.
 String getReduceDebugScript()
          Get the reduce task's debug Script
 Class<? extends Reducer> getReducerClass()
          Get the Reducer class for the job.
 boolean getReduceSpeculativeExecution()
          Should speculative execution be used for this job for reduce tasks? Defaults to true.
 String getSessionId()
          Get the user-specified session identifier.
 boolean getSpeculativeExecution()
          Should speculative execution be used for this job? Defaults to true.
 boolean getUseNewMapper()
          Should the framework use the new context-object code for running the mapper?
 boolean getUseNewReducer()
          Should the framework use the new context-object code for running the reducer?
 String getUser()
          Get the reported username for this job.
 Path getWorkingDirectory()
          Get the current working directory for the default file system.
static long normalizeMemoryConfigValue(long val)
          Normalize the negative values in configuration
 void setCombinerClass(Class<? extends Reducer> theClass)
          Set the user-defined combiner class used to combine map-outputs before being sent to the reducers.
 void setCompressMapOutput(boolean compress)
          Should the map outputs be compressed before transfer? Uses the SequenceFile compression.
 void setInputFormat(Class<? extends InputFormat> theClass)
          Set the InputFormat implementation for the map-reduce job.
 void setJar(String jar)
          Set the user jar for the map-reduce job.
 void setJarByClass(Class cls)
          Set the job's jar file by finding an example class location.
 void setJobEndNotificationURI(String uri)
          Set the uri to be invoked in-order to send a notification after the job has completed (success/failure).
 void setJobName(String name)
          Set the user-specified job name.
 void setJobPriority(JobPriority prio)
          Set JobPriority for this job.
 void setKeepFailedTaskFiles(boolean keep)
          Set whether the framework should keep the intermediate files for failed tasks.
 void setKeepTaskFilesPattern(String pattern)
          Set a regular expression for task names that should be kept.
 void setKeyFieldComparatorOptions(String keySpec)
          Set the KeyFieldBasedComparator options used to compare keys.
 void setKeyFieldPartitionerOptions(String keySpec)
          Set the KeyFieldBasedPartitioner options used for Partitioner
 void setMapDebugScript(String mDbgScript)
          Set the debug script to run when the map tasks fail.
 void setMapOutputCompressorClass(Class<? extends CompressionCodec> codecClass)
          Set the given class as the CompressionCodec for the map outputs.
 void setMapOutputKeyClass(Class<?> theClass)
          Set the key class for the map output data.
 void setMapOutputValueClass(Class<?> theClass)
          Set the value class for the map output data.
 void setMapperClass(Class<? extends Mapper> theClass)
          Set the Mapper class for the job.
 void setMapRunnerClass(Class<? extends MapRunnable> theClass)
          Expert: Set the MapRunnable class for the job.
 void setMapSpeculativeExecution(boolean speculativeExecution)
          Turn speculative execution on or off for this job for map tasks.
 void setMaxMapAttempts(int n)
          Expert: Set the number of maximum attempts that will be made to run a map task.
 void setMaxMapTaskFailuresPercent(int percent)
          Expert: Set the maximum percentage of map tasks that can fail without the job being aborted.
 void setMaxPhysicalMemoryForTask(long mem)
          Deprecated. 
 void setMaxReduceAttempts(int n)
          Expert: Set the number of maximum attempts that will be made to run a reduce task.
 void setMaxReduceTaskFailuresPercent(int percent)
          Set the maximum percentage of reduce tasks that can fail without the job being aborted.
 void setMaxTaskFailuresPerTracker(int noFailures)
          Set the maximum no.
 void setMaxVirtualMemoryForTask(long vmem)
          Deprecated. Use setMemoryForMapTask(long mem) and Use setMemoryForReduceTask(long mem)
 void setMemoryForMapTask(long mem)
           
 void setMemoryForReduceTask(long mem)
           
 void setNumMapTasks(int n)
          Set the number of map tasks for this job.
 void setNumReduceTasks(int n)
          Set the requisite number of reduce tasks for this job.
 void setNumTasksToExecutePerJvm(int numTasks)
          Sets the number of tasks that a spawned task JVM should run before it exits
 void setOutputCommitter(Class<? extends OutputCommitter> theClass)
          Set the OutputCommitter implementation for the map-reduce job.
 void setOutputFormat(Class<? extends OutputFormat> theClass)
          Set the OutputFormat implementation for the map-reduce job.
 void setOutputKeyClass(Class<?> theClass)
          Set the key class for the job output data.
 void setOutputKeyComparatorClass(Class<? extends RawComparator> theClass)
          Set the RawComparator comparator used to compare keys.
 void setOutputValueClass(Class<?> theClass)
          Set the value class for job outputs.
 void setOutputValueGroupingComparator(Class<? extends RawComparator> theClass)
          Set the user defined RawComparator comparator for grouping keys in the input to the reduce.
 void setPartitionerClass(Class<? extends Partitioner> theClass)
          Set the Partitioner class used to partition Mapper-outputs to be sent to the Reducers.
 void setProfileEnabled(boolean newValue)
          Set whether the system should collect profiler information for some of the tasks in this job? The information is stored in the user log directory.
 void setProfileParams(String value)
          Set the profiler configuration arguments.
 void setProfileTaskRange(boolean isMap, String newValue)
          Set the ranges of maps or reduces to profile.
 void setQueueName(String queueName)
          Set the name of the queue to which this job should be submitted.
 void setReduceDebugScript(String rDbgScript)
          Set the debug script to run when the reduce tasks fail.
 void setReducerClass(Class<? extends Reducer> theClass)
          Set the Reducer class for the job.
 void setReduceSpeculativeExecution(boolean speculativeExecution)
          Turn speculative execution on or off for this job for reduce tasks.
 void setSessionId(String sessionId)
          Set the user-specified session identifier.
 void setSpeculativeExecution(boolean speculativeExecution)
          Turn speculative execution on or off for this job.
 void setUseNewMapper(boolean flag)
          Set whether the framework should use the new api for the mapper.
 void setUseNewReducer(boolean flag)
          Set whether the framework should use the new api for the reducer.
 void setUser(String user)
          Set the reported username for this job.
 void setWorkingDirectory(Path dir)
          Set the current working directory for the default file system.
 
Methods inherited from class org.apache.hadoop.conf.Configuration
addDefaultResource, addResource, addResource, addResource, addResource, clear, dumpConfiguration, get, get, getBoolean, getClass, getClass, getClassByName, getClasses, getClassLoader, getConfResourceAsInputStream, getConfResourceAsReader, getEnum, getFile, getFloat, getInstances, getInt, getLocalPath, getLong, getRange, getRaw, getResource, getStringCollection, getStrings, getStrings, getValByRegex, iterator, main, readFields, reloadConfiguration, set, setBoolean, setBooleanIfUnset, setClass, setClassLoader, setEnum, setFloat, setIfUnset, setInt, setLong, setQuietMode, setStrings, size, toString, unset, write, writeXml, writeXml
 
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
 

Field Detail

MAPRED_TASK_MAXVMEM_PROPERTY

@Deprecated
public static final String MAPRED_TASK_MAXVMEM_PROPERTY
Deprecated. Use MAPRED_JOB_MAP_MEMORY_MB_PROPERTY and MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY
See Also:
Constant Field Values

UPPER_LIMIT_ON_TASK_VMEM_PROPERTY

@Deprecated
public static final String UPPER_LIMIT_ON_TASK_VMEM_PROPERTY
Deprecated. 
See Also:
Constant Field Values

MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY

@Deprecated
public static final String MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY
Deprecated. 
See Also:
Constant Field Values

MAPRED_TASK_MAXPMEM_PROPERTY

@Deprecated
public static final String MAPRED_TASK_MAXPMEM_PROPERTY
Deprecated. 
See Also:
Constant Field Values

DISABLED_MEMORY_LIMIT

public static final long DISABLED_MEMORY_LIMIT
A value which if set for memory related configuration options, indicates that the options are turned off.

See Also:
Constant Field Values

MAPRED_LOCAL_DIR_PROPERTY

public static final String MAPRED_LOCAL_DIR_PROPERTY
Property name for the configuration property mapred.local.dir

See Also:
Constant Field Values

DEFAULT_QUEUE_NAME

public static final String DEFAULT_QUEUE_NAME
Name of the queue to which jobs will be submitted, if no queue name is mentioned.

See Also:
Constant Field Values

MAPRED_JOB_MAP_MEMORY_MB_PROPERTY

public static final String MAPRED_JOB_MAP_MEMORY_MB_PROPERTY
See Also:
Constant Field Values

MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY

public static final String MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY
See Also:
Constant Field Values

MAPRED_TASK_JAVA_OPTS

@Deprecated
public static final String MAPRED_TASK_JAVA_OPTS
Deprecated. Use MAPRED_MAP_TASK_JAVA_OPTS or MAPRED_REDUCE_TASK_JAVA_OPTS
Configuration key to set the java command line options for the child map and reduce tasks. Java opts for the task tracker child processes. The following symbol, if present, will be interpolated: @taskid@. It is replaced by current TaskID. Any other occurrences of '@' will go unchanged. For example, to enable verbose gc logging to a file named for the taskid in /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of: -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc The configuration variable MAPRED_TASK_ULIMIT can be used to control the maximum virtual memory of the child processes. The configuration variable MAPRED_TASK_ENV can be used to pass other environment variables to the child processes.

See Also:
Constant Field Values

MAPRED_MAP_TASK_JAVA_OPTS

public static final String MAPRED_MAP_TASK_JAVA_OPTS
Configuration key to set the java command line options for the map tasks. Java opts for the task tracker child map processes. The following symbol, if present, will be interpolated: @taskid@. It is replaced by current TaskID. Any other occurrences of '@' will go unchanged. For example, to enable verbose gc logging to a file named for the taskid in /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of: -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc The configuration variable MAPRED_MAP_TASK_ULIMIT can be used to control the maximum virtual memory of the map processes. The configuration variable MAPRED_MAP_TASK_ENV can be used to pass other environment variables to the map processes.

See Also:
Constant Field Values

MAPRED_REDUCE_TASK_JAVA_OPTS

public static final String MAPRED_REDUCE_TASK_JAVA_OPTS
Configuration key to set the java command line options for the reduce tasks. Java opts for the task tracker child reduce processes. The following symbol, if present, will be interpolated: @taskid@. It is replaced by current TaskID. Any other occurrences of '@' will go unchanged. For example, to enable verbose gc logging to a file named for the taskid in /tmp and to set the heap maximum to be a gigabyte, pass a 'value' of: -Xmx1024m -verbose:gc -Xloggc:/tmp/@taskid@.gc The configuration variable MAPRED_REDUCE_TASK_ULIMIT can be used to control the maximum virtual memory of the reduce processes. The configuration variable MAPRED_REDUCE_TASK_ENV can be used to pass process environment variables to the reduce processes.

See Also:
Constant Field Values

DEFAULT_MAPRED_TASK_JAVA_OPTS

public static final String DEFAULT_MAPRED_TASK_JAVA_OPTS
See Also:
Constant Field Values

MAPRED_TASK_ULIMIT

@Deprecated
public static final String MAPRED_TASK_ULIMIT
Deprecated. Use MAPRED_MAP_TASK_ULIMIT or MAPRED_REDUCE_TASK_ULIMIT
Configuration key to set the maximum virutal memory available to the child map and reduce tasks (in kilo-bytes). Note: This must be greater than or equal to the -Xmx passed to the JavaVM via MAPRED_TASK_JAVA_OPTS, else the VM might not start.

See Also:
Constant Field Values

MAPRED_MAP_TASK_ULIMIT

public static final String MAPRED_MAP_TASK_ULIMIT
Configuration key to set the maximum virutal memory available to the map tasks (in kilo-bytes). Note: This must be greater than or equal to the -Xmx passed to the JavaVM via MAPRED_MAP_TASK_JAVA_OPTS, else the VM might not start.

See Also:
Constant Field Values

MAPRED_REDUCE_TASK_ULIMIT

public static final String MAPRED_REDUCE_TASK_ULIMIT
Configuration key to set the maximum virutal memory available to the reduce tasks (in kilo-bytes). Note: This must be greater than or equal to the -Xmx passed to the JavaVM via MAPRED_REDUCE_TASK_JAVA_OPTS, else the VM might not start.

See Also:
Constant Field Values

MAPRED_TASK_ENV

@Deprecated
public static final String MAPRED_TASK_ENV
Deprecated. Use MAPRED_MAP_TASK_ENV or MAPRED_REDUCE_TASK_ENV
Configuration key to set the environment of the child map/reduce tasks. The format of the value is k1=v1,k2=v2. Further it can reference existing environment variables via $key. Example:

See Also:
Constant Field Values

MAPRED_MAP_TASK_ENV

public static final String MAPRED_MAP_TASK_ENV
Configuration key to set the maximum virutal memory available to the map tasks. The format of the value is k1=v1,k2=v2. Further it can reference existing environment variables via $key. Example:

See Also:
Constant Field Values

MAPRED_REDUCE_TASK_ENV

public static final String MAPRED_REDUCE_TASK_ENV
Configuration key to set the maximum virutal memory available to the reduce tasks. The format of the value is k1=v1,k2=v2. Further it can reference existing environment variables via $key. Example:

See Also:
Constant Field Values

WORKFLOW_ID

public static final String WORKFLOW_ID
See Also:
Constant Field Values

WORKFLOW_NAME

public static final String WORKFLOW_NAME
See Also:
Constant Field Values

WORKFLOW_NODE_NAME

public static final String WORKFLOW_NODE_NAME
See Also:
Constant Field Values

WORKFLOW_ADJACENCY_PREFIX_STRING

public static final String WORKFLOW_ADJACENCY_PREFIX_STRING
See Also:
Constant Field Values

WORKFLOW_ADJACENCY_PREFIX_PATTERN

public static final String WORKFLOW_ADJACENCY_PREFIX_PATTERN
See Also:
Constant Field Values

WORKFLOW_TAGS

public static final String WORKFLOW_TAGS
See Also:
Constant Field Values

MAPREDUCE_RECOVER_JOB

public static final String MAPREDUCE_RECOVER_JOB
See Also:
Constant Field Values

DEFAULT_MAPREDUCE_RECOVER_JOB

public static final boolean DEFAULT_MAPREDUCE_RECOVER_JOB
See Also:
Constant Field Values
Constructor Detail

JobConf

public JobConf()
Construct a map/reduce job configuration.


JobConf

public JobConf(Class exampleClass)
Construct a map/reduce job configuration.

Parameters:
exampleClass - a class whose containing jar is used as the job's jar.

JobConf

public JobConf(Configuration conf)
Construct a map/reduce job configuration.

Parameters:
conf - a Configuration whose settings will be inherited.

JobConf

public JobConf(Configuration conf,
               Class exampleClass)
Construct a map/reduce job configuration.

Parameters:
conf - a Configuration whose settings will be inherited.
exampleClass - a class whose containing jar is used as the job's jar.

JobConf

public JobConf(String config)
Construct a map/reduce configuration.

Parameters:
config - a Configuration-format XML job description file.

JobConf

public JobConf(Path config)
Construct a map/reduce configuration.

Parameters:
config - a Configuration-format XML job description file.

JobConf

public JobConf(boolean loadDefaults)
A new map/reduce configuration where the behavior of reading from the default resources can be turned off.

If the parameter loadDefaults is false, the new instance will not load resources from the default files.

Parameters:
loadDefaults - specifies whether to load from the default files
Method Detail

getCredentials

public Credentials getCredentials()
Get credentials for the job.

Returns:
credentials for the job

getJar

public String getJar()
Get the user jar for the map-reduce job.

Returns:
the user jar for the map-reduce job.

setJar

public void setJar(String jar)
Set the user jar for the map-reduce job.

Parameters:
jar - the user jar for the map-reduce job.

setJarByClass

public void setJarByClass(Class cls)
Set the job's jar file by finding an example class location.

Parameters:
cls - the example class.

getLocalDirs

public String[] getLocalDirs()
                      throws IOException
Throws:
IOException

deleteLocalFiles

public void deleteLocalFiles()
                      throws IOException
Throws:
IOException

deleteLocalFiles

public void deleteLocalFiles(String subdir)
                      throws IOException
Throws:
IOException

getLocalPath

public Path getLocalPath(String pathString)
                  throws IOException
Constructs a local file name. Files are distributed among configured local directories.

Throws:
IOException

getUser

public String getUser()
Get the reported username for this job.

Returns:
the username

setUser

public void setUser(String user)
Set the reported username for this job.

Parameters:
user - the username for this job.

setKeepFailedTaskFiles

public void setKeepFailedTaskFiles(boolean keep)
Set whether the framework should keep the intermediate files for failed tasks.

Parameters:
keep - true if framework should keep the intermediate files for failed tasks, false otherwise.

getKeepFailedTaskFiles

public boolean getKeepFailedTaskFiles()
Should the temporary files for failed tasks be kept?

Returns:
should the files be kept?

setKeepTaskFilesPattern

public void setKeepTaskFilesPattern(String pattern)
Set a regular expression for task names that should be kept. The regular expression ".*_m_000123_0" would keep the files for the first instance of map 123 that ran.

Parameters:
pattern - the java.util.regex.Pattern to match against the task names.

getKeepTaskFilesPattern

public String getKeepTaskFilesPattern()
Get the regular expression that is matched against the task names to see if we need to keep the files.

Returns:
the pattern as a string, if it was set, othewise null.

setWorkingDirectory

public void setWorkingDirectory(Path dir)
Set the current working directory for the default file system.

Parameters:
dir - the new current working directory.

getWorkingDirectory

public Path getWorkingDirectory()
Get the current working directory for the default file system.

Returns:
the directory name.

setNumTasksToExecutePerJvm

public void setNumTasksToExecutePerJvm(int numTasks)
Sets the number of tasks that a spawned task JVM should run before it exits

Parameters:
numTasks - the number of tasks to execute; defaults to 1; -1 signifies no limit

getNumTasksToExecutePerJvm

public int getNumTasksToExecutePerJvm()
Get the number of tasks that a spawned JVM should execute


getInputFormat

public InputFormat getInputFormat()
Get the InputFormat implementation for the map-reduce job, defaults to TextInputFormat if not specified explicity.

Returns:
the InputFormat implementation for the map-reduce job.

setInputFormat

public void setInputFormat(Class<? extends InputFormat> theClass)
Set the InputFormat implementation for the map-reduce job.

Parameters:
theClass - the InputFormat implementation for the map-reduce job.

getOutputFormat

public OutputFormat getOutputFormat()
Get the OutputFormat implementation for the map-reduce job, defaults to TextOutputFormat if not specified explicity.

Returns:
the OutputFormat implementation for the map-reduce job.

getOutputCommitter

public OutputCommitter getOutputCommitter()
Get the OutputCommitter implementation for the map-reduce job, defaults to FileOutputCommitter if not specified explicitly.

Returns:
the OutputCommitter implementation for the map-reduce job.

setOutputCommitter

public void setOutputCommitter(Class<? extends OutputCommitter> theClass)
Set the OutputCommitter implementation for the map-reduce job.

Parameters:
theClass - the OutputCommitter implementation for the map-reduce job.

setOutputFormat

public void setOutputFormat(Class<? extends OutputFormat> theClass)
Set the OutputFormat implementation for the map-reduce job.

Parameters:
theClass - the OutputFormat implementation for the map-reduce job.

setCompressMapOutput

public void setCompressMapOutput(boolean compress)
Should the map outputs be compressed before transfer? Uses the SequenceFile compression.

Parameters:
compress - should the map outputs be compressed?

getCompressMapOutput

public boolean getCompressMapOutput()
Are the outputs of the maps be compressed?

Returns:
true if the outputs of the maps are to be compressed, false otherwise.

setMapOutputCompressorClass

public void setMapOutputCompressorClass(Class<? extends CompressionCodec> codecClass)
Set the given class as the CompressionCodec for the map outputs.

Parameters:
codecClass - the CompressionCodec class that will compress the map outputs.

getMapOutputCompressorClass

public Class<? extends CompressionCodec> getMapOutputCompressorClass(Class<? extends CompressionCodec> defaultValue)
Get the CompressionCodec for compressing the map outputs.

Parameters:
defaultValue - the CompressionCodec to return if not set
Returns:
the CompressionCodec class that should be used to compress the map outputs.
Throws:
IllegalArgumentException - if the class was specified, but not found

getMapOutputKeyClass

public Class<?> getMapOutputKeyClass()
Get the key class for the map output data. If it is not set, use the (final) output key class. This allows the map output key class to be different than the final output key class.

Returns:
the map output key class.

setMapOutputKeyClass

public void setMapOutputKeyClass(Class<?> theClass)
Set the key class for the map output data. This allows the user to specify the map output key class to be different than the final output value class.

Parameters:
theClass - the map output key class.

getMapOutputValueClass

public Class<?> getMapOutputValueClass()
Get the value class for the map output data. If it is not set, use the (final) output value class This allows the map output value class to be different than the final output value class.

Returns:
the map output value class.

setMapOutputValueClass

public void setMapOutputValueClass(Class<?> theClass)
Set the value class for the map output data. This allows the user to specify the map output value class to be different than the final output value class.

Parameters:
theClass - the map output value class.

getOutputKeyClass

public Class<?> getOutputKeyClass()
Get the key class for the job output data.

Returns:
the key class for the job output data.

setOutputKeyClass

public void setOutputKeyClass(Class<?> theClass)
Set the key class for the job output data.

Parameters:
theClass - the key class for the job output data.

getOutputKeyComparator

public RawComparator getOutputKeyComparator()
Get the RawComparator comparator used to compare keys.

Returns:
the RawComparator comparator used to compare keys.

setOutputKeyComparatorClass

public void setOutputKeyComparatorClass(Class<? extends RawComparator> theClass)
Set the RawComparator comparator used to compare keys.

Parameters:
theClass - the RawComparator comparator used to compare keys.
See Also:
setOutputValueGroupingComparator(Class)

setKeyFieldComparatorOptions

public void setKeyFieldComparatorOptions(String keySpec)
Set the KeyFieldBasedComparator options used to compare keys.

Parameters:
keySpec - the key specification of the form -k pos1[,pos2], where, pos is of the form f[.c][opts], where f is the number of the key field to use, and c is the number of the first character from the beginning of the field. Fields and character posns are numbered starting with 1; a character position of zero in pos2 indicates the field's last character. If '.c' is omitted from pos1, it defaults to 1 (the beginning of the field); if omitted from pos2, it defaults to 0 (the end of the field). opts are ordering options. The supported options are: -n, (Sort numerically) -r, (Reverse the result of comparison)

getKeyFieldComparatorOption

public String getKeyFieldComparatorOption()
Get the KeyFieldBasedComparator options


setKeyFieldPartitionerOptions

public void setKeyFieldPartitionerOptions(String keySpec)
Set the KeyFieldBasedPartitioner options used for Partitioner

Parameters:
keySpec - the key specification of the form -k pos1[,pos2], where, pos is of the form f[.c][opts], where f is the number of the key field to use, and c is the number of the first character from the beginning of the field. Fields and character posns are numbered starting with 1; a character position of zero in pos2 indicates the field's last character. If '.c' is omitted from pos1, it defaults to 1 (the beginning of the field); if omitted from pos2, it defaults to 0 (the end of the field).

getKeyFieldPartitionerOption

public String getKeyFieldPartitionerOption()
Get the KeyFieldBasedPartitioner options


getOutputValueGroupingComparator

public RawComparator getOutputValueGroupingComparator()
Get the user defined WritableComparable comparator for grouping keys of inputs to the reduce.

Returns:
comparator set by the user for grouping values.
See Also:
for details.

setOutputValueGroupingComparator

public void setOutputValueGroupingComparator(Class<? extends RawComparator> theClass)
Set the user defined RawComparator comparator for grouping keys in the input to the reduce.

This comparator should be provided if the equivalence rules for keys for sorting the intermediates are different from those for grouping keys before each call to Reducer.reduce(Object, java.util.Iterator, OutputCollector, Reporter).

For key-value pairs (K1,V1) and (K2,V2), the values (V1, V2) are passed in a single call to the reduce function if K1 and K2 compare as equal.

Since setOutputKeyComparatorClass(Class) can be used to control how keys are sorted, this can be used in conjunction to simulate secondary sort on values.

Note: This is not a guarantee of the reduce sort being stable in any sense. (In any case, with the order of available map-outputs to the reduce being non-deterministic, it wouldn't make that much sense.)

Parameters:
theClass - the comparator class to be used for grouping keys. It should implement RawComparator.
See Also:
setOutputKeyComparatorClass(Class)

getUseNewMapper

public boolean getUseNewMapper()
Should the framework use the new context-object code for running the mapper?

Returns:
true, if the new api should be used

setUseNewMapper

public void setUseNewMapper(boolean flag)
Set whether the framework should use the new api for the mapper. This is the default for jobs submitted with the new Job api.

Parameters:
flag - true, if the new api should be used

getUseNewReducer

public boolean getUseNewReducer()
Should the framework use the new context-object code for running the reducer?

Returns:
true, if the new api should be used

setUseNewReducer

public void setUseNewReducer(boolean flag)
Set whether the framework should use the new api for the reducer. This is the default for jobs submitted with the new Job api.

Parameters:
flag - true, if the new api should be used

getOutputValueClass

public Class<?> getOutputValueClass()
Get the value class for job outputs.

Returns:
the value class for job outputs.

setOutputValueClass

public void setOutputValueClass(Class<?> theClass)
Set the value class for job outputs.

Parameters:
theClass - the value class for job outputs.

getMapperClass

public Class<? extends Mapper> getMapperClass()
Get the Mapper class for the job.

Returns:
the Mapper class for the job.

setMapperClass

public void setMapperClass(Class<? extends Mapper> theClass)
Set the Mapper class for the job.

Parameters:
theClass - the Mapper class for the job.

getMapRunnerClass

public Class<? extends MapRunnable> getMapRunnerClass()
Get the MapRunnable class for the job.

Returns:
the MapRunnable class for the job.

setMapRunnerClass

public void setMapRunnerClass(Class<? extends MapRunnable> theClass)
Expert: Set the MapRunnable class for the job. Typically used to exert greater control on Mappers.

Parameters:
theClass - the MapRunnable class for the job.

getPartitionerClass

public Class<? extends Partitioner> getPartitionerClass()
Get the Partitioner used to partition Mapper-outputs to be sent to the Reducers.

Returns:
the Partitioner used to partition map-outputs.

setPartitionerClass

public void setPartitionerClass(Class<? extends Partitioner> theClass)
Set the Partitioner class used to partition Mapper-outputs to be sent to the Reducers.

Parameters:
theClass - the Partitioner used to partition map-outputs.

getReducerClass

public Class<? extends Reducer> getReducerClass()
Get the Reducer class for the job.

Returns:
the Reducer class for the job.

setReducerClass

public void setReducerClass(Class<? extends Reducer> theClass)
Set the Reducer class for the job.

Parameters:
theClass - the Reducer class for the job.

getCombinerClass

public Class<? extends Reducer> getCombinerClass()
Get the user-defined combiner class used to combine map-outputs before being sent to the reducers. Typically the combiner is same as the the Reducer for the job i.e. getReducerClass().

Returns:
the user-defined combiner class used to combine map-outputs.

setCombinerClass

public void setCombinerClass(Class<? extends Reducer> theClass)
Set the user-defined combiner class used to combine map-outputs before being sent to the reducers.

The combiner is an application-specified aggregation operation, which can help cut down the amount of data transferred between the Mapper and the Reducer, leading to better performance.

The framework may invoke the combiner 0, 1, or multiple times, in both the mapper and reducer tasks. In general, the combiner is called as the sort/merge result is written to disk. The combiner must:

Typically the combiner is same as the Reducer for the job i.e. setReducerClass(Class).

Parameters:
theClass - the user-defined combiner class used to combine map-outputs.

getSpeculativeExecution

public boolean getSpeculativeExecution()
Should speculative execution be used for this job? Defaults to true.

Returns:
true if speculative execution be used for this job, false otherwise.

setSpeculativeExecution

public void setSpeculativeExecution(boolean speculativeExecution)
Turn speculative execution on or off for this job.

Parameters:
speculativeExecution - true if speculative execution should be turned on, else false.

getMapSpeculativeExecution

public boolean getMapSpeculativeExecution()
Should speculative execution be used for this job for map tasks? Defaults to true.

Returns:
true if speculative execution be used for this job for map tasks, false otherwise.

setMapSpeculativeExecution

public void setMapSpeculativeExecution(boolean speculativeExecution)
Turn speculative execution on or off for this job for map tasks.

Parameters:
speculativeExecution - true if speculative execution should be turned on for map tasks, else false.

getReduceSpeculativeExecution

public boolean getReduceSpeculativeExecution()
Should speculative execution be used for this job for reduce tasks? Defaults to true.

Returns:
true if speculative execution be used for reduce tasks for this job, false otherwise.

setReduceSpeculativeExecution

public void setReduceSpeculativeExecution(boolean speculativeExecution)
Turn speculative execution on or off for this job for reduce tasks.

Parameters:
speculativeExecution - true if speculative execution should be turned on for reduce tasks, else false.

getNumMapTasks

public int getNumMapTasks()
Get configured the number of reduce tasks for this job. Defaults to 1.

Returns:
the number of reduce tasks for this job.

setNumMapTasks

public void setNumMapTasks(int n)
Set the number of map tasks for this job.

Note: This is only a hint to the framework. The actual number of spawned map tasks depends on the number of InputSplits generated by the job's InputFormat.getSplits(JobConf, int). A custom InputFormat is typically used to accurately control the number of map tasks for the job.

How many maps?

The number of maps is usually driven by the total size of the inputs i.e. total number of blocks of the input files.

The right level of parallelism for maps seems to be around 10-100 maps per-node, although it has been set up to 300 or so for very cpu-light map tasks. Task setup takes awhile, so it is best if the maps take at least a minute to execute.

The default behavior of file-based InputFormats is to split the input into logical InputSplits based on the total size, in bytes, of input files. However, the FileSystem blocksize of the input files is treated as an upper bound for input splits. A lower bound on the split size can be set via mapred.min.split.size.

Thus, if you expect 10TB of input data and have a blocksize of 128MB, you'll end up with 82,000 maps, unless setNumMapTasks(int) is used to set it even higher.

Parameters:
n - the number of map tasks for this job.
See Also:
InputFormat.getSplits(JobConf, int), FileInputFormat, FileSystem.getDefaultBlockSize(), FileStatus.getBlockSize()

getNumReduceTasks

public int getNumReduceTasks()
Get configured the number of reduce tasks for this job. Defaults to 1.

Returns:
the number of reduce tasks for this job.

setNumReduceTasks

public void setNumReduceTasks(int n)
Set the requisite number of reduce tasks for this job.

How many reduces?

The right number of reduces seems to be 0.95 or 1.75 multiplied by (<no. of nodes> * mapred.tasktracker.reduce.tasks.maximum).

With 0.95 all of the reduces can launch immediately and start transfering map outputs as the maps finish. With 1.75 the faster nodes will finish their first round of reduces and launch a second wave of reduces doing a much better job of load balancing.

Increasing the number of reduces increases the framework overhead, but increases load balancing and lowers the cost of failures.

The scaling factors above are slightly less than whole numbers to reserve a few reduce slots in the framework for speculative-tasks, failures etc.

Reducer NONE

It is legal to set the number of reduce-tasks to zero.

In this case the output of the map-tasks directly go to distributed file-system, to the path set by FileOutputFormat.setOutputPath(JobConf, Path). Also, the framework doesn't sort the map-outputs before writing it out to HDFS.

Parameters:
n - the number of reduce tasks for this job.

getMaxMapAttempts

public int getMaxMapAttempts()
Get the configured number of maximum attempts that will be made to run a map task, as specified by the mapred.map.max.attempts property. If this property is not already set, the default is 4 attempts.

Returns:
the max number of attempts per map task.

setMaxMapAttempts

public void setMaxMapAttempts(int n)
Expert: Set the number of maximum attempts that will be made to run a map task.

Parameters:
n - the number of attempts per map task.

getMaxReduceAttempts

public int getMaxReduceAttempts()
Get the configured number of maximum attempts that will be made to run a reduce task, as specified by the mapred.reduce.max.attempts property. If this property is not already set, the default is 4 attempts.

Returns:
the max number of attempts per reduce task.

setMaxReduceAttempts

public void setMaxReduceAttempts(int n)
Expert: Set the number of maximum attempts that will be made to run a reduce task.

Parameters:
n - the number of attempts per reduce task.

getJobName

public String getJobName()
Get the user-specified job name. This is only used to identify the job to the user.

Returns:
the job's name, defaulting to "".

setJobName

public void setJobName(String name)
Set the user-specified job name.

Parameters:
name - the job's new name.

getSessionId

public String getSessionId()
Get the user-specified session identifier. The default is the empty string. The session identifier is used to tag metric data that is reported to some performance metrics system via the org.apache.hadoop.metrics API. The session identifier is intended, in particular, for use by Hadoop-On-Demand (HOD) which allocates a virtual Hadoop cluster dynamically and transiently. HOD will set the session identifier by modifying the mapred-site.xml file before starting the cluster. When not running under HOD, this identifer is expected to remain set to the empty string.

Returns:
the session identifier, defaulting to "".

setSessionId

public void setSessionId(String sessionId)
Set the user-specified session identifier.

Parameters:
sessionId - the new session id.

setMaxTaskFailuresPerTracker

public void setMaxTaskFailuresPerTracker(int noFailures)
Set the maximum no. of failures of a given job per tasktracker. If the no. of task failures exceeds noFailures, the tasktracker is blacklisted for this job.

Parameters:
noFailures - maximum no. of failures of a given job per tasktracker.

getMaxTaskFailuresPerTracker

public int getMaxTaskFailuresPerTracker()
Expert: Get the maximum no. of failures of a given job per tasktracker. If the no. of task failures exceeds this, the tasktracker is blacklisted for this job.

Returns:
the maximum no. of failures of a given job per tasktracker.

getMaxMapTaskFailuresPercent

public int getMaxMapTaskFailuresPercent()
Get the maximum percentage of map tasks that can fail without the job being aborted. Each map task is executed a minimum of getMaxMapAttempts() attempts before being declared as failed. Defaults to zero, i.e. any failed map-task results in the job being declared as JobStatus.FAILED.

Returns:
the maximum percentage of map tasks that can fail without the job being aborted.

setMaxMapTaskFailuresPercent

public void setMaxMapTaskFailuresPercent(int percent)
Expert: Set the maximum percentage of map tasks that can fail without the job being aborted. Each map task is executed a minimum of getMaxMapAttempts() attempts before being declared as failed.

Parameters:
percent - the maximum percentage of map tasks that can fail without the job being aborted.

getMaxReduceTaskFailuresPercent

public int getMaxReduceTaskFailuresPercent()
Get the maximum percentage of reduce tasks that can fail without the job being aborted. Each reduce task is executed a minimum of getMaxReduceAttempts() attempts before being declared as failed. Defaults to zero, i.e. any failed reduce-task results in the job being declared as JobStatus.FAILED.

Returns:
the maximum percentage of reduce tasks that can fail without the job being aborted.

setMaxReduceTaskFailuresPercent

public void setMaxReduceTaskFailuresPercent(int percent)
Set the maximum percentage of reduce tasks that can fail without the job being aborted. Each reduce task is executed a minimum of getMaxReduceAttempts() attempts before being declared as failed.

Parameters:
percent - the maximum percentage of reduce tasks that can fail without the job being aborted.

setJobPriority

public void setJobPriority(JobPriority prio)
Set JobPriority for this job.

Parameters:
prio - the JobPriority for this job.

getJobPriority

public JobPriority getJobPriority()
Get the JobPriority for this job.

Returns:
the JobPriority for this job.

getProfileEnabled

public boolean getProfileEnabled()
Get whether the task profiling is enabled.

Returns:
true if some tasks will be profiled

setProfileEnabled

public void setProfileEnabled(boolean newValue)
Set whether the system should collect profiler information for some of the tasks in this job? The information is stored in the user log directory.

Parameters:
newValue - true means it should be gathered

getProfileParams

public String getProfileParams()
Get the profiler configuration arguments. The default value for this property is "-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s"

Returns:
the parameters to pass to the task child to configure profiling

setProfileParams

public void setProfileParams(String value)
Set the profiler configuration arguments. If the string contains a '%s' it will be replaced with the name of the profiling output file when the task runs. This value is passed to the task child JVM on the command line.

Parameters:
value - the configuration string

getProfileTaskRange

public Configuration.IntegerRanges getProfileTaskRange(boolean isMap)
Get the range of maps or reduces to profile.

Parameters:
isMap - is the task a map?
Returns:
the task ranges

setProfileTaskRange

public void setProfileTaskRange(boolean isMap,
                                String newValue)
Set the ranges of maps or reduces to profile. setProfileEnabled(true) must also be called.

Parameters:
newValue - a set of integer ranges of the map ids

setMapDebugScript

public void setMapDebugScript(String mDbgScript)
Set the debug script to run when the map tasks fail.

The debug script can aid debugging of failed map tasks. The script is given task's stdout, stderr, syslog, jobconf files as arguments.

The debug command, run on the node where the map failed, is:

$script $stdout $stderr $syslog $jobconf.

The script file is distributed through DistributedCache APIs. The script needs to be symlinked.

Here is an example on how to submit a script

 job.setMapDebugScript("./myscript");
 DistributedCache.createSymlink(job);
 DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
 

Parameters:
mDbgScript - the script name

getMapDebugScript

public String getMapDebugScript()
Get the map task's debug script.

Returns:
the debug Script for the mapred job for failed map tasks.
See Also:
setMapDebugScript(String)

setReduceDebugScript

public void setReduceDebugScript(String rDbgScript)
Set the debug script to run when the reduce tasks fail.

The debug script can aid debugging of failed reduce tasks. The script is given task's stdout, stderr, syslog, jobconf files as arguments.

The debug command, run on the node where the map failed, is:

$script $stdout $stderr $syslog $jobconf.

The script file is distributed through DistributedCache APIs. The script file needs to be symlinked

Here is an example on how to submit a script

 job.setReduceDebugScript("./myscript");
 DistributedCache.createSymlink(job);
 DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
 

Parameters:
rDbgScript - the script name

getReduceDebugScript

public String getReduceDebugScript()
Get the reduce task's debug Script

Returns:
the debug script for the mapred job for failed reduce tasks.
See Also:
setReduceDebugScript(String)

getJobEndNotificationURI

public String getJobEndNotificationURI()
Get the uri to be invoked in-order to send a notification after the job has completed (success/failure).

Returns:
the job end notification uri, null if it hasn't been set.
See Also:
setJobEndNotificationURI(String)

setJobEndNotificationURI

public void setJobEndNotificationURI(String uri)
Set the uri to be invoked in-order to send a notification after the job has completed (success/failure).

The uri can contain 2 special parameters: $jobId and $jobStatus. Those, if present, are replaced by the job's identifier and completion-status respectively.

This is typically used by application-writers to implement chaining of Map-Reduce jobs in an asynchronous manner.

Parameters:
uri - the job end notification uri
See Also:
JobStatus, Job Completion and Chaining

getJobLocalDir

public String getJobLocalDir()
Get job-specific shared directory for use as scratch space

When a job starts, a shared directory is created at location ${mapred.local.dir}/taskTracker/$user/jobcache/$jobid/work/ . This directory is exposed to the users through job.local.dir . So, the tasks can use this space as scratch space and share files among them.

This value is available as System property also.

Returns:
The localized job specific shared directory

getMemoryForMapTask

public long getMemoryForMapTask()
Get memory required to run a map task of the job, in MB. If a value is specified in the configuration, it is returned. Else, it returns DISABLED_MEMORY_LIMIT.

For backward compatibility, if the job configuration sets the key MAPRED_TASK_MAXVMEM_PROPERTY to a value different from DISABLED_MEMORY_LIMIT, that value will be used after converting it from bytes to MB.

Returns:
memory required to run a map task of the job, in MB, or DISABLED_MEMORY_LIMIT if unset.

setMemoryForMapTask

public void setMemoryForMapTask(long mem)

getMemoryForReduceTask

public long getMemoryForReduceTask()
Get memory required to run a reduce task of the job, in MB. If a value is specified in the configuration, it is returned. Else, it returns DISABLED_MEMORY_LIMIT.

For backward compatibility, if the job configuration sets the key MAPRED_TASK_MAXVMEM_PROPERTY to a value different from DISABLED_MEMORY_LIMIT, that value will be used after converting it from bytes to MB.

Returns:
memory required to run a reduce task of the job, in MB, or DISABLED_MEMORY_LIMIT if unset.

setMemoryForReduceTask

public void setMemoryForReduceTask(long mem)

getQueueName

public String getQueueName()
Return the name of the queue to which this job is submitted. Defaults to 'default'.

Returns:
name of the queue

setQueueName

public void setQueueName(String queueName)
Set the name of the queue to which this job should be submitted.

Parameters:
queueName - Name of the queue

normalizeMemoryConfigValue

public static long normalizeMemoryConfigValue(long val)
Normalize the negative values in configuration

Parameters:
val -
Returns:
normalized value

getMaxVirtualMemoryForTask

@Deprecated
public long getMaxVirtualMemoryForTask()
Deprecated. Use getMemoryForMapTask() and getMemoryForReduceTask()

Get the memory required to run a task of this job, in bytes. See MAPRED_TASK_MAXVMEM_PROPERTY

This method is deprecated. Now, different memory limits can be set for map and reduce tasks of a job, in MB.

For backward compatibility, if the job configuration sets the key MAPRED_TASK_MAXVMEM_PROPERTY to a value different from DISABLED_MEMORY_LIMIT, that value is returned. Otherwise, this method will return the larger of the values returned by getMemoryForMapTask() and getMemoryForReduceTask() after converting them into bytes.

Returns:
Memory required to run a task of this job, in bytes, or DISABLED_MEMORY_LIMIT, if unset.
See Also:
setMaxVirtualMemoryForTask(long)

setMaxVirtualMemoryForTask

@Deprecated
public void setMaxVirtualMemoryForTask(long vmem)
Deprecated. Use setMemoryForMapTask(long mem) and Use setMemoryForReduceTask(long mem)

Set the maximum amount of memory any task of this job can use. See MAPRED_TASK_MAXVMEM_PROPERTY

mapred.task.maxvmem is split into mapred.job.map.memory.mb and mapred.job.map.memory.mb,mapred each of the new key are set as mapred.task.maxvmem / 1024 as new values are in MB

Parameters:
vmem - Maximum amount of virtual memory in bytes any task of this job can use.
See Also:
getMaxVirtualMemoryForTask()

getMaxPhysicalMemoryForTask

@Deprecated
public long getMaxPhysicalMemoryForTask()
Deprecated. this variable is deprecated and nolonger in use.


setMaxPhysicalMemoryForTask

@Deprecated
public void setMaxPhysicalMemoryForTask(long mem)
Deprecated. 



Copyright © 2009 The Apache Software Foundation