org.apache.hadoop.mapred
Interface RunningJob


@InterfaceAudience.Public
@InterfaceStability.Stable
public interface RunningJob

RunningJob is the user-interface to query for details on a running Map-Reduce job.

Clients can get hold of RunningJob via the JobClient and then query the running-job for details such as name, configuration, progress etc.

See Also:
JobClient

Method Summary
 float cleanupProgress()
          Get the progress of the job's cleanup-tasks, as a float between 0.0 and 1.0.
 Configuration getConfiguration()
          Get the underlying job configuration
 Counters getCounters()
          Gets the counters for this job.
 String getFailureInfo()
          Get failure info for the job.
 String getHistoryUrl()
          Get the url where history file is archived.
 JobID getID()
          Get the job identifier.
 String getJobFile()
          Get the path of the submitted job configuration.
 String getJobID()
          Deprecated. This method is deprecated and will be removed. Applications should rather use getID().
 String getJobName()
          Get the name of the job.
 int getJobState()
          Returns the current state of the Job.
 JobStatus getJobStatus()
          Returns a snapshot of the current status, JobStatus, of the Job.
 TaskCompletionEvent[] getTaskCompletionEvents(int startFrom)
          Get events indicating completion (success/failure) of component tasks.
 String[] getTaskDiagnostics(TaskAttemptID taskid)
          Gets the diagnostic messages for a given task attempt.
 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()
          Check whether the job has been removed from JobTracker memory and retired.
 boolean isSuccessful()
          Check if the job completed successfully.
 void killJob()
          Kill the running job.
 void killTask(String taskId, boolean shouldFail)
          Deprecated. Applications should rather use killTask(TaskAttemptID, boolean)
 void killTask(TaskAttemptID taskId, boolean shouldFail)
          Kill indicated task attempt.
 float mapProgress()
          Get the progress of the job's map-tasks, as a float between 0.0 and 1.0.
 float reduceProgress()
          Get the progress of the job's reduce-tasks, as a float between 0.0 and 1.0.
 void setJobPriority(String priority)
          Set the priority of a running job.
 float setupProgress()
          Get the progress of the job's setup-tasks, as a float between 0.0 and 1.0.
 void waitForCompletion()
          Blocks until the job is complete.
 

Method Detail

getConfiguration

Configuration getConfiguration()
Get the underlying job configuration

Returns:
the configuration of the job.

getID

JobID getID()
Get the job identifier.

Returns:
the job identifier.

getJobID

@Deprecated
String getJobID()
Deprecated. This method is deprecated and will be removed. Applications should rather use getID().


getJobName

String getJobName()
Get the name of the job.

Returns:
the name of the job.

getJobFile

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

Returns:
the path of the submitted job configuration.

getTrackingURL

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

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

mapProgress

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

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

float cleanupProgress()
                      throws IOException
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

setupProgress

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

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

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

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

waitForCompletion

void waitForCompletion()
                       throws IOException
Blocks until the job is complete.

Throws:
IOException

getJobState

int getJobState()
                throws IOException
Returns the current state of the Job. JobStatus

Throws:
IOException

getJobStatus

JobStatus getJobStatus()
                       throws IOException
Returns a snapshot of the current status, JobStatus, of the Job. Need to call again for latest information.

Throws:
IOException

killJob

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

setJobPriority

void setJobPriority(String priority)
                    throws IOException
Set the priority of a running job.

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

getTaskCompletionEvents

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

void killTask(TaskAttemptID taskId,
              boolean shouldFail)
              throws IOException
Kill indicated task attempt.

Parameters:
taskId - the id of the task to be terminated.
shouldFail - if true the task is failed and added to failed tasks list, otherwise it is just killed, w/o affecting job failure status.
Throws:
IOException

killTask

@Deprecated
void killTask(String taskId,
                         boolean shouldFail)
              throws IOException
Deprecated. Applications should rather use killTask(TaskAttemptID, boolean)

Throws:
IOException

getCounters

Counters getCounters()
                     throws IOException
Gets the counters for this job.

Returns:
the counters for this job or null if the job has been retired.
Throws:
IOException

getTaskDiagnostics

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

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

getHistoryUrl

String getHistoryUrl()
                     throws IOException
Get the url where history file is archived. Returns empty string if history file is not available yet.

Returns:
the url where history file is archived
Throws:
IOException

isRetired

boolean isRetired()
                  throws IOException
Check whether the job has been removed from JobTracker memory and retired. On retire, the job history file is copied to a location known by getHistoryUrl()

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

getFailureInfo

String getFailureInfo()
                      throws IOException
Get failure info for the job.

Returns:
the failure info for the job.
Throws:
IOException


Copyright © 2014 Apache Software Foundation. All Rights Reserved.