org.apache.hadoop.mapreduce
Class Job

java.lang.Object
  extended by org.apache.hadoop.mapreduce.task.JobContextImpl
      extended by org.apache.hadoop.mapreduce.Job
All Implemented Interfaces:
JobContext, org.apache.hadoop.mapreduce.MRJobConfig

@InterfaceAudience.Public
@InterfaceStability.Evolving
public class Job
extends org.apache.hadoop.mapreduce.task.JobContextImpl
implements JobContext

The job submitter's view of the Job.

It allows the user to configure the job, submit it, control its execution, and query the state. The set methods only work until the job is submitted, afterwards they will throw an IllegalStateException.

Normally the user creates the application, describes various facets of the job via Job and then submits the job and monitor its progress.

Here is an example on how to submit a job:

     // Create a new Job
     Job job = Job.getInstance();
     job.setJarByClass(MyJob.class);
     
     // Specify various job-specific parameters     
     job.setJobName("myjob");
     
     job.setInputPath(new Path("in"));
     job.setOutputPath(new Path("out"));
     
     job.setMapperClass(MyJob.MyMapper.class);
     job.setReducerClass(MyJob.MyReducer.class);

     // Submit the job, then poll for progress until the job is complete
     job.waitForCompletion(true);
 


Field Summary
static String COMPLETION_POLL_INTERVAL_KEY
          Key in mapred-*.xml that sets completionPollInvervalMillis
static String OUTPUT_FILTER
           
static String PROGRESS_MONITOR_POLL_INTERVAL_KEY
          Key in mapred-*.xml that sets progMonitorPollIntervalMillis
static String SUBMIT_REPLICATION
           
static String USED_GENERIC_PARSER
           
 
Fields inherited from class org.apache.hadoop.mapreduce.task.JobContextImpl
conf, credentials, ugi
 
