|
@@ -29,7 +29,9 @@ import org.apache.hadoop.mapred.TaskCompletionEvent;
|
|
|
|
|
|
/**
|
|
|
* 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.
|
|
|
+ * 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.
|
|
|
*/
|
|
|
public class Job extends JobContext {
|
|
|
|
|
@@ -49,8 +51,9 @@ public class Job extends JobContext {
|
|
|
/**
|
|
|
* Set the number of reduce tasks for the job.
|
|
|
* @param tasks the number of reduce tasks
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setNumReduceTasks(int tasks) {
|
|
|
+ public void setNumReduceTasks(int tasks) throws IllegalStateException {
|
|
|
conf.setInt(NUM_REDUCES_ATTR, tasks);
|
|
|
}
|
|
|
|
|
@@ -58,6 +61,7 @@ public class Job extends JobContext {
|
|
|
* Set the current working directory for the default file system.
|
|
|
*
|
|
|
* @param dir the new current working directory.
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
public void setWorkingDirectory(Path dir) throws IOException {
|
|
|
dir = dir.makeQualified(FileSystem.get(conf));
|
|
@@ -67,48 +71,60 @@ public class Job extends JobContext {
|
|
|
/**
|
|
|
* Set the {@link InputFormat} for the job.
|
|
|
* @param cls the <code>InputFormat</code> to use
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setInputFormatClass(Class<? extends InputFormat<?,?>> cls) {
|
|
|
+ public void setInputFormatClass(Class<? extends InputFormat<?,?>> cls
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(INPUT_FORMAT_CLASS_ATTR, cls, InputFormat.class);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Set the {@link OutputFormat} for the job.
|
|
|
* @param cls the <code>OutputFormat</code> to use
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setOutputFormatClass(Class<? extends OutputFormat<?,?>> cls) {
|
|
|
+ public void setOutputFormatClass(Class<? extends OutputFormat<?,?>> cls
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(OUTPUT_FORMAT_CLASS_ATTR, cls, OutputFormat.class);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Set the {@link Mapper} for the job.
|
|
|
* @param cls the <code>Mapper</code> to use
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setMapperClass(Class<? extends Mapper<?,?,?,?>> cls) {
|
|
|
+ public void setMapperClass(Class<? extends Mapper<?,?,?,?>> cls
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(MAP_CLASS_ATTR, cls, Mapper.class);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Set the combiner class for the job.
|
|
|
* @param cls the combiner to use
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setCombinerClass(Class<? extends Reducer<?,?,?,?>> cls) {
|
|
|
+ public void setCombinerClass(Class<? extends Reducer<?,?,?,?>> cls
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(COMBINE_CLASS_ATTR, cls, Reducer.class);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Set the {@link Reducer} for the job.
|
|
|
* @param cls the <code>Reducer</code> to use
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setReducerClass(Class<? extends Reducer<?,?,?,?>> cls) {
|
|
|
+ public void setReducerClass(Class<? extends Reducer<?,?,?,?>> cls
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(REDUCE_CLASS_ATTR, cls, Reducer.class);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Set the {@link Partitioner} for the job.
|
|
|
* @param cls the <code>Partitioner</code> to use
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setPartitionerClass(Class<? extends Partitioner<?,?>> cls) {
|
|
|
+ public void setPartitionerClass(Class<? extends Partitioner<?,?>> cls
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(PARTITIONER_CLASS_ATTR, cls, Partitioner.class);
|
|
|
}
|
|
|
|
|
@@ -118,8 +134,10 @@ public class Job extends JobContext {
|
|
|
* value class.
|
|
|
*
|
|
|
* @param theClass the map output key class.
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setMapOutputKeyClass(Class<?> theClass) {
|
|
|
+ public void setMapOutputKeyClass(Class<?> theClass
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(MAP_OUTPUT_KEY_CLASS_ATTR, theClass, Object.class);
|
|
|
}
|
|
|
|
|
@@ -129,8 +147,10 @@ public class Job extends JobContext {
|
|
|
* value class.
|
|
|
*
|
|
|
* @param theClass the map output value class.
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setMapOutputValueClass(Class<?> theClass) {
|
|
|
+ public void setMapOutputValueClass(Class<?> theClass
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(MAP_OUTPUT_VALUE_CLASS_ATTR, theClass, Object.class);
|
|
|
}
|
|
|
|
|
@@ -138,8 +158,10 @@ public class Job extends JobContext {
|
|
|
* Set the key class for the job output data.
|
|
|
*
|
|
|
* @param theClass the key class for the job output data.
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setOutputKeyClass(Class<?> theClass) {
|
|
|
+ public void setOutputKeyClass(Class<?> theClass
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(OUTPUT_KEY_CLASS_ATTR, theClass, Object.class);
|
|
|
}
|
|
|
|
|
@@ -147,8 +169,10 @@ public class Job extends JobContext {
|
|
|
* Set the value class for job outputs.
|
|
|
*
|
|
|
* @param theClass the value class for job outputs.
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setOutputValueClass(Class<?> theClass) {
|
|
|
+ public void setOutputValueClass(Class<?> theClass
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(OUTPUT_VALUE_CLASS_ATTR, theClass, Object.class);
|
|
|
}
|
|
|
|
|
@@ -156,8 +180,10 @@ public class Job extends JobContext {
|
|
|
* Define the comparator that controls how the keys are sorted before they
|
|
|
* are passed to the {@link Reducer}.
|
|
|
* @param cls the raw comparator
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setSortComparatorClass(Class<? extends RawComparator<?>> cls) {
|
|
|
+ public void setSortComparatorClass(Class<? extends RawComparator<?>> cls
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(SORT_COMPARATOR_ATTR, cls, RawComparator.class);
|
|
|
}
|
|
|
|
|
@@ -166,8 +192,10 @@ public class Job extends JobContext {
|
|
|
* for a single call to
|
|
|
* {@link Reducer#reduce(Object, Iterable, org.apache.hadoop.mapreduce.Reducer.Context)}
|
|
|
* @param cls the raw comparator to use
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setGroupingComparatorClass(Class<? extends RawComparator<?>> cls){
|
|
|
+ public void setGroupingComparatorClass(Class<? extends RawComparator<?>> cls
|
|
|
+ ) throws IllegalStateException {
|
|
|
conf.setClass(GROUPING_COMPARATOR_ATTR, cls, RawComparator.class);
|
|
|
}
|
|
|
|
|
@@ -175,8 +203,9 @@ public class Job extends JobContext {
|
|
|
* Set the user-specified job name.
|
|
|
*
|
|
|
* @param name the job's new name.
|
|
|
+ * @throws IllegalStateException if the job is submitted
|
|
|
*/
|
|
|
- public void setJobName(String name) {
|
|
|
+ public void setJobName(String name) throws IllegalStateException {
|
|
|
conf.set(JOB_NAME_ATTR, name);
|
|
|
}
|
|
|
|