Browse Source

MAPREDUCE-3674. Invoked with no queueName request param, the jobqueue_details.jsp injects a null queue name into schedulers. (harsh)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1328292 13f79535-47bb-0310-9956-ffa450edef68
Harsh J 13 years ago
parent
commit
b672a813f6

+ 3 - 0
CHANGES.txt

@@ -225,6 +225,9 @@ Release 1.1.0 - unreleased
     MAPREDUCE-4154. streaming MR job succeeds even if the streaming command 
     fails. (Devaraj Das via tgraves)
 
+    MAPREDUCE-3674. Invoked with no queueName request param, the
+    jobqueue_details.jsp injects a null queue name into schedulers. (harsh)
+
 Release 1.0.3 - unreleased
 
   NEW FEATURES

+ 3 - 0
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java

@@ -1020,6 +1020,9 @@ public class FairScheduler extends TaskScheduler {
 
   @Override
   public synchronized Collection<JobInProgress> getJobs(String queueName) {
+    if (queueName == null) {
+      return null;
+    }
     Pool myJobPool = poolMgr.getPool(queueName);
     return myJobPool.getJobs();
   }

+ 3 - 1
src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/Pool.java

@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.metrics.MetricsContext;
 
 /**
  * A schedulable pool of jobs.
@@ -44,6 +43,9 @@ public class Pool {
   private PoolSchedulable reduceSchedulable;
 
   public Pool(FairScheduler scheduler, String name) {
+    if (name == null) {
+      throw new IllegalArgumentException("Passed pool name was null.");
+    }
     this.name = name;
     mapSchedulable = new PoolSchedulable(scheduler, this, TaskType.MAP);
     reduceSchedulable = new PoolSchedulable(scheduler, this, TaskType.REDUCE);

+ 16 - 1
src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairSchedulerPoolNames.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.Pool;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -175,4 +176,18 @@ public class TestFairSchedulerPoolNames {
         "Add pool name to the fair scheduler allocation file"));
   }
 
-}
+  /**
+   * Tests that no Pool object can be created with a null string.
+   */
+  @Test
+  public void testPoolNameNotNull() {
+    try {
+      Pool pool = new Pool(null, null);
+      fail("Pool object got created with a null name somehow.");
+    } catch (IllegalArgumentException e) {
+      // Pass
+    } catch (Exception e) {
+      fail("Pool object got created with a null name and failed only later.");
+    }
+  }
+}

+ 1 - 1
src/webapps/job/jobqueue_details.jsp

@@ -25,7 +25,7 @@ private static final long serialVersionUID = 526456771152222127L;
 <html>
 <head>
 <title>Queue details for
-<%=queueName!=null?queueName:""%> </title>
+<%=queueName!=null?queueName:"(Given queue name was 'null')"%> </title>
 <link rel="stylesheet" type="text/css" href="/static/hadoop.css">
 <script type="text/javascript" src="/static/jobtracker.js"></script>
 </head>