Fields inherited from interface org.apache.hadoop.mapreduce.MRJobConfig
APPLICATION_ATTEMPT_ID, APPLICATION_MASTER_CLASS, CACHE_ARCHIVES, CACHE_ARCHIVES_SIZES, CACHE_ARCHIVES_TIMESTAMPS, CACHE_ARCHIVES_VISIBILITIES, CACHE_FILE_TIMESTAMPS, CACHE_FILE_VISIBILITIES, CACHE_FILES, CACHE_FILES_SIZES, CACHE_LOCALARCHIVES, CACHE_LOCALFILES, CACHE_SYMLINK, CLASSPATH_ARCHIVES, CLASSPATH_FILES, COMBINE_CLASS_ATTR, COMBINE_RECORDS_BEFORE_PROGRESS, COMBINER_GROUP_COMPARATOR_CLASS, COMPLETED_MAPS_FOR_REDUCE_SLOWSTART, COUNTER_GROUP_NAME_MAX_DEFAULT, COUNTER_GROUP_NAME_MAX_KEY, COUNTER_GROUPS_MAX_DEFAULT, COUNTER_GROUPS_MAX_KEY, COUNTER_NAME_MAX_DEFAULT, COUNTER_NAME_MAX_KEY, COUNTERS_MAX_DEFAULT, COUNTERS_MAX_KEY, DEFAULT_JOB_ACL_MODIFY_JOB, DEFAULT_JOB_ACL_VIEW_JOB, DEFAULT_JOB_AM_ACCESS_DISABLED, DEFAULT_JOB_TOKEN_TRACKING_IDS_ENABLED, DEFAULT_LOG_LEVEL, DEFAULT_MAP_CPU_VCORES, DEFAULT_MAP_MEMORY_MB, DEFAULT_MAPRED_ADMIN_JAVA_OPTS, DEFAULT_MAPRED_ADMIN_USER_ENV, DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH, DEFAULT_MAPREDUCE_CROSS_PLATFORM_APPLICATION_CLASSPATH, DEFAULT_MAPREDUCE_JOB_EMIT_TIMELINE_DATA, DEFAULT_MAX_ALLOWED_FETCH_FAILURES_FRACTION, DEFAULT_MAX_FETCH_FAILURES_NOTIFICATIONS, DEFAULT_MAX_SHUFFLE_FETCH_HOST_FAILURES, DEFAULT_MAX_SHUFFLE_FETCH_RETRY_DELAY, DEFAULT_MR_AM_ADMIN_COMMAND_OPTS, DEFAULT_MR_AM_COMMAND_OPTS, DEFAULT_MR_AM_COMMIT_WINDOW_MS, DEFAULT_MR_AM_COMMITTER_CANCEL_TIMEOUT_MS, DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT, DEFAULT_MR_AM_CPU_VCORES, DEFAULT_MR_AM_HISTORY_COMPLETE_EVENT_FLUSH_TIMEOUT_MS, DEFAULT_MR_AM_HISTORY_JOB_COMPLETE_UNFLUSHED_MULTIPLIER, DEFAULT_MR_AM_HISTORY_MAX_UNFLUSHED_COMPLETE_EVENTS, DEFAULT_MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD, DEFAULT_MR_AM_IGNORE_BLACKLISTING_BLACKLISTED_NODE_PERCENT, DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT, DEFAULT_MR_AM_JOB_REDUCE_PREEMPTION_LIMIT, DEFAULT_MR_AM_JOB_REDUCE_RAMP_UP_LIMIT, DEFAULT_MR_AM_LOG_BACKUPS, DEFAULT_MR_AM_LOG_KB, DEFAULT_MR_AM_LOG_LEVEL, DEFAULT_MR_AM_MAX_ATTEMPTS, DEFAULT_MR_AM_NUM_PROGRESS_SPLITS, DEFAULT_MR_AM_PROFILE, DEFAULT_MR_AM_STAGING_DIR, DEFAULT_MR_AM_TASK_ESTIMATOR_SMOOTH_LAMBDA_MS, DEFAULT_MR_AM_TASK_LISTENER_THREAD_COUNT, DEFAULT_MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS, DEFAULT_MR_AM_TO_RM_WAIT_INTERVAL_MS, DEFAULT_MR_AM_VMEM_MB, DEFAULT_MR_CLIENT_MAX_RETRIES, DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES, DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS, DEFAULT_MR_ENCRYPTED_INTERMEDIATE_DATA, DEFAULT_MR_ENCRYPTED_INTERMEDIATE_DATA_BUFFER_KB, DEFAULT_MR_ENCRYPTED_INTERMEDIATE_DATA_KEY_SIZE_BITS, DEFAULT_MR_JOB_END_NOTIFICATION_TIMEOUT, DEFAULT_MR_JOB_REDUCER_PREEMPT_DELAY_SEC, DEFAULT_REDUCE_CPU_VCORES, DEFAULT_REDUCE_MEMORY_MB, DEFAULT_SHELL, DEFAULT_SHUFFLE_FETCH_RETRY_INTERVAL_MS, DEFAULT_SHUFFLE_INPUT_BUFFER_PERCENT, DEFAULT_SPLIT_METAINFO_MAXSIZE, DEFAULT_TASK_LOG_BACKUPS, DEFAULT_TASK_PROFILE_PARAMS, GROUP_COMPARATOR_CLASS, HADOOP_WORK_DIR, ID, INDEX_CACHE_MEMORY_LIMIT, INPUT_FORMAT_CLASS_ATTR, IO_SORT_FACTOR, IO_SORT_MB, JAR, JAR_UNPACK_PATTERN, JOB_ACL_MODIFY_JOB, JOB_ACL_VIEW_JOB, JOB_AM_ACCESS_DISABLED, JOB_CANCEL_DELEGATION_TOKEN, JOB_CONF_FILE, JOB_JAR, JOB_JOBTRACKER_ID, JOB_LOCAL_DIR, JOB_NAME, JOB_NAMENODES, JOB_SPLIT, JOB_SPLIT_METAINFO, JOB_SUBMIT_DIR, JOB_SUBMITHOST, JOB_SUBMITHOSTADDR, JOB_TAGS, JOB_TOKEN_TRACKING_IDS, JOB_TOKEN_TRACKING_IDS_ENABLED, JOB_UBERTASK_ENABLE, JOB_UBERTASK_MAXBYTES, JOB_UBERTASK_MAXMAPS, JOB_UBERTASK_MAXREDUCES, JVM_NUMTASKS_TORUN, KEY_COMPARATOR, MAP_CLASS_ATTR, MAP_COMBINE_MIN_SPILLS, MAP_CPU_VCORES, MAP_DEBUG_SCRIPT, MAP_ENV, MAP_FAILURES_MAX_PERCENT, MAP_INPUT_FILE, MAP_INPUT_PATH, MAP_INPUT_START, MAP_JAVA_OPTS, MAP_LOG_LEVEL, MAP_MAX_ATTEMPTS, MAP_MEMORY_MB, MAP_OUTPUT_COLLECTOR_CLASS_ATTR, MAP_OUTPUT_COMPRESS, MAP_OUTPUT_COMPRESS_CODEC, MAP_OUTPUT_KEY_CLASS, MAP_OUTPUT_KEY_FIELD_SEPERATOR, MAP_OUTPUT_VALUE_CLASS, MAP_SKIP_INCR_PROC_COUNT, MAP_SKIP_MAX_RECORDS, MAP_SORT_SPILL_PERCENT, MAP_SPECULATIVE, MAPRED_ADMIN_USER_ENV, MAPRED_ADMIN_USER_SHELL, MAPRED_MAP_ADMIN_JAVA_OPTS, MAPRED_REDUCE_ADMIN_JAVA_OPTS, MAPREDUCE_APPLICATION_CLASSPATH, MAPREDUCE_APPLICATION_FRAMEWORK_PATH, MAPREDUCE_JOB_CLASSLOADER, MAPREDUCE_JOB_CLASSLOADER_SYSTEM_CLASSES, MAPREDUCE_JOB_CREDENTIALS_BINARY, MAPREDUCE_JOB_DIR, MAPREDUCE_JOB_EMIT_TIMELINE_DATA, MAPREDUCE_JOB_LOG4J_PROPERTIES_FILE, MAPREDUCE_JOB_SHUFFLE_PROVIDER_SERVICES, MAPREDUCE_JOB_USER_CLASSPATH_FIRST, MAPREDUCE_V2_CHILD_CLASS, MAX_ALLOWED_FETCH_FAILURES_FRACTION, MAX_FETCH_FAILURES_NOTIFICATIONS, MAX_SHUFFLE_FETCH_HOST_FAILURES, MAX_SHUFFLE_FETCH_RETRY_DELAY, MAX_TASK_FAILURES_PER_TRACKER, MR_AM_ADMIN_COMMAND_OPTS, MR_AM_ADMIN_USER_ENV, MR_AM_COMMAND_OPTS, MR_AM_COMMIT_WINDOW_MS, MR_AM_COMMITTER_CANCEL_TIMEOUT_MS, MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT, MR_AM_CPU_VCORES, MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR, MR_AM_ENV, MR_AM_HISTORY_COMPLETE_EVENT_FLUSH_TIMEOUT_MS, MR_AM_HISTORY_JOB_COMPLETE_UNFLUSHED_MULTIPLIER, MR_AM_HISTORY_MAX_UNFLUSHED_COMPLETE_EVENTS, MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD, MR_AM_IGNORE_BLACKLISTING_BLACKLISTED_NODE_PERECENT, MR_AM_JOB_CLIENT_PORT_RANGE, MR_AM_JOB_CLIENT_THREAD_COUNT, MR_AM_JOB_NODE_BLACKLISTING_ENABLE, MR_AM_JOB_RECOVERY_ENABLE, MR_AM_JOB_RECOVERY_ENABLE_DEFAULT, MR_AM_JOB_REDUCE_PREEMPTION_LIMIT, MR_AM_JOB_REDUCE_RAMPUP_UP_LIMIT, MR_AM_JOB_SPECULATOR, MR_AM_LOG_BACKUPS, MR_AM_LOG_KB, MR_AM_LOG_LEVEL, MR_AM_MAX_ATTEMPTS, MR_AM_NUM_PROGRESS_SPLITS, MR_AM_PREFIX, MR_AM_PROFILE, MR_AM_PROFILE_PARAMS, MR_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT, MR_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL, MR_AM_STAGING_DIR, MR_AM_TASK_ESTIMATOR, MR_AM_TASK_ESTIMATOR_EXPONENTIAL_RATE_ENABLE, MR_AM_TASK_ESTIMATOR_SMOOTH_LAMBDA_MS, MR_AM_TASK_LISTENER_THREAD_COUNT, MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS, MR_AM_TO_RM_WAIT_INTERVAL_MS, MR_AM_VMEM_MB, MR_APPLICATION_TYPE, MR_CLIENT_MAX_RETRIES, MR_CLIENT_TO_AM_IPC_MAX_RETRIES, MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS, MR_ENCRYPTED_INTERMEDIATE_DATA, MR_ENCRYPTED_INTERMEDIATE_DATA_BUFFER_KB, MR_ENCRYPTED_INTERMEDIATE_DATA_KEY_SIZE_BITS, MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, MR_JOB_END_NOTIFICATION_MAX_RETRY_INTERVAL, MR_JOB_END_NOTIFICATION_PROXY, MR_JOB_END_NOTIFICATION_TIMEOUT, MR_JOB_END_NOTIFICATION_URL, MR_JOB_END_RETRY_ATTEMPTS, MR_JOB_END_RETRY_INTERVAL, MR_JOB_REDUCER_PREEMPT_DELAY_SEC, MR_PREFIX, NUM_MAP_PROFILES, NUM_MAPS, NUM_REDUCE_PROFILES, NUM_REDUCES, OUTPUT, OUTPUT_FORMAT_CLASS_ATTR, OUTPUT_KEY_CLASS, OUTPUT_VALUE_CLASS, PARTITIONER_CLASS_ATTR, PRESERVE_FAILED_TASK_FILES, PRESERVE_FILES_PATTERN, PRIORITY, QUEUE_NAME, RECORDS_BEFORE_PROGRESS, REDUCE_CLASS_ATTR, REDUCE_CPU_VCORES, REDUCE_DEBUG_SCRIPT, REDUCE_ENV, REDUCE_FAILURES_MAXPERCENT, REDUCE_INPUT_BUFFER_PERCENT, REDUCE_JAVA_OPTS, REDUCE_LOG_LEVEL, REDUCE_MARKRESET_BUFFER_PERCENT, REDUCE_MARKRESET_BUFFER_SIZE, REDUCE_MAX_ATTEMPTS, REDUCE_MEMORY_MB, REDUCE_MEMORY_TOTAL_BYTES, REDUCE_MEMTOMEM_ENABLED, REDUCE_MEMTOMEM_THRESHOLD, REDUCE_MERGE_INMEM_THRESHOLD, REDUCE_SKIP_INCR_PROC_COUNT, REDUCE_SKIP_MAXGROUPS, REDUCE_SPECULATIVE, RESERVATION_ID, SETUP_CLEANUP_NEEDED, SHUFFLE_CONNECT_TIMEOUT, SHUFFLE_FETCH_FAILURES, SHUFFLE_FETCH_RETRY_ENABLED, SHUFFLE_FETCH_RETRY_INTERVAL_MS, SHUFFLE_FETCH_RETRY_TIMEOUT_MS, SHUFFLE_INPUT_BUFFER_PERCENT, SHUFFLE_MEMORY_LIMIT_PERCENT, SHUFFLE_MERGE_PERCENT, SHUFFLE_NOTIFY_READERROR, SHUFFLE_PARALLEL_COPIES, SHUFFLE_READ_TIMEOUT, SKIP_OUTDIR, SKIP_RECORDS, SKIP_START_ATTEMPTS, SPECULATIVE_SLOWNODE_THRESHOLD, SPECULATIVE_SLOWTASK_THRESHOLD, SPECULATIVECAP, SPLIT_FILE, SPLIT_METAINFO_MAXSIZE, STDERR_LOGFILE_ENV, STDOUT_LOGFILE_ENV, TASK_ATTEMPT_ID, TASK_CLEANUP_NEEDED, TASK_DEBUGOUT_LINES, TASK_ID, TASK_ISMAP, TASK_LOG_BACKUPS, TASK_MAP_PROFILE_PARAMS, TASK_OUTPUT_DIR, TASK_PARTITION, TASK_PROFILE, TASK_PROFILE_PARAMS, TASK_REDUCE_PROFILE_PARAMS, TASK_TEMP_DIR, TASK_TIMEOUT, TASK_TIMEOUT_CHECK_INTERVAL_MS, TASK_USERLOG_LIMIT, USER_LOG_RETAIN_HOURS, USER_NAME, WORKDIR, WORKFLOW_ADJACENCY_PREFIX_PATTERN, WORKFLOW_ADJACENCY_PREFIX_STRING, WORKFLOW_ID, WORKFLOW_NAME, WORKFLOW_NODE_NAME, WORKFLOW_TAGS, WORKING_DIR
 
