@InterfaceAudience.Public @InterfaceStability.Evolving public class Job extends JobContextImpl implements JobContext, AutoCloseable
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);
 | Modifier and Type | Class and Description | 
|---|---|
| static class  | Job.JobState | 
| static class  | Job.TaskStatusFilter | 
| Modifier and Type | Field and Description | 
|---|---|
| static String | COMPLETION_POLL_INTERVAL_KEYKey in mapred-*.xml that sets completionPollInvervalMillis | 
| static int | DEFAULT_SUBMIT_REPLICATION | 
| static boolean | DEFAULT_USE_WILDCARD_FOR_LIBJARS | 
| static String | OUTPUT_FILTER | 
| static String | PROGRESS_MONITOR_POLL_INTERVAL_KEYKey in mapred-*.xml that sets progMonitorPollIntervalMillis | 
| static String | SUBMIT_REPLICATION | 
| static String | USE_WILDCARD_FOR_LIBJARS | 
| static String | USED_GENERIC_PARSER | 
conf, credentials, ugiAM_NODE_LABEL_EXP, AM_STRICT_LOCALITY, APPLICATION_ATTEMPT_ID, APPLICATION_MASTER_CLASS, ARCHIVES_FOR_SHARED_CACHE, CACHE_ARCHIVES, CACHE_ARCHIVES_SHARED_CACHE_UPLOAD_POLICIES, CACHE_ARCHIVES_SIZES, CACHE_ARCHIVES_TIMESTAMPS, CACHE_ARCHIVES_VISIBILITIES, CACHE_FILE_TIMESTAMPS, CACHE_FILE_VISIBILITIES, CACHE_FILES, CACHE_FILES_SHARED_CACHE_UPLOAD_POLICIES, 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_FINISH_JOB_WHEN_REDUCERS_DONE, DEFAULT_HEAP_MEMORY_MB_RATIO, DEFAULT_IO_SORT_FACTOR, DEFAULT_IO_SORT_MB, DEFAULT_JOB_ACL_MODIFY_JOB, DEFAULT_JOB_ACL_VIEW_JOB, DEFAULT_JOB_AM_ACCESS_DISABLED, DEFAULT_JOB_MAX_MAP, DEFAULT_JOB_RUNNING_MAP_LIMIT, DEFAULT_JOB_RUNNING_REDUCE_LIMIT, DEFAULT_JOB_SINGLE_DISK_LIMIT_BYTES, DEFAULT_JOB_SINGLE_DISK_LIMIT_CHECK_INTERVAL_MS, DEFAULT_JOB_SINGLE_DISK_LIMIT_KILL_LIMIT_EXCEED, 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_MAPREDUCE_JVM_SYSTEM_PROPERTIES_TO_LOG, 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_ADMIN_USER_ENV, 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_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE, DEFAULT_MR_AM_CPU_VCORES, DEFAULT_MR_AM_HARD_KILL_TIMEOUT_MS, 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_STAGING_ERASURECODING_ENABLED, 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_JOB_MAX_RETRIES, DEFAULT_MR_CLIENT_JOB_RETRY_INTERVAL, 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_MR_JOB_REDUCER_UNCONDITIONAL_PREEMPT_DELAY_SEC, DEFAULT_MR_NUM_OPPORTUNISTIC_MAPS_PERCENT, DEFAULT_REDUCE_CPU_VCORES, DEFAULT_REDUCE_MEMORY_MB, DEFAULT_REDUCE_SEPARATE_SHUFFLE_LOG, DEFAULT_SHELL, DEFAULT_SHUFFLE_FETCH_RETRY_INTERVAL_MS, DEFAULT_SHUFFLE_INPUT_BUFFER_PERCENT, DEFAULT_SHUFFLE_LOG_BACKUPS, DEFAULT_SHUFFLE_LOG_KB, DEFAULT_SHUFFLE_MERGE_PERCENT, DEFAULT_SPECULATIVE_MINIMUM_ALLOWED_TASKS, DEFAULT_SPECULATIVE_RETRY_AFTER_NO_SPECULATE, DEFAULT_SPECULATIVE_RETRY_AFTER_SPECULATE, DEFAULT_SPECULATIVECAP_RUNNING_TASKS, DEFAULT_SPECULATIVECAP_TOTAL_TASKS, DEFAULT_SPLIT_METAINFO_MAXSIZE, DEFAULT_TASK_ISMAP, DEFAULT_TASK_LOCAL_WRITE_LIMIT_BYTES, DEFAULT_TASK_LOG_BACKUPS, DEFAULT_TASK_PROFILE_PARAMS, DEFAULT_TASK_TIMEOUT_MILLIS, FILES_FOR_CLASSPATH_AND_SHARED_CACHE, FILES_FOR_SHARED_CACHE, FINISH_JOB_WHEN_REDUCERS_DONE, GROUP_COMPARATOR_CLASS, HADOOP_WORK_DIR, HEAP_MEMORY_MB_RATIO, 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_MAX_MAP, JOB_NAME, JOB_NAMENODES, JOB_NAMENODES_TOKEN_RENEWAL_EXCLUDE, JOB_NODE_LABEL_EXP, JOB_RUNNING_MAP_LIMIT, JOB_RUNNING_REDUCE_LIMIT, JOB_SINGLE_DISK_LIMIT_BYTES, JOB_SINGLE_DISK_LIMIT_CHECK_INTERVAL_MS, JOB_SINGLE_DISK_LIMIT_KILL_LIMIT_EXCEED, 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, JOBJAR_SHARED_CACHE_UPLOAD_POLICY, JOBJAR_SHARED_CACHE_UPLOAD_POLICY_DEFAULT, JOBJAR_VISIBILITY, JOBJAR_VISIBILITY_DEFAULT, 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_NODE_LABEL_EXP, MAP_OUTPUT_COLLECTOR_CLASS_ATTR, MAP_OUTPUT_COMPRESS, MAP_OUTPUT_COMPRESS_CODEC, MAP_OUTPUT_KEY_CLASS, MAP_OUTPUT_KEY_FIELD_SEPARATOR, MAP_OUTPUT_KEY_FIELD_SEPERATOR, MAP_OUTPUT_VALUE_CLASS, MAP_RESOURCE_TYPE_PREFIX, MAP_SKIP_INCR_PROC_COUNT, MAP_SKIP_MAX_RECORDS, MAP_SORT_CLASS, 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_JVM_SYSTEM_PROPERTIES_TO_LOG, MAPREDUCE_V2_CHILD_CLASS, MAX_ALLOWED_FETCH_FAILURES_FRACTION, MAX_FETCH_FAILURES_NOTIFICATIONS, MAX_RESOURCES, MAX_RESOURCES_DEFAULT, MAX_RESOURCES_MB, MAX_RESOURCES_MB_DEFAULT, MAX_SHUFFLE_FETCH_HOST_FAILURES, MAX_SHUFFLE_FETCH_RETRY_DELAY, MAX_SINGLE_RESOURCE_MB, MAX_SINGLE_RESOURCE_MB_DEFAULT, 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_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE, MR_AM_CPU_VCORES, MR_AM_CREATE_JH_INTERMEDIATE_BASE_DIR, MR_AM_ENV, MR_AM_HARD_KILL_TIMEOUT_MS, 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_PREEMPTION_POLICY, MR_AM_PREFIX, MR_AM_PROFILE, MR_AM_PROFILE_PARAMS, MR_AM_RESOURCE_PREFIX, MR_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT, MR_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL, MR_AM_STAGING_DIR, MR_AM_STAGING_DIR_ERASURECODING_ENABLED, 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_AM_WEBAPP_PORT_RANGE, MR_APPLICATION_TYPE, MR_CLIENT_JOB_MAX_RETRIES, MR_CLIENT_JOB_RETRY_INTERVAL, 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_REDACTED_PROPERTIES, MR_JOB_REDUCER_PREEMPT_DELAY_SEC, MR_JOB_REDUCER_UNCONDITIONAL_PREEMPT_DELAY_SEC, MR_JOB_SEND_TOKEN_CONF, MR_NUM_OPPORTUNISTIC_MAPS_PERCENT, 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_NODE_LABEL_EXP, REDUCE_RESOURCE_TYPE_PREFIX, REDUCE_SEPARATE_SHUFFLE_LOG, REDUCE_SKIP_INCR_PROC_COUNT, REDUCE_SKIP_MAXGROUPS, REDUCE_SPECULATIVE, RESERVATION_ID, RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY, RESOURCE_TYPE_NAME_MEMORY, RESOURCE_TYPE_NAME_VCORE, SETUP_CLEANUP_NEEDED, SHARED_CACHE_MODE, SHARED_CACHE_MODE_DEFAULT, 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_LOG_BACKUPS, SHUFFLE_LOG_KB, SHUFFLE_MEMORY_LIMIT_PERCENT, SHUFFLE_MERGE_PERCENT, SHUFFLE_NOTIFY_READERROR, SHUFFLE_PARALLEL_COPIES, SHUFFLE_READ_TIMEOUT, SKIP_OUTDIR, SKIP_RECORDS, SKIP_START_ATTEMPTS, SPECULATIVE_MINIMUM_ALLOWED_TASKS, SPECULATIVE_RETRY_AFTER_NO_SPECULATE, SPECULATIVE_RETRY_AFTER_SPECULATE, SPECULATIVE_SLOWNODE_THRESHOLD, SPECULATIVE_SLOWTASK_THRESHOLD, SPECULATIVECAP, SPECULATIVECAP_RUNNING_TASKS, SPECULATIVECAP_TOTAL_TASKS, SPLIT_FILE, SPLIT_METAINFO_MAXSIZE, STDERR_LOGFILE_ENV, STDOUT_LOGFILE_ENV, TASK_ATTEMPT_ID, TASK_CLEANUP_NEEDED, TASK_DEBUGOUT_LINES, TASK_EXIT_TIMEOUT, TASK_EXIT_TIMEOUT_CHECK_INTERVAL_MS, TASK_EXIT_TIMEOUT_CHECK_INTERVAL_MS_DEFAULT, TASK_EXIT_TIMEOUT_DEFAULT, TASK_ID, TASK_ISMAP, TASK_LOCAL_WRITE_LIMIT_BYTES, TASK_LOG_BACKUPS, TASK_MAP_PROFILE_PARAMS, TASK_OUTPUT_DIR, TASK_PARTITION, TASK_PREEMPTION, TASK_PROFILE, TASK_PROFILE_PARAMS, TASK_PROGRESS_REPORT_INTERVAL, TASK_REDUCE_PROFILE_PARAMS, 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 and Description | 
|---|
| Job()Deprecated. 
 Use  getInstance() | 
