org.apache.hadoop.mapred
Class JobStatus

java.lang.Object
  extended by org.apache.hadoop.mapreduce.JobStatus
      extended by org.apache.hadoop.mapred.JobStatus
All Implemented Interfaces:
Cloneable, Writable

@InterfaceAudience.Public
@InterfaceStability.Stable
public class JobStatus
extends JobStatus

Describes the current status of a job. This is not intended to be a comprehensive piece of data. For that, look at JobProfile.


Field Summary
static int FAILED
           
static int KILLED
           
static int PREP
           
static int RUNNING
           
static int SUCCEEDED
           
 
Constructor Summary
JobStatus()
           
JobStatus(JobID jobid, float setupProgress, float mapProgress, float reduceProgress, float cleanupProgress, int runState, JobPriority jp)
          Deprecated. 
JobStatus(JobID jobid, float setupProgress, float mapProgress, float reduceProgress, float cleanupProgress, int runState, JobPriority jp, String user, String jobName, String jobFile, String trackingUrl)
          Create a job status object for a given jobid.
JobStatus(JobID jobid, float setupProgress, float mapProgress, float reduceProgress, float cleanupProgress, int runState, JobPriority jp, String user, String jobName, String jobFile, String trackingUrl, boolean isUber)
          Create a job status object for a given jobid.
JobStatus(JobID jobid, float setupProgress, float mapProgress, float reduceProgress, float cleanupProgress, int runState, JobPriority jp, String user, String jobName, String queue, String jobFile, String trackingUrl)
          Create a job status object for a given jobid.
JobStatus(JobID jobid, float setupProgress, float mapProgress, float reduceProgress, float cleanupProgress, int runState, JobPriority jp, String user, String jobName, String queue, String jobFile, String trackingUrl, boolean isUber)
          Create a job status object for a given jobid.
JobStatus(JobID jobid, float mapProgress, float reduceProgress, float cleanupProgress, int runState)
          Deprecated. 
JobStatus(JobID jobid, float mapProgress, float reduceProgress, float cleanupProgress, int runState, JobPriority jp)
          Deprecated. 
JobStatus(JobID jobid, float mapProgress, float reduceProgress, float cleanupProgress, int runState, JobPriority jp, String user, String jobName, String jobFile, String trackingUrl)
          Create a job status object for a given jobid.
JobStatus(JobID jobid, float mapProgress, float reduceProgress, float cleanupProgress, int runState, String user, String jobName, String jobFile, String trackingUrl)
          Create a job status object for a given jobid.
JobStatus(JobID jobid, float mapProgress, float reduceProgress, int runState)
          Deprecated. 
JobStatus(JobID jobid, float mapProgress, float reduceProgress, int runState, String user, String jobName, String jobFile, String trackingUrl)
          Create a job status object for a given jobid.
 
Method Summary
 float cleanupProgress()
           
static JobStatus downgrade(JobStatus stat)
           
 String getJobId()
          Deprecated. use getJobID instead
 JobID getJobID()
           
 JobPriority getJobPriority()
          Return the priority of the job
static String getJobRunState(int state)
          Helper method to get human-readable state of the job.
 int getRunState()
           
 float mapProgress()
           
 float reduceProgress()
           
protected  void setCleanupProgress(float p)
          Sets the cleanup progress of this job
 void setFailureInfo(String failureInfo)
          Set diagnostic information.
protected  void setFinishTime(long finishTime)
          Set the finish time of the job
protected  void setHistoryFile(String historyFile)
          Set the job history file url for a completed job
protected  void setJobACLs(Map<org.apache.hadoop.mapreduce.JobACL,org.apache.hadoop.security.authorize.AccessControlList> acls)
          Set the job acls.
 void setJobPriority(JobPriority jp)
          Set the priority of the job, defaulting to NORMAL.
protected  void setMapProgress(float p)
          Sets the map progress of this job
protected  void setReduceProgress(float p)
          Sets the reduce progress of this Job
protected  void setRetired()
          Set the job retire flag to true.
protected  void setSetupProgress(float p)
          Sets the setup progress of this job
protected  void setStartTime(long startTime)
          Set the start time of the job
protected  void setTrackingUrl(String trackingUrl)
          Set the link to the web-ui for details of the job.
 float setupProgress()
           
