/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
import java.io.BufferedReader;
import java.io.BufferedWriter;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.io.OutputStreamWriter;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL;
import java.net.URLConnection;
import java.net.UnknownHostException;
import java.security.PrivilegedExceptionAction;
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.filecache.DistributedCache;
import org.apache.hadoop.filecache.TrackerDistributedCacheManager;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.mapred.Counters.Counter;
import org.apache.hadoop.mapred.Counters.Group;
import org.apache.hadoop.mapred.QueueManager.QueueACL;
import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.JobSubmissionFiles;
import org.apache.hadoop.mapreduce.security.TokenCache;
import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.mapreduce.split.JobSplitWriter;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.codehaus.jackson.JsonParseException;
import org.codehaus.jackson.map.JsonMappingException;
import org.codehaus.jackson.map.ObjectMapper;
/**
* <code>JobClient</code> is the primary interface for the user-job to interact
* with the {@link JobTracker}.
*
* <code>JobClient</code> provides facilities to submit jobs, track their
* progress, access component-tasks' reports/logs, get the Map-Reduce cluster
* status information etc.
*
* <p>The job submission process involves:
* <ol>
* <li>
* Checking the input and output specifications of the job.
* </li>
* <li>
* Computing the {@link InputSplit}s for the job.
* </li>
* <li>
* Setup the requisite accounting information for the {@link DistributedCache}
* of the job, if necessary.
* </li>
* <li>
* Copying the job's jar and configuration to the map-reduce system directory
* on the distributed file-system.
* </li>
* <li>
* Submitting the job to the <code>JobTracker</code> and optionally monitoring
* it's status.
* </li>
* </ol></p>
*
* Normally the user creates the application, describes various facets of the
* job via {@link JobConf} and then uses the <code>JobClient</code> to submit
* the job and monitor its progress.
*
* <p>Here is an example on how to use <code>JobClient</code>:</p>
* <p><blockquote><pre>
* // Create a new JobConf
* JobConf job = new JobConf(new Configuration(), 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
* JobClient.runJob(job);
* </pre></blockquote></p>
*
* <h4 id="JobControl">Job Control</h4>
*
* <p>At times clients would chain map-reduce jobs to accomplish complex tasks
* which cannot be done via a single map-reduce job. This is fairly easy since
* the output of the job, typically, goes to distributed file-system and that
* can be used as the input for the next job.</p>
*
* <p>However, this also means that the onus on ensuring jobs are complete
* (success/failure) lies squarely on the clients. In such situations the
* various job-control options are:
* <ol>
* <li>
* {@link #runJob(JobConf)} : submits the job and returns only after
* the job has completed.
* </li>
* <li>
* {@link #submitJob(JobConf)} : only submits the job, then poll the
* returned handle to the {@link RunningJob} to query status and make
* scheduling decisions.
* </li>
* <li>
* {@link JobConf#setJobEndNotificationURI(String)} : setup a notification
* on job-completion, thus avoiding polling.
* </li>
* </ol></p>
*
* @see JobConf
* @see ClusterStatus
* @see Tool
* @see DistributedCache
*/
public class JobClient extends Configured implements MRConstants, Tool {
private static final Log LOG = LogFactory.getLog(JobClient.class);
public static enum TaskStatusFilter { NONE, KILLED, FAILED, SUCCEEDED, ALL }
private TaskStatusFilter taskOutputFilter = TaskStatusFilter.FAILED;
private static final long MAX_JOBPROFILE_AGE = 1000 * 2;
static{
Configuration.addDefaultResource("mapred-default.xml");
Configuration.addDefaultResource("mapred-site.xml");
}
/**
* A NetworkedJob is an implementation of RunningJob. It holds
* a JobProfile object to provide some info, and interacts with the
* remote service to provide certain functionality.
*/
static class NetworkedJob implements RunningJob {
private JobSubmissionProtocol jobSubmitClient;
JobProfile profile;
JobStatus status;
long statustime;
/**
* We store a JobProfile and a timestamp for when we last
* acquired the job profile. If the job is null, then we cannot
* perform any of the tasks, so we throw an exception.
* The job might be null if the JobTracker has completely forgotten
* about the job. (eg, 24 hours after the job completes.)
*/
public NetworkedJob(JobStatus job, JobProfile prof, JobSubmissionProtocol jobSubmitClient) throws IOException {
this.status = job;
this.profile = prof;
this.jobSubmitClient = jobSubmitClient;
if(this.status == null) {
throw new IOException("The Job status cannot be null");
}
if(this.profile == null) {
throw new IOException("The Job profile cannot be null");
}
if(this.jobSubmitClient == null) {
throw new IOException("The Job Submission Protocol cannot be null");
}
this.statustime = System.currentTimeMillis();
}
/**
* Some methods rely on having a recent job profile object. Refresh
* it, if necessary
*/
synchronized void ensureFreshStatus() throws IOException {
if (System.currentTimeMillis() - statustime > MAX_JOBPROFILE_AGE) {
updateStatus();
}
}
/** Some methods need to update status immediately. So, refresh
* immediately
* @throws IOException
*/
synchronized void updateStatus() throws IOException {
this.status = jobSubmitClient.getJobStatus(profile.getJobID());
if(this.status == null) {
throw new IOException("The job appears to have been removed.");
}
this.statustime = System.currentTimeMillis();
}
/**
* An identifier for the job
*/
public JobID getID() {
return profile.getJobID();
}
/** @deprecated This method is deprecated and will be removed. Applications should
* rather use {@link #getID()}.*/
@Deprecated
public String getJobID() {
return profile.getJobID().toString();
}
/**
* The user-specified job name
*/
public String getJobName() {
return profile.getJobName();
}
/**
* The name of the job file
*/
public String getJobFile() {
return profile.getJobFile();
}
/**
* A URL where the job's status can be seen
*/
public String getTrackingURL() {
return profile.getURL().toString();
}
/**
* A float between 0.0 and 1.0, indicating the % of map work
* completed.
*/
public float mapProgress() throws IOException {
ensureFreshStatus();
return status.mapProgress();
}
/**
* A float between 0.0 and 1.0, indicating the % of reduce work
* completed.
*/
public float reduceProgress() throws IOException {
ensureFreshStatus();
return status.reduceProgress();
}
/**
* A float between 0.0 and 1.0, indicating the % of cleanup work
* completed.
*/
public float cleanupProgress() throws IOException {
ensureFreshStatus();
return status.cleanupProgress();
}
/**
* A float between 0.0 and 1.0, indicating the % of setup work
* completed.
*/
public float setupProgress() throws IOException {
ensureFreshStatus();
return status.setupProgress();
}
/**
* Returns immediately whether the whole job is done yet or not.
*/
public synchronized boolean isComplete() throws IOException {
updateStatus();
return (status.getRunState() == JobStatus.SUCCEEDED ||
status.getRunState() == JobStatus.FAILED ||
status.getRunState() == JobStatus.KILLED);
}
/**
* True iff job completed successfully.
*/
public synchronized boolean isSuccessful() throws IOException {
updateStatus();
return status.getRunState() == JobStatus.SUCCEEDED;
}
/**
* Blocks until the job is finished
*/
public void waitForCompletion() throws IOException {
while (!isComplete()) {
try {
Thread.sleep(5000);
} catch (InterruptedException ie) {
}
}
}
/**
* Tells the service to get the state of the current job.
*/
public synchronized int getJobState() throws IOException {
updateStatus();
return status.getRunState();
}
/**
* Tells the service to terminate the current job.
*/
public synchronized void killJob() throws IOException {
jobSubmitClient.killJob(getID());
}
/** Set the priority of the job.
* @param priority new priority of the job.
*/
public synchronized void setJobPriority(String priority)
throws IOException {
jobSubmitClient.setJobPriority(getID(), priority);
}
/**
* Kill indicated task attempt.
* @param taskId the id of the task to kill.
* @param shouldFail if true the task is failed and added to failed tasks list, otherwise
* it is just killed, w/o affecting job failure status.
*/
public synchronized void killTask(TaskAttemptID taskId, boolean shouldFail) throws IOException {
jobSubmitClient.killTask(taskId, shouldFail);
}
/** @deprecated Applications should rather use {@link #killTask(TaskAttemptID, boolean)}*/
@Deprecated
public synchronized void killTask(String taskId, boolean shouldFail) throws IOException {
killTask(TaskAttemptID.forName(taskId), shouldFail);
}
/**
* Fetch task completion events from jobtracker for this job.
*/
public synchronized TaskCompletionEvent[] getTaskCompletionEvents(
int startFrom) throws IOException{
return jobSubmitClient.getTaskCompletionEvents(
getID(), startFrom, 10);
}
/**
* Dump stats to screen
*/
@Override
public String toString() {
try {
updateStatus();
} catch (IOException e) {
}
return "Job: " + profile.getJobID() + "\n" +
"file: " + profile.getJobFile() + "\n" +
"tracking URL: " + profile.getURL() + "\n" +
"map() completion: " + status.mapProgress() + "\n" +
"reduce() completion: " + status.reduceProgress() + "\n" +
((status.getRunState() == JobStatus.FAILED) ? ("Failure Info: " + status.getFailureInfo()) : "");
}
/**
* Returns the counters for this job
*/
public Counters getCounters() throws IOException {
return jobSubmitClient.getJobCounters(getID());
}
@Override
public String[] getTaskDiagnostics(TaskAttemptID id) throws IOException {
return jobSubmitClient.getTaskDiagnostics(id);
}
@Override
public String getFailureInfo() throws IOException {
//assuming that this is just being called after
//we realized the job failed. SO we try avoiding
//a rpc by not calling updateStatus
ensureFreshStatus();
return status.getFailureInfo();
}
}
private JobSubmissionProtocol jobSubmitClient;
private Path sysDir = null;
private Path stagingAreaDir = null;
private FileSystem fs = null;
private UserGroupInformation ugi;
private static final String TASKLOG_PULL_TIMEOUT_KEY =
"mapreduce.client.tasklog.timeout";
private static final int DEFAULT_TASKLOG_TIMEOUT = 60000;
static int tasklogtimeout;
/**
* Create a job client.
*/
public JobClient() {
}
/**
* Build a job client with the given {@link JobConf}, and connect to the
* default {@link JobTracker}.
*
* @param conf the job configuration.
* @throws IOException
*/
public JobClient(JobConf conf) throws IOException {
setConf(conf);
init(conf);
}
/**
* Connect to the default {@link JobTracker}.
* @param conf the job configuration.
* @throws IOException
*/
public void init(JobConf conf) throws IOException {
String tracker = conf.get("mapred.job.tracker", "local");
tasklogtimeout = conf.getInt(
TASKLOG_PULL_TIMEOUT_KEY, DEFAULT_TASKLOG_TIMEOUT);
this.ugi = UserGroupInformation.getCurrentUser();
if ("local".equals(tracker)) {
conf.setNumMapTasks(1);
this.jobSubmitClient = new LocalJobRunner(conf);
} else {
this.jobSubmitClient = createRPCProxy(JobTracker.getAddress(conf), conf);
}
}
private static JobSubmissionProtocol createRPCProxy(InetSocketAddress addr,
Configuration conf) throws IOException {
return (JobSubmissionProtocol) RPC.getProxy(JobSubmissionProtocol.class,
JobSubmissionProtocol.versionID, addr,
UserGroupInformation.getCurrentUser(), conf,
NetUtils.getSocketFactory(conf, JobSubmissionProtocol.class));
}
@InterfaceAudience.Private
public static class Renewer extends TokenRenewer {
@Override
public boolean handleKind(Text kind) {
return DelegationTokenIdentifier.MAPREDUCE_DELEGATION_KIND.equals(kind);
}
@SuppressWarnings("unchecked")
@Override
public long renew(Token<?> token, Configuration conf
) throws IOException, InterruptedException {
InetSocketAddress addr = SecurityUtil.getTokenServiceAddr(token);
JobSubmissionProtocol jt = createRPCProxy(addr, conf);
return jt.renewDelegationToken((Token<DelegationTokenIdentifier>) token);
}
@SuppressWarnings("unchecked")
@Override
public void cancel(Token<?> token, Configuration conf
) throws IOException, InterruptedException {
InetSocketAddress addr = SecurityUtil.getTokenServiceAddr(token);
JobSubmissionProtocol jt = createRPCProxy(addr, conf);
jt.cancelDelegationToken((Token<DelegationTokenIdentifier>) token);
}
@Override
public boolean isManaged(Token<?> token) throws IOException {
ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
DelegationTokenIdentifier id = new DelegationTokenIdentifier();
id.readFields(new DataInputStream(buf));
// AbstractDelegationToken converts given renewer to a short name, but
// AbstractDelegationTokenSecretManager does not, so we have to do it
String loginUser = UserGroupInformation.getLoginUser().getShortUserName();
return loginUser.equals(id.getRenewer().toString());
}
}
/**
* Build a job client, connect to the indicated job tracker.
*
* @param jobTrackAddr the job tracker to connect to.
* @param conf configuration.
*/
public JobClient(InetSocketAddress jobTrackAddr,
Configuration conf) throws IOException {
this.ugi = UserGroupInformation.getCurrentUser();
jobSubmitClient = createRPCProxy(jobTrackAddr, conf);
}
/**
* Close the <code>JobClient</code>.
*/
public synchronized void close() throws IOException {
if (!(jobSubmitClient instanceof LocalJobRunner)) {
RPC.stopProxy(jobSubmitClient);
}
}
/**
* Get a filesystem handle. We need this to prepare jobs
* for submission to the MapReduce system.
*
* @return the filesystem handle.
* @throws IOException
*/
public synchronized FileSystem getFs() throws IOException {
if (this.fs == null) {
try {
this.fs = ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
public FileSystem run() throws IOException {
Path sysDir = getSystemDir();
return sysDir.getFileSystem(getConf());
}
});
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
}
return this.fs;
}
/* see if two file systems are the same or not
*
*/
private boolean compareFs(FileSystem srcFs, FileSystem destFs) {
URI srcUri = srcFs.getUri();
URI dstUri = destFs.getUri();
if (srcUri.getScheme() == null) {
return false;
}
if (!srcUri.getScheme().equals(dstUri.getScheme())) {
return false;
}
String srcHost = srcUri.getHost();
String dstHost = dstUri.getHost();
if ((srcHost != null) && (dstHost != null)) {
try {
srcHost = InetAddress.getByName(srcHost).getCanonicalHostName();
dstHost = InetAddress.getByName(dstHost).getCanonicalHostName();
} catch(UnknownHostException ue) {
return false;
}
if (!srcHost.equals(dstHost)) {
return false;
}
}
else if (srcHost == null && dstHost != null) {
return false;
}
else if (srcHost != null && dstHost == null) {
return false;
}
//check for ports
if (srcUri.getPort() != dstUri.getPort()) {
return false;
}
return true;
}
// copies a file to the jobtracker filesystem and returns the path where it
// was copied to
private Path copyRemoteFiles(FileSystem jtFs, Path parentDir,
final Path originalPath, final JobConf job, short replication)
throws IOException, InterruptedException {
//check if we do not need to copy the files
// is jt using the same file system.
// just checking for uri strings... doing no dns lookups
// to see if the filesystems are the same. This is not optimal.
// but avoids name resolution.
FileSystem remoteFs = null;
remoteFs = originalPath.getFileSystem(job);
if (compareFs(remoteFs, jtFs)) {
return originalPath;
}
// this might have name collisions. copy will throw an exception
//parse the original path to create new path
Path newPath = new Path(parentDir, originalPath.getName());
FileUtil.copy(remoteFs, originalPath, jtFs, newPath, false, job);
jtFs.setReplication(newPath, replication);
return newPath;
}
private URI getPathURI(Path destPath, String fragment)
throws URISyntaxException {
URI pathURI = destPath.toUri();
if (pathURI.getFragment() == null) {
if (fragment == null) {
pathURI = new URI(pathURI.toString() + "#" + destPath.getName());
} else {
pathURI = new URI(pathURI.toString() + "#" + fragment);
}
}
return pathURI;
}
/**
* configure the jobconf of the user with the command line options of
* -libjars, -files, -archives
* @param job the JobConf
* @param submitJobDir
* @throws IOException
*/
private void copyAndConfigureFiles(JobConf job, Path jobSubmitDir)
throws IOException, InterruptedException {
short replication = (short)job.getInt("mapred.submit.replication", 10);
copyAndConfigureFiles(job, jobSubmitDir, replication);
// Set the working directory
if (job.getWorkingDirectory() == null) {
job.setWorkingDirectory(fs.getWorkingDirectory());
}
}
private void copyAndConfigureFiles(JobConf job, Path submitJobDir,
short replication) throws IOException, InterruptedException {
if (!(job.getBoolean("mapred.used.genericoptionsparser", false))) {
LOG.warn("Use GenericOptionsParser for parsing the arguments. " +
"Applications should implement Tool for the same.");
}
// Retrieve command line arguments placed into the JobConf
// by GenericOptionsParser.
String files = job.get("tmpfiles");
String libjars = job.get("tmpjars");
String archives = job.get("tmparchives");
//
// Figure out what fs the JobTracker is using. Copy the
// job to it, under a temporary name. This allows DFS to work,
// and under the local fs also provides UNIX-like object loading
// semantics. (that is, if the job file is deleted right after
// submission, we can still run the submission to completion)
//
// Create a number of filenames in the JobTracker's fs namespace
FileSystem fs = submitJobDir.getFileSystem(job);
LOG.debug("default FileSystem: " + fs.getUri());
if (fs.exists(submitJobDir)) {
throw new IOException("Not submitting job. Job directory " + submitJobDir
+" already exists!! This is unexpected.Please check what's there in" +
" that directory");
}
submitJobDir = fs.makeQualified(submitJobDir);
FsPermission mapredSysPerms = new FsPermission(JobSubmissionFiles.JOB_DIR_PERMISSION);
FileSystem.mkdirs(fs, submitJobDir, mapredSysPerms);
Path filesDir = JobSubmissionFiles.getJobDistCacheFiles(submitJobDir);
Path archivesDir = JobSubmissionFiles.getJobDistCacheArchives(submitJobDir);
Path libjarsDir = JobSubmissionFiles.getJobDistCacheLibjars(submitJobDir);
// add all the command line files/ jars and archive
// first copy them to jobtrackers filesystem
if (files != null) {
FileSystem.mkdirs(fs, filesDir, mapredSysPerms);
String[] fileArr = files.split(",");
for (String tmpFile: fileArr) {
URI tmpURI;
try {
tmpURI = new URI(tmpFile);
} catch (URISyntaxException e) {
throw new IllegalArgumentException(e);
}
Path tmp = new Path(tmpURI);
Path newPath = copyRemoteFiles(fs,filesDir, tmp, job, replication);
try {
URI pathURI = getPathURI(newPath, tmpURI.getFragment());
DistributedCache.addCacheFile(pathURI, job);
} catch(URISyntaxException ue) {
//should not throw a uri exception
throw new IOException("Failed to create uri for " + tmpFile, ue);
}
DistributedCache.createSymlink(job);
}
}
if (libjars != null) {
FileSystem.mkdirs(fs, libjarsDir, mapredSysPerms);
String[] libjarsArr = libjars.split(",");
for (String tmpjars: libjarsArr) {
Path tmp = new Path(tmpjars);
Path newPath = copyRemoteFiles(fs, libjarsDir, tmp, job, replication);
DistributedCache.addArchiveToClassPath
(new Path(newPath.toUri().getPath()), job, fs);
}
}
if (archives != null) {
FileSystem.mkdirs(fs, archivesDir, mapredSysPerms);
String[] archivesArr = archives.split(",");
for (String tmpArchives: archivesArr) {
URI tmpURI;
try {
tmpURI = new URI(tmpArchives);
} catch (URISyntaxException e) {
throw new IllegalArgumentException(e);
}
Path tmp = new Path(tmpURI);
Path newPath = copyRemoteFiles(fs, archivesDir, tmp, job, replication);
try {
URI pathURI = getPathURI(newPath, tmpURI.getFragment());
DistributedCache.addCacheArchive(pathURI, job);
} catch(URISyntaxException ue) {
//should not throw an uri excpetion
throw new IOException("Failed to create uri for " + tmpArchives, ue);
}
DistributedCache.createSymlink(job);
}
}
// First we check whether the cached archives and files are legal.
TrackerDistributedCacheManager.validate(job);
// set the timestamps of the archives and files
TrackerDistributedCacheManager.determineTimestamps(job);
// set the public/private visibility of the archives and files
TrackerDistributedCacheManager.determineCacheVisibilities(job);
// get DelegationTokens for cache files
TrackerDistributedCacheManager.getDelegationTokens(job,
job.getCredentials());
String originalJarPath = job.getJar();
if (originalJarPath != null) { // copy jar to JobTracker's fs
// use jar name if job is not named.
if ("".equals(job.getJobName())){
job.setJobName(new Path(originalJarPath).getName());
}
Path submitJarFile = JobSubmissionFiles.getJobJar(submitJobDir);
job.setJar(submitJarFile.toString());
fs.copyFromLocalFile(new Path(originalJarPath), submitJarFile);
fs.setReplication(submitJarFile, replication);
fs.setPermission(submitJarFile,
new FsPermission(JobSubmissionFiles.JOB_FILE_PERMISSION));
} else {
LOG.warn("No job jar file set. User classes may not be found. "+
"See JobConf(Class) or JobConf#setJar(String).");
}
}
/**
* Submit a job to the MR system.
*
* This returns a handle to the {@link RunningJob} which can be used to track
* the running-job.
*
* @param jobFile the job configuration.
* @return a handle to the {@link RunningJob} which can be used to track the
* running-job.
* @throws FileNotFoundException
* @throws InvalidJobConfException
* @throws IOException
*/
public RunningJob submitJob(String jobFile) throws FileNotFoundException,
InvalidJobConfException,
IOException {
// Load in the submitted job details
JobConf job = new JobConf(jobFile);
return submitJob(job);
}
/**
* Submit a job to the MR system.
* This returns a handle to the {@link RunningJob} which can be used to track
* the running-job.
*
* @param job the job configuration.
* @return a handle to the {@link RunningJob} which can be used to track the
* running-job.
* @throws FileNotFoundException
* @throws IOException
*/
public RunningJob submitJob(JobConf job) throws FileNotFoundException,
IOException {
try {
return submitJobInternal(job);
} catch (InterruptedException ie) {
throw new IOException("interrupted", ie);
} catch (ClassNotFoundException cnfe) {
throw new IOException("class not found", cnfe);
}
}
/**
* Internal method for submitting jobs to the system.
* @param job the configuration to submit
* @return a proxy object for the running job
* @throws FileNotFoundException
* @throws ClassNotFoundException
* @throws InterruptedException
* @throws IOException
*/
public
RunningJob submitJobInternal(final JobConf job
) throws FileNotFoundException,
ClassNotFoundException,
InterruptedException,
IOException {
/*
* configure the command line options correctly on the submitting dfs
*/
return ugi.doAs(new PrivilegedExceptionAction<RunningJob>() {
public RunningJob run() throws FileNotFoundException,
ClassNotFoundException,
InterruptedException,
IOException{
JobConf jobCopy = job;
Path jobStagingArea = JobSubmissionFiles.getStagingDir(JobClient.this,
jobCopy);
JobID jobId = jobSubmitClient.getNewJobId();
Path submitJobDir = new Path(jobStagingArea, jobId.toString());
jobCopy.set("mapreduce.job.dir", submitJobDir.toString());
JobStatus status = null;
try {
populateTokenCache(jobCopy, jobCopy.getCredentials());
copyAndConfigureFiles(jobCopy, submitJobDir);
// get delegation token for the dir
TokenCache.obtainTokensForNamenodes(jobCopy.getCredentials(),
new Path [] {submitJobDir},
jobCopy);
Path submitJobFile = JobSubmissionFiles.getJobConfPath(submitJobDir);
int reduces = jobCopy.getNumReduceTasks();
InetAddress ip = InetAddress.getLocalHost();
if (ip != null) {
job.setJobSubmitHostAddress(ip.getHostAddress());
job.setJobSubmitHostName(ip.getHostName());
}
JobContext context = new JobContext(jobCopy, jobId);
jobCopy = (JobConf)context.getConfiguration();
// Check the output specification
if (reduces == 0 ? jobCopy.getUseNewMapper() :
jobCopy.getUseNewReducer()) {
org.apache.hadoop.mapreduce.OutputFormat<?,?> output =
ReflectionUtils.newInstance(context.getOutputFormatClass(),
jobCopy);
output.checkOutputSpecs(context);
} else {
jobCopy.getOutputFormat().checkOutputSpecs(fs, jobCopy);
}
// Create the splits for the job
FileSystem fs = submitJobDir.getFileSystem(jobCopy);
LOG.debug("Creating splits at " + fs.makeQualified(submitJobDir));
int maps = writeSplits(context, submitJobDir);
jobCopy.setNumMapTasks(maps);
// write "queue admins of the queue to which job is being submitted"
// to job file.
String queue = jobCopy.getQueueName();
AccessControlList acl = jobSubmitClient.getQueueAdmins(queue);
jobCopy.set(QueueManager.toFullPropertyName(queue,
QueueACL.ADMINISTER_JOBS.getAclName()), acl.getACLString());
// Write job file to JobTracker's fs
FSDataOutputStream out =
FileSystem.create(fs, submitJobFile,
new FsPermission(JobSubmissionFiles.JOB_FILE_PERMISSION));
try {
jobCopy.writeXml(out);
} finally {
out.close();
}
//
// Now, actually submit the job (using the submit name)
//
printTokens(jobId, jobCopy.getCredentials());
status = jobSubmitClient.submitJob(
jobId, submitJobDir.toString(), jobCopy.getCredentials());
JobProfile prof = jobSubmitClient.getJobProfile(jobId);
if (status != null && prof != null) {
return new NetworkedJob(status, prof, jobSubmitClient);
} else {
throw new IOException("Could not launch job");
}
} finally {
if (status == null) {
LOG.info("Cleaning up the staging area " + submitJobDir);
if (fs != null && submitJobDir != null)
fs.delete(submitJobDir, true);
}
}
}
});
}
@SuppressWarnings("unchecked")
private void printTokens(JobID jobId,
Credentials credentials) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Printing tokens for job: " + jobId);
for(Token<?> token: credentials.getAllTokens()) {
if (token.getKind().toString().equals("HDFS_DELEGATION_TOKEN")) {
LOG.debug("Submitting with " +
DFSClient.stringifyToken((Token<org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier>) token));
}
}
}
}
@SuppressWarnings("unchecked")
private <T extends InputSplit>
int writeNewSplits(JobContext job, Path jobSubmitDir) throws IOException,
InterruptedException, ClassNotFoundException {
Configuration conf = job.getConfiguration();
InputFormat<?, ?> input =
ReflectionUtils.newInstance(job.getInputFormatClass(), conf);
List<InputSplit> splits = input.getSplits(job);
T[] array = (T[]) splits.toArray(new InputSplit[splits.size()]);
// sort the splits into order based on size, so that the biggest
// go first
Arrays.sort(array, new SplitComparator());
JobSplitWriter.createSplitFiles(jobSubmitDir, conf,
jobSubmitDir.getFileSystem(conf), array);
return array.length;
}
private int writeSplits(org.apache.hadoop.mapreduce.JobContext job,
Path jobSubmitDir) throws IOException,
InterruptedException, ClassNotFoundException {
JobConf jConf = (JobConf)job.getConfiguration();
int maps;
if (jConf.getUseNewMapper()) {
maps = writeNewSplits(job, jobSubmitDir);
} else {
maps = writeOldSplits(jConf, jobSubmitDir);
}
return maps;
}
//method to write splits for old api mapper.
private int writeOldSplits(JobConf job, Path jobSubmitDir)
throws IOException {
org.apache.hadoop.mapred.InputSplit[] splits =
job.getInputFormat().getSplits(job, job.getNumMapTasks());
// sort the splits into order based on size, so that the biggest
// go first
Arrays.sort(splits, new Comparator<org.apache.hadoop.mapred.InputSplit>() {
public int compare(org.apache.hadoop.mapred.InputSplit a,
org.apache.hadoop.mapred.InputSplit b) {
try {
long left = a.getLength();
long right = b.getLength();
if (left == right) {
return 0;
} else if (left < right) {
return 1;
} else {
return -1;
}
} catch (IOException ie) {
throw new RuntimeException("Problem getting input split size", ie);
}
}
});
JobSplitWriter.createSplitFiles(jobSubmitDir, job,
jobSubmitDir.getFileSystem(job), splits);
return splits.length;
}
private static class SplitComparator implements Comparator<InputSplit> {
@Override
public int compare(InputSplit o1, InputSplit o2) {
try {
long len1 = o1.getLength();
long len2 = o2.getLength();
if (len1 < len2) {
return 1;
} else if (len1 == len2) {
return 0;
} else {
return -1;
}
} catch (IOException ie) {
throw new RuntimeException("exception in compare", ie);
} catch (InterruptedException ie) {
throw new RuntimeException("exception in compare", ie);
}
}
}
/**
* Checks if the job directory is clean and has all the required components
* for (re) starting the job
*/
public static boolean isJobDirValid(Path jobDirPath, FileSystem fs)
throws IOException {
FileStatus[] contents = fs.listStatus(jobDirPath);
int matchCount = 0;
if (contents != null && contents.length >=2) {
for (FileStatus status : contents) {
if ("job.xml".equals(status.getPath().getName())) {
++matchCount;
}
if ("job.split".equals(status.getPath().getName())) {
++matchCount;
}
}
if (matchCount == 2) {
return true;
}
}
return false;
}
/**
* Get an {@link RunningJob} object to track an ongoing job. Returns
* null if the id does not correspond to any known job.
*
* @param jobid the jobid of the job.
* @return the {@link RunningJob} handle to track the job, null if the
* <code>jobid</code> doesn't correspond to any known job.
* @throws IOException
*/
public RunningJob getJob(JobID jobid) throws IOException {
JobStatus status = jobSubmitClient.getJobStatus(jobid);
JobProfile profile = jobSubmitClient.getJobProfile(jobid);
if (status != null && profile != null) {
return new NetworkedJob(status, profile, jobSubmitClient);
} else {
return null;
}
}
/**@deprecated Applications should rather use {@link #getJob(JobID)}.
*/
@Deprecated
public RunningJob getJob(String jobid) throws IOException {
return getJob(JobID.forName(jobid));
}
/**
* Get the information of the current state of the map tasks of a job.
*
* @param jobId the job to query.
* @return the list of all of the map tips.
* @throws IOException
*/
public TaskReport[] getMapTaskReports(JobID jobId) throws IOException {
return jobSubmitClient.getMapTaskReports(jobId);
}
/**@deprecated Applications should rather use {@link #getMapTaskReports(JobID)}*/
@Deprecated
public TaskReport[] getMapTaskReports(String jobId) throws IOException {
return getMapTaskReports(JobID.forName(jobId));
}
/**
* Get the information of the current state of the reduce tasks of a job.
*
* @param jobId the job to query.
* @return the list of all of the reduce tips.
* @throws IOException
*/
public TaskReport[] getReduceTaskReports(JobID jobId) throws IOException {
return jobSubmitClient.getReduceTaskReports(jobId);
}
/**
* Get the information of the current state of the cleanup tasks of a job.
*
* @param jobId the job to query.
* @return the list of all of the cleanup tips.
* @throws IOException
*/
public TaskReport[] getCleanupTaskReports(JobID jobId) throws IOException {
return jobSubmitClient.getCleanupTaskReports(jobId);
}
/**
* Get the information of the current state of the setup tasks of a job.
*
* @param jobId the job to query.
* @return the list of all of the setup tips.
* @throws IOException
*/
public TaskReport[] getSetupTaskReports(JobID jobId) throws IOException {
return jobSubmitClient.getSetupTaskReports(jobId);
}
/**@deprecated Applications should rather use {@link #getReduceTaskReports(JobID)}*/
@Deprecated
public TaskReport[] getReduceTaskReports(String jobId) throws IOException {
return getReduceTaskReports(JobID.forName(jobId));
}
/**
* Display the information about a job's tasks, of a particular type and
* in a particular state
*
* @param jobId the ID of the job
* @param type the type of the task (map/reduce/setup/cleanup)
* @param state the state of the task
* (pending/running/completed/failed/killed)
*/
public void displayTasks(JobID jobId, String type, String state)
throws IOException {
TaskReport[] reports = new TaskReport[0];
if (type.equals("map")) {
reports = getMapTaskReports(jobId);
} else if (type.equals("reduce")) {
reports = getReduceTaskReports(jobId);
} else if (type.equals("setup")) {
reports = getSetupTaskReports(jobId);
} else if (type.equals("cleanup")) {
reports = getCleanupTaskReports(jobId);
}
for (TaskReport report : reports) {
TIPStatus status = report.getCurrentStatus();
if ((state.equals("pending") && status ==TIPStatus.PENDING) ||
(state.equals("running") && status ==TIPStatus.RUNNING) ||
(state.equals("completed") && status == TIPStatus.COMPLETE) ||
(state.equals("failed") && status == TIPStatus.FAILED) ||
(state.equals("killed") && status == TIPStatus.KILLED)) {
printTaskAttempts(report);
}
}
}
private void printTaskAttempts(TaskReport report) {
if (report.getCurrentStatus() == TIPStatus.COMPLETE) {
System.out.println(report.getSuccessfulTaskAttempt());
} else if (report.getCurrentStatus() == TIPStatus.RUNNING) {
for (TaskAttemptID t :
report.getRunningTaskAttempts()) {
System.out.println(t);
}
}
}
/**
* Get status information about the Map-Reduce cluster.
*
* @return the status information about the Map-Reduce cluster as an object
* of {@link ClusterStatus}.
* @throws IOException
*/
public ClusterStatus getClusterStatus() throws IOException {
return getClusterStatus(false);
}
/**
* Get status information about the Map-Reduce cluster.
*
* @param detailed if true then get a detailed status including the
* tracker names and memory usage of the JobTracker
* @return the status information about the Map-Reduce cluster as an object
* of {@link ClusterStatus}.
* @throws IOException
*/
public ClusterStatus getClusterStatus(boolean detailed) throws IOException {
return jobSubmitClient.getClusterStatus(detailed);
}
/**
* Grab the jobtracker's view of the staging directory path where
* job-specific files will be placed.
*
* @return the staging directory where job-specific files are to be placed.
*/
public Path getStagingAreaDir() throws IOException {
if (stagingAreaDir == null) {
stagingAreaDir = new Path(jobSubmitClient.getStagingAreaDir());
}
return stagingAreaDir;
}
/**
* Get the jobs that are not completed and not failed.
*
* @return array of {@link JobStatus} for the running/to-be-run jobs.
* @throws IOException
*/
public JobStatus[] jobsToComplete() throws IOException {
return jobSubmitClient.jobsToComplete();
}
private static void downloadProfile(TaskCompletionEvent e
) throws IOException {
URLConnection connection =
new URL(getTaskLogURL(e.getTaskAttemptId(), e.getTaskTrackerHttp()) +
"&filter=profile").openConnection();
InputStream in = connection.getInputStream();
OutputStream out = new FileOutputStream(e.getTaskAttemptId() + ".profile");
IOUtils.copyBytes(in, out, 64 * 1024, true);
}
/**
* Get the jobs that are submitted.
*
* @return array of {@link JobStatus} for the submitted jobs.
* @throws IOException
*/
public JobStatus[] getAllJobs() throws IOException {
return jobSubmitClient.getAllJobs();
}
/**
* Utility that submits a job, then polls for progress until the job is
* complete.
*
* @param job the job configuration.
* @throws IOException if the job fails
*/
public static RunningJob runJob(JobConf job) throws IOException {
JobClient jc = new JobClient(job);
RunningJob rj = jc.submitJob(job);
try {
if (!jc.monitorAndPrintJob(job, rj)) {
LOG.info("Job Failed: " + rj.getFailureInfo());
throw new IOException("Job failed!");
}
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
}
return rj;
}
/**
* Monitor a job and print status in real-time as progress is made and tasks
* fail.
* @param conf the job's configuration
* @param job the job to track
* @return true if the job succeeded
* @throws IOException if communication to the JobTracker fails
*/
public boolean monitorAndPrintJob(JobConf conf,
RunningJob job
) throws IOException, InterruptedException {
String lastReport =