| Job(org.apache.hadoop.conf.Configuration conf)Deprecated. 
 | 
| Job(org.apache.hadoop.conf.Configuration conf,
   String jobName)Deprecated. 
 | 
| Modifier and Type | Method and Description | 
|---|---|
| void | addArchiveToClassPath(org.apache.hadoop.fs.Path archive)Add an archive path to the current set of classpath entries. | 
| static boolean | addArchiveToSharedCache(URI resource,
                       org.apache.hadoop.conf.Configuration conf)Add an archive to job config for shared cache processing. | 
| void | addCacheArchive(URI uri)Add a archives to be localized | 
| void | addCacheFile(URI uri)Add a file to be localized | 
| void | addFileToClassPath(org.apache.hadoop.fs.Path file)Add an file path to the current set of classpath entries It adds the file
 to cache as well. | 
| static boolean | addFileToSharedCache(URI resource,
                    org.apache.hadoop.conf.Configuration conf)Add a file to job config for shared cache processing. | 
| static boolean | addFileToSharedCacheAndClasspath(URI resource,
                                org.apache.hadoop.conf.Configuration conf)Add a file to job config for shared cache processing. | 
| float | cleanupProgress()Get the progress of the job's cleanup-tasks, as a float between 0.0 
 and 1.0. | 