Constructor Summary
Job()
          Deprecated. Use getInstance()
Job(Configuration conf)
          Deprecated. Use getInstance(Configuration)
Job(Configuration conf, String jobName)
          Deprecated. Use getInstance(Configuration, String)
 
Method Summary
 void addArchiveToClassPath(Path archive)
          Add an archive path to the current set of classpath entries.
 void addCacheArchive(URI uri)
          Add a archives to be localized
 void addCacheFile(URI uri)
          Add a file to be localized
 void addFileToClassPath(Path file)
          Add an file path to the current set of classpath entries It adds the file to cache as well.
 float cleanupProgress()
          Get the progress of the job's cleanup-tasks, as a float between 0.0 and 1.0.
 void createSymlink()
          Deprecated. 
 void failTask(TaskAttemptID taskId)
          Fail indicated task attempt.
static int getCompletionPollInterval(Configuration conf)
          The interval at which waitForCompletion() should check.
 Counters getCounters()
          Gets the counters for this job.
 long getFinishTime()
          Get finish time of the job.
 String getHistoryUrl()
           
static Job getInstance()
          Creates a new Job with no particular Cluster .
static Job getInstance(Cluster ignored)
          Deprecated. Use getInstance()
static Job getInstance(Cluster ignored, Configuration conf)
          Deprecated. Use getInstance(Configuration)
