You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by om...@apache.org on 2011/03/04 05:30:59 UTC

svn commit: r1077577 - in /hadoop/common/branches/branch-0.20-security-patches/src/test/system: aop/org/apache/hadoop/mapred/ conf/ java/org/apache/hadoop/mapred/ java/org/apache/hadoop/mapreduce/test/system/ scripts/

Author: omalley
Date: Fri Mar  4 04:30:59 2011
New Revision: 1077577

URL: http://svn.apache.org/viewvc?rev=1077577&view=rev
Log:
commit e32ca4e3630ae80c7e2d87fac06d7d60c0f5f825
Author: Balaji Rajagopalan <ba...@yahoo-inc.com>
Date:   Wed Jul 21 18:35:51 2010 -0700

    MAPREDUCE-1854 from https://issues.apache.org/jira/secure/attachment/12449063/health_script_y20_1.patch

Added:
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/HealthScriptHelper.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptError.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptPathError.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptTimeout.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptError
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptTimeout
Modified:
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JTProtocolAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/conf/system-test.xml
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/JTProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/MRCluster.java

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JTProtocolAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JTProtocolAspect.aj?rev=1077577&r1=1077576&r2=1077577&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JTProtocolAspect.aj (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JTProtocolAspect.aj Fri Mar  4 04:30:59 2011
@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TaskID;
@@ -79,4 +78,8 @@ public aspect JTProtocolAspect {
   public String JTProtocol.getJobHistoryLocationForRetiredJob(JobID jobID) throws IOException {
     return "";
   }
+  
+  public boolean JTProtocol.isBlackListed(String trackerID) throws IOException {
+    return false;
+  }
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj?rev=1077577&r1=1077576&r2=1077577&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj Fri Mar  4 04:30:59 2011
@@ -155,6 +155,10 @@ public privileged aspect JobTrackerAspec
     return retireJobs.get(
         org.apache.hadoop.mapred.JobID.downgrade(id))!=null?true:false;
   }
+  
+  public boolean JobTracker.isBlackListed(String trackerName) throws IOException {
+    return isBlacklisted(trackerName);
+  }
 
   public String JobTracker.getJobHistoryLocationForRetiredJob(
       JobID id) throws IOException {

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/system/conf/system-test.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/conf/system-test.xml?rev=1077577&r1=1077576&r2=1077577&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/conf/system-test.xml (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/conf/system-test.xml Fri Mar  4 04:30:59 2011
@@ -90,7 +90,15 @@ neutral at the forward-port stage -->
   Command for resuming the given suspended process.
   </description>
 </property>
-
+<property>
+  <name>test.system.hdrc.healthscript.path</name>
+  <value>/tmp</value>
+  <description>
+  This location is used by health script test cases to configure the
+  health script in remote cluster where the error inducing health script
+  will be copied.
+  </description>
+</property>
 
 <!-- Mandatory keys to be set for the multi user support to be enabled.  -->
 

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/HealthScriptHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/HealthScriptHelper.java?rev=1077577&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/HealthScriptHelper.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/HealthScriptHelper.java Fri Mar  4 04:30:59 2011
@@ -0,0 +1,166 @@
+package org.apache.hadoop.mapred;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import org.apache.hadoop.mapred.UtilsForTests;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.test.system.MRCluster;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.apache.hadoop.mapreduce.test.system.TTClient;
+import org.apache.hadoop.mapreduce.test.system.JTClient;
+import org.apache.hadoop.mapreduce.test.system.MRCluster.Role;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.mapred.TaskTrackerStatus;
+import org.apache.hadoop.test.system.process.HadoopDaemonRemoteCluster;
+import org.junit.AfterClass;
+import org.junit.Assert;
+
+
+/** 
+ * This is helper class that is used by the health script test cases
+ *
+ */
+public class HealthScriptHelper  {
+
+  static final Log LOG = LogFactory.getLog(HealthScriptHelper.class);
+  
+  /**
+   * Will verify that given task tracker is not blacklisted
+   * @param client tasktracker info
+   * @param conf modified configuration object
+   * @param cluster mrcluster instance
+   * @throws IOException thrown if verification fails
+   */
+  public void verifyTTNotBlackListed(TTClient client, Configuration conf,
+      MRCluster cluster) throws IOException {        
+    int interval = conf.getInt("mapred.healthChecker.interval",0);
+    Assert.assertTrue("Interval cannot be zero.",interval != 0);
+    UtilsForTests.waitFor(interval+2000);
+    String defaultHealthScript = conf.get("mapred.healthChecker.script.path");    
+    Assert.assertTrue("Task tracker is not healthy",
+        nodeHealthStatus(client, true) == true);
+    TaskTrackerStatus status = client.getStatus();
+    JTClient jclient = cluster.getJTClient();
+    Assert.assertTrue("Failed to move task tracker to healthy list",
+        jclient.getProxy().isBlackListed(status.getTrackerName()) == false);        
+    Assert.assertTrue("Health script was not set",defaultHealthScript != null);
+    
+  }
+  
+  /**
+   * Verifies that the given task tracker is blacklisted
+   * @param conf modified Configuration object
+   * @param client tasktracker info
+   * @param errorMessage that needs to be asserted
+   * @param cluster mr cluster instance
+   * @throws IOException is thrown when verification fails
+   */
+  public void verifyTTBlackList(Configuration conf, TTClient client,
+      String errorMessage, MRCluster cluster) throws IOException{   
+    int interval = conf.getInt("mapred.healthChecker.interval",0);
+    Assert.assertTrue("Interval cannot be zero.",interval != 0);
+    UtilsForTests.waitFor(interval+2000);
+    //TaskTrackerStatus status = client.getStatus();
+    Assert.assertTrue("Task tracker was never blacklisted ",
+        nodeHealthStatus(client, false) == true);
+    TaskTrackerStatus status = client.getStatus();
+    Assert.assertTrue("The custom error message did not appear",
+        status.getHealthStatus().getHealthReport().trim().
+        equals(errorMessage));
+    JTClient jClient = cluster.getJTClient();    
+    Assert.assertTrue("Failed to move task tracker to blacklisted list",
+        jClient.getProxy().isBlackListed(status.getTrackerName()) == true);    
+  }
+  
+  /**
+   * The method return true from the task tracker if it is unhealthy/healthy
+   * depending the blacklisted status
+   * @param client the tracker tracker instance
+   * @param health status information. 
+   * @return status of task tracker
+   * @throws IOException failed to get the status of task tracker
+   */
+  public boolean nodeHealthStatus(TTClient client,boolean hStatus) throws IOException {
+    int counter = 0;
+    TaskTrackerStatus status = client.getStatus();
+    while (counter < 60) {
+      LOG.info("isNodeHealthy "+status.getHealthStatus().isNodeHealthy());
+      if (status.getHealthStatus().isNodeHealthy() == hStatus) {
+        break;
+      } else {
+        UtilsForTests.waitFor(3000);
+        status = client.getStatus();
+        Assert.assertNotNull("Task tracker status is null",status);
+      }
+      counter++;
+    }
+    if(counter != 60) {
+      return true;
+    }
+    return false;
+  }
+  
+  /**
+   * This will copy the error inducing health script from local node running
+   * the system tests to the node where the task tracker runs
+   * @param scriptName name of the scirpt to be copied
+   * @param hostname identifies the task tracker
+   * @param remoteLocation location in remote task tracker node
+   * @param cluster mrcluster instance
+   * @throws IOException thrown if copy file fails. 
+   */
+  public void copyFileToRemoteHost(String scriptName, String hostname,
+      String remoteLocation,MRCluster cluster) throws IOException {        
+    ArrayList<String> cmdArgs = new ArrayList<String>();
+    String scriptDir = cluster.getConf().get(
+        HadoopDaemonRemoteCluster.CONF_SCRIPTDIR);
+    StringBuffer localFile = new StringBuffer();    
+    localFile.append(scriptDir).append(File.separator).append(scriptName);
+    cmdArgs.add("scp");
+    cmdArgs.add(localFile.toString());
+    StringBuffer remoteFile = new StringBuffer();
+    remoteFile.append(hostname).append(":");
+    remoteFile.append(remoteLocation).append(File.separator).append(scriptName);
+    cmdArgs.add(remoteFile.toString());
+    executeCommand(cmdArgs);
+  }
+  
+  private void executeCommand(ArrayList<String> cmdArgs) throws IOException{
+    String[] cmd = (String[]) cmdArgs.toArray(new String[cmdArgs.size()]);
+    ShellCommandExecutor executor = new ShellCommandExecutor(cmd);
+    LOG.info(executor.toString());
+    executor.execute();
+    String output = executor.getOutput();    
+    if (!output.isEmpty()) { //getOutput() never returns null value
+      if (output.toLowerCase().contains("error")) {
+        LOG.warn("Error is detected.");
+        throw new IOException("Start error\n" + output);
+      }
+    }
+  }
+  
+  /**
+   * cleans up the error inducing health script in the remote node
+   * @param path the script that needs to be deleted
+   * @param hostname where the script resides. 
+   */
+  public void deleteFileOnRemoteHost(String path, String hostname) {
+    try {
+      ArrayList<String> cmdArgs = new ArrayList<String>();
+      cmdArgs.add("ssh");
+      cmdArgs.add(hostname);
+      cmdArgs.add("if [ -f "+ path+
+      "  ];\n then echo Will remove existing file "+path+";  rm -f "+
+      path+";\n  fi");
+      executeCommand(cmdArgs);
+    }
+    catch (IOException io) {
+      LOG.error("Failed to remove the script "+path+" on remote host "+hostname);
+    }
+  }
+
+}

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptError.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptError.java?rev=1077577&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptError.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptError.java Fri Mar  4 04:30:59 2011
@@ -0,0 +1,76 @@
+package org.apache.hadoop.mapred;
+
+import java.io.File;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.test.system.MRCluster;
+import org.apache.hadoop.mapreduce.test.system.TTClient;
+import org.apache.hadoop.mapreduce.test.system.MRCluster.Role;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestHealthScriptError {
+  public static MRCluster cluster;
+  public static HealthScriptHelper helper;
+  public static String remotePath;
+  public String healthScriptError="healthScriptError";
+  public static String remoteHSPath = "test.system.hdrc.healthscript.path";
+  static final Log LOG = LogFactory.getLog(TestHealthScriptError.class);
+  
+  @BeforeClass
+  public static void setUp() throws java.lang.Exception {
+    String [] expExcludeList = new String[2];
+    expExcludeList[0] = "java.net.ConnectException";
+    expExcludeList[1] = "java.io.IOException";
+    cluster = MRCluster.createCluster(new Configuration());
+    cluster.setExcludeExpList(expExcludeList);
+    cluster.setUp();
+    remotePath = cluster.getConf().get(remoteHSPath);
+    helper = new HealthScriptHelper();
+  }
+  
+  /**
+   * The test will induce the ERROR with health script, asserts the task tracker
+   * is unhealthy and then revert backs to non error condition and verifies
+   * the task tracker is healthy. When the task tracker is marked unhealthy
+   * also verifies that is marked as blacklisted, and reverse is true when it is
+   * healthy. Also this verifies the custom error message that is set when
+   * the task tracker is marked unhealthy.
+   * @throws Exception in case of test errors
+   */
+  @Test
+  public void testInduceError() throws Exception { 
+    LOG.info("running testInduceError");
+    TTClient client = cluster.getTTClient();
+    Configuration tConf= client.getProxy().getDaemonConf();    
+    tConf.set("mapred.task.tracker.report.address",
+        cluster.getConf().get("mapred.task.tracker.report.address"));
+    String defaultHealthScript = tConf.get("mapred.healthChecker.script.path");
+    Assert.assertTrue("Health script was not set", defaultHealthScript != null);        
+    tConf.set("mapred.healthChecker.script.path", remotePath+File.separator+
+        healthScriptError);
+    tConf.setInt("mapred.healthChecker.interval", 1000);
+    helper.copyFileToRemoteHost(healthScriptError, client.getHostName(), 
+        remotePath, cluster);
+    cluster.restartDaemonWithNewConfig(client, "mapred-site.xml", tConf, 
+        Role.TT);
+    //make sure the TT is blacklisted
+    helper.verifyTTBlackList(tConf, client,
+        "ERROR Task Tracker status is fatal", cluster);
+    //Now put back the task tracker in a healthy state
+    cluster.restart(client, Role.TT);
+    //now do the opposite of blacklist verification
+    tConf = client.getProxy().getDaemonConf();
+    helper.deleteFileOnRemoteHost(remotePath+File.separator+healthScriptError,
+        client.getHostName());
+  } 
+  
+  @AfterClass
+  public static void tearDown() throws Exception {    
+    cluster.tearDown();
+  }
+}

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptPathError.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptPathError.java?rev=1077577&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptPathError.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptPathError.java Fri Mar  4 04:30:59 2011
@@ -0,0 +1,65 @@
+package org.apache.hadoop.mapred;
+
+import java.io.File;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.test.system.MRCluster;
+import org.apache.hadoop.mapreduce.test.system.TTClient;
+import org.apache.hadoop.mapreduce.test.system.MRCluster.Role;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestHealthScriptPathError {
+  public static MRCluster cluster;
+  public static HealthScriptHelper helper;
+  public static String remotePath;
+  public String invalidHealthScript="invalidHealthScript";
+  public static String remoteHSPath = "test.system.hdrc.healthscript.path";
+  static final Log LOG = LogFactory.getLog(TestHealthScriptPathError.class);
+  
+  @BeforeClass
+  public static void setUp() throws java.lang.Exception {
+    String [] expExcludeList = new String[2];
+    expExcludeList[0] = "java.net.ConnectException";
+    expExcludeList[1] = "java.io.IOException";
+    cluster = MRCluster.createCluster(new Configuration());
+    cluster.setExcludeExpList(expExcludeList);
+    cluster.setUp();
+    remotePath = cluster.getConf().get(remoteHSPath);
+    helper = new HealthScriptHelper();
+  }
+  /**
+   * Error in the test path and script will not run, the TT will not be marked
+   * unhealthy
+   * @throws Exception in case of test errors
+   */
+  @Test
+  public void testHealthScriptPathError() throws Exception {
+    LOG.info("running testHealthScriptPathError");
+    TTClient client = cluster.getTTClient();
+    Configuration tConf= client.getProxy().getDaemonConf();    
+    tConf.set("mapred.task.tracker.report.address",
+        cluster.getConf().get("mapred.task.tracker.report.address"));
+    String defaultHealthScript = tConf.get("mapred.healthChecker.script.path");
+    Assert.assertTrue("Health script was not set", defaultHealthScript != null);    
+    tConf.set("mapred.healthChecker.script.path", remotePath+File.separator+
+        invalidHealthScript);
+    tConf.setInt("mapred.healthChecker.interval",1000);
+    cluster.restartDaemonWithNewConfig(client, "mapred-site.xml", tConf, 
+        Role.TT);
+    //For a invalid health script the TT remains healthy
+    helper.verifyTTNotBlackListed( client, tConf, cluster);
+    cluster.restart(client, Role.TT);    
+    tConf = client.getProxy().getDaemonConf();
+  } 
+  
+  @AfterClass
+  public static void tearDown() throws Exception {    
+    cluster.tearDown();
+  }
+  
+}

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptTimeout.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptTimeout.java?rev=1077577&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptTimeout.java (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestHealthScriptTimeout.java Fri Mar  4 04:30:59 2011
@@ -0,0 +1,73 @@
+package org.apache.hadoop.mapred;
+
+import java.io.File;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.test.system.MRCluster;
+import org.apache.hadoop.mapreduce.test.system.TTClient;
+import org.apache.hadoop.mapreduce.test.system.MRCluster.Role;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestHealthScriptTimeout {
+  public static String remotePath;
+  public static MRCluster cluster;
+  public static HealthScriptHelper helper;
+  public String healthScriptTimeout="healthScriptTimeout";
+  public static String remoteHSPath = "test.system.hdrc.healthscript.path";
+  static final Log LOG = LogFactory.getLog(TestHealthScriptTimeout.class);
+  
+  @BeforeClass
+  public static void setUp() throws java.lang.Exception {
+    String [] expExcludeList = new String[2];
+    expExcludeList[0] = "java.net.ConnectException";
+    expExcludeList[1] = "java.io.IOException";
+    cluster = MRCluster.createCluster(new Configuration());
+    cluster.setExcludeExpList(expExcludeList);
+    cluster.setUp();
+    remotePath = cluster.getConf().get(remoteHSPath);    
+    helper = new HealthScriptHelper();
+  }
+  
+  /**
+   * In this case the test times out the task tracker will get blacklisted  . 
+   * @throws Exception in case of test errors 
+   */
+  @Test
+  public void testScriptTimeout() throws Exception {
+    LOG.info("running testScriptTimeout");
+    TTClient client = cluster.getTTClient();
+    Configuration tConf= client.getProxy().getDaemonConf();
+    int defaultTimeout = tConf.getInt("mapred.healthChecker.script.timeout", 0);
+    tConf.set("mapred.task.tracker.report.address",
+        cluster.getConf().get("mapred.task.tracker.report.address"));
+    Assert.assertTrue("Health script timeout was not set",defaultTimeout != 0);     
+    tConf.set("mapred.healthChecker.script.path", remotePath+File.separator+
+        healthScriptTimeout);
+    tConf.setInt("mapred.healthChecker.script.timeout", 100);
+    tConf.setInt("mapred.healthChecker.interval",1000);    
+    helper.copyFileToRemoteHost(healthScriptTimeout, client.getHostName(),
+        remotePath, cluster);
+    cluster.restartDaemonWithNewConfig(client, "mapred-site.xml", tConf, 
+        Role.TT);
+    //make sure the TT is blacklisted
+    helper.verifyTTBlackList(tConf, client, "Node health script timed out",
+        cluster);
+    //Now put back the task tracker in a health state
+    cluster.restart(client, Role.TT);
+    tConf = client.getProxy().getDaemonConf();
+    //now do the opposite of blacklist verification
+    helper.deleteFileOnRemoteHost(remotePath+File.separator+healthScriptTimeout,
+        client.getHostName());
+    
+  } 
+  
+  @AfterClass
+  public static void tearDown() throws Exception {    
+    cluster.tearDown();
+  }
+}

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/JTProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/JTProtocol.java?rev=1077577&r1=1077576&r2=1077577&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/JTProtocol.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/JTProtocol.java Fri Mar  4 04:30:59 2011
@@ -118,4 +118,12 @@ public interface JTProtocol extends Daem
    * @throws IOException
    */
   String getJobHistoryLocationForRetiredJob(JobID jobID) throws IOException;
+  
+  /**
+   * This directly calls the JobTracker public with no modifications
+   * @param trackerID uniquely indentifies the task tracker
+   * @return
+   * @throws IOException is thrown in case of RPC error
+   */
+  public boolean isBlackListed(String trackerID) throws IOException;
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/MRCluster.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/MRCluster.java?rev=1077577&r1=1077576&r2=1077577&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/MRCluster.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/MRCluster.java Fri Mar  4 04:30:59 2011
@@ -60,7 +60,7 @@ public class MRCluster extends AbstractD
   private static String TT_hostFileName;
   private static String jtHostName;
 
-  protected enum Role {JT, TT};
+  public enum Role {JT, TT};
 
   static{
     Configuration.addDefaultResource("mapred-default.xml");
@@ -134,6 +134,19 @@ public class MRCluster extends AbstractD
     }
     return null;
   }
+  
+  /**
+   * This function will give access to one of many TTClient present
+   * @return an Instance of TTclient 
+   */
+  public TTClient getTTClient() {
+    for (TTClient c: getTTClients()) {
+      if (c != null){
+        return c;
+      }
+    }
+    return null;
+  }
 
   @Override
   public void ensureClean() throws IOException {

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptError
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptError?rev=1077577&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptError (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptError Fri Mar  4 04:30:59 2011
@@ -0,0 +1,19 @@
+#!/usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#This will mark the task tracker unhealthy with the custom error message
+echo "ERROR Task Tracker status is fatal"

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptTimeout
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptTimeout?rev=1077577&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptTimeout (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/healthScriptTimeout Fri Mar  4 04:30:59 2011
@@ -0,0 +1,19 @@
+#!/usr/bin/env bash
+
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#This will let the script sleep for 2 seconds
+sleep 2