| void | close()Close the  Job. | 
| void | createSymlink()Deprecated.  | 
| void | failTask(TaskAttemptID taskId)Fail indicated task attempt. | 
| static Map<String,Boolean> | getArchiveSharedCacheUploadPolicies(org.apache.hadoop.conf.Configuration conf)This is to get the shared cache upload policies for archives. | 
| Cluster | getCluster() | 
| static int | getCompletionPollInterval(org.apache.hadoop.conf.Configuration conf)The interval at which waitForCompletion() should check. | 
| Counters | getCounters()Gets the counters for this job. | 
| static Map<String,Boolean> | getFileSharedCacheUploadPolicies(org.apache.hadoop.conf.Configuration conf)This is to get the shared cache upload policies for files. | 
| long | getFinishTime()Get finish time of the job. | 
| String | getHistoryUrl() | 
| static Job | getInstance() | 
| static Job | getInstance(Cluster ignored)Deprecated. 
 Use  getInstance() | 
| static Job | getInstance(Cluster ignored,
           org.apache.hadoop.conf.Configuration conf)Deprecated. 
 | 
| static Job | getInstance(Cluster cluster,
           JobStatus status,
           org.apache.hadoop.conf.Configuration conf) | 
| static Job | getInstance(org.apache.hadoop.conf.Configuration conf) | 
| static Job | getInstance(org.apache.hadoop.conf.Configuration conf,
           String jobName) | 