static Job getInstance(Configuration conf)
          Creates a new Job with no particular Cluster and a given Configuration.
static Job getInstance(Configuration conf, String jobName)
          Creates a new Job with no particular Cluster and a given jobName.
static Job getInstance(JobStatus status, Configuration conf)
          Creates a new Job with no particular Cluster and given Configuration and JobStatus.
 String getJobFile()
          Get the path of the submitted job configuration.
 String getJobName()
          The user-specified job name.
 org.apache.hadoop.mapreduce.JobStatus.State getJobState()
          Returns the current state of the Job.
 JobPriority getPriority()
          Get scheduling info of the job.
static int getProgressPollInterval(Configuration conf)
          The interval at which monitorAndPrintJob() prints status
 ReservationId getReservationId()
          Get the reservation to which the job is submitted to, if any
 String getSchedulingInfo()
          Get scheduling info of the job.
 long getStartTime()
          Get start time of the job.
 JobStatus getStatus()
           
 TaskCompletionEvent[] getTaskCompletionEvents(int startFrom)
          Get events indicating completion (success/failure) of component tasks.
 TaskCompletionEvent[] getTaskCompletionEvents(int startFrom, int numEvents)
          Get events indicating completion (success/failure) of component tasks.
 String[] getTaskDiagnostics(TaskAttemptID taskid)
          Gets the diagnostic messages for a given task attempt.