protected  void setUsername(String userName)
           
 
Methods inherited from class org.apache.hadoop.mapreduce.JobStatus
clone, getCleanupProgress, getFailureInfo, getFinishTime, getHistoryFile, getJobACLs, getJobFile, getJobName, getMapProgress, getNeededMem, getNumReservedSlots, getNumUsedSlots, getPriority, getQueue, getReduceProgress, getReservedMem, getSchedulingInfo, getSetupProgress, getStartTime, getState, getTrackingUrl, getUsedMem, getUsername, isJobComplete, isRetired, isUber, readFields, setNeededMem, setNumReservedSlots, setNumUsedSlots, setPriority, setQueue, setReservedMem, setState, setUber, setUsedMem, toString, write
 
Methods inherited from class java.lang.Object
equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
 

Field Detail

RUNNING

public static final int RUNNING

SUCCEEDED

public static final int SUCCEEDED

FAILED

public static final int FAILED

PREP

public static final int PREP

KILLED

public static final int KILLED
Constructor Detail

JobStatus

public JobStatus()

JobStatus

@Deprecated
public JobStatus(JobID jobid,
                            float mapProgress,
                            float reduceProgress,
                            float cleanupProgress,
                            int runState)
Deprecated. 


JobStatus

@Deprecated
public JobStatus(JobID jobid,
                            float mapProgress,
                            float reduceProgress,
                            int runState)
Deprecated. 

Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
runState - The current state of the job

JobStatus

@Deprecated
public JobStatus(JobID jobid,
                            float mapProgress,
                            float reduceProgress,
                            float cleanupProgress,
                            int runState,
                            JobPriority jp)
Deprecated. 

Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
runState - The current state of the job
jp - Priority of the job.

JobStatus

@Deprecated
public JobStatus(JobID jobid,
                            float setupProgress,
                            float mapProgress,
                            float reduceProgress,
                            float cleanupProgress,
                            int runState,
                            JobPriority jp)
Deprecated. 

Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
setupProgress - The progress made on the setup
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
cleanupProgress - The progress made on the cleanup
runState - The current state of the job
jp - Priority of the job.

JobStatus

public JobStatus(JobID jobid,
                 float mapProgress,
                 float reduceProgress,
                 float cleanupProgress,
                 int runState,
                 String user,
                 String jobName,
                 String jobFile,
                 String trackingUrl)
Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
cleanupProgress - The progress made on cleanup
runState - The current state of the job
user - userid of the person who submitted the job.
jobName - user-specified job name.
jobFile - job configuration file.
trackingUrl - link to the web-ui for details of the job.

JobStatus

public JobStatus(JobID jobid,
                 float mapProgress,
                 float reduceProgress,
                 int runState,
                 String user,
                 String jobName,
                 String jobFile,
                 String trackingUrl)
Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
runState - The current state of the job
user - userid of the person who submitted the job.
jobName - user-specified job name.
jobFile - job configuration file.
trackingUrl - link to the web-ui for details of the job.

JobStatus

public JobStatus(JobID jobid,
                 float mapProgress,
                 float reduceProgress,
                 float cleanupProgress,
                 int runState,
                 JobPriority jp,
                 String user,
                 String jobName,
                 String jobFile,
                 String trackingUrl)
Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
runState - The current state of the job
jp - Priority of the job.
user - userid of the person who submitted the job.
jobName - user-specified job name.
jobFile - job configuration file.
trackingUrl - link to the web-ui for details of the job.

JobStatus

public JobStatus(JobID jobid,
                 float setupProgress,
                 float mapProgress,
                 float reduceProgress,
                 float cleanupProgress,
                 int runState,
                 JobPriority jp,
                 String user,
                 String jobName,
                 String jobFile,
                 String trackingUrl)
Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
setupProgress - The progress made on the setup
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
cleanupProgress - The progress made on the cleanup
runState - The current state of the job
jp - Priority of the job.
user - userid of the person who submitted the job.
jobName - user-specified job name.
jobFile - job configuration file.
trackingUrl - link to the web-ui for details of the job.

JobStatus

public JobStatus(JobID jobid,
                 float setupProgress,
                 float mapProgress,
                 float reduceProgress,
                 float cleanupProgress,
                 int runState,
                 JobPriority jp,
                 String user,
                 String jobName,
                 String jobFile,
                 String trackingUrl,
                 boolean isUber)
Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
setupProgress - The progress made on the setup
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
cleanupProgress - The progress made on the cleanup
runState - The current state of the job
jp - Priority of the job.
user - userid of the person who submitted the job.
jobName - user-specified job name.
jobFile - job configuration file.
trackingUrl - link to the web-ui for details of the job.
isUber - Whether job running in uber mode

JobStatus