| static Job | getInstance(JobStatus status,
           org.apache.hadoop.conf.Configuration conf) | 
| String | getJobFile()Get the path of the submitted job configuration. | 
| String | getJobName()The user-specified job name. | 
| JobStatus.State | getJobState()Returns the current state of the Job. | 
| org.apache.hadoop.mapreduce.JobSubmitter | getJobSubmitter(org.apache.hadoop.fs.FileSystem fs,
               ClientProtocol submitClient)Only for mocking via unit tests. | 
| JobPriority | getPriority()Get scheduling info of the job. | 
| static int | getProgressPollInterval(org.apache.hadoop.conf.Configuration conf)The interval at which monitorAndPrintJob() prints status | 
| org.apache.hadoop.yarn.api.records.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 Job.TaskStatusFilter | getTaskOutputFilter(org.apache.hadoop.conf.Configuration conf)Get the task output filter. | 
| 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. | 
| boolean | 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. | 
| 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. | 
| static void | setArchiveSharedCacheUploadPolicies(org.apache.hadoop.conf.Configuration conf,
                                   Map<String,Boolean> policies)This is to set the shared cache upload policies for archives. | 
| 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 org.apache.hadoop.io.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) | 
| static void | setFileSharedCacheUploadPolicies(org.apache.hadoop.conf.Configuration conf,
                                Map<String,Boolean> policies)This is to set the shared cache upload policies for files. | 