static org.apache.hadoop.mapreduce.Job.TaskStatusFilter getTaskOutputFilter(Configuration conf)
          Get the task output filter.
 org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(TaskType type)
          Get the information of the current state of the tasks of a job.
 String getTrackingURL()
          Get the URL where some job progress information will be displayed.
 boolean isComplete()
          Check if the job is finished or not.
 boolean isRetired()
           
 boolean isSuccessful()
          Check if the job completed successfully.
 boolean isUber()
           
 void killJob()
          Kill the running job.
 void killTask(TaskAttemptID taskId)
          Kill indicated task attempt.
 float mapProgress()
          Get the progress of the job's map-tasks, as a float between 0.0 and 1.0.
 boolean monitorAndPrintJob()
          Monitor a job and print status in real-time as progress is made and tasks fail.
 float reduceProgress()
          Get the progress of the job's reduce-tasks, as a float between 0.0 and 1.0.
 void setCacheArchives(URI[] archives)
          Set the given set of archives
 void setCacheFiles(URI[] files)
          Set the given set of files
 void setCancelDelegationTokenUponJobCompletion(boolean value)
          Sets the flag that will allow the JobTracker to cancel the HDFS delegation tokens upon job completion.
 void setCombinerClass(Class<? extends Reducer> cls)
          Set the combiner class for the job.
 void setCombinerKeyGroupingComparatorClass(Class<? extends RawComparator> cls)
          Define the comparator that controls which keys are grouped together for a single call to combiner, Reducer.reduce(Object, Iterable, org.apache.hadoop.mapreduce.Reducer.Context)
 void setGroupingComparatorClass(Class<? extends RawComparator> cls)
          Define the comparator that controls which keys are grouped together for a single call to Reducer.reduce(Object, Iterable, org.apache.hadoop.mapreduce.Reducer.Context)
 void setInputFormatClass(Class<? extends InputFormat> cls)
          Set the InputFormat for the job.
 void setJar(String jar)
          Set the job jar
 void setJarByClass(Class<?> cls)
          Set the Jar by finding where a given class came from.
 void setJobName(String name)
          Set the user-specified job name.
 void setJobSetupCleanupNeeded(boolean needed)
          Specify whether job-setup and job-cleanup is needed for the job
 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> cls)
          Set the Mapper 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 setMaxReduceAttempts(int n)
          Expert: Set the number of maximum attempts that will be made to run a reduce task.
 void setNumReduceTasks(int tasks)
          Set the number of reduce tasks for the job.
 void setOutputFormatClass(Class<? extends OutputFormat> cls)
          Set the OutputFormat for the job.
 void setOutputKeyClass(Class<?> theClass)
          Set the key class for the job output data.
 void setOutputValueClass(Class<?> theClass)
          Set the value class for job outputs.
 void setPartitionerClass(Class<? extends Partitioner> cls)
          Set the Partitioner for the job.
 void setPriority(JobPriority priority)
          Set the priority of a running job.
 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 setReducerClass(Class<? extends Reducer> cls)
          Set the Reducer for the job.
 void setReduceSpeculativeExecution(boolean speculativeExecution)
          Turn speculative execution on or off for this job for reduce tasks.
 void setReservationId(ReservationId reservationId)
          Set the reservation to which the job is submitted to
 void setSortComparatorClass(Class<? extends RawComparator> cls)
          Define the comparator that controls how the keys are sorted before they are passed to the Reducer.
 void setSpeculativeExecution(boolean speculativeExecution)
          Turn speculative execution on or off for this job.
static void setTaskOutputFilter(Configuration conf, org.apache.hadoop.mapreduce.Job.TaskStatusFilter newValue)
          Modify the Configuration to set the task output filter.
 float setupProgress()
          Get the progress of the job's setup-tasks, as a float between 0.0 and 1.0.
 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.
 void submit()
          Submit the job to the cluster and return immediately.
 String toString()
          Dump stats to screen.
 boolean waitForCompletion(boolean verbose)
          Submit the job to the cluster and wait for it to finish.
 
Methods inherited from class org.apache.hadoop.mapreduce.task.JobContextImpl
getArchiveClassPaths, getArchiveTimestamps, getCacheArchives, getCacheFiles, getCombinerClass, getCombinerKeyGroupingComparator, getConfiguration, getCredentials, getFileClassPaths, getFileTimestamps, getGroupingComparator, getInputFormatClass, getJar, getJobID, getJobSetupCleanupNeeded, getLocalCacheArchives, getLocalCacheFiles, getMapOutputKeyClass, getMapOutputValueClass, getMapperClass, getMaxMapAttempts, getMaxReduceAttempts, getNumReduceTasks, getOutputFormatClass, getOutputKeyClass, getOutputValueClass, getPartitionerClass, getProfileEnabled, getProfileParams, getProfileTaskRange, getReducerClass, getSortComparator, getSymlink, getTaskCleanupNeeded, getUser, getWorkingDirectory, setJobID
 
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
 
Methods inherited from interface org.apache.hadoop.mapreduce.JobContext
getArchiveClassPaths, getArchiveTimestamps, getCacheArchives, getCacheFiles, getCombinerClass, getCombinerKeyGroupingComparator, getConfiguration, getCredentials, getFileClassPaths, getFileTimestamps, getGroupingComparator, getInputFormatClass, getJar, getJobID, getJobSetupCleanupNeeded, getLocalCacheArchives, getLocalCacheFiles, getMapOutputKeyClass, getMapOutputValueClass, getMapperClass, getMaxMapAttempts, getMaxReduceAttempts, getNumReduceTasks, getOutputFormatClass, getOutputKeyClass, getOutputValueClass, getPartitionerClass, getProfileEnabled, getProfileParams, getProfileTaskRange, getReducerClass, getSortComparator, getSymlink, getTaskCleanupNeeded, getUser, getWorkingDirectory
 

Field Detail

OUTPUT_FILTER

public static final String OUTPUT_FILTER
See Also:
Constant Field Values

COMPLETION_POLL_INTERVAL_KEY

public static final String COMPLETION_POLL_INTERVAL_KEY
Key in mapred-*.xml that sets completionPollInvervalMillis

See Also:
Constant Field Values

PROGRESS_MONITOR_POLL_INTERVAL_KEY

public static final String PROGRESS_MONITOR_POLL_INTERVAL_KEY
Key in mapred-*.xml that sets progMonitorPollIntervalMillis

See Also:
Constant Field Values

USED_GENERIC_PARSER

public static final String USED_GENERIC_PARSER
See Also:
Constant Field Values

SUBMIT_REPLICATION

public static final String SUBMIT_REPLICATION
See Also:
Constant Field Values
Constructor Detail

Job

@Deprecated
public Job()
    throws IOException
Deprecated. Use getInstance()

Throws:
IOException

Job

@Deprecated
public Job(Configuration conf)
    throws IOException
Deprecated. Use getInstance(Configuration)

Throws:
IOException

Job

@Deprecated
public Job(Configuration conf,
                      String jobName)
    throws IOException
