org.apache.hadoop.mapreduce.lib.jobcontrol
Class ControlledJob

java.lang.Object
  extended by org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob
Direct Known Subclasses:
Job

@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ControlledJob
extends Object

This class encapsulates a MapReduce job and its dependency. It monitors the states of the depending jobs and updates the state of this job. A job starts in the WAITING state. If it does not have any depending jobs, or all of the depending jobs are in SUCCESS state, then the job state will become READY. If any depending jobs fail, the job will fail too. When in READY state, the job can be submitted to Hadoop for execution, with the state changing into RUNNING state. From RUNNING state, the job can get into SUCCESS or FAILED state, depending the status of the job execution.


Field Summary
static String CREATE_DIR
           
 
Constructor Summary
ControlledJob(Configuration conf)
          Construct a job.
ControlledJob(Job job, List<ControlledJob> dependingJobs)
          Construct a job.
 
Method Summary
 boolean addDependingJob(ControlledJob dependingJob)
          Add a job to this jobs' dependency list.
 void failJob(String message)
           
 List<ControlledJob> getDependentJobs()
           
 Job getJob()
           
 String getJobID()
           
 String getJobName()
           
 org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob.State getJobState()
           
 JobID getMapredJobId()
           
 String getMessage()
           
 boolean isCompleted()
           
 boolean isReady()
           
 void killJob()
           
 void setJob(Job job)
          Set the mapreduce job
 void setJobID(String id)
          Set the job ID for this job.
 void setJobName(String jobName)
          Set the job name for this job.
protected  void setJobState(org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob.State state)
          Set the state for this job.
 void setMessage(String message)
          Set the message for this job.
protected  void submit()
          Submit this job to mapred.
 String toString()
           
 
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
 

Field Detail

CREATE_DIR

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

ControlledJob

public ControlledJob(Job job,
                     List<ControlledJob> dependingJobs)
              throws IOException
Construct a job.

Parameters:
job - a mapreduce job to be executed.
dependingJobs - an array of jobs the current job depends on
Throws:
IOException

ControlledJob

public ControlledJob(Configuration conf)
              throws IOException
Construct a job.

Parameters:
conf - mapred job configuration representing a job to be executed.
Throws:
IOException
Method Detail

toString

public String toString()
Overrides:
toString in class Object

getJobName

public String getJobName()
Returns:
the job name of this job

setJobName

public void setJobName(String jobName)
Set the job name for this job.

Parameters:
jobName - the job name

getJobID

public String getJobID()
Returns:
the job ID of this job assigned by JobControl

setJobID

public void setJobID(String id)
Set the job ID for this job.

Parameters:
id - the job ID

getMapredJobId

public JobID getMapredJobId()
Returns:
the mapred ID of this job as assigned by the mapred framework.

getJob

public Job getJob()
Returns:
the mapreduce job

setJob

public void setJob(Job job)
Set the mapreduce job

Parameters:
job - the mapreduce job for this job.

getJobState

public org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob.State getJobState()
Returns:
the state of this job

setJobState

protected void setJobState(org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob.State state)
Set the state for this job.

Parameters:
state - the new state for this job.

getMessage

public String getMessage()
Returns:
the message of this job

setMessage

public void setMessage(String message)
Set the message for this job.

Parameters:
message - the message for this job.

getDependentJobs

public List<ControlledJob> getDependentJobs()
Returns:
the depending jobs of this job

addDependingJob

public boolean addDependingJob(ControlledJob dependingJob)
Add a job to this jobs' dependency list. Dependent jobs can only be added while a Job is waiting to run, not during or afterwards.

Parameters:
dependingJob - Job that this Job depends on.
Returns:
true if the Job was added.

isCompleted

public boolean isCompleted()
Returns:
true if this job is in a complete state

isReady

public boolean isReady()
Returns:
true if this job is in READY state

killJob

public void killJob()
             throws IOException,
                    InterruptedException
Throws:
IOException
InterruptedException

failJob

public void failJob(String message)
             throws IOException,
                    InterruptedException
Throws:
IOException
InterruptedException

submit

protected void submit()
Submit this job to mapred. The state becomes RUNNING if submission is successful, FAILED otherwise.



Copyright © 2014 Apache Software Foundation. All Rights Reserved.