| void | setGroupingComparatorClass(Class<? extends org.apache.hadoop.io.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  InputFormatfor 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  Mapperfor 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  OutputFormatfor 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  Partitionerfor the job. | 
| void | setPriority(JobPriority jobPriority)Set the priority of a running job. | 
| void | setPriorityAsInteger(int jobPriority)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  Reducerfor the job. | 
| void | setReduceSpeculativeExecution(boolean speculativeExecution)Turn speculative execution on or off for this job for reduce tasks. | 
| void | setReservationId(org.apache.hadoop.yarn.api.records.ReservationId reservationId)Set the reservation to which the job is submitted to | 
| void | setSortComparatorClass(Class<? extends org.apache.hadoop.io.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(org.apache.hadoop.conf.Configuration conf,
                   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(org.apache.hadoop.fs.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. | 
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, setJobIDclone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, waitgetArchiveClassPaths, 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, getWorkingDirectorypublic static final String OUTPUT_FILTER
public static final String COMPLETION_POLL_INTERVAL_KEY
public static final String PROGRESS_MONITOR_POLL_INTERVAL_KEY
public static final String USED_GENERIC_PARSER
public static final String SUBMIT_REPLICATION
public static final int DEFAULT_SUBMIT_REPLICATION
public static final String USE_WILDCARD_FOR_LIBJARS
public static final boolean DEFAULT_USE_WILDCARD_FOR_LIBJARS
@Deprecated public Job() throws IOException
getInstance()IOException@Deprecated public Job(org.apache.hadoop.conf.Configuration conf) throws IOException
getInstance(Configuration)IOException@Deprecated public Job(org.apache.hadoop.conf.Configuration conf, String jobName) throws IOException
getInstance(Configuration, String)IOExceptionpublic static Job getInstance() throws IOException
Job with no particular Cluster .
 A Cluster will be created with a generic Configuration.Job , with no connection to a cluster yet.IOExceptionpublic static Job getInstance(org.apache.hadoop.conf.Configuration conf) throws IOException
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.conf - the configurationJob , with no connection to a cluster yet.IOExceptionpublic static Job getInstance(org.apache.hadoop.conf.Configuration conf, String jobName) throws IOException
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.conf - the configurationJob , with no connection to a cluster yet.IOExceptionpublic static Job getInstance(JobStatus status, org.apache.hadoop.conf.Configuration conf) throws IOException
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.status - job statusconf - job configurationJob , with no connection to a cluster yet.IOException@Deprecated public static Job getInstance(Cluster ignored) throws IOException
getInstance()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.ignored - Job , with no connection to a cluster yet.IOException@Deprecated public static Job getInstance(Cluster ignored, org.apache.hadoop.conf.Configuration conf) throws IOException
getInstance(Configuration)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.ignored - conf - job configurationJob , with no connection to a cluster yet.IOException@InterfaceAudience.Private public static Job getInstance(Cluster cluster, JobStatus status, org.apache.hadoop.conf.Configuration conf) throws IOException
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.cluster - clusterstatus - job statusconf - job configurationJob , with no connection to a cluster yet.IOExceptionpublic JobStatus getStatus() throws IOException, InterruptedException
IOExceptionInterruptedExceptionpublic JobStatus.State getJobState() throws IOException, InterruptedException
IOExceptionInterruptedExceptionpublic String getTrackingURL()
public String getJobFile()
public long getStartTime()
public long getFinishTime()
                   throws IOException,
                          InterruptedException
IOExceptionInterruptedExceptionpublic String getSchedulingInfo()
public JobPriority getPriority() throws IOException, InterruptedException
IOExceptionInterruptedExceptionpublic String getJobName()
getJobName in interface JobContextgetJobName in class JobContextImplpublic String getHistoryUrl() throws IOException, InterruptedException
IOExceptionInterruptedExceptionpublic boolean isRetired()
                  throws IOException,
                         InterruptedException
IOExceptionInterruptedException@InterfaceAudience.Private public Cluster getCluster()
public String toString()
toString in class JobContextImplpublic TaskReport[] getTaskReports(TaskType type) throws IOException, InterruptedException
type - Type of the taskIOExceptionInterruptedExceptionpublic float mapProgress()
                  throws IOException
IOExceptionpublic float reduceProgress()
                     throws IOException
IOExceptionpublic float cleanupProgress()
                      throws IOException,
                             InterruptedException
IOExceptionInterruptedExceptionpublic float setupProgress()
                    throws IOException
IOExceptionpublic boolean isComplete()
                   throws IOException
true if the job is complete, else false.IOExceptionpublic boolean isSuccessful()
                     throws IOException
true if the job succeeded, else false.IOExceptionpublic void killJob()
             throws IOException
IOExceptionpublic void setPriority(JobPriority jobPriority) throws IOException, InterruptedException
jobPriority - the new priority for the job.IOExceptionInterruptedExceptionpublic void setPriorityAsInteger(int jobPriority)
                          throws IOException,
                                 InterruptedException
jobPriority - the new priority for the job.IOExceptionInterruptedExceptionpublic TaskCompletionEvent[] getTaskCompletionEvents(int startFrom, int numEvents) throws IOException, InterruptedException
startFrom - index to start fetching events fromnumEvents - number of events to fetchTaskCompletionEventsIOExceptionInterruptedExceptionpublic TaskCompletionEvent[] getTaskCompletionEvents(int startFrom) throws IOException
startFrom - index to start fetching events fromTaskCompletionEventsIOException@InterfaceAudience.Private public boolean killTask(TaskAttemptID taskId, boolean shouldFail) throws IOException
taskId - the id of the task to kill.shouldFail - if true the task is failed and added
                   to failed tasks list, otherwise it is just killed,
                   w/o affecting job failure status.IOExceptionpublic void killTask(TaskAttemptID taskId) throws IOException
taskId - the id of the task to be terminated.IOExceptionpublic void failTask(TaskAttemptID taskId) throws IOException
taskId - the id of the task to be terminated.IOExceptionpublic Counters getCounters() throws IOException
IOExceptionpublic String[] getTaskDiagnostics(TaskAttemptID taskid) throws IOException, InterruptedException
taskid - IOExceptionInterruptedExceptionpublic void setNumReduceTasks(int tasks)
                       throws IllegalStateException
tasks - the number of reduce tasksIllegalStateException - if the job is submittedpublic void setWorkingDirectory(org.apache.hadoop.fs.Path dir)
                         throws IOException
dir - the new current working directory.IllegalStateException - if the job is submittedIOExceptionpublic void setInputFormatClass(Class<? extends InputFormat> cls) throws IllegalStateException
InputFormat for the job.cls - the InputFormat to useIllegalStateException - if the job is submittedpublic void setOutputFormatClass(Class<? extends OutputFormat> cls) throws IllegalStateException
OutputFormat for the job.cls - the OutputFormat to useIllegalStateException - if the job is submittedpublic void setMapperClass(Class<? extends Mapper> cls) throws IllegalStateException
Mapper for the job.cls - the Mapper to useIllegalStateException - if the job is submittedpublic void setJarByClass(Class<?> cls)
cls - the example classpublic void setJar(String jar)
public void setUser(String user)
user - the username for this job.public void setCombinerClass(Class<? extends Reducer> cls) throws IllegalStateException
cls - the combiner to useIllegalStateException - if the job is submittedpublic void setReducerClass(Class<? extends Reducer> cls) throws IllegalStateException
Reducer for the job.cls - the Reducer to useIllegalStateException - if the job is submittedpublic void setPartitionerClass(Class<? extends Partitioner> cls) throws IllegalStateException
Partitioner for the job.cls - the Partitioner to useIllegalStateException - if the job is submittedpublic void setMapOutputKeyClass(Class<?> theClass) throws IllegalStateException
theClass - the map output key class.IllegalStateException - if the job is submittedpublic void setMapOutputValueClass(Class<?> theClass) throws IllegalStateException
theClass - the map output value class.IllegalStateException - if the job is submittedpublic void setOutputKeyClass(Class<?> theClass) throws IllegalStateException
theClass - the key class for the job output data.IllegalStateException - if the job is submittedpublic void setOutputValueClass(Class<?> theClass) throws IllegalStateException
theClass - the value class for job outputs.IllegalStateException - if the job is submittedpublic void setCombinerKeyGroupingComparatorClass(Class<? extends org.apache.hadoop.io.RawComparator> cls) throws IllegalStateException
Reducer.reduce(Object, Iterable,
 org.apache.hadoop.mapreduce.Reducer.Context)cls - the raw comparator to useIllegalStateException - if the job is submittedpublic void setSortComparatorClass(Class<? extends org.apache.hadoop.io.RawComparator> cls) throws IllegalStateException
Reducer.cls - the raw comparatorIllegalStateException - if the job is submittedsetCombinerKeyGroupingComparatorClass(Class)public void setGroupingComparatorClass(Class<? extends org.apache.hadoop.io.RawComparator> cls) throws IllegalStateException
Reducer.reduce(Object, Iterable, 
                       org.apache.hadoop.mapreduce.Reducer.Context)cls - the raw comparator to useIllegalStateException - if the job is submittedsetCombinerKeyGroupingComparatorClass(Class)public void setJobName(String name) throws IllegalStateException
name - the job's new name.IllegalStateException - if the job is submittedpublic void setSpeculativeExecution(boolean speculativeExecution)
speculativeExecution - true if speculative execution 
                             should be turned on, else false.public void setMapSpeculativeExecution(boolean speculativeExecution)
speculativeExecution - true if speculative execution 
                             should be turned on for map tasks,
                             else false.public void setReduceSpeculativeExecution(boolean speculativeExecution)
speculativeExecution - true if speculative execution 
                             should be turned on for reduce tasks,
                             else false.public void setJobSetupCleanupNeeded(boolean needed)
needed - If true, job-setup and job-cleanup will be
               considered from OutputCommitter 
               else ignored.public void setCacheArchives(URI[] archives)
archives - The list of archives that need to be localizedpublic void setCacheFiles(URI[] files)
files - The list of files that need to be localizedpublic void addCacheArchive(URI uri)
uri - The uri of the cache to be localizedpublic void addCacheFile(URI uri)
uri - The uri of the cache to be localizedpublic void addFileToClassPath(org.apache.hadoop.fs.Path file)
                        throws IOException
addArchiveToClassPath(Path)
 method instead.file - Path of the file to be addedIOExceptionpublic void addArchiveToClassPath(org.apache.hadoop.fs.Path archive)
                           throws IOException
archive - Path of the archive to be addedIOException@Deprecated public void createSymlink()
public void setMaxMapAttempts(int n)
n - the number of attempts per map task.public void setMaxReduceAttempts(int n)
n - the number of attempts per reduce task.public void setProfileEnabled(boolean newValue)
newValue - true means it should be gatheredpublic void setProfileParams(String value)
value - the configuration stringpublic void setProfileTaskRange(boolean isMap,
                                String newValue)
newValue - a set of integer ranges of the map idspublic void setCancelDelegationTokenUponJobCompletion(boolean value)
@InterfaceStability.Unstable public static boolean addFileToSharedCache(URI resource, org.apache.hadoop.conf.Configuration conf)
resource - The resource that Job Submitter will process later using
          shared cache.conf - Configuration to add the resource to@InterfaceStability.Unstable public static boolean addFileToSharedCacheAndClasspath(URI resource, org.apache.hadoop.conf.Configuration conf)
resource - The resource that Job Submitter will process later using
          shared cache.conf - Configuration to add the resource to@InterfaceStability.Unstable public static boolean addArchiveToSharedCache(URI resource, org.apache.hadoop.conf.Configuration conf)
resource - The resource that Job Submitter will process later using
          shared cache.conf - Configuration to add the resource to@InterfaceStability.Unstable
public static void setFileSharedCacheUploadPolicies(org.apache.hadoop.conf.Configuration conf,
                                                                                 Map<String,Boolean> policies)
conf - Configuration which stores the shared cache upload policiespolicies - A map containing the shared cache upload policies for a set
          of resources. The key is the url of the resource and the value is
          the upload policy. True if it should be uploaded, false otherwise.@InterfaceStability.Unstable
public static void setArchiveSharedCacheUploadPolicies(org.apache.hadoop.conf.Configuration conf,
                                                                                    Map<String,Boolean> policies)
conf - Configuration which stores the shared cache upload policiespolicies - A map containing the shared cache upload policies for a set
          of resources. The key is the url of the resource and the value is
          the upload policy. True if it should be uploaded, false otherwise.@InterfaceStability.Unstable public static Map<String,Boolean> getFileSharedCacheUploadPolicies(org.apache.hadoop.conf.Configuration conf)
conf - Configuration which stores the shared cache upload policies@InterfaceStability.Unstable public static Map<String,Boolean> getArchiveSharedCacheUploadPolicies(org.apache.hadoop.conf.Configuration conf)
conf - Configuration which stores the shared cache upload policies@InterfaceAudience.Private
public org.apache.hadoop.mapreduce.JobSubmitter getJobSubmitter(org.apache.hadoop.fs.FileSystem fs,
                                                                                           ClientProtocol submitClient)
                                                                                    throws IOException
IOExceptionpublic void submit()
            throws IOException,
                   InterruptedException,
                   ClassNotFoundException
public boolean waitForCompletion(boolean verbose)
                          throws IOException,
                                 InterruptedException,
                                 ClassNotFoundException
verbose - print the progress to the userIOException - thrown if the communication with the 
         JobTracker is lostInterruptedExceptionClassNotFoundExceptionpublic boolean monitorAndPrintJob()
                           throws IOException,
                                  InterruptedException
IOException - if communication to the JobTracker failsInterruptedExceptionpublic static int getProgressPollInterval(org.apache.hadoop.conf.Configuration conf)
public static int getCompletionPollInterval(org.apache.hadoop.conf.Configuration conf)
public static Job.TaskStatusFilter getTaskOutputFilter(org.apache.hadoop.conf.Configuration conf)
conf - the configuration.public static void setTaskOutputFilter(org.apache.hadoop.conf.Configuration conf,
                                       Job.TaskStatusFilter newValue)
conf - the Configuration to modify.newValue - the value to set.public boolean isUber()
               throws IOException,
                      InterruptedException
IOExceptionInterruptedExceptionpublic org.apache.hadoop.yarn.api.records.ReservationId getReservationId()
public void setReservationId(org.apache.hadoop.yarn.api.records.ReservationId reservationId)
reservationId - the reservationId to setpublic void close()
           throws IOException
Job.close in interface AutoCloseableIOException - if fail to close.Copyright © 2008–2018 Apache Software Foundation. All rights reserved.