Deprecated. Use getInstance(Configuration, String)

Throws:
IOException
Method Detail

getInstance

public static Job getInstance()
                       throws IOException
Creates a new Job with no particular Cluster . A Cluster will be created with a generic Configuration.

Returns:
the Job , with no connection to a cluster yet.
Throws:
IOException

getInstance

public static Job getInstance(Configuration conf)
                       throws IOException
Creates a new Job with no particular Cluster and a given Configuration. The Job makes a copy of the Configuration so that any necessary internal modifications do not reflect on the incoming parameter. A Cluster will be created from the conf parameter only when it's needed.

Parameters:
conf - the configuration
Returns:
the Job , with no connection to a cluster yet.
Throws:
IOException

getInstance

public static Job getInstance(Configuration conf,
                              String jobName)
                       throws IOException
Creates a new Job with no particular Cluster and a given jobName. A Cluster will be created from the conf parameter only when it's needed. The Job makes a copy of the Configuration so that any necessary internal modifications do not reflect on the incoming parameter.

Parameters:
conf - the configuration
Returns:
the Job , with no connection to a cluster yet.
Throws:
IOException

getInstance

public static Job getInstance(JobStatus status,
                              Configuration conf)
                       throws IOException
Creates a new Job with no particular Cluster and given Configuration and JobStatus. A Cluster will be created from the conf parameter only when it's needed. The Job makes a copy of the Configuration so that any necessary internal modifications do not reflect on the incoming parameter.

Parameters:
status - job status
conf - job configuration
Returns:
the Job , with no connection to a cluster yet.
Throws:
IOException

getInstance

@Deprecated
public static Job getInstance(Cluster ignored)
                       throws IOException
Deprecated. Use getInstance()

Creates a new Job with no particular Cluster. A Cluster will be created from the conf parameter only when it's needed. The Job makes a copy of the Configuration so that any necessary internal modifications do not reflect on the incoming parameter.

Parameters:
ignored -
Returns:
the Job , with no connection to a cluster yet.
Throws:
IOException

getInstance

@Deprecated
public static Job getInstance(Cluster ignored,
                                         Configuration conf)
                       throws IOException
Deprecated. Use getInstance(Configuration)

Creates a new Job with no particular Cluster and given Configuration. A Cluster will be created from the conf parameter only when it's needed. The Job makes a copy of the Configuration so that any necessary internal modifications do not reflect on the incoming parameter.

Parameters:
ignored -
conf - job configuration
Returns:
the Job , with no connection to a cluster yet.
Throws:
IOException

getStatus

public JobStatus getStatus()
                    throws IOException,
                           InterruptedException
Throws:
IOException
InterruptedException

getJobState

public org.apache.hadoop.mapreduce.JobStatus.State getJobState()
                                                        throws IOException,
                                                               InterruptedException
Returns the current state of the Job.

Returns:
JobStatus#State
Throws:
IOException
InterruptedException

getTrackingURL

public String getTrackingURL()
Get the URL where some job progress information will be displayed.

Returns:
the URL where some job progress information will be displayed.

getJobFile

public String getJobFile()
Get the path of the submitted job configuration.

Returns:
the path of the submitted job configuration.

getStartTime

public long getStartTime()
Get start time of the job.

Returns:
the start time of the job

getFinishTime

public long getFinishTime()
                   throws IOException,
                          InterruptedException
Get finish time of the job.

Returns:
the finish time of the job
Throws:
IOException
InterruptedException

getSchedulingInfo

public String getSchedulingInfo()
Get scheduling info of the job.

Returns:
the scheduling info of the job

getPriority

public JobPriority getPriority()
                        throws IOException,
                               InterruptedException
Get scheduling info of the job.

Returns:
the scheduling info of the job
Throws:
IOException
InterruptedException

getJobName

public String getJobName()
The user-specified job name.

Specified by:
getJobName in interface JobContext
Overrides:
getJobName in class org.apache.hadoop.mapreduce.task.JobContextImpl
Returns:
the job's name, defaulting to "".

getHistoryUrl

public String getHistoryUrl()
                     throws IOException,
                            InterruptedException
Throws:
IOException
InterruptedException

isRetired

public boolean isRetired()
                  throws IOException,
                         InterruptedException
Throws:
IOException
InterruptedException

toString

public String toString()
Dump stats to screen.

Overrides:
toString in class Object

getTaskReports

public org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(TaskType type)
                                                        throws IOException,
                                                               InterruptedException
Get the information of the current state of the tasks of a job.

Parameters:
type - Type of the task
Returns:
the list of all of the map tips.
Throws:
IOException
InterruptedException

mapProgress

public float mapProgress()
                  throws IOException
Get the progress of the job's map-tasks, as a float between 0.0 and 1.0. When all map tasks have completed, the function returns 1.0.

Returns:
the progress of the job's map-tasks.
Throws:
IOException

reduceProgress

public float reduceProgress()
                     throws IOException
Get the progress of the job's reduce-tasks, as a float between 0.0 and 1.0. When all reduce tasks have completed, the function returns 1.0.

Returns:
the progress of the job's reduce-tasks.
Throws:
IOException

cleanupProgress

public float cleanupProgress()
                      throws IOException,
                             InterruptedException
Get the progress of the job's cleanup-tasks, as a float between 0.0 and 1.0. When all cleanup tasks have completed, the function returns 1.0.

