Browse Source

commit 160c37114ee95c749c5a96d706a80d09345c4797
Author: Vinay Kumar Thota <vinayt@yahoo-inc.com>
Date: Fri Jun 4 09:04:10 2010 +0000

MAPREDUCE:1693 from https://issues.apache.org/jira/secure/attachment/12444913/1693-ydist_security.patch


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1077490 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 years ago
parent
commit
8595355fee

+ 550 - 0
src/test/system/java/org/apache/hadoop/mapred/TestTaskChildsKilling.java

@@ -0,0 +1,550 @@
+/**
+ * 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.
+ */
+
+package org.apache.hadoop.mapred;
+
+import org.apache.commons.logging.Log;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.mapreduce.test.system.JTProtocol;
+import org.apache.hadoop.mapreduce.test.system.JobInfo;
+import org.apache.hadoop.mapreduce.test.system.MRCluster;
+import org.apache.hadoop.mapreduce.test.system.TTClient;
+import org.apache.hadoop.mapreduce.test.system.JTClient;
+import org.apache.hadoop.mapreduce.test.system.TTProtocol;
+import org.apache.hadoop.mapreduce.test.system.TTTaskInfo;
+import org.apache.hadoop.mapreduce.test.system.TaskInfo;
+import org.apache.hadoop.mapreduce.test.system.FinishTaskControlAction;
+import org.apache.hadoop.mapred.JobClient.NetworkedJob;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.util.Tool;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Assert;
+import org.junit.Test;
+import java.io.IOException;
+import java.io.DataOutputStream;
+import java.util.Collection;
+import testjar.GenerateTaskChildProcess;
+import java.util.Hashtable;
+
+/**
+ * Submit a job which would spawn child processes and 
+ * verify whether the task child processes are cleaned up 
+ * or not after either job killed or task killed or task failed.
+ */
+public class TestTaskChildsKilling {
+  private static final Log LOG = LogFactory
+      .getLog(TestTaskChildsKilling.class);
+  private static MRCluster cluster;
+  private static Path inputDir = new Path("input");
+  private static Path outputDir = new Path("output");
+  private static Configuration conf = new Configuration();
+  private static String confFile = "mapred-site.xml";
+
+  @BeforeClass
+  public static void before() throws Exception {
+    Hashtable<String,Long> prop = new Hashtable<String,Long>();
+    prop.put("mapred.map.max.attempts", 1L);
+    String [] expExcludeList = {"java.net.ConnectException",
+        "java.io.IOException"};
+    cluster = MRCluster.createCluster(conf);
+    cluster.setExcludeExpList(expExcludeList);
+    cluster.setUp();
+    cluster.restartClusterWithNewConfig(prop, confFile);
+    UtilsForTests.waitFor(1000);
+    conf = cluster.getJTClient().getProxy().getDaemonConf();
+    createInput(inputDir, conf);
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    cleanup(inputDir, conf);
+    cleanup(outputDir, conf);
+    cluster.tearDown();
+    cluster.restart();
+    UtilsForTests.waitFor(1000);
+  }
+
+  /**
+   * Verifying the process tree cleanup of a task after task is killed
+   * by using -kill-task option.
+   */
+  @Test
+  public void testProcessTreeCleanupOfKilledTask1() throws 
+      Exception {
+    TaskInfo taskInfo = null;
+    TaskID tID = null;
+    TTTaskInfo [] ttTaskinfo = null;
+    String pid = null;
+    TTProtocol ttIns = null;
+    TTClient ttClientIns = null;
+    int counter = 0;
+
+    JobConf jobConf = new JobConf(conf);
+    jobConf.setJobName("Message Display");
+    jobConf.setJarByClass(GenerateTaskChildProcess.class);
+    jobConf.setMapperClass(GenerateTaskChildProcess.StrDisplayMapper.class);
+    jobConf.setNumMapTasks(1);
+    jobConf.setNumReduceTasks(0);
+    jobConf.setMaxMapAttempts(1);
+    cleanup(outputDir, conf);
+    FileInputFormat.setInputPaths(jobConf, inputDir);
+    FileOutputFormat.setOutputPath(jobConf, outputDir);
+    
+    JTClient jtClient = cluster.getJTClient();
+    JobClient client = jtClient.getClient();
+    JTProtocol wovenClient = cluster.getJTClient().getProxy();
+    RunningJob runJob = client.submitJob(jobConf);
+    JobID id = runJob.getID();
+    JobInfo jInfo = wovenClient.getJobInfo(id);
+    Assert.assertNotNull("Job information is null",jInfo);
+
+    Assert.assertTrue("Job has not been started for 1 min.", 
+        jtClient.isJobStarted(id));
+
+    TaskInfo[] taskInfos = wovenClient.getTaskInfo(id);
+    for (TaskInfo taskinfo : taskInfos) {
+      if (!taskinfo.isSetupOrCleanup()) {
+        taskInfo = taskinfo;
+        break;
+      }
+    }
+
+    Assert.assertTrue("Task has not been started for 1 min.", 
+        jtClient.isTaskStarted(taskInfo));
+
+    tID = TaskID.downgrade(taskInfo.getTaskID());
+    TaskAttemptID tAttID = new TaskAttemptID(tID,0);
+    FinishTaskControlAction action = new FinishTaskControlAction(tID);
+
+    Collection<TTClient> ttClients = cluster.getTTClients();
+    for (TTClient ttClient : ttClients) {
+      TTProtocol tt = ttClient.getProxy();
+      tt.sendAction(action);
+      ttTaskinfo = tt.getTasks();
+      for (TTTaskInfo tttInfo : ttTaskinfo) {
+        if (!tttInfo.isTaskCleanupTask()) {
+          pid = tttInfo.getPid();
+          ttClientIns = ttClient;
+          ttIns = tt;
+          break;
+        }
+      }
+      if (ttClientIns != null) {
+        break;
+      }
+    }
+
+
+    Assert.assertTrue("Map process tree is not alive before task kills.", 
+        ttIns.isProcessTreeAlive(pid));
+
+    String args[] = new String[] { "-kill-task", tAttID.toString() };
+    int exitCode = runTool(jobConf, client, args);
+    Assert.assertEquals("Exit Code:", 0, exitCode);
+    
+    LOG.info("Waiting till the task is killed...");
+    counter = 0;
+    while (counter < 30) {
+      if (taskInfo.getTaskStatus().length > 0) {
+        if (taskInfo.getTaskStatus()[0].getRunState() == 
+            TaskStatus.State.KILLED) {
+          break;
+        }
+      }
+      UtilsForTests.waitFor(1000);
+      taskInfo = wovenClient.getTaskInfo(taskInfo.getTaskID());
+      counter ++;
+    }
+
+    runJob.killJob();
+    LOG.info("Waiting till the job is completed...");
+    counter = 0;
+    while (counter < 60) {
+      if (jInfo.getStatus().isJobComplete()) {
+        break;
+      } 
+      UtilsForTests.waitFor(1000);
+      jInfo = wovenClient.getJobInfo(id);
+      counter ++;
+    }
+    Assert.assertTrue("Job has not  been completed for 1 min.", 
+        counter != 60 );
+    ttIns = ttClientIns.getProxy();
+    UtilsForTests.waitFor(1000);
+    Assert.assertTrue("Map process is still alive after task has been killed.", 
+        !ttIns.isProcessTreeAlive(pid));
+  }
+
+  /**
+   * Verifying the process tree cleanup of a particular task 
+   * after task is killed.
+   */
+  @Test
+  public void testProcessTreeCleanupOfKilledTask2() throws 
+      IOException {
+    TaskInfo taskInfo = null;
+    TaskID tID = null;
+    TaskAttemptID taskAttID = null;
+    TTTaskInfo [] ttTaskinfo = null;
+    String pid = null;
+    TTProtocol ttIns = null;
+    TTClient ttClientIns = null;
+    int counter = 0;
+
+    JobConf jobConf = new JobConf(conf);
+    jobConf.setJobName("Message Display");
+    jobConf.setJarByClass(GenerateTaskChildProcess.class);
+    jobConf.setMapperClass(GenerateTaskChildProcess.StrDisplayMapper.class);
+    jobConf.setNumMapTasks(1);
+    jobConf.setNumReduceTasks(0);
+    cleanup(outputDir, conf);
+    FileInputFormat.setInputPaths(jobConf, inputDir);
+    FileOutputFormat.setOutputPath(jobConf, outputDir);
+ 
+    JTClient jtClient = cluster.getJTClient();
+    JobClient client = jtClient.getClient();
+    JTProtocol wovenClient = cluster.getJTClient().getProxy();
+    RunningJob runJob = client.submitJob(jobConf);
+    JobID id = runJob.getID();
+    JobInfo jInfo = wovenClient.getJobInfo(id);    
+    Assert.assertNotNull("Job information is null", jInfo);
+
+    Assert.assertTrue("Job has not been started for 1 min.", 
+        jtClient.isJobStarted(id));
+
+    TaskInfo[] taskInfos = wovenClient.getTaskInfo(id);
+    for (TaskInfo taskinfo : taskInfos) {
+      if (!taskinfo.isSetupOrCleanup()) {
+        taskInfo = taskinfo;
+        break;
+      }
+    }
+
+    Assert.assertTrue("Task has not been started for 1 min.", 
+        jtClient.isTaskStarted(taskInfo));
+
+    tID = TaskID.downgrade(taskInfo.getTaskID());
+    taskAttID = new TaskAttemptID(tID,0);
+    FinishTaskControlAction action = new FinishTaskControlAction(tID);
+    Collection<TTClient> ttClients = cluster.getTTClients();
+    for (TTClient ttClient : ttClients) {
+      TTProtocol tt = ttClient.getProxy();
+      tt.sendAction(action);
+      ttTaskinfo = tt.getTasks();
+      for (TTTaskInfo tttInfo : ttTaskinfo) {
+        if (!tttInfo.isTaskCleanupTask()) {
+          pid = tttInfo.getPid();
+          ttClientIns = ttClient;
+          ttIns = tt;
+          break;
+        }
+      }
+      if (ttClientIns != null) {
+        break;
+      }
+    }
+
+
+    Assert.assertTrue("Map process is not alive before task kills.", 
+        ttIns.isProcessTreeAlive(pid));
+
+    NetworkedJob networkJob = client.new NetworkedJob(jInfo.getStatus());
+    networkJob.killTask(taskAttID, false);
+
+    LOG.info("Waiting till the task is killed...");
+    taskInfo = wovenClient.getTaskInfo(taskInfo.getTaskID());
+    counter = 0;
+    while (counter < 30) {
+      if (taskInfo.getTaskStatus()[0].getRunState() == 
+              TaskStatus.State.KILLED) {
+        break;
+      } 
+      UtilsForTests.waitFor(1000);
+      taskInfo = wovenClient.getTaskInfo(taskInfo.getTaskID());
+      counter ++;
+    }
+    runJob.killJob();
+    LOG.info("Waiting till the job is completed...");
+    counter = 0;
+    while (counter < 60) {
+      if (jInfo.getStatus().isJobComplete()) {
+        break;
+      }
+      UtilsForTests.waitFor(1000);
+      jInfo = wovenClient.getJobInfo(id);
+      counter ++;
+    }
+    Assert.assertTrue("Job has not been completed for 1 min.", 
+        counter != 60);
+    UtilsForTests.waitFor(2000);
+    ttIns = ttClientIns.getProxy();    
+    Assert.assertTrue("Map process is still alive after task has been killed.", 
+        !ttIns.isProcessTreeAlive(pid));
+  }
+
+  /**
+   * Verifying the child process tree clean up of a task which fails due
+   * to an exception. 
+   */
+  @Test
+  public void testProcessTreeCleanupOfFailedTask1() throws IOException {
+    TaskInfo taskInfo = null;
+    TaskID tID = null;
+    TTTaskInfo [] ttTaskinfo = null;
+    String pid = null;
+    TTProtocol ttIns = null;
+    TTClient ttClientIns = null;
+    int counter = 0;
+
+    JobConf jobConf = new JobConf(conf);
+    jobConf.setJobName("Message Display");
+    jobConf.setJarByClass(GenerateTaskChildProcess.class);
+    jobConf.setMapperClass(GenerateTaskChildProcess.FailedMapper.class);
+    jobConf.setNumMapTasks(1);
+    jobConf.setNumReduceTasks(0);
+    cleanup(outputDir, conf);
+    FileInputFormat.setInputPaths(jobConf, inputDir);
+    FileOutputFormat.setOutputPath(jobConf, outputDir);
+
+    JTClient jtClient = cluster.getJTClient();
+    JobClient client = jtClient.getClient();
+    JTProtocol wovenClient = cluster.getJTClient().getProxy();
+    RunningJob runJob = client.submitJob(jobConf);
+    JobID id = runJob.getID();
+    JobInfo jInfo = wovenClient.getJobInfo(id);
+    Assert.assertNotNull("Job information is null", jInfo);
+
+    Assert.assertTrue("Job has not been started for 1 min.", 
+        jtClient.isJobStarted(id));
+
+    TaskInfo[] taskInfos = wovenClient.getTaskInfo(id);
+    for (TaskInfo taskinfo : taskInfos) {
+      if (!taskinfo.isSetupOrCleanup()) {
+        taskInfo = taskinfo;
+        break;
+      }
+    }
+
+    Assert.assertTrue("Task has not been started for 1 min.", 
+        jtClient.isTaskStarted(taskInfo));
+
+    tID = TaskID.downgrade(taskInfo.getTaskID());    
+    FinishTaskControlAction action = new FinishTaskControlAction(tID);
+    
+    Collection<TTClient> ttClients = cluster.getTTClients();
+    for (TTClient ttClient : ttClients) {
+      TTProtocol tt = ttClient.getProxy();
+      tt.sendAction(action);
+      ttTaskinfo = tt.getTasks();
+      for (TTTaskInfo tttInfo : ttTaskinfo) {
+        if (!tttInfo.isTaskCleanupTask()) {
+          pid = tttInfo.getPid();
+          ttClientIns = ttClient;
+          ttIns = tt;
+          break;
+        }
+      }
+      if (ttClientIns != null) {
+        break;
+      }
+    }
+
+    Assert.assertTrue("Map process is not alive before task fails.", 
+            ttIns.isProcessTreeAlive(pid));
+
+    LOG.info("Waiting till the task is failed...");
+    counter = 0;
+    while (counter < 60) {
+      if (taskInfo.getTaskStatus().length > 0) {
+        if (taskInfo.getTaskStatus()[0].getRunState() == 
+            TaskStatus.State.FAILED) {
+          break;
+        }
+      }
+      UtilsForTests.waitFor(1000);
+      taskInfo = wovenClient.getTaskInfo(taskInfo.getTaskID());
+      counter++;
+    }
+
+    LOG.info("Waiting till the job is completed...");
+    counter = 0;
+    while (counter < 60) {
+      if (jInfo.getStatus().isJobComplete()) {
+        break;
+      }
+      UtilsForTests.waitFor(1000);
+      jInfo = wovenClient.getJobInfo(id);
+      counter ++;
+    }
+    Assert.assertTrue("Job has not been completed for 1 min.", 
+        counter != 60);
+    ttIns = ttClientIns.getProxy();
+    UtilsForTests.waitFor(2000);
+    Assert.assertTrue("Map process is still alive after task has been failed.", 
+            !ttIns.isProcessTreeAlive(pid));
+  }
+
+  /**
+   * Verifying the process tree cleanup of a task after task is failed
+   * by using -fail-task option.
+   */
+  @Test
+  public void testProcessTreeCleanupOfFailedTask2() throws 
+      Exception {
+    TaskInfo taskInfo = null;
+    TaskID tID = null;
+    TTTaskInfo [] ttTaskinfo = null;
+    String pid = null;
+    TTProtocol ttIns = null;
+    TTClient ttClientIns = null;
+    int counter = 0;
+    
+    JobConf jobConf = new JobConf(conf);
+    jobConf.setJobName("Message Display");
+    jobConf.setJarByClass(GenerateTaskChildProcess.class);
+    jobConf.setMapperClass(GenerateTaskChildProcess.StrDisplayMapper.class);
+    jobConf.setNumMapTasks(1);
+    jobConf.setNumReduceTasks(0);
+    cleanup(outputDir, conf);
+    FileInputFormat.setInputPaths(jobConf, inputDir);
+    FileOutputFormat.setOutputPath(jobConf, outputDir);
+
+    JTClient jtClient = cluster.getJTClient();
+    JobClient client = jtClient.getClient();
+    JTProtocol wovenClient = cluster.getJTClient().getProxy();
+    RunningJob runJob = client.submitJob(jobConf);
+    JobID id = runJob.getID();
+    JobInfo jInfo = wovenClient.getJobInfo(id);
+    Assert.assertNotNull("Job information is null", jInfo);
+
+    Assert.assertTrue("Job has not been started for 1 min.", 
+        jtClient.isJobStarted(id));
+
+    TaskInfo[] taskInfos = wovenClient.getTaskInfo(id);
+    for (TaskInfo taskinfo : taskInfos) {
+      if (!taskinfo.isSetupOrCleanup()) {
+        taskInfo = taskinfo;
+        break;
+      }
+    }
+
+    Assert.assertTrue("Task has not been started for 1 min.",
+        jtClient.isTaskStarted(taskInfo));
+
+    tID = TaskID.downgrade(taskInfo.getTaskID());
+    TaskAttemptID tAttID = new TaskAttemptID(tID,0);
+    FinishTaskControlAction action = new FinishTaskControlAction(tID);
+
+    Collection<TTClient> ttClients = cluster.getTTClients();
+    for (TTClient ttClient : ttClients) {
+      TTProtocol tt = ttClient.getProxy();
+      tt.sendAction(action);
+      ttTaskinfo = tt.getTasks();
+      for (TTTaskInfo tttInfo : ttTaskinfo) {
+        if (!tttInfo.isTaskCleanupTask()) {
+          pid = tttInfo.getPid();
+          ttClientIns = ttClient;
+          ttIns = tt;
+          break;
+        }
+      }
+      if (ttClientIns != null) {
+        break;
+      }
+    }
+
+
+    Assert.assertTrue("Map process is not alive before task fails.", 
+        ttIns.isProcessTreeAlive(pid));
+
+    String args[] = new String[] { "-fail-task", tAttID.toString() };
+    int exitCode = runTool(jobConf, client, args);
+    Assert.assertEquals("Exit Code:", 0, exitCode);
+
+    LOG.info("Waiting till the task is failed...");
+    taskInfo = wovenClient.getTaskInfo(taskInfo.getTaskID());
+    counter = 0;
+    while (counter < 60) {
+      if (taskInfo.getTaskStatus().length > 0) {
+        if (taskInfo.getTaskStatus()[0].getRunState() ==
+            TaskStatus.State.FAILED) {
+          break;
+        } 
+      }
+      UtilsForTests.waitFor(1000);
+      taskInfo = wovenClient.getTaskInfo(taskInfo.getTaskID());
+      counter ++;
+    }
+    counter = 0;
+    LOG.info("Waiting till the job is completed...");
+    while (counter < 60) {
+      if (jInfo.getStatus().isJobComplete()) {
+        break;
+      }
+      UtilsForTests.waitFor(1000);
+      jInfo = wovenClient.getJobInfo(id);
+      counter ++;
+    }
+
+    Assert.assertTrue("Job has not been completed for 1 min",
+        counter != 60);
+    ttIns = ttClientIns.getProxy();
+    UtilsForTests.waitFor(1000);
+    Assert.assertTrue("Map process is still alive after task has been failed.", 
+        !ttIns.isProcessTreeAlive(pid));
+  }
+
+  private int runTool(Configuration job, Tool tool, 
+      String[] jobArgs) throws Exception {
+    int returnStatus = ToolRunner.run(job, tool, jobArgs);
+    return returnStatus;
+  }
+
+  private static void cleanup(Path dir, Configuration conf) throws 
+      IOException {
+    FileSystem fs = dir.getFileSystem(conf);
+    fs.delete(dir, true);
+  }
+  private static void createInput(Path inDir, Configuration conf) throws 
+      IOException {
+    String input = "Hadoop is framework for data intensive distributed " 
+        + "applications.\n Hadoop enables applications" 
+        + " to work with thousands of nodes.";
+    FileSystem fs = inDir.getFileSystem(conf);
+    if (!fs.mkdirs(inDir)) {
+      throw new IOException("Failed to create the input directory:" 
+          + inDir.toString());
+    }
+    fs.setPermission(inDir, new FsPermission(FsAction.ALL, 
+        FsAction.ALL, FsAction.ALL));
+    DataOutputStream file = fs.create(new Path(inDir, "data.txt"));
+    int i = 0;
+    while(i < 10) {
+      file.writeBytes(input);
+      i++;
+    }
+    file.close();
+  }
+}