public JobStatus(JobID jobid,
                 float setupProgress,
                 float mapProgress,
                 float reduceProgress,
                 float cleanupProgress,
                 int runState,
                 JobPriority jp,
                 String user,
                 String jobName,
                 String queue,
                 String jobFile,
                 String trackingUrl)
Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
setupProgress - The progress made on the setup
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
cleanupProgress - The progress made on the cleanup
runState - The current state of the job
jp - Priority of the job.
user - userid of the person who submitted the job.
jobName - user-specified job name.
queue - job queue name.
jobFile - job configuration file.
trackingUrl - link to the web-ui for details of the job.

JobStatus

public JobStatus(JobID jobid,
                 float setupProgress,
                 float mapProgress,
                 float reduceProgress,
                 float cleanupProgress,
                 int runState,
                 JobPriority jp,
                 String user,
                 String jobName,
                 String queue,
                 String jobFile,
                 String trackingUrl,
                 boolean isUber)
Create a job status object for a given jobid.

Parameters:
jobid - The jobid of the job
setupProgress - The progress made on the setup
mapProgress - The progress made on the maps
reduceProgress - The progress made on the reduces
cleanupProgress - The progress made on the cleanup
runState - The current state of the job
jp - Priority of the job.
user - userid of the person who submitted the job.
jobName - user-specified job name.
queue - job queue name.
jobFile - job configuration file.
trackingUrl - link to the web-ui for details of the job.
isUber - Whether job running in uber mode
Method Detail

getJobRunState

public static String getJobRunState(int state)
Helper method to get human-readable state of the job.

Parameters:
state - job state
Returns:
human-readable state of the job

downgrade

public static JobStatus downgrade(JobStatus stat)

getJobId

@Deprecated
public String getJobId()
Deprecated. use getJobID instead


getJobID

public JobID getJobID()
Overrides:
getJobID in class JobStatus
Returns:
The jobid of the Job

getJobPriority

public JobPriority getJobPriority()
Return the priority of the job

Returns:
job priority

setMapProgress

protected void setMapProgress(float p)
Sets the map progress of this job

Overrides:
setMapProgress in class JobStatus
Parameters:
p - The value of map progress to set to

setCleanupProgress

protected void setCleanupProgress(float p)
Sets the cleanup progress of this job

Overrides:
setCleanupProgress in class JobStatus
Parameters:
p - The value of cleanup progress to set to

setSetupProgress

protected void setSetupProgress(float p)
Sets the setup progress of this job

Overrides:
setSetupProgress in class JobStatus
Parameters:
p - The value of setup progress to set to

setReduceProgress

protected void setReduceProgress(float p)
Sets the reduce progress of this Job

Overrides:
setReduceProgress in class JobStatus
Parameters:
p - The value of reduce progress to set to

setFinishTime

protected void setFinishTime(long finishTime)
Set the finish time of the job

Overrides:
setFinishTime in class JobStatus
Parameters:
finishTime - The finishTime of the job

setHistoryFile

protected void setHistoryFile(String historyFile)
Set the job history file url for a completed job

Overrides:
setHistoryFile in class JobStatus

setTrackingUrl

protected void setTrackingUrl(String trackingUrl)
Set the link to the web-ui for details of the job.

Overrides:
setTrackingUrl in class JobStatus

setRetired

protected void setRetired()
Set the job retire flag to true.

Overrides:
setRetired in class JobStatus

getRunState

public int getRunState()
Returns:
running state of the job

setStartTime

protected void setStartTime(long startTime)
Set the start time of the job

Overrides:
setStartTime in class JobStatus
Parameters:
startTime - The startTime of the job

setUsername

protected void setUsername(String userName)
Overrides:
setUsername in class JobStatus
Parameters:
userName - The username of the job

setJobACLs

protected void setJobACLs(Map<org.apache.hadoop.mapreduce.JobACL,org.apache.hadoop.security.authorize.AccessControlList> acls)
Description copied from class: JobStatus
Set the job acls.

Overrides:
setJobACLs in class JobStatus
Parameters:
acls - Map from JobACL to AccessControlList

setFailureInfo

public void setFailureInfo(String failureInfo)
Description copied from class: JobStatus
Set diagnostic information.

Overrides:
setFailureInfo in class JobStatus
Parameters:
failureInfo - diagnostic information

setJobPriority

public void setJobPriority(JobPriority jp)
Set the priority of the job, defaulting to NORMAL.

Parameters:
jp - new job priority

mapProgress

public float mapProgress()
Returns:
Percentage of progress in maps

cleanupProgress

public float cleanupProgress()
Returns:
Percentage of progress in cleanup

setupProgress

public float setupProgress()
Returns:
Percentage of progress in setup

reduceProgress

public float reduceProgress()
Returns:
Percentage of progress in reduce


Copyright © 2014 Apache Software Foundation. All Rights Reserved.