Author: acmurthy Date: Thu Jan 10 11:52:37 2008 New Revision: 610902 URL: http://svn.apache.org/viewvc?rev=610902&view=rev Log: HADOOP-2487. Added an option to get statuses for all submitted/run jobs. This information can be used to develop tools for analysing jobs. Contributed by Amareshwari Sri Ramadasu.
Modified: lucene/hadoop/trunk/CHANGES.txt lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=610902&r1=610901&r2=610902&view=diff ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Thu Jan 10 11:52:37 2008 @@ -66,6 +66,10 @@ HADOOP-2447. HDFS can be configured to limit the total number of objects (inodes and blocks) in the file system. (dhruba) + HADOOP-2487. Added an option to get statuses for all submitted/run jobs. + This information can be used to develop tools for analysing jobs. + (Amareshwari Sri Ramadasu via acmurthy) + IMPROVEMENTS HADOOP-2045. Change committer list on website to a table, so that Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java?rev=610902&r1=610901&r2=610902&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java Thu Jan 10 11:52:37 2008 @@ -728,6 +728,16 @@ } /** + * Get the jobs that are submitted. + * + * @return array of [EMAIL PROTECTED] 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. * @@ -947,6 +957,7 @@ System.out.printf("\t-kill\t<job-id>\n"); System.out.printf("\t-events\t<job-id> <from-event-#> <#-of-events>\n"); System.out.printf("\t-list\n"); + System.out.printf("\t-list\tall\n"); System.out.printf("\t-kill-task <task-id>\n"); System.out.printf("\t-fail-task <task-id>\n\n"); ToolRunner.printGenericCommandUsage(System.out); @@ -964,6 +975,7 @@ boolean killJob = false; boolean listEvents = false; boolean listJobs = false; + boolean listAllJobs = false; boolean killTask = false; boolean failTask = false; @@ -992,7 +1004,13 @@ nEvents = Integer.parseInt(argv[3]); listEvents = true; } else if ("-list".equals(argv[0])) { - listJobs = true; + if (argv.length != 1 && !(argv.length == 2 && "all".equals(argv[1]))) + displayUsage(); + if (argv.length == 2 && "all".equals(argv[1])) { + listAllJobs = true; + } else { + listJobs = true; + } } else if("-kill-task".equals(argv[0])) { if(argv.length != 2) displayUsage(); @@ -1047,6 +1065,9 @@ } else if (listJobs) { listJobs(); exitCode = 0; + } else if (listAllJobs) { + listAllJobs(); + exitCode = 0; } else if(killTask) { if(jobSubmitClient.killTask(taskid, false)) { System.out.println("Killed task " + taskid); @@ -1105,6 +1126,25 @@ } } + /** + * Dump a list of all jobs submitted. + * @throws IOException + */ + private void listAllJobs() throws IOException { + JobStatus[] jobs = getAllJobs(); + if (jobs == null) + jobs = new JobStatus[0]; + + System.out.printf("%d jobs submitted\n", jobs.length); + System.out.printf("States are:\n\tRunning : 1\tSucceded : 2" + + "\tFailed : 3\tPrep : 4\n"); + System.out.printf("JobId\tState\tStartTime\tUserName\n"); + for (JobStatus job : jobs) { + System.out.printf("%s\t%d\t%d\t%s\n", job.getJobId(), job.getRunState(), + job.getStartTime(), job.getUsername()); + } + } + /** */ public static void main(String argv[]) throws Exception { Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java?rev=610902&r1=610901&r2=610902&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobSubmissionProtocol.java Thu Jan 10 11:52:37 2008 @@ -36,8 +36,9 @@ *Version 5: max_tasks in ClusterStatus is replaced by * max_map_tasks and max_reduce_tasks for HADOOP-1274 * Version 6: change the counters representation for HADOOP-2248 + * Version 7: added getAllJobs for HADOOP-2487 */ - public static final long versionID = 6L; + public static final long versionID = 7L; /** * Allocate a name for the job. @@ -114,6 +115,12 @@ */ public JobStatus[] jobsToComplete() throws IOException; + /** + * Get all the jobs submitted. + * @return array of JobStatus for the submitted jobs + */ + public JobStatus[] getAllJobs() throws IOException; + /** * Get task completion events for the jobid, starting from fromEventId. * Returns empty aray if no events are available. Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java?rev=610902&r1=610901&r2=610902&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java Thu Jan 10 11:52:37 2008 @@ -1767,6 +1767,18 @@ } return v.toArray(new JobStatus[v.size()]); } + + public JobStatus[] getAllJobs() { + Vector<JobStatus> v = new Vector<JobStatus>(); + for (Iterator it = jobs.values().iterator(); it.hasNext();) { + JobInProgress jip = (JobInProgress) it.next(); + JobStatus status = jip.getStatus(); + status.setStartTime(jip.getStartTime()); + status.setUsername(jip.getProfile().getUser()); + v.add(status); + } + return v.toArray(new JobStatus[v.size()]); + } /////////////////////////////////////////////////////////////// // JobTracker methods Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java?rev=610902&r1=610901&r2=610902&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java Thu Jan 10 11:52:37 2008 @@ -329,6 +329,9 @@ } public JobStatus[] jobsToComplete() {return null;} + + public JobStatus[] getAllJobs() {return null;} + public TaskCompletionEvent[] getTaskCompletionEvents( String jobid, int fromEventId, int maxEvents) throws IOException{ return TaskCompletionEvent.EMPTY_ARRAY;