MAPREDUCE-2331. Add coverage of task graph servlet to fair scheduler system test. Contributed by Todd Lipcon

git-svn-id: https://svn.apache.org/repos/asf/hadoop/mapreduce/trunk@1078708 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/CHANGES.txt b/CHANGES.txt
index 4d337b1..4f0e925 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -57,6 +57,9 @@
     MAPREDUCE-2239. BlockPlacementPolicyRaid should call getBlockLocations
     only when necessary. (schen)
 
+    MAPREDUCE-2331. Add coverage of task graph servlet to fair scheduler system
+    test. (todd)
+
   OPTIMIZATIONS
     
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and
diff --git a/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairSchedulerSystem.java b/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairSchedulerSystem.java
index 77e659b..db38a88 100644
--- a/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairSchedulerSystem.java
+++ b/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairSchedulerSystem.java
@@ -102,6 +102,8 @@
           }));
     }
 
+    JobClient jc = new JobClient(mr.createJobConf(null));
+
     // Wait for the tasks to finish, and visit the scheduler servlet
     // every few seconds while waiting.
     for (Future<Void> future : futures) {
@@ -114,6 +116,16 @@
         }
         checkServlet(true);
         checkServlet(false);
+
+        JobStatus jobs[] = jc.getAllJobs();
+        if (jobs == null) {
+          System.err.println("No jobs running, not checking tasklog servlet");
+          continue;
+        }
+        for (JobStatus j : jobs) {
+          System.err.println("Checking task log for " + j.getJobID());
+          checkTaskGraphServlet(j.getJobID());
+        }
       }
     }
   }
@@ -144,5 +156,32 @@
 
     String contents = sb.toString();
     assertTrue(contents.contains("Fair Scheduler Administration"));
- }
+  }
+
+  private void checkTaskGraphServlet(JobID job) throws Exception {
+    String jtURL = "http://localhost:" +
+      mr.getJobTrackerRunner().getJobTrackerInfoPort();
+    URL url = new URL(jtURL + "/taskgraph?jobid=" + job.toString() + "&type=map");
+    HttpURLConnection connection = (HttpURLConnection)url.openConnection();
+    connection.setRequestMethod("GET");
+    connection.connect();
+    assertEquals(200, connection.getResponseCode());
+
+    // Just to be sure, slurp the content and make sure it looks like the scheduler
+    String contents = slurpContents(connection);
+    assertTrue(contents.contains("</svg>"));
+  }
+
+  private String slurpContents(HttpURLConnection connection) throws Exception {
+    BufferedReader reader = new BufferedReader(
+      new InputStreamReader(connection.getInputStream()));
+    StringBuilder sb = new StringBuilder();
+
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      sb.append(line).append('\n');
+    }
+
+    return sb.toString();
+  }
 }