You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by yh...@apache.org on 2009/12/08 06:05:19 UTC

svn commit: r888262 - in /hadoop/mapreduce/branches/branch-0.21: ./ src/java/org/apache/hadoop/mapred/ src/test/mapred/org/apache/hadoop/mapred/

Author: yhemanth
Date: Tue Dec  8 05:05:18 2009
New Revision: 888262

URL: http://svn.apache.org/viewvc?rev=888262&view=rev
Log:
Merge -r 888256:888257 from trunk to branch-0.21 to fix MAPREDUCE-1075.

Modified:
    hadoop/mapreduce/branches/branch-0.21/CHANGES.txt
    hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobClient.java
    hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobQueueClient.java
    hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobTracker.java
    hadoop/mapreduce/branches/branch-0.21/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueClient.java

Modified: hadoop/mapreduce/branches/branch-0.21/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/branch-0.21/CHANGES.txt?rev=888262&r1=888261&r2=888262&view=diff
==============================================================================
--- hadoop/mapreduce/branches/branch-0.21/CHANGES.txt (original)
+++ hadoop/mapreduce/branches/branch-0.21/CHANGES.txt Tue Dec  8 05:05:18 2009
@@ -852,3 +852,7 @@
     (Owen O'Malley via cdouglas)
 
     MAPREDUCE-1244. Fix eclipse-plugin's build dependencies. (gkesavan)
+
+    MAPREDUCE-1075. Fix JobTracker to not throw an NPE for a non-existent
+    queue. (V.V.Chaitanya Krishna via yhemanth)
+

Modified: hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobClient.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobClient.java?rev=888262&r1=888261&r2=888262&view=diff
==============================================================================
--- hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobClient.java (original)
+++ hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobClient.java Tue Dec  8 05:05:18 2009
@@ -1001,8 +1001,12 @@
   
   public JobStatus[] getJobsFromQueue(String queueName) throws IOException {
     try {
+      QueueInfo queue = cluster.getQueue(queueName);
+      if (queue == null) {
+        return null;
+      }
       org.apache.hadoop.mapreduce.JobStatus[] stats = 
-        cluster.getQueue(queueName).getJobStatuses();
+        queue.getJobStatuses();
       JobStatus[] ret = new JobStatus[stats.length];
       for (int i = 0 ; i < stats.length; i++ ) {
         ret[i] = JobStatus.downgrade(stats[i]);
@@ -1022,7 +1026,11 @@
    */
   public JobQueueInfo getQueueInfo(String queueName) throws IOException {
     try {
-      return new JobQueueInfo(cluster.getQueue(queueName));
+      QueueInfo queueInfo = cluster.getQueue(queueName);
+      if (queueInfo != null) {
+        return new JobQueueInfo(queueInfo);
+      }
+      return null;
     } catch (InterruptedException ie) {
       throw new IOException(ie);
     }

Modified: hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobQueueClient.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobQueueClient.java?rev=888262&r1=888261&r2=888262&view=diff
==============================================================================
--- hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobQueueClient.java (original)
+++ hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobQueueClient.java Tue Dec  8 05:05:18 2009
@@ -175,6 +175,11 @@
   private void displayQueueInfo(String queue, boolean showJobs)
       throws IOException {
     JobQueueInfo jobQueueInfo = jc.getQueueInfo(queue);
+    
+    if (jobQueueInfo == null) {
+      System.out.println("Queue \"" + queue + "\" does not exist.");
+      return;
+    }
     printJobQueueInfo(jobQueueInfo, new PrintWriter(System.out));
     if (showJobs && (jobQueueInfo.getChildren() == null ||
         jobQueueInfo.getChildren().size() == 0)) {

Modified: hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobTracker.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobTracker.java?rev=888262&r1=888261&r2=888262&view=diff
==============================================================================
--- hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobTracker.java (original)
+++ hadoop/mapreduce/branches/branch-0.21/src/java/org/apache/hadoop/mapred/JobTracker.java Tue Dec  8 05:05:18 2009
@@ -3883,7 +3883,9 @@
   @Override
   public QueueInfo getQueue(String queue) throws IOException {
     JobQueueInfo jqueue = queueManager.getJobQueueInfo(queue);
-    jqueue.setJobStatuses(getJobsFromQueue(jqueue.getQueueName()));
+    if (jqueue != null) {
+      jqueue.setJobStatuses(getJobsFromQueue(jqueue.getQueueName()));
+    }
     return jqueue;
   }
 

Modified: hadoop/mapreduce/branches/branch-0.21/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueClient.java
URL: http://svn.apache.org/viewvc/hadoop/mapreduce/branches/branch-0.21/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueClient.java?rev=888262&r1=888261&r2=888262&view=diff
==============================================================================
--- hadoop/mapreduce/branches/branch-0.21/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueClient.java (original)
+++ hadoop/mapreduce/branches/branch-0.21/src/test/mapred/org/apache/hadoop/mapred/TestJobQueueClient.java Tue Dec  8 05:05:18 2009
@@ -17,20 +17,33 @@
  */
 package org.apache.hadoop.mapred;
 
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.CONFIG;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.checkForConfigFile;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.createDocument;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.createSimpleDocumentWithAcls;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.miniMRCluster;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.setUpCluster;
+import static org.apache.hadoop.mapred.QueueManagerTestUtils.writeToFile;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
+import java.io.IOException;
 import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.junit.After;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.Cluster;
+import org.apache.hadoop.mapreduce.QueueInfo;
 import org.junit.Test;
+import org.w3c.dom.Document;
 
 public class TestJobQueueClient {
   @Test
   public void testQueueOrdering() throws Exception {
-    System.out.println("in test queue ordering");
     // create some sample queues in a hierarchy..
     JobQueueInfo[] roots = new JobQueueInfo[2];
     roots[0] = new JobQueueInfo("q1", "q1 scheduling info");
@@ -53,7 +66,6 @@
   
   @Test
   public void testQueueInfoPrinting() throws Exception {
-    System.out.println("in test queue info printing");
     // create a test queue with children.
     // create some sample queues in a hierarchy..
     JobQueueInfo root = new JobQueueInfo("q1", "q1 scheduling info");
@@ -76,4 +88,24 @@
     
     assertEquals(sb.toString(), writer.toString());
   }
-}
\ No newline at end of file
+  
+  @Test
+  public void testGetQueue() throws Exception {
+    checkForConfigFile();
+    Document doc = createDocument();
+    createSimpleDocumentWithAcls(doc, "true");
+    writeToFile(doc, CONFIG);
+    Configuration conf = new Configuration();
+    conf.addResource(CONFIG);
+    setUpCluster(conf);
+    JobClient jc = new JobClient(miniMRCluster.createJobConf());
+    // test for existing queue
+    QueueInfo queueInfo = jc.getQueueInfo("q1");
+    assertEquals("q1",queueInfo.getQueueName());
+    // try getting a non-existing queue
+    queueInfo = jc.getQueueInfo("queue");
+    assertNull(queueInfo);
+
+    new File(CONFIG).delete();
+  }
+}