Returns:
the progress of the job's cleanup-tasks.
Throws:
IOException
InterruptedException

setupProgress

public float setupProgress()
                    throws IOException
Get the progress of the job's setup-tasks, as a float between 0.0 and 1.0. When all setup tasks have completed, the function returns 1.0.

Returns:
the progress of the job's setup-tasks.
Throws:
IOException

isComplete

public boolean isComplete()
                   throws IOException
Check if the job is finished or not. This is a non-blocking call.

Returns:
true if the job is complete, else false.
Throws:
IOException

isSuccessful

public boolean isSuccessful()
                     throws IOException
Check if the job completed successfully.

Returns:
true if the job succeeded, else false.
Throws:
IOException

killJob

public void killJob()
             throws IOException
Kill the running job. Blocks until all job tasks have been killed as well. If the job is no longer running, it simply returns.

Throws:
IOException

setPriority

public void setPriority(JobPriority priority)
                 throws IOException,
                        InterruptedException
Set the priority of a running job.

Parameters:
priority - the new priority for the job.
Throws:
IOException
InterruptedException

getTaskCompletionEvents

public TaskCompletionEvent[] getTaskCompletionEvents(int startFrom,
                                                     int numEvents)
                                              throws IOException,
                                                     InterruptedException
Get events indicating completion (success/failure) of component tasks.

Parameters:
startFrom - index to start fetching events from
numEvents - number of events to fetch
Returns:
an array of TaskCompletionEvents
Throws:
IOException
InterruptedException

getTaskCompletionEvents

public TaskCompletionEvent[] getTaskCompletionEvents(int startFrom)
                                              throws IOException
Get events indicating completion (success/failure) of component tasks.

Parameters:
startFrom - index to start fetching events from
Returns:
an array of TaskCompletionEvents
Throws:
IOException

killTask

public void killTask(TaskAttemptID taskId)
              throws IOException
Kill indicated task attempt.

Parameters:
taskId - the id of the task to be terminated.
Throws:
IOException

failTask

public void failTask(TaskAttemptID taskId)
              throws IOException
Fail indicated task attempt.

Parameters:
taskId - the id of the task to be terminated.
Throws:
IOException

getCounters

public Counters getCounters()
                     throws IOException
Gets the counters for this job. May return null if the job has been retired and the job is no longer in the completed job store.

Returns:
the counters for this job.
Throws:
IOException

getTaskDiagnostics

public String[] getTaskDiagnostics(TaskAttemptID taskid)
                            throws IOException,
                                   InterruptedException
Gets the diagnostic messages for a given task attempt.

Parameters:
taskid -
Returns:
the list of diagnostic messages for the task
Throws:
IOException
InterruptedException

setNumReduceTasks

public void setNumReduceTasks(int tasks)
                       throws IllegalStateException
Set the number of reduce tasks for the job.

Parameters:
tasks - the number of reduce tasks
Throws:
IllegalStateException - if the job is submitted

setWorkingDirectory

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

Parameters:
dir - the new current working directory.
Throws:
IllegalStateException - if the job is submitted
IOException

setInputFormatClass

public void setInputFormatClass(Class<? extends InputFormat> cls)
                         throws IllegalStateException
Set the InputFormat for the job.

Parameters:
cls - the InputFormat to use
Throws:
IllegalStateException - if the job is submitted

setOutputFormatClass

public void setOutputFormatClass(Class<? extends OutputFormat> cls)
                          throws IllegalStateException
Set the OutputFormat for the job.

Parameters:
cls - the OutputFormat to use
Throws:
IllegalStateException - if the job is submitted

setMapperClass

public void setMapperClass(Class<? extends Mapper> cls)
                    throws IllegalStateException
Set the Mapper for the job.

Parameters:
cls - the Mapper to use
Throws:
IllegalStateException - if the job is submitted

setJarByClass

public void setJarByClass(Class<?> cls)
Set the Jar by finding where a given class came from.

Parameters:
cls - the example class

setJar

public void setJar(String jar)
Set the job jar


setUser

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

Parameters:
user - the username for this job.

setCombinerClass

public void setCombinerClass(Class<? extends Reducer> cls)
                      throws IllegalStateException
Set the combiner class for the job.

Parameters:
cls - the combiner to use
Throws:
IllegalStateException - if the job is submitted

setReducerClass

public void setReducerClass(Class<? extends Reducer> cls)
                     throws IllegalStateException
Set the Reducer for the job.

Parameters:
cls - the Reducer to use
Throws:
IllegalStateException - if the job is submitted

setPartitionerClass

public void setPartitionerClass(Class<? extends Partitioner> cls)
                         throws IllegalStateException
Set the Partitioner for the job.

Parameters:
cls - the Partitioner to use
Throws:
IllegalStateException - if the job is submitted

setMapOutputKeyClass

public void setMapOutputKeyClass(Class<?> theClass)
                          throws IllegalStateException
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.
Throws:
IllegalStateException - if the job is submitted

setMapOutputValueClass

public void setMapOutputValueClass(Class<?> theClass)
                            throws IllegalStateException
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.
Throws:
IllegalStateException - if the job is submitted

setOutputKeyClass

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

Parameters:
theClass - the key class for the job output data.
Throws:
IllegalStateException - if the job is submitted

setOutputValueClass

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

Parameters:
theClass - the value class for job outputs.
Throws:
IllegalStateException - if the job is submitted

setCombinerKeyGroupingComparatorClass

