/** * 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(); } @Override public JobStatus getJobStatus() throws IOException { updateStatus(); return status; } } private JobSubmissionProtocol rpcJobSubmitClient; 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; public static final String MAPREDUCE_CLIENT_RETRY_POLICY_ENABLED_KEY = "mapreduce.jobclient.retry.policy.enabled"; public static final boolean MAPREDUCE_CLIENT_RETRY_POLICY_ENABLED_DEFAULT = false; public static final String MAPREDUCE_CLIENT_RETRY_POLICY_SPEC_KEY = "mapreduce.jobclient.retry.policy.spec"; public static final String MAPREDUCE_CLIENT_RETRY_POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,... /** * Create a job client. */ public JobClient() { } /** * Build a job client with the given {@link JobConf}, and connect to the * default {@link JobTracker}. * 用指定的jobconf创建一个jobclient 并且连接默认的jobtracker * * @param conf the job configuration. * @throws IOException */ public JobClient(JobConf conf) throws IOException { setConf(conf); init(conf); } /** * Connect to the default {@link JobTracker}. * 连接默认的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)) {//如果tracker是在本地则 开启本地jobrunner conf.setNumMapTasks(1); this.jobSubmitClient = new LocalJobRunner(conf); } else { this.rpcJobSubmitClient = createRPCProxy(JobTracker.getAddress(conf), conf); this.jobSubmitClient = createProxy(this.rpcJobSubmitClient, conf); } } private static JobSubmissionProtocol createRPCProxy(InetSocketAddress addr, Configuration conf) throws IOException { JobSubmissionProtocol rpcJobSubmitClient = (JobSubmissionProtocol)RPC.getProxy( JobSubmissionProtocol.class, JobSubmissionProtocol.versionID, addr, UserGroupInformation.getCurrentUser(), conf, NetUtils.getSocketFactory(conf, JobSubmissionProtocol.class), 0, RetryUtils.getMultipleLinearRandomRetry( conf, MAPREDUCE_CLIENT_RETRY_POLICY_ENABLED_KEY, MAPREDUCE_CLIENT_RETRY_POLICY_ENABLED_DEFAULT, MAPREDUCE_CLIENT_RETRY_POLICY_SPEC_KEY, MAPREDUCE_CLIENT_RETRY_POLICY_SPEC_DEFAULT ), false); return rpcJobSubmitClient; } private static JobSubmissionProtocol createProxy( JobSubmissionProtocol rpcJobSubmitClient, Configuration conf) throws IOException { /* * Default is to retry on JobTrackerNotYetInitializedException * i.e. wait for JobTracker to get to RUNNING state and for * SafeModeException */ @SuppressWarnings("unchecked") RetryPolicy defaultPolicy = RetryUtils.getDefaultRetryPolicy( conf, MAPREDUCE_CLIENT_RETRY_POLICY_ENABLED_KEY, MAPREDUCE_CLIENT_RETRY_POLICY_ENABLED_DEFAULT, MAPREDUCE_CLIENT_RETRY_POLICY_SPEC_KEY, MAPREDUCE_CLIENT_RETRY_POLICY_SPEC_DEFAULT, JobTrackerNotYetInitializedException.class, SafeModeException.class ); /* * Method specific retry policies for killJob and killTask... * * No retries on any exception including * ConnectionException and SafeModeException */ Map<String,RetryPolicy> methodNameToPolicyMap = new HashMap<String,RetryPolicy>(); methodNameToPolicyMap.put("killJob", RetryPolicies.TRY_ONCE_THEN_FAIL); methodNameToPolicyMap.put("killTask", RetryPolicies.TRY_ONCE_THEN_FAIL); final JobSubmissionProtocol jsp = (JobSubmissionProtocol) RetryProxy.create( JobSubmissionProtocol.class, rpcJobSubmitClient, defaultPolicy, methodNameToPolicyMap); RPC.checkVersion(JobSubmissionProtocol.class, JobSubmissionProtocol.versionID, jsp); return jsp; } @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 = createProxy(createRPCProxy(addr, conf), 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 = createProxy(createRPCProxy(addr, conf), conf); jt.cancelDelegationToken((Token<DelegationTokenIdentifier>) token); }