+ 245 - 0
src/test/testjar/GenerateTaskChildProcess.java

@@ -0,0 +1,245 @@
+/**
+ * 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.
+ */
+
+package testjar;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import java.util.Random;
+import java.io.IOException;
+import java.io.DataOutputStream;
+import java.io.File;
+
+/**
+ * It uses for defining a various types of mapperes with child processes. 
+ */
+public class GenerateTaskChildProcess {
+  private static final Log LOG = LogFactory
+          .getLog(GenerateTaskChildProcess.class);
+  /**
+   * It uses for defining the string appending mapper with 
+   * child processes.It's keep appending the string and increases
+   * the memory continuously. 
+   */
+  public static class StrAppendMapper extends MapReduceBase implements 
+          Mapper<LongWritable, Text, NullWritable, NullWritable> {
+    private JobConf conf;
+
+    public void configure(JobConf conf) {
+          this.conf = conf;
+    }
+
+    public void map(LongWritable key, Text value,
+            OutputCollector<NullWritable, NullWritable> out,
+                    Reporter reporter) throws IOException {
+      int counter = 0;
+      while (counter < 30) {
+        try {
+          reporter.progress();
+          synchronized (this) {
+            this.wait(1000);
+          }          
+        } catch (InterruptedException iexp) {
+          iexp.printStackTrace();
+          LOG.warn("Interrupted while the map was waiting.");
+        }
+        counter ++;
+      }
+      try {
+          createChildProcess(conf, "AppendStr");
+        } catch (Exception exp) {
+          exp.printStackTrace();
+          LOG.warn("Exception thrown while creating the child processes");
+      }
+    }
+
+    public void close() {
+    }
+  }
+  
+  /**
+   * It uses for defining the String display mapper with child processes.
+   */
+  public static class StrDisplayMapper extends MapReduceBase implements
+          Mapper<LongWritable, Text, NullWritable, NullWritable> {
+    private JobConf conf;
+    public void configure(JobConf conf) {
+      this.conf = conf;
+    }
+
+    public void map(LongWritable key, Text value,
+            OutputCollector<NullWritable, NullWritable> out,
+                    Reporter reporter) throws IOException {
+      int counter = 0;
+      while (counter < 30) {
+        try {
+          reporter.progress();
+          synchronized (this) {
+            this.wait(1000);
+          }
+        } catch (InterruptedException iexp) {
+          iexp.printStackTrace();
+          LOG.warn("Interrupted while the map was waiting.");
+          break;
+        }
+        counter ++;
+      }
+      try {
+        createChildProcess(conf, "DispStr");
+      } catch (Exception exp) {
+          exp.printStackTrace();
+          LOG.warn("Exception thrown while creating the child processes.");
+      }
+    }
+  }
+
+  /**
+   * It uses for defining a failed mapper with child processes.
+   *
+   */
+  public static class FailedMapper extends MapReduceBase implements
+          Mapper<LongWritable, Text, NullWritable, NullWritable> {
+    private JobConf conf;
+    public void configure(JobConf conf) {
+      try {
+        createChildProcess(conf, "failedmapper");
+      } catch (Exception exp) {
+        exp.printStackTrace();
+        LOG.warn("Exception throw while creating the child processes");
+      }
+    }
+
+    public void map(LongWritable key, Text value,
+          OutputCollector<NullWritable, NullWritable> out,
+                  Reporter reporter) throws IOException {
+      int counter = 0;
+      while (counter < 30) {
+        try {
+          reporter.progress();
+          synchronized (this) {
+            this.wait(1000);
+          }
+        } catch (InterruptedException iexp) {
+          iexp.printStackTrace();
+          LOG.warn("Interrupted while the map was waiting.");
+          break;
+        }
+        counter ++;
+      }
+      throw new RuntimeException("Mapper failed.");
+    }
+  }
+
+  /** 
+   * It uses for creating the child processes for a task.
+   * @param conf configuration for a job.
+   * @param jobName the name of the mapper job.
+   * @throws IOException if an I/O error occurs.
+   */
+  private static void createChildProcess(JobConf conf, String jobName)
+          throws IOException {
+    FileSystem fs = FileSystem.getLocal(conf);
+    File TMP_ROOT_DIR = new File("/tmp");
+    String TEST_ROOT_DIR = TMP_ROOT_DIR.getAbsolutePath() 
+            + Path.SEPARATOR + "ChildProc_" + jobName;
+    Path scriptDir = new Path(TEST_ROOT_DIR);
+    int numOfChildProcesses = 2;
+
+    if (fs.exists(scriptDir)) {
+      fs.delete(scriptDir, true);
+    }
+    fs.mkdirs(scriptDir);
+    fs.setPermission(scriptDir, new FsPermission(FsAction.ALL,
+            FsAction.ALL, FsAction.ALL));
+
+    String scriptDirName = scriptDir.toUri().getPath();
+    Random rm = new Random();
+    String scriptName = "ShellScript_" + jobName + "_" 
+            + rm.nextInt() + ".sh";
+    Path scriptPath = new Path(scriptDirName, scriptName);
+    String shellScript = scriptPath.toString();
+    String script = null;
+    if (jobName.equals("AppendStr")) {
+      script =  "#!/bin/sh\n" 
+              + "umask 000\n" 
+              + "StrVal=\"Hadoop is framework for data intensive "
+              + "distributed applications.\"\n"
+              + "StrVal=\"${StrVal}Hadoop enables applications to work "
+              + "with thousands of nodes.\"\n"
+              + "echo $StrVal\n"
+              + "if [ \"X$1\" != \"X0\" ]\nthen\n" 
+              + "  sh " + shellScript + " $(($1-1))\n"
+              + "else\n"
+              + "  while(true)\n" 
+              + "  do\n"
+              + "    StrVal=\"$StrVal Hadoop \"\n"
+              + "  done\n"
+              + "fi";
+    } else if (jobName.equals("DispStr")) {
+      script =  "#!/bin/sh\n" 
+              + "umask 000\n" 
+              + "msg=Welcome\n"
+              + "echo $msg\n"
+              + " if [ \"X$1\" != \"X0\" ]\nthen\n" 
+              + "  sh " + shellScript + " $(($1-1))\n" 
+              + "else\n" 
+              + "  while(true)\n"
+              + "  do\n"
+              + "    sleep 2 \n" 
+              + "  done\n" 
+              + "fi";
+    }else {
+     script =  "#!/bin/sh\n" 
+             + "umask 000\n" 
+             + "msg=Welcome\n"
+             + "echo $msg\n"
+             + " if [ \"X$1\" != \"X0\" ]\nthen\n" 
+             + "  sh " + shellScript + " $(($1-1))\n" 
+             + "else\n"
+             + "  for count in {1..1000}\n"
+             + "  do\n"
+             + "    echo \"$msg_$count\" \n" 
+             + "  done\n" 
+             + "fi";
+    }
+    DataOutputStream file = fs.create(scriptPath);
+    file.writeBytes(script);
+    file.close();
+    File scriptFile = new File(scriptDirName,scriptName);
+    scriptFile.setExecutable(true);
+    LOG.info("script absolute path:" + scriptFile.getAbsolutePath());
+    String [] cmd = new String[]{scriptFile.getAbsolutePath(), 
+            String.valueOf(numOfChildProcesses)};
+    ShellCommandExecutor shellExec = new ShellCommandExecutor(cmd);
+    shellExec.execute();
+  }
+
+}