public void setCombinerKeyGroupingComparatorClass(Class<? extends RawComparator> cls)
                                           throws IllegalStateException
Define the comparator that controls which keys are grouped together for a single call to combiner, Reducer.reduce(Object, Iterable, org.apache.hadoop.mapreduce.Reducer.Context)

Parameters:
cls - the raw comparator to use
Throws:
IllegalStateException - if the job is submitted

setSortComparatorClass

public void setSortComparatorClass(Class<? extends RawComparator> cls)
                            throws IllegalStateException
Define the comparator that controls how the keys are sorted before they are passed to the Reducer.

Parameters:
cls - the raw comparator
Throws:
IllegalStateException - if the job is submitted
See Also:
setCombinerKeyGroupingComparatorClass(Class)

setGroupingComparatorClass

public void setGroupingComparatorClass(Class<? extends RawComparator> cls)
                                throws IllegalStateException
Define the comparator that controls which keys are grouped together for a single call to Reducer.reduce(Object, Iterable, org.apache.hadoop.mapreduce.Reducer.Context)

Parameters:
cls - the raw comparator to use
Throws:
IllegalStateException - if the job is submitted
See Also:
setCombinerKeyGroupingComparatorClass(Class)

setJobName

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

Parameters:
name - the job's new name.
Throws:
IllegalStateException - if the job is submitted

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.

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.

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.

setJobSetupCleanupNeeded

public void setJobSetupCleanupNeeded(boolean needed)
Specify whether job-setup and job-cleanup is needed for the job

Parameters:
needed - If true, job-setup and job-cleanup will be considered from OutputCommitter else ignored.

setCacheArchives

public void setCacheArchives(URI[] archives)
Set the given set of archives

Parameters:
archives - The list of archives that need to be localized

setCacheFiles

public void setCacheFiles(URI[] files)
Set the given set of files

Parameters:
files - The list of files that need to be localized

addCacheArchive

public void addCacheArchive(URI uri)
Add a archives to be localized

Parameters:
uri - The uri of the cache to be localized

addCacheFile

public void addCacheFile(URI uri)
Add a file to be localized

Parameters:
uri - The uri of the cache to be localized

addFileToClassPath

public void addFileToClassPath(Path file)
                        throws IOException
Add an file path to the current set of classpath entries It adds the file to cache as well. Files added with this method will not be unpacked while being added to the classpath. To add archives to classpath, use the addArchiveToClassPath(Path) method instead.

Parameters:
file - Path of the file to be added
Throws:
IOException

addArchiveToClassPath

public void addArchiveToClassPath(Path archive)
                           throws IOException
Add an archive path to the current set of classpath entries. It adds the archive to cache as well. Archive files will be unpacked and added to the classpath when being distributed.

Parameters:
archive - Path of the archive to be added
Throws:
IOException

createSymlink

@Deprecated
public void createSymlink()
Deprecated. 

Originally intended to enable symlinks, but currently symlinks cannot be disabled.


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.

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.

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

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

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

setCancelDelegationTokenUponJobCompletion

public void setCancelDelegationTokenUponJobCompletion(boolean value)
Sets the flag that will allow the JobTracker to cancel the HDFS delegation tokens upon job completion. Defaults to true.


submit

public void submit()
            throws IOException,
                   InterruptedException,
                   ClassNotFoundException
Submit the job to the cluster and return immediately.

Throws:
IOException
InterruptedException
ClassNotFoundException

waitForCompletion

public boolean waitForCompletion(boolean verbose)
                          throws IOException,
                                 InterruptedException,
                                 ClassNotFoundException
Submit the job to the cluster and wait for it to finish.

Parameters:
verbose - print the progress to the user
Returns:
true if the job succeeded
Throws:
IOException - thrown if the communication with the JobTracker is lost
InterruptedException
ClassNotFoundException

monitorAndPrintJob

public boolean monitorAndPrintJob()
                           throws IOException,
                                  InterruptedException
Monitor a job and print status in real-time as progress is made and tasks fail.

Returns:
true if the job succeeded
Throws:
IOException - if communication to the JobTracker fails
InterruptedException

getProgressPollInterval

public static int getProgressPollInterval(Configuration conf)
The interval at which monitorAndPrintJob() prints status


getCompletionPollInterval

public static int getCompletionPollInterval(Configuration conf)
The interval at which waitForCompletion() should check.


getTaskOutputFilter

public static org.apache.hadoop.mapreduce.Job.TaskStatusFilter getTaskOutputFilter(Configuration conf)
Get the task output filter.

Parameters:
conf - the configuration.
Returns:
the filter level.

setTaskOutputFilter

public static void setTaskOutputFilter(Configuration conf,
                                       org.apache.hadoop.mapreduce.Job.TaskStatusFilter newValue)
Modify the Configuration to set the task output filter.

Parameters:
conf - the Configuration to modify.
newValue - the value to set.

isUber

public boolean isUber()
               throws IOException,
                      InterruptedException
Throws:
IOException
InterruptedException

getReservationId

public ReservationId getReservationId()
Get the reservation to which the job is submitted to, if any

Returns:
the reservationId the identifier of the job's reservation, null if the job does not have any reservation associated with it

setReservationId

public void setReservationId(ReservationId reservationId)
Set the reservation to which the job is submitted to

Parameters:
reservationId - the reservationId to set


Copyright © 2014 Apache Software Foundation. All Rights Reserved.