Explorar o código

commit 53521eb587b76d342a75f81f5b1095826e01c946
Author: Hemanth Yamijala <yhemanth@yahoo-inc.com>
Date: Tue Jan 19 12:11:07 2010 +0530

MAPREDUCE:842 from https://issues.apache.org/jira/secure/attachment/12430697/842.20S-4.patch

+++ b/YAHOO-CHANGES.txt
+ MAPREDUCE-842. Setup secure permissions for localized job files,
+ intermediate outputs and log files on tasktrackers.
+ (Vinod Kumar Vavilapalli via yhemanth)
+


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

Owen O'Malley %!s(int64=14) %!d(string=hai) anos
pai
achega
5905832730
Modificáronse 40 ficheiros con 2950 adicións e 1044 borrados
  1. 20 7
      build.xml
  2. 2 3
      conf/taskcontroller.cfg
  3. 9 1
      src/c++/task-controller/Makefile.in
  4. 19 10
      src/c++/task-controller/configuration.c
  5. 2 2
      src/c++/task-controller/configuration.h.in
  6. 7 2
      src/c++/task-controller/configure.ac
  7. 42 21
      src/c++/task-controller/main.c
  8. 599 84
      src/c++/task-controller/task-controller.c
  9. 43 15
      src/c++/task-controller/task-controller.h
  10. 180 0
      src/c++/task-controller/tests/test-task-controller.c
  11. 1 1
      src/contrib/build-contrib.xml
  12. 1 1
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingAsDifferentUser.java
  13. 8 3
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestUlimit.java
  14. 37 14
      src/docs/src/documentation/content/xdocs/cluster_setup.xml
  15. 5 3
      src/mapred/org/apache/hadoop/mapred/Child.java
  16. 3 8
      src/mapred/org/apache/hadoop/mapred/DefaultTaskController.java
  17. 9 2
      src/mapred/org/apache/hadoop/mapred/IsolationRunner.java
  18. 1 1
      src/mapred/org/apache/hadoop/mapred/JobConf.java
  19. 26 14
      src/mapred/org/apache/hadoop/mapred/JvmManager.java
  20. 125 181
      src/mapred/org/apache/hadoop/mapred/LinuxTaskController.java
  21. 28 17
      src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
  22. 100 97
      src/mapred/org/apache/hadoop/mapred/MapOutputFile.java
  23. 29 21
      src/mapred/org/apache/hadoop/mapred/MapTask.java
  24. 2 2
      src/mapred/org/apache/hadoop/mapred/MapTaskRunner.java
  25. 10 11
      src/mapred/org/apache/hadoop/mapred/ReduceTask.java
  26. 2 2
      src/mapred/org/apache/hadoop/mapred/ReduceTaskRunner.java
  27. 0 2
      src/mapred/org/apache/hadoop/mapred/Task.java
  28. 111 43
      src/mapred/org/apache/hadoop/mapred/TaskController.java
  29. 8 3
      src/mapred/org/apache/hadoop/mapred/TaskLog.java
  30. 365 236
      src/mapred/org/apache/hadoop/mapred/TaskRunner.java
  31. 428 163
      src/mapred/org/apache/hadoop/mapred/TaskTracker.java
  32. 47 17
      src/test/org/apache/hadoop/mapred/ClusterWithLinuxTaskController.java
  33. 1 1
      src/test/org/apache/hadoop/mapred/MiniMRCluster.java
  34. 9 9
      src/test/org/apache/hadoop/mapred/TestIsolationRunner.java
  35. 25 36
      src/test/org/apache/hadoop/mapred/TestJobExecutionAsDifferentUser.java
  36. 18 4
      src/test/org/apache/hadoop/mapred/TestKillSubProcesses.java
  37. 243 0
      src/test/org/apache/hadoop/mapred/TestLocalizationWithLinuxTaskController.java
  38. 2 4
      src/test/org/apache/hadoop/mapred/TestMapRed.java
  39. 3 3
      src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java
  40. 380 0
      src/test/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java

+ 20 - 7
build.xml

@@ -740,8 +740,8 @@
       <sysproperty key="test.debug.data" value="${test.debug.data}"/>
       <sysproperty key="test.debug.data" value="${test.debug.data}"/>
       <sysproperty key="hadoop.log.dir" value="${test.log.dir}"/>
       <sysproperty key="hadoop.log.dir" value="${test.log.dir}"/>
       <sysproperty key="test.src.dir" value="${test.src.dir}"/>
       <sysproperty key="test.src.dir" value="${test.src.dir}"/>
-    	<sysproperty key="taskcontroller-path" value="${taskcontroller-path}"/>
-    	<sysproperty key="taskcontroller-user" value="${taskcontroller-user}"/>
+      <sysproperty key="taskcontroller-path" value="${taskcontroller-path}"/>
+      <sysproperty key="taskcontroller-ugi" value="${taskcontroller-ugi}"/>
       <sysproperty key="test.build.extraconf" value="${test.build.extraconf}" />
       <sysproperty key="test.build.extraconf" value="${test.build.extraconf}" />
       <sysproperty key="hadoop.policy.file" value="hadoop-policy.xml"/>
       <sysproperty key="hadoop.policy.file" value="hadoop-policy.xml"/>
       <sysproperty key="java.library.path"
       <sysproperty key="java.library.path"
@@ -1820,10 +1820,8 @@
   <target name="init-task-controller-build">
   <target name="init-task-controller-build">
     <mkdir dir="${build.c++.task-controller}" />
     <mkdir dir="${build.c++.task-controller}" />
     <copy todir="${build.c++.task-controller}">
     <copy todir="${build.c++.task-controller}">
-      <fileset dir="${c++.task-controller.src}" includes="*.c">
-      </fileset>
-      <fileset dir="${c++.task-controller.src}" includes="*.h">
-      </fileset>
+      <fileset dir="${c++.task-controller.src}" includes="*.c"/>
+      <fileset dir="${c++.task-controller.src}" includes="*.h"/>
     </copy>
     </copy>
     <chmod file="${c++.task-controller.src}/configure" perm="ugo+x"/> 
     <chmod file="${c++.task-controller.src}/configure" perm="ugo+x"/> 
     <condition property="task-controller.conf.dir.passed">
     <condition property="task-controller.conf.dir.passed">
@@ -1863,5 +1861,20 @@
       <arg value="install" />
       <arg value="install" />
     </exec>
     </exec>
   </target>
   </target>
-  <!-- end of task-controller target -->
+  <target name="test-task-controller" depends="task-controller">
+    <copy todir="${build.c++.task-controller}" verbose="true">
+      <fileset dir="${c++.task-controller.src}" includes="tests/"/>
+    </copy>
+    <exec executable="${make.cmd}" dir="${build.c++.task-controller}" 
+        searchpath="yes" failonerror="yes">
+      <arg value="clean" />
+      <arg value="test" />
+    </exec>
+    <exec executable="${build.c++.task-controller}/tests/test-task-controller"
+        dir="${build.c++.task-controller}/tests/"
+        failonerror="yes">
+    </exec>
+  </target>
+
+  <!-- end of task-controller targets -->
 </project>
 </project>

+ 2 - 3
conf/taskcontroller.cfg

@@ -1,4 +1,3 @@
-mapred.local.dir=#configured value of hadoop.tmp.dir it can be a list of paths comma seperated
-hadoop.pid.dir=#configured HADOOP_PID_DIR
-hadoop.indent.str=#configured HADOOP_IDENT_STR
+mapred.local.dir=#configured value of mapred.local.dir. It can be a list of comma separated paths.
+hadoop.log.dir=#configured value of hadoop.log.dir.
 mapred.tasktracker.tasks.sleeptime-before-sigkill=#sleep time before sig kill is to be sent to process group after sigterm is sent. Should be in seconds
 mapred.tasktracker.tasks.sleeptime-before-sigkill=#sleep time before sig kill is to be sent to process group after sigterm is sent. Should be in seconds

+ 9 - 1
src/c++/task-controller/Makefile.in

@@ -21,6 +21,9 @@ CFLAGS = @CFLAGS@
 BINARY=task-controller
 BINARY=task-controller
 installdir = @prefix@
 installdir = @prefix@
 
 
+testdir = tests
+TESTOBJS=${testdir}/test-task-controller.o task-controller.o configuration.o
+TESTBINARY=${testdir}/test-task-controller
 
 
 all: $(OBJS)
 all: $(OBJS)
 	$(CC) $(CFLAG) -o $(BINARY) $(OBJS)
 	$(CC) $(CFLAG) -o $(BINARY) $(OBJS)
@@ -34,9 +37,14 @@ task-controller.o: task-controller.c task-controller.h
 configuration.o: configuration.h configuration.c
 configuration.o: configuration.h configuration.c
 	$(CC) $(CFLAG) -o configuration.o -c configuration.c
 	$(CC) $(CFLAG) -o configuration.o -c configuration.c
 
 
+${testdir}/test-task-controller.o: task-controller.c task-controller.h
+	$(CC) $(CFLAG) -o ${testdir}/test-task-controller.o -c ${testdir}/test-task-controller.c
+
+test: $(TESTOBJS)
+	$(CC) $(CFLAG) -o $(TESTBINARY) $(TESTOBJS)
 
 
 clean:
 clean:
-	rm -rf $(BINARY) $(OBJS)
+	rm -rf $(BINARY) $(OBJS) $(TESTOBJS)
 
 
 install: all
 install: all
 	cp $(BINARY) $(installdir)
 	cp $(BINARY) $(installdir)

+ 19 - 10
src/c++/task-controller/configuration.c

@@ -72,8 +72,9 @@ void get_configs() {
 #endif
 #endif
 
 
 #ifdef DEBUG
 #ifdef DEBUG
-  fprintf(LOGFILE,"get_configs :Conf file name is : %s \n", file_name);
+  fprintf(LOGFILE, "get_configs :Conf file name is : %s \n", file_name);
 #endif
 #endif
+
   //allocate space for ten configuration items.
   //allocate space for ten configuration items.
   config.confdetails = (struct confentry **) malloc(sizeof(struct confentry *)
   config.confdetails = (struct confentry **) malloc(sizeof(struct confentry *)
       * MAX_SIZE);
       * MAX_SIZE);
@@ -87,7 +88,7 @@ void get_configs() {
   while(!feof(conf_file)) {
   while(!feof(conf_file)) {
     line = (char *) malloc(linesize);
     line = (char *) malloc(linesize);
     if(line == NULL) {
     if(line == NULL) {
-      fprintf(LOGFILE,"malloc failed while reading configuration file.\n");
+      fprintf(LOGFILE, "malloc failed while reading configuration file.\n");
       goto cleanup;
       goto cleanup;
     }
     }
     size_read = getline(&line,&linesize,conf_file);
     size_read = getline(&line,&linesize,conf_file);
@@ -123,9 +124,11 @@ void get_configs() {
           "Failed allocating memory for single configuration item\n");
           "Failed allocating memory for single configuration item\n");
       goto cleanup;
       goto cleanup;
     }
     }
+
 #ifdef DEBUG
 #ifdef DEBUG
-    fprintf(LOGFILE,"get_configs : Adding conf key : %s \n", equaltok);
+    fprintf(LOGFILE, "get_configs : Adding conf key : %s \n", equaltok);
 #endif
 #endif
+
     memset(config.confdetails[config.size], 0, sizeof(struct confentry));
     memset(config.confdetails[config.size], 0, sizeof(struct confentry));
     config.confdetails[config.size]->key = (char *) malloc(
     config.confdetails[config.size]->key = (char *) malloc(
             sizeof(char) * (strlen(equaltok)+1));
             sizeof(char) * (strlen(equaltok)+1));
@@ -142,9 +145,11 @@ void get_configs() {
       free(config.confdetails[config.size]);
       free(config.confdetails[config.size]);
       continue;
       continue;
     }
     }
+
 #ifdef DEBUG
 #ifdef DEBUG
-    fprintf(LOGFILE,"get_configs : Adding conf value : %s \n", equaltok);
+    fprintf(LOGFILE, "get_configs : Adding conf value : %s \n", equaltok);
 #endif
 #endif
+
     config.confdetails[config.size]->value = (char *) malloc(
     config.confdetails[config.size]->value = (char *) malloc(
             sizeof(char) * (strlen(equaltok)+1));
             sizeof(char) * (strlen(equaltok)+1));
     strcpy((char *)config.confdetails[config.size]->value, equaltok);
     strcpy((char *)config.confdetails[config.size]->value, equaltok);
@@ -184,8 +189,7 @@ void get_configs() {
  * array, next time onwards used the populated array.
  * array, next time onwards used the populated array.
  *
  *
  */
  */
-
-const char * get_value(char* key) {
+const char * get_value(const char* key) {
   int count;
   int count;
   if (config.size == 0) {
   if (config.size == 0) {
     get_configs();
     get_configs();
@@ -196,22 +200,27 @@ const char * get_value(char* key) {
   }
   }
   for (count = 0; count < config.size; count++) {
   for (count = 0; count < config.size; count++) {
     if (strcmp(config.confdetails[count]->key, key) == 0) {
     if (strcmp(config.confdetails[count]->key, key) == 0) {
-      return config.confdetails[count]->value;
+      return strdup(config.confdetails[count]->value);
     }
     }
   }
   }
   return NULL;
   return NULL;
 }
 }
 
 
-const char ** get_values(char * key) {
+/**
+ * Function to return an array of values for a key.
+ * Value delimiter is assumed to be a comma.
+ */
+const char ** get_values(const char * key) {
+  const char ** toPass = NULL;
+  const char *value = get_value(key);
   char *tempTok = NULL;
   char *tempTok = NULL;
   char *tempstr = NULL;
   char *tempstr = NULL;
   int size = 0;
   int size = 0;
   int toPassSize = MAX_SIZE;
   int toPassSize = MAX_SIZE;
-  const char** toPass = (const char **) malloc(sizeof(char *) * toPassSize);
 
 
   //first allocate any array of 10
   //first allocate any array of 10
-  const char * value = get_value(key);
   if(value != NULL) {
   if(value != NULL) {
+    toPass = (const char **) malloc(sizeof(char *) * toPassSize);
     tempTok = strtok_r((char *)value, ",", &tempstr);
     tempTok = strtok_r((char *)value, ",", &tempstr);
     while (tempTok != NULL) {
     while (tempTok != NULL) {
       toPass[size++] = tempTok;
       toPass[size++] = tempTok;

+ 2 - 2
src/c++/task-controller/configuration.h.in

@@ -53,10 +53,10 @@ extern struct configuration config;
   extern char *hadoop_conf_dir;
   extern char *hadoop_conf_dir;
 #endif
 #endif
 //method exposed to get the configurations
 //method exposed to get the configurations
-const char * get_value(char* key);
+const char * get_value(const char* key);
 //method to free allocated configuration
 //method to free allocated configuration
 void free_configurations();
 void free_configurations();
 
 
 //function to return array of values pointing to the key. Values are
 //function to return array of values pointing to the key. Values are
 //comma seperated strings.
 //comma seperated strings.
-const char ** get_values(char* key);
+const char ** get_values(const char* key);

+ 7 - 2
src/c++/task-controller/configure.ac

@@ -38,7 +38,7 @@ AC_PROG_CC
 
 
 # Checks for header files.
 # Checks for header files.
 AC_HEADER_STDC
 AC_HEADER_STDC
-AC_CHECK_HEADERS([stdlib.h string.h unistd.h])
+AC_CHECK_HEADERS([stdlib.h string.h unistd.h fcntl.h])
 
 
 #check for HADOOP_CONF_DIR
 #check for HADOOP_CONF_DIR
 
 
@@ -50,12 +50,17 @@ fi
 # Checks for typedefs, structures, and compiler characteristics.
 # Checks for typedefs, structures, and compiler characteristics.
 AC_C_CONST
 AC_C_CONST
 AC_TYPE_PID_T
 AC_TYPE_PID_T
+AC_TYPE_MODE_T
+AC_TYPE_SIZE_T
 
 
 # Checks for library functions.
 # Checks for library functions.
 AC_FUNC_MALLOC
 AC_FUNC_MALLOC
 AC_FUNC_REALLOC
 AC_FUNC_REALLOC
-AC_CHECK_FUNCS([strerror])
+AC_FUNC_CHOWN
+AC_CHECK_FUNCS([strerror memset mkdir rmdir strdup])
 
 
 AC_CONFIG_FILES([Makefile])
 AC_CONFIG_FILES([Makefile])
 AC_OUTPUT
 AC_OUTPUT
 
 
+AC_HEADER_STDBOOL
+AC_PROG_MAKE_SET

+ 42 - 21
src/c++/task-controller/main.c

@@ -17,6 +17,32 @@
  */
  */
 #include "task-controller.h"
 #include "task-controller.h"
 
 
+void open_log_file(const char *log_file) {
+  if (log_file == NULL) {
+    LOGFILE = stdout;
+  } else {
+    LOGFILE = fopen(log_file, "a");
+    if (LOGFILE == NULL) {
+      fprintf(stdout, "Unable to open LOGFILE : %s \n", log_file);
+      LOGFILE = stdout;
+    }
+    if (LOGFILE != stdout) {
+      if (chmod(log_file, S_IREAD | S_IEXEC | S_IWRITE | S_IROTH | S_IWOTH
+          | S_IRGRP | S_IWGRP) < 0) {
+        fprintf(stdout, "Unable to change permission of the log file %s \n",
+            log_file);
+        fclose(LOGFILE);
+        fprintf(stdout, "changing log file to stdout");
+        LOGFILE = stdout;
+      }
+    }
+  }
+}
+
+void display_usage(FILE *stream) {
+  fprintf(stream,
+      "Usage: task-controller [-l logfile] user command command-args\n");
+}
 
 
 int main(int argc, char **argv) {
 int main(int argc, char **argv) {
   int command;
   int command;
@@ -24,6 +50,7 @@ int main(int argc, char **argv) {
   const char * job_id = NULL;
   const char * job_id = NULL;
   const char * task_id = NULL;
   const char * task_id = NULL;
   const char * tt_root = NULL;
   const char * tt_root = NULL;
+  const char *log_dir = NULL;
   int exit_code = 0;
   int exit_code = 0;
   const char * task_pid = NULL;
   const char * task_pid = NULL;
   const char* const short_options = "l:";
   const char* const short_options = "l:";
@@ -35,7 +62,7 @@ int main(int argc, char **argv) {
   //Minimum number of arguments required to run the task-controller
   //Minimum number of arguments required to run the task-controller
   //command-name user command tt-root
   //command-name user command tt-root
   if (argc < 3) {
   if (argc < 3) {
-    display_usage(stderr);
+    display_usage(stdout);
     return INVALID_ARGUMENT_NUMBER;
     return INVALID_ARGUMENT_NUMBER;
   }
   }
 
 
@@ -54,24 +81,9 @@ int main(int argc, char **argv) {
       break;
       break;
     }
     }
   } while (next_option != -1);
   } while (next_option != -1);
-  if (log_file == NULL) {
-    LOGFILE = stderr;
-  } else {
-    LOGFILE = fopen(log_file, "a");
-    if (LOGFILE == NULL) {
-      fprintf(stderr, "Unable to open LOGFILE : %s \n", log_file);
-      LOGFILE = stderr;
-    }
-    if (LOGFILE != stderr) {
-      if (chmod(log_file, S_IREAD | S_IEXEC | S_IWRITE | S_IROTH | S_IWOTH
-          | S_IRGRP | S_IWGRP) < 0) {
-        fprintf(stderr, "Unable to change permission of the log file %s \n",
-            log_file);
-        fprintf(stderr, "changing log file to stderr");
-        LOGFILE = stderr;
-      }
-    }
-  }
+
+  open_log_file(log_file);
+
   //checks done for user name
   //checks done for user name
   //checks done if the user is root or not.
   //checks done if the user is root or not.
   if (argv[optind] == NULL) {
   if (argv[optind] == NULL) {
@@ -88,11 +100,15 @@ int main(int argc, char **argv) {
   }
   }
   optind = optind + 1;
   optind = optind + 1;
   command = atoi(argv[optind++]);
   command = atoi(argv[optind++]);
-#ifdef DEBUG
+
   fprintf(LOGFILE, "main : command provided %d\n",command);
   fprintf(LOGFILE, "main : command provided %d\n",command);
   fprintf(LOGFILE, "main : user is %s\n", user_detail->pw_name);
   fprintf(LOGFILE, "main : user is %s\n", user_detail->pw_name);
-#endif
+
   switch (command) {
   switch (command) {
+  case INITIALIZE_JOB:
+    job_id = argv[optind++];
+    exit_code = initialize_job(job_id, user_detail->pw_name);
+    break;
   case LAUNCH_TASK_JVM:
   case LAUNCH_TASK_JVM:
     tt_root = argv[optind++];
     tt_root = argv[optind++];
     job_id = argv[optind++];
     job_id = argv[optind++];
@@ -100,6 +116,11 @@ int main(int argc, char **argv) {
     exit_code
     exit_code
         = run_task_as_user(user_detail->pw_name, job_id, task_id, tt_root);
         = run_task_as_user(user_detail->pw_name, job_id, task_id, tt_root);
     break;
     break;
+  case INITIALIZE_TASK:
+    job_id = argv[optind++];
+    task_id = argv[optind++];
+    exit_code = initialize_task(job_id, task_id, user_detail->pw_name);
+    break;
   case TERMINATE_TASK_JVM:
   case TERMINATE_TASK_JVM:
     task_pid = argv[optind++];
     task_pid = argv[optind++];
     exit_code = kill_user_task(user_detail->pw_name, task_pid, SIGTERM);
     exit_code = kill_user_task(user_detail->pw_name, task_pid, SIGTERM);

+ 599 - 84
src/c++/task-controller/task-controller.c

@@ -71,103 +71,428 @@ int change_user(const char * user) {
   return 0;
   return 0;
 }
 }
 
 
-// function to check if the passed tt_root is present in hadoop.tmp.dir
-int check_tt_root(const char *tt_root) {
-  char ** mapred_local_dir;
-  char ** iter;
-  int found = -1;
+/**
+ * Checks the passed value for the variable config_key against the values in
+ * the configuration.
+ * Returns 0 if the passed value is found in the configuration,
+ *        -1 otherwise
+ */
+int check_variable_against_config(const char *config_key,
+    const char *passed_value) {
 
 
-  if (tt_root == NULL) {
+  if (config_key == NULL || passed_value == NULL) {
     return -1;
     return -1;
   }
   }
 
 
-  mapred_local_dir = (char **)get_values(TT_SYS_DIR_KEY);
-  iter = mapred_local_dir;
+  int found = -1;
+
+  const char **config_value = get_values(config_key);
 
 
-  if (mapred_local_dir == NULL) {
+  if (config_value == NULL) {
+    fprintf(LOGFILE, "%s is not configured.\n", config_key);
     return -1;
     return -1;
   }
   }
 
 
-  while(*iter != NULL) {
-    if(strcmp(*iter, tt_root) == 0) {
+  char *full_config_value = (char *)get_value(config_key);
+
+  char **config_val_ptr = (char **) config_value;
+  while (*config_val_ptr != NULL) {
+    if (strcmp(*config_val_ptr, passed_value) == 0) {
       found = 0;
       found = 0;
       break;
       break;
     }
     }
-    ++iter; 
+    config_val_ptr++;
   }
   }
-  free(mapred_local_dir);
+
+  if (found != 0) {
+    fprintf(
+        LOGFILE,
+        "Invalid value passed: \
+        Configured value of %s is %s. \
+        Passed value is %s.\n",
+        config_key, full_config_value, passed_value);
+  }
+  free(full_config_value);
+  free(config_value);
   return found;
   return found;
 }
 }
 
 
 /**
 /**
- * Function to check if the constructed path and absolute
- * path resolve to one and same.
+ * Utility function to concatenate argB to argA using the concat_pattern
  */
  */
+char *concatenate(const char *argA, const char *argB, char *concat_pattern,
+    char *return_path_name) {
+  if (argA == NULL || argB == NULL) {
+    fprintf(LOGFILE, "One of the arguments passed for %s in null.\n",
+        return_path_name);
+    return NULL;
+  }
+
+  char *return_path = NULL;
+  int str_len = strlen(concat_pattern) + strlen(argA) + strlen(argB);
+
+  return_path = (char *) malloc(sizeof(char) * (str_len + 1));
+  if (return_path == NULL) {
+    fprintf(LOGFILE, "Unable to allocate memory for %s.\n", return_path_name);
+    return NULL;
+  }
+  memset(return_path, '\0', str_len + 1);
+  snprintf(return_path, str_len, concat_pattern, argA, argB);
+  return return_path;
+}
+
+/**
+ * Get the job-directory path from tt_root and job-id
+ */
+char *get_job_directory(const char * tt_root, const char *jobid) {
+  return concatenate(tt_root, jobid, TT_JOB_DIR_PATTERN, "job_dir_path");
+}
+
+char *get_job_work_directory(const char *job_dir) {
+  return concatenate(job_dir, "", JOB_DIR_TO_JOB_WORK_PATTERN,
+      "job_work_dir_path");
+}
+/**
+ * Get the attempt directory for the given attempt_id
+ */
+char *get_attempt_directory(const char *job_dir, const char *attempt_id) {
+  return concatenate(job_dir, attempt_id, JOB_DIR_TO_ATTEMPT_DIR_PATTERN,
+      "attempt_dir_path");
+}
+
+/*
+ * Get the path to the task launcher file which is created by the TT
+ */
+char *get_task_launcher_file(const char *job_dir, const char *attempt_dir) {
+  return concatenate(job_dir, attempt_dir, TASK_SCRIPT_PATTERN,
+      "task_script_path");
+}
 
 
-int check_path(char *path) {
+/**
+ * Get the log directory for the given attempt.
+ */
+char *get_task_log_dir(const char *log_dir, const char *attempt_id) {
+  return concatenate(log_dir, attempt_id, ATTEMPT_LOG_DIR_PATTERN,
+      "task_log_dir");
+}
+
+/**
+ * Function to check if the passed tt_root is present in mapred.local.dir
+ * the task-controller is configured with.
+ */
+int check_tt_root(const char *tt_root) {
+  return check_variable_against_config(TT_SYS_DIR_KEY, tt_root);
+}
+
+/**
+ * Function to check if the constructed path and absolute path of the task
+ * launcher file resolve to one and same. This is done so as to avoid
+ * security pitfalls because of relative path components in the file name.
+ */
+int check_task_launcher_path(char *path) {
   char * resolved_path = (char *) canonicalize_file_name(path);
   char * resolved_path = (char *) canonicalize_file_name(path);
-  if(resolved_path == NULL) {
+  if (resolved_path == NULL) {
+    fprintf(LOGFILE,
+        "Error resolving the task launcher file path: %s. Passed path: %s\n",
+        strerror(errno), path);
     return ERROR_RESOLVING_FILE_PATH;
     return ERROR_RESOLVING_FILE_PATH;
   }
   }
-  if(strcmp(resolved_path, path) !=0) {
+  if (strcmp(resolved_path, path) != 0) {
+    fprintf(LOGFILE,
+        "Relative path components in the file path: %s. Resolved path: %s\n",
+        path, resolved_path);
     free(resolved_path);
     free(resolved_path);
     return RELATIVE_PATH_COMPONENTS_IN_FILE_PATH;
     return RELATIVE_PATH_COMPONENTS_IN_FILE_PATH;
   }
   }
   free(resolved_path);
   free(resolved_path);
   return 0;
   return 0;
 }
 }
+
 /**
 /**
- * Function to check if a user actually owns the file.
+ * Function to change the owner/group of a given path.
  */
  */
-int check_owner(uid_t uid, char *path) {
-  struct stat filestat;
-  if(stat(path, &filestat)!=0) {
-    return UNABLE_TO_STAT_FILE;
+static int change_owner(const char *path, uid_t uid, gid_t gid) {
+  int exit_code = chown(path, uid, gid);
+  if (exit_code != 0) {
+    fprintf(LOGFILE, "chown %d:%d for path %s failed: %s.\n", uid, gid, path,
+        strerror(errno));
   }
   }
-  //check owner.
-  if(uid != filestat.st_uid){
-    return FILE_NOT_OWNED_BY_TASKTRACKER;
+  return exit_code;
+}
+
+/**
+ * Function to change the mode of a given path.
+ */
+static int change_mode(const char *path, mode_t mode) {
+  int exit_code = chmod(path, mode);
+  if (exit_code != 0) {
+    fprintf(LOGFILE, "chown %d of path %s failed: %s.\n", mode, path,
+        strerror(errno));
   }
   }
-  return 0;
+  return exit_code;
 }
 }
 
 
-/*
- * function to provide path to the task file which is created by the tt
- *
- *Check TT_LOCAL_TASK_SCRIPT_PATTERN for pattern
+/**
+ * Function to secure the given path. It does the following recursively:
+ *    1) changes the owner/group of the paths to the passed owner/group
+ *    2) changes the file permission to the passed file_mode and directory
+ *       permission to the passed dir_mode
  */
  */
-void get_task_file_path(const char * jobid, const char * taskid,
-    const char * tt_root, char **task_script_path) {
-  const char ** mapred_local_dir = get_values(TT_SYS_DIR_KEY);
-  *task_script_path = NULL;
-  int str_len = strlen(TT_LOCAL_TASK_SCRIPT_PATTERN) + strlen(jobid) + (strlen(
-      taskid)) + strlen(tt_root);
+static int secure_path(const char *path, uid_t uid, gid_t gid,
+    mode_t file_mode, mode_t dir_mode) {
+  FTS *tree = NULL; // the file hierarchy
+  FTSENT *entry = NULL; // a file in the hierarchy
+  char *paths[] = { (char *) path };
+  int process_path = 0;
+  int dir = 0;
+  int error_code = 0;
+  int done = 0;
+
+  // Get physical locations and don't resolve the symlinks.
+  // Don't change directory while walking the directory.
+  int ftsoptions = FTS_PHYSICAL | FTS_NOCHDIR;
+
+  tree = fts_open(paths, ftsoptions, NULL);
+  if (tree == NULL) {
+    fprintf(LOGFILE,
+        "Cannot open file traversal structure for the path %s:%s.\n", path,
+        strerror(errno));
+    return -1;
+  }
+
+  while (((entry = fts_read(tree)) != NULL) && !done) {
+    dir = 0;
+    switch (entry->fts_info) {
+    case FTS_D:
+      // A directory being visited in pre-order.
+      // We change ownership of directories in post-order.
+      // so ignore the pre-order visit.
+      process_path = 0;
+      break;
+    case FTS_DC:
+      // A directory that causes a cycle in the tree
+      // We don't expect cycles, ignore.
+      process_path = 0;
+      break;
+    case FTS_DNR:
+      // A directory which cannot be read
+      // Ignore and set error code.
+      process_path = 0;
+      error_code = -1;
+      break;
+    case FTS_DOT:
+      // "."  or ".."
+      process_path = 0;
+      break;
+    case FTS_F:
+      // A regular file
+      process_path = 1;
+      break;
+    case FTS_DP:
+      // A directory being visited in post-order
+      if (entry->fts_level == 0) {
+        // root directory. Done with traversing.
+        done = 1;
+      }
+      process_path = 1;
+      dir = 1;
+      break;
+    case FTS_SL:
+      // A symbolic link
+      process_path = 1;
+      break;
+    case FTS_SLNONE:
+      // A symbolic link with a nonexistent target
+      process_path = 1;
+      break;
+    case FTS_NS:
+      // A  file for which no stat(2) information was available
+      // Ignore and set error code
+      process_path = 0;
+      error_code = -1;
+      break;
+    case FTS_ERR:
+      // An error return. Ignore and set error code.
+      process_path = 0;
+      error_code = -1;
+      break;
+    case FTS_DEFAULT:
+      // File that doesn't belong to any of the above type. Ignore.
+      process_path = 0;
+      break;
+    default:
+      // None of the above. Ignore and set error code
+      process_path = 0;
+      error_code = -1;
+    }
+
+    if (error_code != 0) {
+      break;
+    }
+    if (!process_path) {
+      continue;
+    }
+
+    if (check_ownership(getuid(), getgid(), entry->fts_path) != 0) {
+      fprintf(LOGFILE,
+          "Invalid file path. %s not user/group owned by the tasktracker.\n",
+          entry->fts_path);
+      error_code = -1;
+    } else if (change_owner(entry->fts_path, uid, gid) != 0) {
+      fprintf(LOGFILE, "couldn't change the ownership of %s\n",
+          entry->fts_path);
+      error_code = -3;
+    } else if (change_mode(entry->fts_path, (dir ? dir_mode : file_mode)) != 0) {
+      fprintf(LOGFILE, "couldn't change the permissions of %s\n",
+          entry->fts_path);
+      error_code = -3;
+    }
+  }
+  if (fts_close(tree) != 0) {
+    fprintf(LOGFILE, "couldn't close file traversal structure:%s.\n",
+        strerror(errno));
+  }
+  return error_code;
+}
+
+/**
+ * Function to prepare the attempt directories for the task JVM.
+ * This is done by changing the ownership of the attempt directory recursively
+ * to the job owner. We do the following:
+ *     *  sudo chown user:mapred -R taskTracker/jobcache/$jobid/$attemptid/
+ *     *  sudo chmod 2770 -R taskTracker/jobcache/$jobid/$attemptid/
+ */
+int prepare_attempt_directories(const char *job_id, const char *attempt_id,
+    const char *user) {
+  if (job_id == NULL || attempt_id == NULL || user == NULL) {
+    fprintf(LOGFILE, "Either attempt_id is null or the user passed is null.\n");
+    return INVALID_ARGUMENT_NUMBER;
+  }
 
 
-  if (mapred_local_dir == NULL) {
-    return;
+  if (get_user_details(user) < 0) {
+    fprintf(LOGFILE, "Couldn't get the user details of %s.\n", user);
+    return INVALID_USER_NAME;
   }
   }
 
 
-  *task_script_path = (char *) malloc(sizeof(char) * (str_len + 1));
-  if (*task_script_path == NULL) {
-    fprintf(LOGFILE, "Unable to allocate memory for task_script_path \n");
-    free(mapred_local_dir);
-    return;
+  int tasktracker_gid = getgid();
+
+  char **local_dir = (char **) get_values(TT_SYS_DIR_KEY);
+
+  if (local_dir == NULL) {
+    fprintf(LOGFILE, "%s is not configured.\n", TT_SYS_DIR_KEY);
+    cleanup();
+    return PREPARE_ATTEMPT_DIRECTORIES_FAILED;
   }
   }
 
 
-  memset(*task_script_path,'\0',str_len+1);
-  snprintf(*task_script_path, str_len, TT_LOCAL_TASK_SCRIPT_PATTERN, tt_root,
-      jobid, taskid);
+  char *full_local_dir_str = (char *) get_value(TT_SYS_DIR_KEY);
+#ifdef DEBUG
+  fprintf(LOGFILE, "Value from config for %s is %s.\n", TT_SYS_DIR_KEY,
+      full_local_dir_str);
+#endif
+
+  char *job_dir;
+  char *attempt_dir;
+  char **local_dir_ptr = local_dir;
+  int failed = 0;
+  while (*local_dir_ptr != NULL) {
+    job_dir = get_job_directory(*local_dir_ptr, job_id);
+    if (job_dir == NULL) {
+      fprintf(LOGFILE, "Couldn't get job directory for %s.\n", job_id);
+      failed = 1;
+      break;
+    }
+
+    // prepare attempt-dir in each of the mapred_local_dir
+    attempt_dir = get_attempt_directory(job_dir, attempt_id);
+    if (attempt_dir == NULL) {
+      fprintf(LOGFILE, "Couldn't get attempt directory for %s.\n", attempt_id);
+      failed = 1;
+      free(job_dir);
+      break;
+    }
+
+    struct stat filestat;
+    if (stat(attempt_dir, &filestat) != 0) {
+      if (errno == ENOENT) {
 #ifdef DEBUG
 #ifdef DEBUG
-  fprintf(LOGFILE, "get_task_file_path : task script path = %s\n", *task_script_path);
-  fflush(LOGFILE);
+        fprintf(LOGFILE,
+            "attempt_dir %s doesn't exist. Not doing anything.\n", attempt_dir);
 #endif
 #endif
-  free(mapred_local_dir);
+      } else {
+        // stat failed because of something else!
+        fprintf(LOGFILE, "Failed to stat the attempt_dir %s\n", attempt_dir);
+        failed = 1;
+        free(attempt_dir);
+        free(job_dir);
+        break;
+      }
+    } else if (secure_path(attempt_dir, user_detail->pw_uid, tasktracker_gid,
+        S_IRWXU | S_IRWXG, S_ISGID | S_IRWXU | S_IRWXG) != 0) {
+      // No setgid on files and setgid on dirs, 770
+      fprintf(LOGFILE, "Failed to secure the attempt_dir %s\n", attempt_dir);
+      failed = 1;
+      free(attempt_dir);
+      free(job_dir);
+      break;
+    }
+
+    local_dir_ptr++;
+    free(attempt_dir);
+    free(job_dir);
+  }
+  free(local_dir);
+  free(full_local_dir_str);
+
+  cleanup();
+  if (failed) {
+    return PREPARE_ATTEMPT_DIRECTORIES_FAILED;
+  }
+  return 0;
 }
 }
 
 
-//end of private functions
-void display_usage(FILE *stream) {
-  fprintf(stream,
-      "Usage: task-controller [-l logfile] user command command-args\n");
+/**
+ * Function to prepare the task logs for the child. It gives the user
+ * ownership of the attempt's log-dir to the user and group ownership to the
+ * user running tasktracker.
+ *     *  sudo chown user:mapred log-dir/userlogs/$attemptid
+ *     *  sudo chmod -R 2770 log-dir/userlogs/$attemptid
+ */
+int prepare_task_logs(const char *log_dir, const char *task_id) {
+
+  char *task_log_dir = get_task_log_dir(log_dir, task_id);
+  if (task_log_dir == NULL) {
+    fprintf(LOGFILE, "Couldn't get task_log directory %s.\n", task_log_dir);
+    return -1;
+  }
+
+  struct stat filestat;
+  if (stat(task_log_dir, &filestat) != 0) {
+    if (errno == ENOENT) {
+      // See TaskRunner.java to see that an absent log-dir doesn't fail the task.
+      // Task log dir for cleanup tasks will not have the name
+      // task-attempt-id.cleanup. Instead a log.index.cleanup is created in
+      // task-attempt log dir. We check if the directory exists and return if
+      // it doesn't. So the following will work for cleanup attempts too.
+#ifdef DEBUG
+      fprintf(LOGFILE, "task_log_dir %s doesn't exist. Not doing anything.\n",
+          task_log_dir);
+#endif
+      return 0;
+    } else {
+      // stat failed because of something else!
+      fprintf(LOGFILE, "Failed to stat the task_log_dir %s\n", task_log_dir);
+      return -1;
+    }
+  }
+
+  int tasktracker_gid = getgid();
+  if (secure_path(task_log_dir, user_detail->pw_uid, tasktracker_gid, S_IRWXU
+      | S_IRWXG, S_ISGID | S_IRWXU | S_IRWXG) != 0) {
+    // setgid on dirs but not files, 770. As of now, there are no files though
+    fprintf(LOGFILE, "Failed to secure the log_dir %s\n", task_log_dir);
+    return -1;
+  }
+  return 0;
 }
 }
 
 
 //function used to populate and user_details structure.
 //function used to populate and user_details structure.
@@ -184,30 +509,197 @@ int get_user_details(const char *user) {
 }
 }
 
 
 /*
 /*
- *Function used to launch a task as the provided user.
- * First the function checks if the tt_root passed is found in
- * hadoop.temp.dir
- * Uses get_task_file_path to fetch the task script file path.
- * Does an execlp on the same in order to replace the current image with
- * task image.
+ * Function to check if a user/group actually owns the file.
+  */
+int check_ownership(uid_t uid, gid_t gid, char *path) {
+  struct stat filestat;
+  if (stat(path, &filestat) != 0) {
+    return UNABLE_TO_STAT_FILE;
+  }
+  // check user/group.
+  if (uid != filestat.st_uid || gid != filestat.st_gid) {
+    return FILE_NOT_OWNED_BY_TASKTRACKER;
+  }
+  return 0;
+}
+
+/**
+ * Function to prepare the job directories for the task JVM.
+ * We do the following:
+ *     *  sudo chown user:mapred -R taskTracker/jobcache/$jobid
+ *     *  sudo chmod 2570 -R taskTracker/jobcache/$jobid
+ *     *  sudo chmod 2770 taskTracker/jobcache/$jobid/work
  */
  */
+int initialize_job(const char *jobid, const char *user) {
+  if (jobid == NULL || user == NULL) {
+    fprintf(LOGFILE, "Either jobid is null or the user passed is null.\n");
+    return INVALID_ARGUMENT_NUMBER;
+  }
 
 
+  if (get_user_details(user) < 0) {
+    fprintf(LOGFILE, "Couldn't get the user details of %s", user);
+    return INVALID_USER_NAME;
+  }
+
+  gid_t tasktracker_gid = getgid(); // TaskTracker's group-id
+
+  char **local_dir = (char **) get_values(TT_SYS_DIR_KEY);
+  if (local_dir == NULL) {
+    fprintf(LOGFILE, "%s is not configured.\n", TT_SYS_DIR_KEY);
+    cleanup();
+    return INVALID_TT_ROOT;
+  }
+
+  char *full_local_dir_str = (char *) get_value(TT_SYS_DIR_KEY);
+#ifdef DEBUG
+  fprintf(LOGFILE, "Value from config for %s is %s.\n", TT_SYS_DIR_KEY,
+      full_local_dir_str);
+#endif
+
+  char *job_dir, *job_work_dir;
+  char **local_dir_ptr = local_dir;
+  int failed = 0;
+  while (*local_dir_ptr != NULL) {
+    job_dir = get_job_directory(*local_dir_ptr, jobid);
+    if (job_dir == NULL) {
+      fprintf(LOGFILE, "Couldn't get job directory for %s.\n", jobid);
+      failed = 1;
+      break;
+    }
+
+    struct stat filestat;
+    if (stat(job_dir, &filestat) != 0) {
+      if (errno == ENOENT) {
+#ifdef DEBUG
+        fprintf(LOGFILE, "job_dir %s doesn't exist. Not doing anything.\n",
+            job_dir);
+#endif
+      } else {
+        // stat failed because of something else!
+        fprintf(LOGFILE, "Failed to stat the job_dir %s\n", job_dir);
+        failed = 1;
+        free(job_dir);
+        break;
+      }
+    } else if (secure_path(job_dir, user_detail->pw_uid, tasktracker_gid,
+        S_IRUSR | S_IXUSR | S_IRWXG, S_ISGID | S_IRUSR | S_IXUSR | S_IRWXG)
+        != 0) {
+      // No setgid on files and setgid on dirs, 570
+      fprintf(LOGFILE, "Failed to secure the job_dir %s\n", job_dir);
+      failed = 1;
+      free(job_dir);
+      break;
+    } else {
+      job_work_dir = get_job_work_directory(job_dir);
+      if (job_work_dir == NULL) {
+        fprintf(LOGFILE, "Couldn't get job-work directory for %s.\n", jobid);
+        failed = 1;
+        break;
+      }
+
+      // Set 2770 on the job-work directory
+      if (stat(job_work_dir, &filestat) != 0) {
+        if (errno == ENOENT) {
+#ifdef DEBUG
+          fprintf(LOGFILE,
+              "job_work_dir %s doesn't exist. Not doing anything.\n",
+              job_work_dir);
+#endif
+        } else {
+          // stat failed because of something else!
+          fprintf(LOGFILE, "Failed to stat the job_work_dir %s\n",
+              job_work_dir);
+          failed = 1;
+          free(job_work_dir);
+          free(job_dir);
+          break;
+        }
+      } else if (change_mode(job_work_dir, S_ISGID | S_IRWXU | S_IRWXG) != 0) {
+        fprintf(LOGFILE,
+            "couldn't change the permissions of job_work_dir %s\n",
+            job_work_dir);
+        failed = 1;
+        free(job_work_dir);
+        free(job_dir);
+        break;
+      }
+    }
+
+    local_dir_ptr++;
+    free(job_dir);
+  }
+  free(local_dir);
+  free(full_local_dir_str);
+  cleanup();
+  if (failed) {
+    return INITIALIZE_JOB_FAILED;
+  }
+  return 0;
+}
+
+/**
+ * Function used to initialize task. Prepares attempt_dir, jars_dir and
+ * log_dir to be accessible by the child
+ */
+int initialize_task(const char *jobid, const char *taskid, const char *user) {
+  int exit_code = 0;
+#ifdef DEBUG
+  fprintf(LOGFILE, "job-id passed to initialize_task : %s.\n", jobid);
+  fprintf(LOGFILE, "task-d passed to initialize_task : %s.\n", taskid);
+#endif
+
+  if (prepare_attempt_directories(jobid, taskid, user) != 0) {
+    fprintf(LOGFILE,
+        "Couldn't prepare the attempt directories for %s of user %s.\n",
+        taskid, user);
+    exit_code = PREPARE_ATTEMPT_DIRECTORIES_FAILED;
+    goto cleanup;
+  }
+
+  char *log_dir = (char *) get_value(TT_LOG_DIR_KEY);
+  if (log_dir == NULL) {
+    fprintf(LOGFILE, "Log directory is not configured.\n");
+    exit_code = INVALID_TT_LOG_DIR;
+    goto cleanup;
+  }
+
+  if (prepare_task_logs(log_dir, taskid) != 0) {
+    fprintf(LOGFILE, "Couldn't prepare task logs directory %s for %s.\n",
+        log_dir, taskid);
+    exit_code = PREPARE_TASK_LOGS_FAILED;
+  }
+
+  cleanup:
+  // free configurations
+  cleanup();
+  if (log_dir != NULL) {
+    free(log_dir);
+  }
+  return exit_code;
+}
+
+/*
+ * Function used to launch a task as the provided user. It does the following :
+ * 1) Checks if the tt_root passed is found in mapred.local.dir
+ * 2) Prepares attempt_dir and log_dir to be accessible by the child
+ * 3) Uses get_task_launcher_file to fetch the task script file path
+ * 4) Does an execlp on the same in order to replace the current image with
+ * task image.
+ */
 int run_task_as_user(const char * user, const char *jobid, const char *taskid,
 int run_task_as_user(const char * user, const char *jobid, const char *taskid,
     const char *tt_root) {
     const char *tt_root) {
-  char *task_script_path = NULL;
   int exit_code = 0;
   int exit_code = 0;
-  uid_t uid = getuid();
 
 
-  if(jobid == NULL || taskid == NULL) {
+  if (jobid == NULL || taskid == NULL || tt_root == NULL) {
     return INVALID_ARGUMENT_NUMBER;
     return INVALID_ARGUMENT_NUMBER;
   }
   }
 
 
 #ifdef DEBUG
 #ifdef DEBUG
-  fprintf(LOGFILE,"run_task_as_user : Job id : %s \n", jobid);
-  fprintf(LOGFILE,"run_task_as_user : task id : %s \n", taskid);
-  fprintf(LOGFILE,"run_task_as_user : tt_root : %s \n", tt_root);
-  fflush(LOGFILE);
+  fprintf(LOGFILE, "Job-id passed to run_task_as_user : %s.\n", jobid);
+  fprintf(LOGFILE, "task-d passed to run_task_as_user : %s.\n", taskid);
+  fprintf(LOGFILE, "tt_root passed to run_task_as_user : %s.\n", tt_root);
 #endif
 #endif
+
   //Check tt_root before switching the user, as reading configuration
   //Check tt_root before switching the user, as reading configuration
   //file requires privileged access.
   //file requires privileged access.
   if (check_tt_root(tt_root) < 0) {
   if (check_tt_root(tt_root) < 0) {
@@ -216,34 +708,48 @@ int run_task_as_user(const char * user, const char *jobid, const char *taskid,
     return INVALID_TT_ROOT;
     return INVALID_TT_ROOT;
   }
   }
 
 
-  //change the user
-  fclose(LOGFILE);
-  fcloseall();
-  umask(0);
-  if (change_user(user) != 0) {
-    cleanup();
-    return SETUID_OPER_FAILED;
+  char *job_dir = NULL, *task_script_path = NULL;
+
+  if ((exit_code = initialize_task(jobid, taskid, user)) != 0) {
+    fprintf(LOGFILE, "Couldn't initialise the task %s of user %s.\n", taskid,
+        user);
+    goto cleanup;
+  }
+
+  job_dir = get_job_directory(tt_root, jobid);
+  if (job_dir == NULL) {
+    fprintf(LOGFILE, "Couldn't obtain job_dir for %s in %s.\n", jobid, tt_root);
+    exit_code = OUT_OF_MEMORY;
+    goto cleanup;
   }
   }
 
 
-  get_task_file_path(jobid, taskid, tt_root, &task_script_path);
+  task_script_path = get_task_launcher_file(job_dir, taskid);
   if (task_script_path == NULL) {
   if (task_script_path == NULL) {
-    cleanup();
-    return INVALID_TASK_SCRIPT_PATH;
+    fprintf(LOGFILE, "Couldn't obtain task_script_path in %s.\n", job_dir);
+    exit_code = OUT_OF_MEMORY;
+    goto cleanup;
   }
   }
+
   errno = 0;
   errno = 0;
-  exit_code = check_path(task_script_path);
+  exit_code = check_task_launcher_path(task_script_path);
   if(exit_code != 0) {
   if(exit_code != 0) {
     goto cleanup;
     goto cleanup;
   }
   }
-  errno = 0;
-  exit_code = check_owner(uid, task_script_path);
-  if(exit_code != 0) {
+
+  //change the user
+  fcloseall();
+  free(job_dir);
+  umask(0007);
+  if (change_user(user) != 0) {
+    exit_code = SETUID_OPER_FAILED;
     goto cleanup;
     goto cleanup;
   }
   }
+
   errno = 0;
   errno = 0;
   cleanup();
   cleanup();
   execlp(task_script_path, task_script_path, NULL);
   execlp(task_script_path, task_script_path, NULL);
   if (errno != 0) {
   if (errno != 0) {
+    fprintf(LOGFILE, "Couldn't execute the task jvm file: %s", strerror(errno));
     free(task_script_path);
     free(task_script_path);
     exit_code = UNABLE_TO_EXECUTE_TASK_SCRIPT;
     exit_code = UNABLE_TO_EXECUTE_TASK_SCRIPT;
   }
   }
@@ -251,6 +757,9 @@ int run_task_as_user(const char * user, const char *jobid, const char *taskid,
   return exit_code;
   return exit_code;
 
 
 cleanup:
 cleanup:
+  if (job_dir != NULL) {
+    free(job_dir);
+  }
   if (task_script_path != NULL) {
   if (task_script_path != NULL) {
     free(task_script_path);
     free(task_script_path);
   }
   }
@@ -264,19 +773,25 @@ cleanup:
  * The function sends appropriate signal to the process group
  * The function sends appropriate signal to the process group
  * specified by the task_pid.
  * specified by the task_pid.
  */
  */
-
 int kill_user_task(const char *user, const char *task_pid, int sig) {
 int kill_user_task(const char *user, const char *task_pid, int sig) {
   int pid = 0;
   int pid = 0;
 
 
   if(task_pid == NULL) {
   if(task_pid == NULL) {
     return INVALID_ARGUMENT_NUMBER;
     return INVALID_ARGUMENT_NUMBER;
   }
   }
+
+#ifdef DEBUG
+  fprintf(LOGFILE, "user passed to kill_user_task : %s.\n", user);
+  fprintf(LOGFILE, "task-pid passed to kill_user_task : %s.\n", task_pid);
+  fprintf(LOGFILE, "signal passed to kill_user_task : %d.\n", sig);
+#endif
+
   pid = atoi(task_pid);
   pid = atoi(task_pid);
 
 
   if(pid <= 0) {
   if(pid <= 0) {
     return INVALID_TASK_PID;
     return INVALID_TASK_PID;
   }
   }
-  fclose(LOGFILE);
+
   fcloseall();
   fcloseall();
   if (change_user(user) != 0) {
   if (change_user(user) != 0) {
     cleanup();
     cleanup();
@@ -286,6 +801,7 @@ int kill_user_task(const char *user, const char *task_pid, int sig) {
   //Don't continue if the process-group is not alive anymore.
   //Don't continue if the process-group is not alive anymore.
   if(kill(-pid,0) < 0) {
   if(kill(-pid,0) < 0) {
     errno = 0;
     errno = 0;
+    cleanup();
     return 0;
     return 0;
   }
   }
 
 
@@ -300,4 +816,3 @@ int kill_user_task(const char *user, const char *task_pid, int sig) {
   cleanup();
   cleanup();
   return 0;
   return 0;
 }
 }
-

+ 43 - 15
src/c++/task-controller/task-controller.h

@@ -28,14 +28,20 @@
 #include <sys/stat.h>
 #include <sys/stat.h>
 #include <sys/signal.h>
 #include <sys/signal.h>
 #include <getopt.h>
 #include <getopt.h>
-#include<grp.h>
+#include <grp.h>
+#include <dirent.h>
+#include <fcntl.h>
+#include <fts.h>
+
 #include "configuration.h"
 #include "configuration.h"
 
 
 //command definitions
 //command definitions
 enum command {
 enum command {
+  INITIALIZE_JOB,
   LAUNCH_TASK_JVM,
   LAUNCH_TASK_JVM,
+  INITIALIZE_TASK,
   TERMINATE_TASK_JVM,
   TERMINATE_TASK_JVM,
-  KILL_TASK_JVM
+  KILL_TASK_JVM,
 };
 };
 
 
 enum errorcodes {
 enum errorcodes {
@@ -45,23 +51,33 @@ enum errorcodes {
   SUPER_USER_NOT_ALLOWED_TO_RUN_TASKS, //4
   SUPER_USER_NOT_ALLOWED_TO_RUN_TASKS, //4
   INVALID_TT_ROOT, //5
   INVALID_TT_ROOT, //5
   SETUID_OPER_FAILED, //6
   SETUID_OPER_FAILED, //6
-  INVALID_TASK_SCRIPT_PATH, //7
-  UNABLE_TO_EXECUTE_TASK_SCRIPT, //8
-  UNABLE_TO_KILL_TASK, //9
-  INVALID_PROCESS_LAUNCHING_TASKCONTROLLER, //10
-  INVALID_TASK_PID, //11
-  ERROR_RESOLVING_FILE_PATH, //12
-  RELATIVE_PATH_COMPONENTS_IN_FILE_PATH, //13
-  UNABLE_TO_STAT_FILE, //14
-  FILE_NOT_OWNED_BY_TASKTRACKER //15
+  UNABLE_TO_EXECUTE_TASK_SCRIPT, //7
+  UNABLE_TO_KILL_TASK, //8
+  INVALID_TASK_PID, //9
+  ERROR_RESOLVING_FILE_PATH, //10
+  RELATIVE_PATH_COMPONENTS_IN_FILE_PATH, //11
+  UNABLE_TO_STAT_FILE, //12
+  FILE_NOT_OWNED_BY_TASKTRACKER, //13
+  PREPARE_ATTEMPT_DIRECTORIES_FAILED, //14
+  INITIALIZE_JOB_FAILED, //15
+  PREPARE_TASK_LOGS_FAILED, //16
+  INVALID_TT_LOG_DIR, //17
+  OUT_OF_MEMORY, //18
 };
 };
 
 
+#define TT_JOB_DIR_PATTERN "%s/taskTracker/jobcache/%s"
+
+#define JOB_DIR_TO_JOB_WORK_PATTERN "%s/work"
+
+#define JOB_DIR_TO_ATTEMPT_DIR_PATTERN "%s/%s"
 
 
-#define TT_LOCAL_TASK_SCRIPT_PATTERN "%s/taskTracker/jobcache/%s/%s/taskjvm.sh"
+#define ATTEMPT_LOG_DIR_PATTERN "%s/userlogs/%s"
+
+#define TASK_SCRIPT_PATTERN "%s/%s/taskjvm.sh"
 
 
 #define TT_SYS_DIR_KEY "mapred.local.dir"
 #define TT_SYS_DIR_KEY "mapred.local.dir"
 
 
-#define MAX_ITEMS 10
+#define TT_LOG_DIR_KEY "hadoop.log.dir"
 
 
 #ifndef HADOOP_CONF_DIR
 #ifndef HADOOP_CONF_DIR
   #define EXEC_PATTERN "/bin/task-controller"
   #define EXEC_PATTERN "/bin/task-controller"
@@ -72,10 +88,22 @@ extern struct passwd *user_detail;
 
 
 extern FILE *LOGFILE;
 extern FILE *LOGFILE;
 
 
-void display_usage(FILE *stream);
+int run_task_as_user(const char * user, const char *jobid, const char *taskid,
+    const char *tt_root);
+
+int initialize_task(const char *jobid, const char *taskid, const char *user);
 
 
-int run_task_as_user(const char * user, const char *jobid, const char *taskid, const char *tt_root);
+int initialize_job(const char *jobid, const char *user);
 
 
 int kill_user_task(const char *user, const char *task_pid, int sig);
 int kill_user_task(const char *user, const char *task_pid, int sig);
 
 
+int prepare_attempt_directory(const char *attempt_dir, const char *user);
+
+// The following functions are exposed for testing
+
+int check_variable_against_config(const char *config_key,
+    const char *passed_value);
+
 int get_user_details(const char *user);
 int get_user_details(const char *user);
+
+char *get_task_launcher_file(const char *job_dir, const char *attempt_dir);

+ 180 - 0
src/c++/task-controller/tests/test-task-controller.c

@@ -0,0 +1,180 @@
+/**
+ * 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.
+ */
+#include "../task-controller.h"
+
+#define HADOOP_CONF_DIR "/tmp"
+
+int write_config_file(char *file_name) {
+  FILE *file;
+  char const *str =
+      "mapred.local.dir=/tmp/testing1,/tmp/testing2,/tmp/testing3,/tmp/testing4\n";
+
+  file = fopen(file_name, "w");
+  if (file == NULL) {
+    printf("Failed to open %s.\n", file_name);
+    return EXIT_FAILURE;
+  }
+  fwrite(str, 1, strlen(str), file);
+  fclose(file);
+  return 0;
+}
+
+void test_check_variable_against_config() {
+
+  // A temporary configuration directory
+  char *conf_dir_templ = "/tmp/test-task-controller-conf-dir-XXXXXX";
+
+  // To accomodate "/conf/taskcontroller.cfg"
+  char template[strlen(conf_dir_templ) + strlen("/conf/taskcontroller.cfg")];
+
+  strcpy(template, conf_dir_templ);
+  char *temp_dir = mkdtemp(template);
+  if (temp_dir == NULL) {
+    printf("Couldn't create a temporary dir for conf.\n");
+    goto cleanup;
+  }
+
+  // Set the configuration directory
+  hadoop_conf_dir = strdup(temp_dir);
+
+  // create the configuration directory
+  strcat(template, "/conf");
+  char *conf_dir = strdup(template);
+  mkdir(conf_dir, S_IRWXU);
+
+  // create the configuration file
+  strcat(template, "/taskcontroller.cfg");
+  if (write_config_file(template) != 0) {
+    printf("Couldn't write the configuration file.\n");
+    goto cleanup;
+  }
+
+  // Test obtaining a value for a key from the config
+  char *config_values[4] = { "/tmp/testing1", "/tmp/testing2",
+      "/tmp/testing3", "/tmp/testing4" };
+  char *value = (char *) get_value("mapred.local.dir");
+  if (strcmp(value, "/tmp/testing1,/tmp/testing2,/tmp/testing3,/tmp/testing4")
+      != 0) {
+    printf("Obtaining a value for a key from the config failed.\n");
+    goto cleanup;
+  }
+
+  // Test the parsing of a multiple valued key from the config
+  char **values = (char **)get_values("mapred.local.dir");
+  char **values_ptr = values;
+  int i = 0;
+  while (*values_ptr != NULL) {
+    printf(" value : %s\n", *values_ptr);
+    if (strcmp(*values_ptr, config_values[i++]) != 0) {
+      printf("Configured values are not read out properly. Test failed!");
+      goto cleanup;;
+    }
+    values_ptr++;
+  }
+
+  if (check_variable_against_config("mapred.local.dir", "/tmp/testing5") == 0) {
+    printf("Configuration should not contain /tmp/testing5! \n");
+    goto cleanup;
+  }
+
+  if (check_variable_against_config("mapred.local.dir", "/tmp/testing4") != 0) {
+    printf("Configuration should contain /tmp/testing4! \n");
+    goto cleanup;
+  }
+
+  cleanup: if (value != NULL) {
+    free(value);
+  }
+  if (values != NULL) {
+    free(values);
+  }
+  if (hadoop_conf_dir != NULL) {
+    free(hadoop_conf_dir);
+  }
+  unlink(template);
+  rmdir(conf_dir);
+  rmdir(hadoop_conf_dir);
+}
+
+void test_get_job_directory() {
+  char *job_dir = (char *) get_job_directory("/tmp", "job_200906101234_0001");
+  printf("job_dir obtained is %s\n", job_dir);
+  int ret = 0;
+  if (strcmp(job_dir, "/tmp/taskTracker/jobcache/job_200906101234_0001") != 0) {
+    ret = -1;
+  }
+  free(job_dir);
+  assert(ret == 0);
+}
+
+void test_get_attempt_directory() {
+  char *attempt_dir = (char *) get_attempt_directory(
+      "/tmp/taskTracker/jobcache/job_200906101234_0001",
+      "attempt_200906112028_0001_m_000000_0");
+  printf("attempt_dir obtained is %s\n", attempt_dir);
+  int ret = 0;
+  if (strcmp(
+      attempt_dir,
+      "/tmp/taskTracker/jobcache/job_200906101234_0001/attempt_200906112028_0001_m_000000_0")
+      != 0) {
+    ret = -1;
+  }
+  free(attempt_dir);
+  assert(ret == 0);
+}
+
+void test_get_task_launcher_file() {
+  char *task_file = (char *) get_task_launcher_file(
+      "/tmp/taskTracker/jobcache/job_200906101234_0001",
+      "attempt_200906112028_0001_m_000000_0");
+  printf("task_file obtained is %s\n", task_file);
+  int ret = 0;
+  if (strcmp(
+      task_file,
+      "/tmp/taskTracker/jobcache/job_200906101234_0001/attempt_200906112028_0001_m_000000_0/taskjvm.sh")
+      != 0) {
+    ret = -1;
+  }
+  free(task_file);
+  assert(ret == 0);
+}
+
+void test_get_task_log_dir() {
+  char *logdir = (char *) get_task_log_dir("/tmp/testing",
+      "attempt_200906112028_0001_m_000000_0");
+  printf("logdir obtained is %s\n", logdir);
+  int ret = 0;
+  if (strcmp(logdir,
+      "/tmp/testing/userlogs/attempt_200906112028_0001_m_000000_0") != 0) {
+    ret = -1;
+  }
+  free(logdir);
+  assert(ret == 0);
+}
+
+int main(int argc, char **argv) {
+  printf("Starting tests\n");
+  LOGFILE = stdout;
+  test_check_variable_against_config();
+  test_get_job_directory();
+  test_get_attempt_directory();
+  test_get_task_launcher_file();
+  test_get_task_log_dir();
+  printf("Finished tests\n");
+  return 0;
+}

+ 1 - 1
src/contrib/build-contrib.xml

@@ -234,7 +234,7 @@
       <sysproperty key="hadoop.test.localoutputfile" value="${hadoop.test.localoutputfile}"/>
       <sysproperty key="hadoop.test.localoutputfile" value="${hadoop.test.localoutputfile}"/>
       <sysproperty key="hadoop.log.dir" value="${hadoop.log.dir}"/> 
       <sysproperty key="hadoop.log.dir" value="${hadoop.log.dir}"/> 
       <sysproperty key="taskcontroller-path" value="${taskcontroller-path}"/>
       <sysproperty key="taskcontroller-path" value="${taskcontroller-path}"/>
-      <sysproperty key="taskcontroller-user" value="${taskcontroller-user}"/>
+      <sysproperty key="taskcontroller-ugi" value="${taskcontroller-ugi}"/>
       <classpath refid="test.classpath"/>
       <classpath refid="test.classpath"/>
       <formatter type="${test.junit.output.format}" />
       <formatter type="${test.junit.output.format}" />
       <batchtest todir="${build.test}" unless="testcase">
       <batchtest todir="${build.test}" unless="testcase">

+ 1 - 1
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingAsDifferentUser.java

@@ -65,7 +65,7 @@ public class TestStreamingAsDifferentUser extends
             "stream.tmpdir=" + System.getProperty("test.build.data", "/tmp") };
             "stream.tmpdir=" + System.getProperty("test.build.data", "/tmp") };
     StreamJob streamJob = new StreamJob(args, true);
     StreamJob streamJob = new StreamJob(args, true);
     streamJob.setConf(myConf);
     streamJob.setConf(myConf);
-    streamJob.go();
+    assertTrue("Job has not succeeded", streamJob.go() == 0);
     assertOwnerShip(outputPath);
     assertOwnerShip(outputPath);
   }
   }
 }
 }

+ 8 - 3
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestUlimit.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.hadoop.mapred.TestMiniMRWithDFS;
 import org.apache.hadoop.mapred.TestMiniMRWithDFS;
+import org.apache.hadoop.util.StringUtils;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
@@ -89,10 +90,14 @@ public class TestUlimit extends TestCase {
       assertFalse("output not cleaned up", fs.exists(outputPath));
       assertFalse("output not cleaned up", fs.exists(outputPath));
       mr.waitUntilIdle();
       mr.waitUntilIdle();
     } catch(IOException e) {
     } catch(IOException e) {
-      fail(e.toString());
+      fail(StringUtils.stringifyException(e));
     } finally {
     } finally {
-      mr.shutdown();
-      dfs.shutdown();
+      if (mr != null) {
+        mr.shutdown();
+      }
+      if (dfs != null) {
+        dfs.shutdown();
+      }
     }
     }
   }
   }
 
 

+ 37 - 14
src/docs/src/documentation/content/xdocs/cluster_setup.xml

@@ -509,11 +509,13 @@
             distribution. The task tracker uses this executable to 
             distribution. The task tracker uses this executable to 
             launch and kill tasks. The setuid executable switches to
             launch and kill tasks. The setuid executable switches to
             the user who has submitted the job and launches or kills
             the user who has submitted the job and launches or kills
-            the tasks. Currently, this task controller 
-            opens up permissions to local files and directories used 
-            by the tasks such as the job jar files, distributed archive 
-            files, intermediate files and task log files. In future,
-            it is expected that stricter file permissions are used.
+            the tasks. For maximum security, this task controller 
+            sets up restricted permissions and user/group ownership of
+            local files and directories used by the tasks such as the
+            job jar files, intermediate files and task log files. Currently
+            permissions on distributed cache files are opened up to be
+            accessible by all users. In future, it is expected that stricter
+            file permissions are set for these files too.
             </td>
             </td>
             </tr>
             </tr>
             </table>
             </table>
@@ -555,18 +557,32 @@
             </p>
             </p>
             
             
             <p>
             <p>
-            The executable must be deployed as a setuid executable, by changing
-            the ownership to <em>root</em>, group ownership to that of tasktracker
-            and giving it permissions <em>4510</em>.Please take a note that,
-            group which owns task-controller should contain only tasktracker
-            as its memeber and not users who submit jobs.
+            The executable must have specific permissions as follows. The
+            executable should have <em>6050 or --Sr-s---</em> permissions
+            user-owned by root(super-user) and group-owned by a group 
+            of which only the TaskTracker's user is the sole group member. 
+            For example, let's say that the TaskTracker is run as user
+            <em>mapred</em> who is part of the groups <em>users</em> and
+            <em>mapredGroup</em> any of them being the primary group.
+            Let also be that <em>users</em> has both <em>mapred</em> and
+            another user <em>X</em> as its members, while <em>mapredGroup</em>
+            has only <em>mapred</em> as its member. Going by the above
+            description, the setuid/setgid executable should be set
+            <em>6050 or --Sr-s---</em> with user-owner as <em>mapred</em> and
+            group-owner as <em>mapredGroup</em> which has
+            only <em>mapred</em> as its member(and not <em>users</em> which has
+            <em>X</em> also as its member besides <em>mapred</em>).
             </p>
             </p>
             
             
             <p>The executable requires a configuration file called 
             <p>The executable requires a configuration file called 
             <em>taskcontroller.cfg</em> to be
             <em>taskcontroller.cfg</em> to be
             present in the configuration directory passed to the ant target 
             present in the configuration directory passed to the ant target 
             mentioned above. If the binary was not built with a specific 
             mentioned above. If the binary was not built with a specific 
-            conf directory, the path defaults to <em>/path-to-binary/../conf</em>.
+            conf directory, the path defaults to
+            <em>/path-to-binary/../conf</em>. The configuration file must be
+            owned by the user running TaskTracker (user <em>mapred</em> in the
+            above example), group-owned by anyone and should have the
+            permissions <em>0400 or r--------</em>.
             </p>
             </p>
             
             
             <p>The executable requires following configuration items to be 
             <p>The executable requires following configuration items to be 
@@ -581,13 +597,20 @@
             validate paths passed to the setuid executable in order to prevent
             validate paths passed to the setuid executable in order to prevent
             arbitrary paths being passed to it.</td>
             arbitrary paths being passed to it.</td>
             </tr>
             </tr>
+            <tr>
+            <td>hadoop.log.dir</td>
+            <td>Path to hadoop log directory. Should be same as the value which
+            the TaskTracker is started with. This is required to set proper
+            permissions on the log files so that they can be written to by the user's
+            tasks and read by the TaskTracker for serving on the web UI.</td>
+            </tr>
             </table>
             </table>
 
 
             <p>
             <p>
-            The LinuxTaskController requires that paths leading up to
+            The LinuxTaskController requires that paths including and leading up to
             the directories specified in
             the directories specified in
-            <em>mapred.local.dir</em> and <em>hadoop.log.dir</em> to be 755
-            and directories themselves having 777 permissions.
+            <em>mapred.local.dir</em> and <em>hadoop.log.dir</em> to be set 755
+            permissions.
             </p>
             </p>
             </section>
             </section>
             
             

+ 5 - 3
src/mapred/org/apache/hadoop/mapred/Child.java

@@ -154,6 +154,10 @@ class Child {
         // set the jobTokenFile into task
         // set the jobTokenFile into task
         task.setJobTokenSecret(JobTokenSecretManager.createSecretKey(jt.getPassword()));
         task.setJobTokenSecret(JobTokenSecretManager.createSecretKey(jt.getPassword()));
 
 
+        // setup the child's mapred-local-dir. The child is now sandboxed and
+        // can only see files down and under attemtdir only.
+        TaskRunner.setupChildMapredLocalDirs(task, job);
+
         //setupWorkDir actually sets up the symlinks for the distributed
         //setupWorkDir actually sets up the symlinks for the distributed
         //cache. After a task exits we wipe the workdir clean, and hence
         //cache. After a task exits we wipe the workdir clean, and hence
         //the symlinks have to be rebuilt.
         //the symlinks have to be rebuilt.
@@ -165,8 +169,6 @@ class Child {
 
 
         task.setConf(job);
         task.setConf(job);
 
 
-        defaultConf.addResource(new Path(task.getJobFile()));
-
         // Initiate Java VM metrics
         // Initiate Java VM metrics
         JvmMetrics.init(task.getPhase().toString(), job.getSessionId());
         JvmMetrics.init(task.getPhase().toString(), job.getSessionId());
         // use job-specified working directory
         // use job-specified working directory
@@ -191,7 +193,7 @@ class Child {
           task.taskCleanup(umbilical);
           task.taskCleanup(umbilical);
         }
         }
       } catch (Exception e) {
       } catch (Exception e) {
-        LOG.info("Error cleaning up" + e);
+        LOG.info("Error cleaning up", e);
       }
       }
       // Report back any failures, for diagnostic purposes
       // Report back any failures, for diagnostic purposes
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
       ByteArrayOutputStream baos = new ByteArrayOutputStream();

+ 3 - 8
src/mapred/org/apache/hadoop/mapred/DefaultTaskController.java

@@ -49,6 +49,8 @@ class DefaultTaskController extends TaskController {
    */
    */
   void launchTaskJVM(TaskController.TaskControllerContext context) 
   void launchTaskJVM(TaskController.TaskControllerContext context) 
                                       throws IOException {
                                       throws IOException {
+    initializeTask(context);
+
     JvmEnv env = context.env;
     JvmEnv env = context.env;
     List<String> wrappedCommand = 
     List<String> wrappedCommand = 
       TaskLog.captureOutAndError(env.setup, env.vargs, env.stdout, env.stderr,
       TaskLog.captureOutAndError(env.setup, env.vargs, env.stdout, env.stderr,
@@ -73,20 +75,13 @@ class DefaultTaskController extends TaskController {
     // So this is a dummy method.
     // So this is a dummy method.
     return;
     return;
   }
   }
-  
-
-  @Override
-  void setup() {
-    // nothing to setup
-    return;
-  }
 
 
   /*
   /*
    * No need to do anything as we don't need to do as we dont need anything
    * No need to do anything as we don't need to do as we dont need anything
    * extra from what TaskTracker has done.
    * extra from what TaskTracker has done.
    */
    */
   @Override
   @Override
-  void initializeJob(JobID jobId) {
+  void initializeJob(JobInitializationContext context) {
   }
   }
 
 
   @Override
   @Override

+ 9 - 2
src/mapred/org/apache/hadoop/mapred/IsolationRunner.java

@@ -167,17 +167,24 @@ public class IsolationRunner {
     // setup the local and user working directories
     // setup the local and user working directories
     FileSystem local = FileSystem.getLocal(conf);
     FileSystem local = FileSystem.getLocal(conf);
     LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
     LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
+
     File workDirName = TaskRunner.formWorkDir(lDirAlloc, taskId, false, conf);
     File workDirName = TaskRunner.formWorkDir(lDirAlloc, taskId, false, conf);
     local.setWorkingDirectory(new Path(workDirName.toString()));
     local.setWorkingDirectory(new Path(workDirName.toString()));
     FileSystem.get(conf).setWorkingDirectory(conf.getWorkingDirectory());
     FileSystem.get(conf).setWorkingDirectory(conf.getWorkingDirectory());
     
     
     // set up a classloader with the right classpath
     // set up a classloader with the right classpath
-    ClassLoader classLoader = makeClassLoader(conf, workDirName);
+    ClassLoader classLoader =
+        makeClassLoader(conf, new File(workDirName.toString()));
     Thread.currentThread().setContextClassLoader(classLoader);
     Thread.currentThread().setContextClassLoader(classLoader);
     conf.setClassLoader(classLoader);
     conf.setClassLoader(classLoader);
     
     
+    // split.dta file is used only by IsolationRunner. The file can now be in
+    // any of the configured local disks, so use LocalDirAllocator to find out
+    // where it is.
     Path localMetaSplit = 
     Path localMetaSplit = 
-      new Path(new Path(jobFilename.toString()).getParent(), "split.info");
+        new LocalDirAllocator("mapred.local.dir").getLocalPathToRead(
+            TaskTracker.getLocalSplitFile(taskId.getJobID().toString(), taskId
+                .toString()), conf);
     DataInputStream splitFile = FileSystem.getLocal(conf).open(localMetaSplit);
     DataInputStream splitFile = FileSystem.getLocal(conf).open(localMetaSplit);
     TaskSplitIndex splitIndex = new TaskSplitIndex();
     TaskSplitIndex splitIndex = new TaskSplitIndex();
     splitIndex.readFields(splitFile);
     splitIndex.readFields(splitFile);

+ 1 - 1
src/mapred/org/apache/hadoop/mapred/JobConf.java

@@ -1425,7 +1425,7 @@ public class JobConf extends Configuration {
    * @return The localized job specific shared directory
    * @return The localized job specific shared directory
    */
    */
   public String getJobLocalDir() {
   public String getJobLocalDir() {
-    return get("job.local.dir");
+    return get(TaskTracker.JOB_LOCAL_DIR);
   }
   }
 
 
   /**
   /**

+ 26 - 14
src/mapred/org/apache/hadoop/mapred/JvmManager.java

@@ -112,7 +112,8 @@ class JvmManager {
     }
     }
   }
   }
 
 
-  public TaskInProgress getTaskForJvm(JVMId jvmId) {
+  public TaskInProgress getTaskForJvm(JVMId jvmId)
+      throws IOException {
     if (jvmId.isMapJVM()) {
     if (jvmId.isMapJVM()) {
       return mapJvmManager.getTaskForJvm(jvmId);
       return mapJvmManager.getTaskForJvm(jvmId);
     } else {
     } else {
@@ -177,7 +178,8 @@ class JvmManager {
       jvmIdToRunner.get(jvmId).setBusy(true);
       jvmIdToRunner.get(jvmId).setBusy(true);
     }
     }
     
     
-    synchronized public TaskInProgress getTaskForJvm(JVMId jvmId) {
+    synchronized public TaskInProgress getTaskForJvm(JVMId jvmId)
+        throws IOException {
       if (jvmToRunningTask.containsKey(jvmId)) {
       if (jvmToRunningTask.containsKey(jvmId)) {
         //Incase of JVM reuse, tasks are returned to previously launched
         //Incase of JVM reuse, tasks are returned to previously launched
         //JVM via this method. However when a new task is launched
         //JVM via this method. However when a new task is launched
@@ -185,15 +187,24 @@ class JvmManager {
         TaskRunner taskRunner = jvmToRunningTask.get(jvmId);
         TaskRunner taskRunner = jvmToRunningTask.get(jvmId);
         JvmRunner jvmRunner = jvmIdToRunner.get(jvmId);
         JvmRunner jvmRunner = jvmIdToRunner.get(jvmId);
         Task task = taskRunner.getTaskInProgress().getTask();
         Task task = taskRunner.getTaskInProgress().getTask();
-        TaskControllerContext context = 
-          new TaskController.TaskControllerContext();
+
+        // Initialize task dirs
+        TaskControllerContext context =
+            new TaskController.TaskControllerContext();
         context.env = jvmRunner.env;
         context.env = jvmRunner.env;
         context.task = task;
         context.task = task;
-        //If we are returning the same task as which the JVM was launched
-        //we don't initialize task once again.
-        if(!jvmRunner.env.conf.get("mapred.task.id").
-            equals(task.getTaskID().toString())) {
-          tracker.getTaskController().initializeTask(context);
+        // If we are returning the same task as which the JVM was launched
+        // we don't initialize task once again.
+        if (!jvmRunner.env.conf.get("mapred.task.id").equals(
+            task.getTaskID().toString())) {
+          try {
+            tracker.getTaskController().initializeTask(context);
+          } catch (IOException e) {
+            LOG.warn("Failed to initialize the new task "
+                + task.getTaskID().toString() + " to be given to JVM with id "
+                + jvmId);
+            throw e;
+          }
         }
         }
 
 
         jvmRunner.taskGiven(task);
         jvmRunner.taskGiven(task);
@@ -405,7 +416,6 @@ class JvmManager {
           //Launch the task controller to run task JVM
           //Launch the task controller to run task JVM
           initalContext.task = jvmToRunningTask.get(jvmId).getTask();
           initalContext.task = jvmToRunningTask.get(jvmId).getTask();
           initalContext.env = env;
           initalContext.env = env;
-          tracker.getTaskController().initializeTask(initalContext);
           tracker.getTaskController().launchTaskJVM(initalContext);
           tracker.getTaskController().launchTaskJVM(initalContext);
         } catch (IOException ioe) {
         } catch (IOException ioe) {
           // do nothing
           // do nothing
@@ -415,13 +425,13 @@ class JvmManager {
           if (shexec == null) {
           if (shexec == null) {
             return;
             return;
           }
           }
-          
+
           kill();
           kill();
-          
+
           int exitCode = shexec.getExitCode();
           int exitCode = shexec.getExitCode();
           updateOnJvmExit(jvmId, exitCode);
           updateOnJvmExit(jvmId, exitCode);
-          LOG.info("JVM : " + jvmId +" exited. Number of tasks it ran: " + 
-              numTasksRan);
+          LOG.info("JVM : " + jvmId + " exited with exit code " + exitCode
+              + ". Number of tasks it ran: " + numTasksRan);
           try {
           try {
             // In case of jvm-reuse,
             // In case of jvm-reuse,
             //the task jvm cleans up the common workdir for every 
             //the task jvm cleans up the common workdir for every 
@@ -445,6 +455,8 @@ class JvmManager {
             initalContext.sleeptimeBeforeSigkill = tracker.getJobConf()
             initalContext.sleeptimeBeforeSigkill = tracker.getJobConf()
               .getLong("mapred.tasktracker.tasks.sleeptime-before-sigkill",
               .getLong("mapred.tasktracker.tasks.sleeptime-before-sigkill",
                   ProcessTree.DEFAULT_SLEEPTIME_BEFORE_SIGKILL);
                   ProcessTree.DEFAULT_SLEEPTIME_BEFORE_SIGKILL);
+
+            // Destroy the task jvm
             controller.destroyTaskJVM(initalContext);
             controller.destroyTaskJVM(initalContext);
           } else {
           } else {
             LOG.info(String.format("JVM Not killed %s but just removed", jvmId
             LOG.info(String.format("JVM Not killed %s but just removed", jvmId

+ 125 - 181
src/mapred/org/apache/hadoop/mapred/LinuxTaskController.java

@@ -24,12 +24,11 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.mapred.JvmManager.JvmEnv;
 import org.apache.hadoop.mapred.JvmManager.JvmEnv;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
@@ -73,52 +72,27 @@ class LinuxTaskController extends TaskController {
         new File(hadoopBin, "task-controller").getAbsolutePath();
         new File(hadoopBin, "task-controller").getAbsolutePath();
   }
   }
   
   
-  // The list of directory paths specified in the
-  // variable mapred.local.dir. This is used to determine
-  // which among the list of directories is picked up
-  // for storing data for a particular task.
-  private String[] mapredLocalDirs;
-  
-  // permissions to set on files and directories created.
-  // When localized files are handled securely, this string
-  // will change to something more restrictive. Until then,
-  // it opens up the permissions for all, so that the tasktracker
-  // and job owners can access files together.
-  private static final String FILE_PERMISSIONS = "ugo+rwx";
-  
-  // permissions to set on components of the path leading to
-  // localized files and directories. Read and execute permissions
-  // are required for different users to be able to access the
-  // files.
-  private static final String PATH_PERMISSIONS = "go+rx";
-  
   public LinuxTaskController() {
   public LinuxTaskController() {
     super();
     super();
   }
   }
   
   
-  @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-    mapredLocalDirs = conf.getStrings("mapred.local.dir");
-    //Setting of the permissions of the local directory is done in 
-    //setup()
-  }
-  
   /**
   /**
    * List of commands that the setuid script will execute.
    * List of commands that the setuid script will execute.
    */
    */
   enum TaskCommands {
   enum TaskCommands {
+    INITIALIZE_JOB,
     LAUNCH_TASK_JVM,
     LAUNCH_TASK_JVM,
+    INITIALIZE_TASK,
     TERMINATE_TASK_JVM,
     TERMINATE_TASK_JVM,
-    KILL_TASK_JVM
+    KILL_TASK_JVM,
   }
   }
-  
+
   /**
   /**
    * Launch a task JVM that will run as the owner of the job.
    * Launch a task JVM that will run as the owner of the job.
    * 
    * 
-   * This method launches a task JVM by executing a setuid
-   * executable that will switch to the user and run the
-   * task.
+   * This method launches a task JVM by executing a setuid executable that will
+   * switch to the user and run the task. Also does initialization of the first
+   * task in the same setuid process launch.
    */
    */
   @Override
   @Override
   void launchTaskJVM(TaskController.TaskControllerContext context) 
   void launchTaskJVM(TaskController.TaskControllerContext context) 
@@ -150,48 +124,103 @@ class LinuxTaskController extends TaskController {
     ShellCommandExecutor shExec =  buildTaskControllerExecutor(
     ShellCommandExecutor shExec =  buildTaskControllerExecutor(
                                     TaskCommands.LAUNCH_TASK_JVM, 
                                     TaskCommands.LAUNCH_TASK_JVM, 
                                     env.conf.getUser(),
                                     env.conf.getUser(),
-                                    launchTaskJVMArgs, env);
+                                    launchTaskJVMArgs, env.workDir, env.env);
     context.shExec = shExec;
     context.shExec = shExec;
     try {
     try {
       shExec.execute();
       shExec.execute();
     } catch (Exception e) {
     } catch (Exception e) {
-      LOG.warn("Exception thrown while launching task JVM : " + 
-          StringUtils.stringifyException(e));
-      LOG.warn("Exit code from task is : " + shExec.getExitCode());
-      LOG.warn("Output from task-contoller is : " + shExec.getOutput());
+      int exitCode = shExec.getExitCode();
+      LOG.warn("Exit code from task is : " + exitCode);
+      // 143 (SIGTERM) and 137 (SIGKILL) exit codes means the task was
+      // terminated/killed forcefully. In all other cases, log the
+      // task-controller output
+      if (exitCode != 143 && exitCode != 137) {
+        LOG.warn("Exception thrown while launching task JVM : "
+            + StringUtils.stringifyException(e));
+        LOG.info("Output from LinuxTaskController's launchTaskJVM follows:");
+        logOutput(shExec.getOutput());
+      }
       throw new IOException(e);
       throw new IOException(e);
     }
     }
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("output after executing task jvm = " + shExec.getOutput()); 
+    if (LOG.isDebugEnabled()) {
+      LOG.info("Output from LinuxTaskController's launchTaskJVM follows:");
+      logOutput(shExec.getOutput());
     }
     }
   }
   }
 
 
   /**
   /**
-   * Returns list of arguments to be passed while launching task VM.
-   * See {@code buildTaskControllerExecutor(TaskCommands, 
-   * String, List<String>, JvmEnv)} documentation.
+   * Helper method that runs a LinuxTaskController command
+   * 
+   * @param taskCommand
+   * @param user
+   * @param cmdArgs
+   * @param env
+   * @throws IOException
+   */
+  private void runCommand(TaskCommands taskCommand, String user,
+      List<String> cmdArgs, File workDir, Map<String, String> env)
+      throws IOException {
+
+    ShellCommandExecutor shExec =
+        buildTaskControllerExecutor(taskCommand, user, cmdArgs, workDir, env);
+    try {
+      shExec.execute();
+    } catch (Exception e) {
+      LOG.warn("Exit code from " + taskCommand.toString() + " is : "
+          + shExec.getExitCode());
+      LOG.warn("Exception thrown by " + taskCommand.toString() + " : "
+          + StringUtils.stringifyException(e));
+      LOG.info("Output from LinuxTaskController's " + taskCommand.toString()
+          + " follows:");
+      logOutput(shExec.getOutput());
+      throw new IOException(e);
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.info("Output from LinuxTaskController's " + taskCommand.toString()
+          + " follows:");
+      logOutput(shExec.getOutput());
+    }
+  }
+
+  /**
+   * Returns list of arguments to be passed while initializing a new task. See
+   * {@code buildTaskControllerExecutor(TaskCommands, String, List<String>,
+   * JvmEnv)} documentation.
+   * 
    * @param context
    * @param context
    * @return Argument to be used while launching Task VM
    * @return Argument to be used while launching Task VM
    */
    */
-  private List<String> buildLaunchTaskArgs(TaskControllerContext context) {
+  private List<String> buildInitializeTaskArgs(TaskControllerContext context) {
     List<String> commandArgs = new ArrayList<String>(3);
     List<String> commandArgs = new ArrayList<String>(3);
     String taskId = context.task.getTaskID().toString();
     String taskId = context.task.getTaskID().toString();
     String jobId = getJobId(context);
     String jobId = getJobId(context);
-    LOG.debug("getting the task directory as: " 
-        + getTaskCacheDirectory(context));
-    commandArgs.add(getDirectoryChosenForTask(
-        new File(getTaskCacheDirectory(context)), 
-        context));
     commandArgs.add(jobId);
     commandArgs.add(jobId);
-    if(!context.task.isTaskCleanupTask()) {
+    if (!context.task.isTaskCleanupTask()) {
       commandArgs.add(taskId);
       commandArgs.add(taskId);
-    }else {
+    } else {
       commandArgs.add(taskId + TaskTracker.TASK_CLEANUP_SUFFIX);
       commandArgs.add(taskId + TaskTracker.TASK_CLEANUP_SUFFIX);
     }
     }
     return commandArgs;
     return commandArgs;
   }
   }
-  
-  // get the Job ID from the information in the TaskControllerContext
+
+  @Override
+  void initializeTask(TaskControllerContext context)
+      throws IOException {
+    LOG.debug("Going to do " + TaskCommands.INITIALIZE_TASK.toString()
+        + " for " + context.task.getTaskID().toString());
+    runCommand(TaskCommands.INITIALIZE_TASK, context.env.conf.getUser(),
+        buildInitializeTaskArgs(context), context.env.workDir, context.env.env);
+  }
+
+  private void logOutput(String output) {
+    String shExecOutput = output;
+    if (shExecOutput != null) {
+      for (String str : shExecOutput.split("\n")) {
+        LOG.info(str);
+      }
+    }
+  }
+
   private String getJobId(TaskControllerContext context) {
   private String getJobId(TaskControllerContext context) {
     String taskId = context.task.getTaskID().toString();
     String taskId = context.task.getTaskID().toString();
     TaskAttemptID tId = TaskAttemptID.forName(taskId);
     TaskAttemptID tId = TaskAttemptID.forName(taskId);
@@ -199,6 +228,27 @@ class LinuxTaskController extends TaskController {
     return jobId;
     return jobId;
   }
   }
 
 
+  /**
+   * Returns list of arguments to be passed while launching task VM.
+   * See {@code buildTaskControllerExecutor(TaskCommands, 
+   * String, List<String>, JvmEnv)} documentation.
+   * @param context
+   * @return Argument to be used while launching Task VM
+   */
+  private List<String> buildLaunchTaskArgs(TaskControllerContext context) {
+    List<String> commandArgs = new ArrayList<String>(3);
+    LOG.debug("getting the task directory as: " 
+        + getTaskCacheDirectory(context));
+    LOG.debug("getting the tt_root as " +getDirectoryChosenForTask(
+        new File(getTaskCacheDirectory(context)), 
+        context) );
+    commandArgs.add(getDirectoryChosenForTask(
+        new File(getTaskCacheDirectory(context)), 
+        context));
+    commandArgs.addAll(buildInitializeTaskArgs(context));
+    return commandArgs;
+  }
+
   // Get the directory from the list of directories configured
   // Get the directory from the list of directories configured
   // in mapred.local.dir chosen for storing data pertaining to
   // in mapred.local.dir chosen for storing data pertaining to
   // this task.
   // this task.
@@ -208,8 +258,8 @@ class LinuxTaskController extends TaskController {
     String taskId = context.task.getTaskID().toString();
     String taskId = context.task.getTaskID().toString();
     for (String dir : mapredLocalDirs) {
     for (String dir : mapredLocalDirs) {
       File mapredDir = new File(dir);
       File mapredDir = new File(dir);
-      File taskDir = new File(mapredDir, TaskTracker.getLocalTaskDir(
-          jobId, taskId, context.task.isTaskCleanupTask()));
+      File taskDir = new File(mapredDir, TaskTracker.getTaskWorkDir(
+          jobId, taskId, context.task.isTaskCleanupTask())).getParentFile();
       if (directory.equals(taskDir)) {
       if (directory.equals(taskDir)) {
         return dir;
         return dir;
       }
       }
@@ -219,68 +269,7 @@ class LinuxTaskController extends TaskController {
     throw new IllegalArgumentException("invalid task cache directory "
     throw new IllegalArgumentException("invalid task cache directory "
                 + directory.getAbsolutePath());
                 + directory.getAbsolutePath());
   }
   }
-  
-  /**
-   * Setup appropriate permissions for directories and files that
-   * are used by the task.
-   * 
-   * As the LinuxTaskController launches tasks as a user, different
-   * from the daemon, all directories and files that are potentially 
-   * used by the tasks are setup with appropriate permissions that
-   * will allow access.
-   * 
-   * Until secure data handling is implemented (see HADOOP-4491 and
-   * HADOOP-4493, for e.g.), the permissions are set up to allow
-   * read, write and execute access for everyone. This will be 
-   * changed to restricted access as data is handled securely.
-   */
-  void initializeTask(TaskControllerContext context) {
-    // Setup permissions for the job and task cache directories.
-    setupTaskCacheFileAccess(context);
-    // setup permissions for task log directory
-    setupTaskLogFileAccess(context);    
-  }
-  
-  // Allows access for the task to create log files under 
-  // the task log directory
-  private void setupTaskLogFileAccess(TaskControllerContext context) {
-    TaskAttemptID taskId = context.task.getTaskID();
-    File f = TaskLog.getTaskLogFile(taskId, TaskLog.LogName.SYSLOG);
-    String taskAttemptLogDir = f.getParentFile().getAbsolutePath();
-    changeDirectoryPermissions(taskAttemptLogDir, FILE_PERMISSIONS, false);
-  }
 
 
-  // Allows access for the task to read, write and execute 
-  // the files under the job and task cache directories
-  private void setupTaskCacheFileAccess(TaskControllerContext context) {
-    String taskId = context.task.getTaskID().toString();
-    JobID jobId = JobID.forName(getJobId(context));
-    //Change permission for the task across all the disks
-    for(String localDir : mapredLocalDirs) {
-      File f = new File(localDir);
-      File taskCacheDir = new File(f,TaskTracker.getLocalTaskDir(
-          jobId.toString(), taskId, context.task.isTaskCleanupTask()));
-      if(taskCacheDir.exists()) {
-        changeDirectoryPermissions(taskCacheDir.getPath(), 
-            FILE_PERMISSIONS, true);
-      }          
-    }//end of local directory Iteration 
-  }
-
-  // convenience method to execute chmod.
-  private void changeDirectoryPermissions(String dir, String mode, 
-                                              boolean isRecursive) {
-    int ret = 0;
-    try {
-      ret = FileUtil.chmod(dir, mode, isRecursive);
-    } catch (Exception e) {
-      LOG.warn("Exception in changing permissions for directory " + dir + 
-                  ". Exception: " + e.getMessage());
-    }
-    if (ret != 0) {
-      LOG.warn("Could not change permissions for directory " + dir);
-    }
-  }
   /**
   /**
    * Builds the command line for launching/terminating/killing task JVM.
    * Builds the command line for launching/terminating/killing task JVM.
    * Following is the format for launching/terminating/killing task JVM
    * Following is the format for launching/terminating/killing task JVM
@@ -295,14 +284,15 @@ class LinuxTaskController extends TaskController {
    * @param command command to be executed.
    * @param command command to be executed.
    * @param userName user name
    * @param userName user name
    * @param cmdArgs list of extra arguments
    * @param cmdArgs list of extra arguments
+   * @param workDir working directory for the task-controller
    * @param env JVM environment variables.
    * @param env JVM environment variables.
    * @return {@link ShellCommandExecutor}
    * @return {@link ShellCommandExecutor}
    * @throws IOException
    * @throws IOException
    */
    */
-  private ShellCommandExecutor buildTaskControllerExecutor(TaskCommands command, 
-                                          String userName, 
-                                          List<String> cmdArgs, JvmEnv env) 
-                                    throws IOException {
+  private ShellCommandExecutor buildTaskControllerExecutor(
+      TaskCommands command, String userName, List<String> cmdArgs,
+      File workDir, Map<String, String> env)
+      throws IOException {
     String[] taskControllerCmd = new String[3 + cmdArgs.size()];
     String[] taskControllerCmd = new String[3 + cmdArgs.size()];
     taskControllerCmd[0] = getTaskControllerExecutablePath();
     taskControllerCmd[0] = getTaskControllerExecutablePath();
     taskControllerCmd[1] = userName;
     taskControllerCmd[1] = userName;
@@ -317,9 +307,9 @@ class LinuxTaskController extends TaskController {
       }
       }
     }
     }
     ShellCommandExecutor shExec = null;
     ShellCommandExecutor shExec = null;
-    if(env.workDir != null && env.workDir.exists()) {
+    if(workDir != null && workDir.exists()) {
       shExec = new ShellCommandExecutor(taskControllerCmd,
       shExec = new ShellCommandExecutor(taskControllerCmd,
-          env.workDir, env.env);
+          workDir, env);
     } else {
     } else {
       shExec = new ShellCommandExecutor(taskControllerCmd);
       shExec = new ShellCommandExecutor(taskControllerCmd);
     }
     }
@@ -376,66 +366,20 @@ class LinuxTaskController extends TaskController {
     return taskControllerExe;
     return taskControllerExe;
   }  
   }  
 
 
-  /**
-   * Sets up the permissions of the following directories:
-   * 
-   * Job cache directory
-   * Archive directory
-   * Hadoop log directories
-   * 
-   */
-  @Override
-  void setup() {
-    //set up job cache directory and associated permissions
-    String localDirs[] = this.mapredLocalDirs;
-    for(String localDir : localDirs) {
-      //Cache root
-      File cacheDirectory = new File(localDir,TaskTracker.getCacheSubdir());
-      File jobCacheDirectory = new File(localDir,TaskTracker.getJobCacheSubdir());
-      if(!cacheDirectory.exists()) {
-        if(!cacheDirectory.mkdirs()) {
-          LOG.warn("Unable to create cache directory : " + 
-              cacheDirectory.getPath());
-        }
-      }
-      if(!jobCacheDirectory.exists()) {
-        if(!jobCacheDirectory.mkdirs()) {
-          LOG.warn("Unable to create job cache directory : " + 
-              jobCacheDirectory.getPath());
-        }
-      }
-      //Give world writable permission for every directory under
-      //mapred-local-dir.
-      //Child tries to write files under it when executing.
-      changeDirectoryPermissions(localDir, FILE_PERMISSIONS, true);
-    }//end of local directory manipulations
-    //setting up perms for user logs
-    File taskLog = TaskLog.getUserLogDir();
-    changeDirectoryPermissions(taskLog.getPath(), FILE_PERMISSIONS,false);
+  private List<String> buildInitializeJobCommandArgs(
+      JobInitializationContext context) {
+    List<String> initJobCmdArgs = new ArrayList<String>();
+    initJobCmdArgs.add(context.jobid.toString());
+    return initJobCmdArgs;
   }
   }
 
 
-  /*
-   * Create Job directories across disks and set their permissions to 777
-   * This way when tasks are run we just need to setup permissions for
-   * task folder.
-   */
   @Override
   @Override
-  void initializeJob(JobID jobid) {
-    for(String localDir : this.mapredLocalDirs) {
-      File jobDirectory = new File(localDir, 
-          TaskTracker.getLocalJobDir(jobid.toString()));
-      if(!jobDirectory.exists()) {
-        if(!jobDirectory.mkdir()) {
-          LOG.warn("Unable to create job cache directory : " 
-              + jobDirectory.getPath());
-          continue;
-        }
-      }
-      //Should be recursive because the jar and work folders might be 
-      //present under the job cache directory
-      changeDirectoryPermissions(
-          jobDirectory.getPath(), FILE_PERMISSIONS, true);
-    }
+  void initializeJob(JobInitializationContext context)
+      throws IOException {
+    LOG.debug("Going to initialize job " + context.jobid.toString()
+        + " on the TT");
+    runCommand(TaskCommands.INITIALIZE_JOB, context.user,
+        buildInitializeJobCommandArgs(context), context.workDir, null);
   }
   }
   
   
   /**
   /**
@@ -470,7 +414,7 @@ class LinuxTaskController extends TaskController {
     }
     }
     ShellCommandExecutor shExec = buildTaskControllerExecutor(
     ShellCommandExecutor shExec = buildTaskControllerExecutor(
         command, context.env.conf.getUser(), 
         command, context.env.conf.getUser(), 
-        buildKillTaskCommandArgs(context), context.env);
+        buildKillTaskCommandArgs(context), context.env.workDir, context.env.env);
     try {
     try {
       shExec.execute();
       shExec.execute();
     } catch (Exception e) {
     } catch (Exception e) {

+ 28 - 17
src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -21,12 +21,17 @@ package org.apache.hadoop.mapred;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Random;
 import java.util.Random;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
 import org.apache.hadoop.mapred.JobTrackerMetricsInst;
 import org.apache.hadoop.mapred.JobTrackerMetricsInst;
 import org.apache.hadoop.mapred.JvmTask;
 import org.apache.hadoop.mapred.JvmTask;
 import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
 import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
@@ -62,7 +67,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
 
 
     private JobStatus status;
     private JobStatus status;
     private ArrayList<TaskAttemptID> mapIds = new ArrayList<TaskAttemptID>();
     private ArrayList<TaskAttemptID> mapIds = new ArrayList<TaskAttemptID>();
-    private MapOutputFile mapoutputFile;
+
     private JobProfile profile;
     private JobProfile profile;
     private Path localFile;
     private Path localFile;
     private FileSystem localFs;
     private FileSystem localFs;
@@ -83,8 +88,6 @@ class LocalJobRunner implements JobSubmissionProtocol {
       this.systemJobDir = new Path(jobSubmitDir);
       this.systemJobDir = new Path(jobSubmitDir);
       this.file = new Path(systemJobDir, "job.xml");
       this.file = new Path(systemJobDir, "job.xml");
       this.id = jobid;
       this.id = jobid;
-      this.mapoutputFile = new MapOutputFile(jobid);
-      this.mapoutputFile.setConf(conf);
 
 
       this.localFile = new JobConf(conf).getLocalPath(jobDir+id+".xml");
       this.localFile = new JobConf(conf).getLocalPath(jobDir+id+".xml");
       this.localFs = FileSystem.getLocal(conf);
       this.localFs = FileSystem.getLocal(conf);
@@ -120,7 +123,9 @@ class LocalJobRunner implements JobSubmissionProtocol {
         }
         }
         outputCommitter.setupJob(jContext);
         outputCommitter.setupJob(jContext);
         status.setSetupProgress(1.0f);
         status.setSetupProgress(1.0f);
-        
+
+        Map<TaskAttemptID, MapOutputFile> mapOutputFiles =
+          new HashMap<TaskAttemptID, MapOutputFile>();
         for (int i = 0; i < taskSplitMetaInfos.length; i++) {
         for (int i = 0; i < taskSplitMetaInfos.length; i++) {
           if (!this.isInterrupted()) {
           if (!this.isInterrupted()) {
             TaskAttemptID mapId = new TaskAttemptID(new TaskID(jobId, true, i),0);  
             TaskAttemptID mapId = new TaskAttemptID(new TaskID(jobId, true, i),0);  
@@ -129,6 +134,12 @@ class LocalJobRunner implements JobSubmissionProtocol {
                                       mapId, i,
                                       mapId, i,
                                       taskSplitMetaInfos[i].getSplitIndex(), 1);
                                       taskSplitMetaInfos[i].getSplitIndex(), 1);
             JobConf localConf = new JobConf(job);
             JobConf localConf = new JobConf(job);
+            TaskRunner.setupChildMapredLocalDirs(map, localConf);
+
+            MapOutputFile mapOutput = new MapOutputFile();
+            mapOutput.setConf(localConf);
+            mapOutputFiles.put(mapId, mapOutput);
+
             map.setJobFile(localFile.toString());
             map.setJobFile(localFile.toString());
             map.localizeConfiguration(localConf);
             map.localizeConfiguration(localConf);
             map.setConf(localConf);
             map.setConf(localConf);
@@ -146,14 +157,21 @@ class LocalJobRunner implements JobSubmissionProtocol {
           new TaskAttemptID(new TaskID(jobId, false, 0), 0);
           new TaskAttemptID(new TaskID(jobId, false, 0), 0);
         try {
         try {
           if (numReduceTasks > 0) {
           if (numReduceTasks > 0) {
+            ReduceTask reduce =
+                new ReduceTask(file.toString(), reduceId, 0, mapIds.size(),
+                    1);
+            JobConf localConf = new JobConf(job);
+            TaskRunner.setupChildMapredLocalDirs(reduce, localConf);
             // move map output to reduce input  
             // move map output to reduce input  
             for (int i = 0; i < mapIds.size(); i++) {
             for (int i = 0; i < mapIds.size(); i++) {
               if (!this.isInterrupted()) {
               if (!this.isInterrupted()) {
                 TaskAttemptID mapId = mapIds.get(i);
                 TaskAttemptID mapId = mapIds.get(i);
-                Path mapOut = this.mapoutputFile.getOutputFile(mapId);
-                Path reduceIn = this.mapoutputFile.getInputFileForWrite(
-                                  mapId.getTaskID(),reduceId,
-                                  localFs.getLength(mapOut));
+                Path mapOut = mapOutputFiles.get(mapId).getOutputFile();
+                MapOutputFile localOutputFile = new MapOutputFile();
+                localOutputFile.setConf(localConf);
+                Path reduceIn =
+                  localOutputFile.getInputFileForWrite(mapId.getTaskID(),
+                        localFs.getFileStatus(mapOut).getLen());
                 if (!localFs.mkdirs(reduceIn.getParent())) {
                 if (!localFs.mkdirs(reduceIn.getParent())) {
                   throw new IOException("Mkdirs failed to create "
                   throw new IOException("Mkdirs failed to create "
                       + reduceIn.getParent().toString());
                       + reduceIn.getParent().toString());
@@ -165,10 +183,6 @@ class LocalJobRunner implements JobSubmissionProtocol {
               }
               }
             }
             }
             if (!this.isInterrupted()) {
             if (!this.isInterrupted()) {
-              ReduceTask reduce = new ReduceTask(file.toString(), 
-                                                 reduceId, 0, mapIds.size(), 
-                                                 1);
-              JobConf localConf = new JobConf(job);
               reduce.setJobFile(localFile.toString());
               reduce.setJobFile(localFile.toString());
               reduce.localizeConfiguration(localConf);
               reduce.localizeConfiguration(localConf);
               reduce.setConf(localConf);
               reduce.setConf(localConf);
@@ -183,11 +197,8 @@ class LocalJobRunner implements JobSubmissionProtocol {
             }
             }
           }
           }
         } finally {
         } finally {
-          for (TaskAttemptID mapId: mapIds) {
-            this.mapoutputFile.removeAll(mapId);
-          }
-          if (numReduceTasks == 1) {
-            this.mapoutputFile.removeAll(reduceId);
+          for (MapOutputFile output : mapOutputFiles.values()) {
+            output.removeAll();
           }
           }
         }
         }
         // delete the temporary directory in output directory
         // delete the temporary directory in output directory

+ 100 - 97
src/mapred/org/apache/hadoop/mapred/MapOutputFile.java

@@ -30,144 +30,152 @@ import org.apache.hadoop.fs.Path;
 class MapOutputFile {
 class MapOutputFile {
 
 
   private JobConf conf;
   private JobConf conf;
-  private JobID jobId;
-  
-  MapOutputFile() {
-  }
 
 
-  MapOutputFile(JobID jobId) {
-    this.jobId = jobId;
+  static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
+
+  MapOutputFile() {
   }
   }
 
 
   private LocalDirAllocator lDirAlloc = 
   private LocalDirAllocator lDirAlloc = 
                             new LocalDirAllocator("mapred.local.dir");
                             new LocalDirAllocator("mapred.local.dir");
   
   
-  /** Return the path to local map output file created earlier
-   * @param mapTaskId a map task id
+  /**
+   * Return the path to local map output file created earlier
+   * 
+   * @return path
+   * @throws IOException
    */
    */
-  public Path getOutputFile(TaskAttemptID mapTaskId)
-    throws IOException {
-    return lDirAlloc.getLocalPathToRead(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), mapTaskId.toString())
-                       + "/file.out", conf);
+  public Path getOutputFile()
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(TaskTracker.OUTPUT + Path.SEPARATOR
+        + "file.out", conf);
   }
   }
 
 
-  /** Create a local map output file name.
-   * @param mapTaskId a map task id
+  /**
+   * Create a local map output file name.
+   * 
    * @param size the size of the file
    * @param size the size of the file
+   * @return path
+   * @throws IOException
    */
    */
-  public Path getOutputFileForWrite(TaskAttemptID mapTaskId, long size)
-    throws IOException {
-    return lDirAlloc.getLocalPathForWrite(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), mapTaskId.toString())
-                       + "/file.out", size, conf);
+  public Path getOutputFileForWrite(long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(TaskTracker.OUTPUT + Path.SEPARATOR
+        + "file.out", size, conf);
   }
   }
 
 
-  /** Return the path to a local map output index file created earlier
-   * @param mapTaskId a map task id
+  /**
+   * Return the path to a local map output index file created earlier
+   * 
+   * @return path
+   * @throws IOException
    */
    */
-  public Path getOutputIndexFile(TaskAttemptID mapTaskId)
-    throws IOException {
-    return lDirAlloc.getLocalPathToRead(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), mapTaskId.toString())
-                       + "/file.out.index", conf);
+  public Path getOutputIndexFile()
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(TaskTracker.OUTPUT + Path.SEPARATOR
+        + "file.out.index", conf);
   }
   }
 
 
-  /** Create a local map output index file name.
-   * @param mapTaskId a map task id
+  /**
+   * Create a local map output index file name.
+   * 
    * @param size the size of the file
    * @param size the size of the file
+   * @return path
+   * @throws IOException
    */
    */
-  public Path getOutputIndexFileForWrite(TaskAttemptID mapTaskId, long size)
-    throws IOException {
-    return lDirAlloc.getLocalPathForWrite(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), mapTaskId.toString())
-                       + "/file.out.index", 
-                       size, conf);
+  public Path getOutputIndexFileForWrite(long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(TaskTracker.OUTPUT + Path.SEPARATOR
+        + "file.out.index", size, conf);
   }
   }
 
 
-  /** Return a local map spill file created earlier.
-   * @param mapTaskId a map task id
+  /**
+   * Return a local map spill file created earlier.
+   * 
    * @param spillNumber the number
    * @param spillNumber the number
+   * @return path
+   * @throws IOException
    */
    */
-  public Path getSpillFile(TaskAttemptID mapTaskId, int spillNumber)
-    throws IOException {
-    return lDirAlloc.getLocalPathToRead(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), mapTaskId.toString())
-                       + "/spill" 
-                       + spillNumber + ".out", conf);
+  public Path getSpillFile(int spillNumber)
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(TaskTracker.OUTPUT + "/spill"
+        + spillNumber + ".out", conf);
   }
   }
 
 
-  /** Create a local map spill file name.
-   * @param mapTaskId a map task id
+  /**
+   * Create a local map spill file name.
+   * 
    * @param spillNumber the number
    * @param spillNumber the number
    * @param size the size of the file
    * @param size the size of the file
+   * @return path
+   * @throws IOException
    */
    */
-  public Path getSpillFileForWrite(TaskAttemptID mapTaskId, int spillNumber, 
-         long size) throws IOException {
-    return lDirAlloc.getLocalPathForWrite(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), mapTaskId.toString())
-                       + "/spill" + 
-                       spillNumber + ".out", size, conf);
+  public Path getSpillFileForWrite(int spillNumber, long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(TaskTracker.OUTPUT + "/spill"
+        + spillNumber + ".out", size, conf);
   }
   }
 
 
-  /** Return a local map spill index file created earlier
-   * @param mapTaskId a map task id
+  /**
+   * Return a local map spill index file created earlier
+   * 
    * @param spillNumber the number
    * @param spillNumber the number
+   * @return path
+   * @throws IOException
    */
    */
-  public Path getSpillIndexFile(TaskAttemptID mapTaskId, int spillNumber)
-    throws IOException {
-    return lDirAlloc.getLocalPathToRead(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), mapTaskId.toString())
-                       + "/spill" + 
-                       spillNumber + ".out.index", conf);
+  public Path getSpillIndexFile(int spillNumber)
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(TaskTracker.OUTPUT + "/spill"
+        + spillNumber + ".out.index", conf);
   }
   }
 
 
-  /** Create a local map spill index file name.
-   * @param mapTaskId a map task id
+  /**
+   * Create a local map spill index file name.
+   * 
    * @param spillNumber the number
    * @param spillNumber the number
    * @param size the size of the file
    * @param size the size of the file
+   * @return path
+   * @throws IOException
    */
    */
-  public Path getSpillIndexFileForWrite(TaskAttemptID mapTaskId, int spillNumber,
-         long size) throws IOException {
-    return lDirAlloc.getLocalPathForWrite(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), mapTaskId.toString())
-                       + "/spill" + spillNumber + 
-                       ".out.index", size, conf);
+  public Path getSpillIndexFileForWrite(int spillNumber, long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(TaskTracker.OUTPUT + "/spill"
+        + spillNumber + ".out.index", size, conf);
   }
   }
 
 
-  /** Return a local reduce input file created earlier
-   * @param mapTaskId a map task id
-   * @param reduceTaskId a reduce task id
+  /**
+   * Return a local reduce input file created earlier
+   * 
+   * @param mapId a map task id
+   * @return path
+   * @throws IOException 
    */
    */
-  public Path getInputFile(int mapId, TaskAttemptID reduceTaskId)
-    throws IOException {
-    // TODO *oom* should use a format here
-    return lDirAlloc.getLocalPathToRead(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), reduceTaskId.toString())
-                       + "/map_" + mapId + ".out",
-                       conf);
+  public Path getInputFile(int mapId)
+      throws IOException {
+    return lDirAlloc.getLocalPathToRead(String.format(
+        REDUCE_INPUT_FILE_FORMAT_STRING, TaskTracker.OUTPUT, Integer
+            .valueOf(mapId)), conf);
   }
   }
 
 
-  /** Create a local reduce input file name.
-   * @param mapTaskId a map task id
-   * @param reduceTaskId a reduce task id
+  /**
+   * Create a local reduce input file name.
+   * 
+   * @param mapId a map task id
    * @param size the size of the file
    * @param size the size of the file
+   * @return path
+   * @throws IOException
    */
    */
-  public Path getInputFileForWrite(TaskID mapId, TaskAttemptID reduceTaskId, 
-                                   long size)
-    throws IOException {
-    // TODO *oom* should use a format here
-    return lDirAlloc.getLocalPathForWrite(TaskTracker.getIntermediateOutputDir(
-                       jobId.toString(), reduceTaskId.toString())
-                       + "/map_" + mapId.getId() + ".out", 
-                       size, conf);
+  public Path getInputFileForWrite(TaskID mapId, long size)
+      throws IOException {
+    return lDirAlloc.getLocalPathForWrite(String.format(
+        REDUCE_INPUT_FILE_FORMAT_STRING, TaskTracker.OUTPUT, mapId.getId()),
+        size, conf);
   }
   }
 
 
   /** Removes all of the files related to a task. */
   /** Removes all of the files related to a task. */
-  public void removeAll(TaskAttemptID taskId) throws IOException {
-    conf.deleteLocalFiles(TaskTracker.getIntermediateOutputDir(
-                          jobId.toString(), taskId.toString())
-);
+  public void removeAll()
+      throws IOException {
+    conf.deleteLocalFiles(TaskTracker.OUTPUT);
   }
   }
 
 
   public void setConf(Configuration conf) {
   public void setConf(Configuration conf) {
@@ -177,9 +185,4 @@ class MapOutputFile {
       this.conf = new JobConf(conf);
       this.conf = new JobConf(conf);
     }
     }
   }
   }
-  
-  public void setJobId(JobID jobId) {
-    this.jobId = jobId;
-  }
-
 }
 }

+ 29 - 21
src/mapred/org/apache/hadoop/mapred/MapTask.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.BytesWritable;
@@ -77,7 +78,6 @@ class MapTask extends Task {
    * The size of each record in the index file for the map-outputs.
    * The size of each record in the index file for the map-outputs.
    */
    */
   public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
   public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
-  
 
 
   private TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
   private TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
   private String splitClass;
   private String splitClass;
@@ -106,12 +106,20 @@ class MapTask extends Task {
   }
   }
 
 
   @Override
   @Override
-  public void localizeConfiguration(JobConf conf) throws IOException {
+  public void localizeConfiguration(JobConf conf)
+      throws IOException {
     super.localizeConfiguration(conf);
     super.localizeConfiguration(conf);
+    // split.info file is used only by IsolationRunner.
+    // Write the split file to the local disk if it is a normal map task (not a
+    // job-setup or a job-cleanup task) and if the user wishes to run
+    // IsolationRunner either by setting keep.failed.tasks.files to true or by
+    // using keep.tasks.files.pattern
     if (supportIsolationRunner(conf) && isMapOrReduce()) {
     if (supportIsolationRunner(conf) && isMapOrReduce()) {
       // localize the split meta-information
       // localize the split meta-information
-      Path localSplitMeta = new Path(new Path(getJobFile()).getParent(), 
-                                 "split.info");
+      Path localSplitMeta =
+        new LocalDirAllocator("mapred.local.dir").getLocalPathForWrite(
+            TaskTracker.getLocalSplitFile(getJobID().toString(), getTaskID()
+                .toString()), conf);
       LOG.debug("Writing local split to " + localSplitMeta);
       LOG.debug("Writing local split to " + localSplitMeta);
       DataOutputStream out = FileSystem.getLocal(conf).create(localSplitMeta);
       DataOutputStream out = FileSystem.getLocal(conf).create(localSplitMeta);
       splitMetaInfo.write(out);
       splitMetaInfo.write(out);
@@ -1228,8 +1236,8 @@ class MapTask extends Task {
       try {
       try {
         // create spill file
         // create spill file
         final SpillRecord spillRec = new SpillRecord(partitions);
         final SpillRecord spillRec = new SpillRecord(partitions);
-        final Path filename = mapOutputFile.getSpillFileForWrite(getTaskID(),
-            numSpills, size);
+        final Path filename =
+            mapOutputFile.getSpillFileForWrite(numSpills, size);
         out = rfs.create(filename);
         out = rfs.create(filename);
 
 
         final int endPosition = (kvend > kvstart)
         final int endPosition = (kvend > kvstart)
@@ -1293,9 +1301,9 @@ class MapTask extends Task {
 
 
         if (totalIndexCacheMemory >= INDEX_CACHE_MEMORY_LIMIT) {
         if (totalIndexCacheMemory >= INDEX_CACHE_MEMORY_LIMIT) {
           // create spill index file
           // create spill index file
-          Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
-              getTaskID(), numSpills,
-              partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH);
+          Path indexFilename =
+              mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
+                  * MAP_OUTPUT_INDEX_RECORD_LENGTH);
           spillRec.writeToFile(indexFilename, job);
           spillRec.writeToFile(indexFilename, job);
         } else {
         } else {
           indexCacheList.add(spillRec);
           indexCacheList.add(spillRec);
@@ -1321,8 +1329,8 @@ class MapTask extends Task {
       try {
       try {
         // create spill file
         // create spill file
         final SpillRecord spillRec = new SpillRecord(partitions);
         final SpillRecord spillRec = new SpillRecord(partitions);
-        final Path filename = mapOutputFile.getSpillFileForWrite(getTaskID(),
-            numSpills, size);
+        final Path filename =
+            mapOutputFile.getSpillFileForWrite(numSpills, size);
         out = rfs.create(filename);
         out = rfs.create(filename);
         
         
         // we don't run the combiner for a single record
         // we don't run the combiner for a single record
@@ -1358,9 +1366,9 @@ class MapTask extends Task {
         }
         }
         if (totalIndexCacheMemory >= INDEX_CACHE_MEMORY_LIMIT) {
         if (totalIndexCacheMemory >= INDEX_CACHE_MEMORY_LIMIT) {
           // create spill index file
           // create spill index file
-          Path indexFilename = mapOutputFile.getSpillIndexFileForWrite(
-              getTaskID(), numSpills,
-              partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH);
+          Path indexFilename =
+              mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
+                  * MAP_OUTPUT_INDEX_RECORD_LENGTH);
           spillRec.writeToFile(indexFilename, job);
           spillRec.writeToFile(indexFilename, job);
         } else {
         } else {
           indexCacheList.add(spillRec);
           indexCacheList.add(spillRec);
@@ -1450,14 +1458,14 @@ class MapTask extends Task {
       final TaskAttemptID mapId = getTaskID();
       final TaskAttemptID mapId = getTaskID();
 
 
       for(int i = 0; i < numSpills; i++) {
       for(int i = 0; i < numSpills; i++) {
-        filename[i] = mapOutputFile.getSpillFile(mapId, i);
+        filename[i] = mapOutputFile.getSpillFile(i);
         finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
         finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
       }
       }
       if (numSpills == 1) { //the spill is the final output
       if (numSpills == 1) { //the spill is the final output
         rfs.rename(filename[0],
         rfs.rename(filename[0],
             new Path(filename[0].getParent(), "file.out"));
             new Path(filename[0].getParent(), "file.out"));
         if (indexCacheList.size() == 0) {
         if (indexCacheList.size() == 0) {
-          rfs.rename(mapOutputFile.getSpillIndexFile(mapId, 0),
+          rfs.rename(mapOutputFile.getSpillIndexFile(0),
               new Path(filename[0].getParent(),"file.out.index"));
               new Path(filename[0].getParent(),"file.out.index"));
         } else {
         } else {
           indexCacheList.get(0).writeToFile(
           indexCacheList.get(0).writeToFile(
@@ -1468,7 +1476,7 @@ class MapTask extends Task {
 
 
       // read in paged indices
       // read in paged indices
       for (int i = indexCacheList.size(); i < numSpills; ++i) {
       for (int i = indexCacheList.size(); i < numSpills; ++i) {
-        Path indexFileName = mapOutputFile.getSpillIndexFile(mapId, i);
+        Path indexFileName = mapOutputFile.getSpillIndexFile(i);
         indexCacheList.add(new SpillRecord(indexFileName, job));
         indexCacheList.add(new SpillRecord(indexFileName, job));
       }
       }
 
 
@@ -1476,10 +1484,10 @@ class MapTask extends Task {
       //lengths for each partition
       //lengths for each partition
       finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
       finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
       finalIndexFileSize = partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH;
       finalIndexFileSize = partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH;
-      Path finalOutputFile = mapOutputFile.getOutputFileForWrite(mapId,
-                             finalOutFileSize);
-      Path finalIndexFile = mapOutputFile.getOutputIndexFileForWrite(
-                            mapId, finalIndexFileSize);
+      Path finalOutputFile =
+          mapOutputFile.getOutputFileForWrite(finalOutFileSize);
+      Path finalIndexFile =
+          mapOutputFile.getOutputIndexFileForWrite(finalIndexFileSize);
 
 
       //The output stream for the final single output file
       //The output stream for the final single output file
       FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
       FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);

+ 2 - 2
src/mapred/org/apache/hadoop/mapred/MapTaskRunner.java

@@ -34,13 +34,13 @@ class MapTaskRunner extends TaskRunner {
       return false;
       return false;
     }
     }
     
     
-    mapOutputFile.removeAll(getTask().getTaskID());
+    mapOutputFile.removeAll();
     return true;
     return true;
   }
   }
 
 
   /** Delete all of the temporary map output files. */
   /** Delete all of the temporary map output files. */
   public void close() throws IOException {
   public void close() throws IOException {
     LOG.info(getTask()+" done; removing files.");
     LOG.info(getTask()+" done; removing files.");
-    mapOutputFile.removeAll(getTask().getTaskID());
+    mapOutputFile.removeAll();
   }
   }
 }
 }

+ 10 - 11
src/mapred/org/apache/hadoop/mapred/ReduceTask.java

@@ -213,7 +213,7 @@ class ReduceTask extends Task {
     if (isLocal) {
     if (isLocal) {
       // for local jobs
       // for local jobs
       for(int i = 0; i < numMaps; ++i) {
       for(int i = 0; i < numMaps; ++i) {
-        fileList.add(mapOutputFile.getInputFile(i, getTaskID()));
+        fileList.add(mapOutputFile.getInputFile(i));
       }
       }
     } else {
     } else {
       // for non local jobs
       // for non local jobs
@@ -1287,12 +1287,11 @@ class ReduceTask extends Task {
         // else, we will check the localFS to find a suitable final location
         // else, we will check the localFS to find a suitable final location
         // for this path
         // for this path
         TaskAttemptID reduceId = reduceTask.getTaskID();
         TaskAttemptID reduceId = reduceTask.getTaskID();
-        Path filename = new Path("/" + TaskTracker.getIntermediateOutputDir(
-                                 reduceId.getJobID().toString(),
-                                 reduceId.toString()) 
-                                 + "/map_" +
-                                 loc.getTaskId().getId() + ".out");
-        
+        Path filename =
+            new Path(String.format(
+                MapOutputFile.REDUCE_INPUT_FILE_FORMAT_STRING,
+                TaskTracker.OUTPUT, loc.getTaskId().getId()));
+
         // Copy the map output to a temp file whose name is unique to this attempt 
         // Copy the map output to a temp file whose name is unique to this attempt 
         Path tmpMapOutput = new Path(filename+"-"+id);
         Path tmpMapOutput = new Path(filename+"-"+id);
         
         
@@ -2350,8 +2349,8 @@ class ReduceTask extends Task {
         if (numMemDiskSegments > 0 &&
         if (numMemDiskSegments > 0 &&
               ioSortFactor > mapOutputFilesOnDisk.size()) {
               ioSortFactor > mapOutputFilesOnDisk.size()) {
           // must spill to disk, but can't retain in-mem for intermediate merge
           // must spill to disk, but can't retain in-mem for intermediate merge
-          final Path outputPath = mapOutputFile.getInputFileForWrite(mapId,
-                            reduceTask.getTaskID(), inMemToDiskBytes);
+          final Path outputPath =
+              mapOutputFile.getInputFileForWrite(mapId, inMemToDiskBytes);
           final RawKeyValueIterator rIter = Merger.merge(job, fs,
           final RawKeyValueIterator rIter = Merger.merge(job, fs,
               keyClass, valueClass, memDiskSegments, numMemDiskSegments,
               keyClass, valueClass, memDiskSegments, numMemDiskSegments,
               tmpDir, comparator, reporter, spilledRecordsCounter, null);
               tmpDir, comparator, reporter, spilledRecordsCounter, null);
@@ -2649,8 +2648,8 @@ class ReduceTask extends Task {
         long mergeOutputSize = createInMemorySegments(inMemorySegments, 0);
         long mergeOutputSize = createInMemorySegments(inMemorySegments, 0);
         int noInMemorySegments = inMemorySegments.size();
         int noInMemorySegments = inMemorySegments.size();
 
 
-        Path outputPath = mapOutputFile.getInputFileForWrite(mapId, 
-                          reduceTask.getTaskID(), mergeOutputSize);
+        Path outputPath =
+            mapOutputFile.getInputFileForWrite(mapId, mergeOutputSize);
 
 
         Writer writer = 
         Writer writer = 
           new Writer(conf, rfs, outputPath,
           new Writer(conf, rfs, outputPath,

+ 2 - 2
src/mapred/org/apache/hadoop/mapred/ReduceTaskRunner.java

@@ -37,7 +37,7 @@ class ReduceTaskRunner extends TaskRunner {
     }
     }
     
     
     // cleanup from failures
     // cleanup from failures
-    mapOutputFile.removeAll(getTask().getTaskID());
+    mapOutputFile.removeAll();
     return true;
     return true;
   }
   }
   
   
@@ -46,6 +46,6 @@ class ReduceTaskRunner extends TaskRunner {
   public void close() throws IOException {
   public void close() throws IOException {
     LOG.info(getTask()+" done; removing files.");
     LOG.info(getTask()+" done; removing files.");
     getTask().getProgress().setStatus("closed");
     getTask().getProgress().setStatus("closed");
-    mapOutputFile.removeAll(getTask().getTaskID());
+    mapOutputFile.removeAll();
   }
   }
 }
 }

+ 0 - 2
src/mapred/org/apache/hadoop/mapred/Task.java

@@ -179,7 +179,6 @@ abstract public class Task implements Writable, Configurable {
                                                     TaskStatus.Phase.MAP : 
                                                     TaskStatus.Phase.MAP : 
                                                     TaskStatus.Phase.SHUFFLE, 
                                                     TaskStatus.Phase.SHUFFLE, 
                                                   counters);
                                                   counters);
-    this.mapOutputFile.setJobId(taskId.getJobID());
     spilledRecordsCounter = counters.findCounter(Counter.SPILLED_RECORDS);
     spilledRecordsCounter = counters.findCounter(Counter.SPILLED_RECORDS);
   }
   }
 
 
@@ -405,7 +404,6 @@ abstract public class Task implements Writable, Configurable {
     partition = in.readInt();
     partition = in.readInt();
     numSlotsRequired = in.readInt();
     numSlotsRequired = in.readInt();
     taskStatus.readFields(in);
     taskStatus.readFields(in);
-    this.mapOutputFile.setJobId(taskId.getJobID()); 
     skipRanges.readFields(in);
     skipRanges.readFields(in);
     currentRecIndexIterator = skipRanges.skipRangeIterator();
     currentRecIndexIterator = skipRanges.skipRangeIterator();
     currentRecStartIndex = currentRecIndexIterator.next();
     currentRecStartIndex = currentRecIndexIterator.next();

+ 111 - 43
src/mapred/org/apache/hadoop/mapred/TaskController.java

@@ -17,13 +17,17 @@
 */
 */
 package org.apache.hadoop.mapred;
 package org.apache.hadoop.mapred;
 
 
+import java.io.File;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.Map;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.mapred.JvmManager.JvmEnv;
 import org.apache.hadoop.mapred.JvmManager.JvmEnv;
+import org.apache.hadoop.mapred.TaskTracker.PermissionsHandler;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 
 
@@ -45,27 +49,95 @@ abstract class TaskController implements Configurable {
   public Configuration getConf() {
   public Configuration getConf() {
     return conf;
     return conf;
   }
   }
-  
+
+  // The list of directory paths specified in the variable mapred.local.dir.
+  // This is used to determine which among the list of directories is picked up
+  // for storing data for a particular task.
+  protected String[] mapredLocalDirs;
+
   public void setConf(Configuration conf) {
   public void setConf(Configuration conf) {
     this.conf = conf;
     this.conf = conf;
+    mapredLocalDirs = conf.getStrings("mapred.local.dir");
   }
   }
-  
+
   /**
   /**
-   * Setup task controller component.
+   * Sets up the permissions of the following directories on all the configured
+   * disks:
+   * <ul>
+   * <li>mapred-local directories</li>
+   * <li>Job cache directories</li>
+   * <li>Archive directories</li>
+   * <li>Hadoop log directories</li>
+   * </ul>
+   */
+  void setup() {
+    for (String localDir : this.mapredLocalDirs) {
+      // Set up the mapred-local directories.
+      File mapredlocalDir = new File(localDir);
+      if (!mapredlocalDir.exists() && !mapredlocalDir.mkdirs()) {
+        LOG.warn("Unable to create mapred-local directory : "
+            + mapredlocalDir.getPath());
+      } else {
+        PermissionsHandler.setPermissions(mapredlocalDir,
+            PermissionsHandler.sevenFiveFive);
+      }
+
+      // Set up the cache directory used for distributed cache files
+      File distributedCacheDir =
+          new File(localDir, TaskTracker.getDistributedCacheDir());
+      if (!distributedCacheDir.exists() && !distributedCacheDir.mkdirs()) {
+        LOG.warn("Unable to create cache directory : "
+            + distributedCacheDir.getPath());
+      } else {
+        PermissionsHandler.setPermissions(distributedCacheDir,
+            PermissionsHandler.sevenFiveFive);
+      }
+
+      // Set up the jobcache directory
+      File jobCacheDir = new File(localDir, TaskTracker.getJobCacheSubdir());
+      if (!jobCacheDir.exists() && !jobCacheDir.mkdirs()) {
+        LOG.warn("Unable to create job cache directory : "
+            + jobCacheDir.getPath());
+      } else {
+        PermissionsHandler.setPermissions(jobCacheDir,
+            PermissionsHandler.sevenFiveFive);
+      }
+    }
+
+    // Set up the user log directory
+    File taskLog = TaskLog.getUserLogDir();
+    if (!taskLog.exists() && !taskLog.mkdirs()) {
+      LOG.warn("Unable to create taskLog directory : " + taskLog.getPath());
+    } else {
+      PermissionsHandler.setPermissions(taskLog,
+          PermissionsHandler.sevenFiveFive);
+    }
+  }
+
+  /**
+   * Take task-controller specific actions to initialize job. This involves
+   * setting appropriate permissions to job-files so as to secure the files to
+   * be accessible only by the user's tasks.
    * 
    * 
+   * @throws IOException
    */
    */
-  abstract void setup();
-  
-  
+  abstract void initializeJob(JobInitializationContext context) throws IOException;
+
   /**
   /**
    * Launch a task JVM
    * Launch a task JVM
    * 
    * 
-   * This method defines how a JVM will be launched to run a task.
+   * This method defines how a JVM will be launched to run a task. Each
+   * task-controller should also do an
+   * {@link #initializeTask(TaskControllerContext)} inside this method so as to
+   * initialize the task before launching it. This is for reasons of
+   * task-controller specific optimizations w.r.t combining initialization and
+   * launching of tasks.
+   * 
    * @param context the context associated to the task
    * @param context the context associated to the task
    */
    */
   abstract void launchTaskJVM(TaskControllerContext context)
   abstract void launchTaskJVM(TaskControllerContext context)
                                       throws IOException;
                                       throws IOException;
-  
+
   /**
   /**
    * Top level cleanup a task JVM method.
    * Top level cleanup a task JVM method.
    *
    *
@@ -90,47 +162,44 @@ abstract class TaskController implements Configurable {
     }
     }
     killTask(context);
     killTask(context);
   }
   }
-  
-  /**
-   * Perform initializing actions required before a task can run.
-   * 
-   * For instance, this method can be used to setup appropriate
-   * access permissions for files and directories that will be
-   * used by tasks. Tasks use the job cache, log, PID and distributed cache
-   * directories and files as part of their functioning. Typically,
-   * these files are shared between the daemon and the tasks
-   * themselves. So, a TaskController that is launching tasks
-   * as different users can implement this method to setup
-   * appropriate ownership and permissions for these directories
-   * and files.
-   */
-  abstract void initializeTask(TaskControllerContext context);
-  
-  
+
+  /** Perform initializing actions required before a task can run.
+    * 
+    * For instance, this method can be used to setup appropriate
+    * access permissions for files and directories that will be
+    * used by tasks. Tasks use the job cache, log, and distributed cache
+    * directories and files as part of their functioning. Typically,
+    * these files are shared between the daemon and the tasks
+    * themselves. So, a TaskController that is launching tasks
+    * as different users can implement this method to setup
+    * appropriate ownership and permissions for these directories
+    * and files.
+    */
+  abstract void initializeTask(TaskControllerContext context)
+      throws IOException;
+
   /**
   /**
    * Contains task information required for the task controller.  
    * Contains task information required for the task controller.  
    */
    */
   static class TaskControllerContext {
   static class TaskControllerContext {
     // task being executed
     // task being executed
-    Task task; 
-    // the JVM environment for the task
-    JvmEnv env;
-    // the Shell executor executing the JVM for this task
-    ShellCommandExecutor shExec; 
-    // process handle of task JVM
-    String pid;
-    // waiting time before sending SIGKILL to task JVM after sending SIGTERM
-    long sleeptimeBeforeSigkill;
+    Task task;
+    ShellCommandExecutor shExec;     // the Shell executor executing the JVM for this task.
+
+    // Information used only when this context is used for launching new tasks.
+    JvmEnv env;     // the JVM environment for the task.
+
+    // Information used only when this context is used for destroying a task jvm.
+    String pid; // process handle of task JVM.
+    long sleeptimeBeforeSigkill; // waiting time before sending SIGKILL to task JVM after sending SIGTERM
+  }
+
+  static class JobInitializationContext {
+    JobID jobid;
+    File workDir;
+    String user;
   }
   }
 
 
-  /**
-   * Method which is called after the job is localized so that task controllers
-   * can implement their own job localization logic.
-   * 
-   * @param tip  Task of job for which localization happens.
-   */
-  abstract void initializeJob(JobID jobId);
-  
   /**
   /**
    * Sends a graceful terminate signal to taskJVM and it sub-processes. 
    * Sends a graceful terminate signal to taskJVM and it sub-processes. 
    *   
    *   
@@ -144,6 +213,5 @@ abstract class TaskController implements Configurable {
    * 
    * 
    * @param context task context
    * @param context task context
    */
    */
-  
   abstract void killTask(TaskControllerContext context);
   abstract void killTask(TaskControllerContext context);
 }
 }

+ 8 - 3
src/mapred/org/apache/hadoop/mapred/TaskLog.java

@@ -57,9 +57,10 @@ public class TaskLog {
   private static final Log LOG =
   private static final Log LOG =
     LogFactory.getLog(TaskLog.class);
     LogFactory.getLog(TaskLog.class);
 
 
+  static final String USERLOGS_DIR_NAME = "userlogs";
+
   private static final File LOG_DIR = 
   private static final File LOG_DIR = 
-    new File(System.getProperty("hadoop.log.dir"), 
-             "userlogs").getAbsoluteFile();
+    new File(getBaseLogDir(), USERLOGS_DIR_NAME).getAbsoluteFile();
   
   
   // localFS is set in (and used by) writeToIndexFile()
   // localFS is set in (and used by) writeToIndexFile()
   static LocalFileSystem localFS = null;
   static LocalFileSystem localFS = null;
@@ -178,7 +179,11 @@ public class TaskLog {
       return new File(getBaseDir(taskid), "log.index");
       return new File(getBaseDir(taskid), "log.index");
     }
     }
   }
   }
-  
+
+  static String getBaseLogDir() {
+    return System.getProperty("hadoop.log.dir");
+  }
+
   static File getBaseDir(String taskid) {
   static File getBaseDir(String taskid) {
     return new File(LOG_DIR, taskid);
     return new File(LOG_DIR, taskid);
   }
   }

+ 365 - 236
src/mapred/org/apache/hadoop/mapred/TaskRunner.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.TaskTracker.PermissionsHandler;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -78,7 +79,7 @@ abstract class TaskRunner extends Thread {
     this.t = tip.getTask();
     this.t = tip.getTask();
     this.tracker = tracker;
     this.tracker = tracker;
     this.conf = conf;
     this.conf = conf;
-    this.mapOutputFile = new MapOutputFile(t.getJobID());
+    this.mapOutputFile = new MapOutputFile();
     this.mapOutputFile.setConf(conf);
     this.mapOutputFile.setConf(conf);
     this.jvmManager = tracker.getJvmManagerInstance();
     this.jvmManager = tracker.getJvmManagerInstance();
   }
   }
@@ -125,223 +126,41 @@ abstract class TaskRunner extends Thread {
       
       
       URI[] archives = DistributedCache.getCacheArchives(conf);
       URI[] archives = DistributedCache.getCacheArchives(conf);
       URI[] files = DistributedCache.getCacheFiles(conf);
       URI[] files = DistributedCache.getCacheFiles(conf);
+      // We don't create any symlinks yet, so presence/absence of workDir
+      // actually on the file system doesn't matter.
       setupDistributedCache(lDirAlloc, workDir, archives, files);
       setupDistributedCache(lDirAlloc, workDir, archives, files);
-          
+      
+      // Set up the child task's configuration. After this call, no localization
+      // of files should happen in the TaskTracker's process space. Any changes to
+      // the conf object after this will NOT be reflected to the child.
+      setupChildTaskConfiguration(lDirAlloc);
+      
       if (!prepare()) {
       if (!prepare()) {
         return;
         return;
       }
       }
-
-      // Accumulates class paths for child.
-      List<String> classPaths = new ArrayList<String>();
-      // start with same classpath as parent process
-      appendSystemClasspaths(classPaths);
-
-      if (!workDir.mkdirs()) {
-        if (!workDir.isDirectory()) {
-          LOG.fatal("Mkdirs failed to create " + workDir.toString());
-        }
-      }
-
-      // include the user specified classpath
-      appendJobJarClasspaths(conf.getJar(), classPaths);
-  		
-      // Distributed cache paths
-      appendDistributedCacheClasspaths(conf, archives, files, classPaths);
-      
-      // Include the working dir too
-      classPaths.add(workDir.toString());
       
       
       // Build classpath
       // Build classpath
+      List<String> classPaths = getClassPaths(conf, workDir, archives, files);
       
       
+      long logSize = TaskLog.getTaskLogLength(conf);
       
       
       //  Build exec child JVM args.
       //  Build exec child JVM args.
-      Vector<String> vargs = new Vector<String>(8);
-      File jvm =                                  // use same jvm as parent
-        new File(new File(System.getProperty("java.home"), "bin"), "java");
-
-      vargs.add(jvm.toString());
-
-      // Add child (task) java-vm options.
-      //
-      // The following symbols if present in mapred.child.java.opts value are
-      // replaced:
-      // + @taskid@ is interpolated with value of TaskID.
-      // Other occurrences of @ will not be altered.
-      //
-      // Example with multiple arguments and substitutions, showing
-      // jvm GC logging, and start of a passwordless JVM JMX agent so can
-      // connect with jconsole and the likes to watch child memory, threads
-      // and get thread dumps.
-      //
-      //  <property>
-      //    <name>mapred.child.java.opts</name>
-      //    <value>-verbose:gc -Xloggc:/tmp/@taskid@.gc \
-      //           -Dcom.sun.management.jmxremote.authenticate=false \
-      //           -Dcom.sun.management.jmxremote.ssl=false \
-      //    </value>
-      //  </property>
-      //
-      String javaOpts = conf.get("mapred.child.java.opts", "-Xmx200m");
-      javaOpts = javaOpts.replace("@taskid@", taskid.toString());
-      String [] javaOptsSplit = javaOpts.split(" ");
-      
-      // Add java.library.path; necessary for loading native libraries.
-      //
-      // 1. To support native-hadoop library i.e. libhadoop.so, we add the 
-      //    parent processes' java.library.path to the child. 
-      // 2. We also add the 'cwd' of the task to it's java.library.path to help 
-      //    users distribute native libraries via the DistributedCache.
-      // 3. The user can also specify extra paths to be added to the 
-      //    java.library.path via mapred.child.java.opts.
-      //
-      String libraryPath = System.getProperty("java.library.path");
-      if (libraryPath == null) {
-        libraryPath = workDir.getAbsolutePath();
-      } else {
-        libraryPath += SYSTEM_PATH_SEPARATOR + workDir;
-      }
-      boolean hasUserLDPath = false;
-      for(int i=0; i<javaOptsSplit.length ;i++) { 
-        if(javaOptsSplit[i].startsWith("-Djava.library.path=")) {
-          javaOptsSplit[i] += SYSTEM_PATH_SEPARATOR + libraryPath;
-          hasUserLDPath = true;
-          break;
-        }
-      }
-      if(!hasUserLDPath) {
-        vargs.add("-Djava.library.path=" + libraryPath);
-      }
-      for (int i = 0; i < javaOptsSplit.length; i++) {
-        vargs.add(javaOptsSplit[i]);
-      }
-
-      // add java.io.tmpdir given by mapred.child.tmp
-      String tmp = conf.get("mapred.child.tmp", "./tmp");
-      Path tmpDir = new Path(tmp);
-      
-      // if temp directory path is not absolute 
-      // prepend it with workDir.
-      if (!tmpDir.isAbsolute()) {
-        tmpDir = new Path(workDir.toString(), tmp);
-      }
-      FileSystem localFs = FileSystem.getLocal(conf);
-      if (!localFs.mkdirs(tmpDir) && !localFs.getFileStatus(tmpDir).isDir()) {
-        throw new IOException("Mkdirs failed to create " + tmpDir.toString());
-      }
-      vargs.add("-Djava.io.tmpdir=" + tmpDir.toString());
-
-      // Add classpath.
-      vargs.add("-classpath");
-      String classPath = StringUtils.join(SYSTEM_PATH_SEPARATOR, classPaths);
-      vargs.add(classPath);
-
-      // Setup the log4j prop
-      long logSize = TaskLog.getTaskLogLength(conf);
-      vargs.add("-Dhadoop.log.dir=" + 
-          new File(System.getProperty("hadoop.log.dir")
-          ).getAbsolutePath());
-      vargs.add("-Dhadoop.root.logger=INFO,TLA");
-      vargs.add("-Dhadoop.tasklog.taskid=" + taskid);
-      vargs.add("-Dhadoop.tasklog.totalLogFileSize=" + logSize);
-
-      if (conf.getProfileEnabled()) {
-        if (conf.getProfileTaskRange(t.isMapTask()
-                                     ).isIncluded(t.getPartition())) {
-          File prof = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.PROFILE);
-          vargs.add(String.format(conf.getProfileParams(), prof.toString()));
-        }
-      }
-
-      // Add main class and its arguments 
-      vargs.add(Child.class.getName());  // main of Child
-      // pass umbilical address
-      InetSocketAddress address = tracker.getTaskTrackerReportAddress();
-      vargs.add(address.getAddress().getHostAddress()); 
-      vargs.add(Integer.toString(address.getPort())); 
-      vargs.add(taskid.toString());                      // pass task identifier
+      Vector<String> vargs = getVMArgs(taskid, workDir, classPaths, logSize);
       
       
       tracker.addToMemoryManager(t.getTaskID(), t.isMapTask(), conf);
       tracker.addToMemoryManager(t.getTaskID(), t.isMapTask(), conf);
 
 
       // set memory limit using ulimit if feasible and necessary ...
       // set memory limit using ulimit if feasible and necessary ...
-      String[] ulimitCmd = Shell.getUlimitMemoryCommand(conf);
-      List<String> setup = null;
-      if (ulimitCmd != null) {
-        setup = new ArrayList<String>();
-        for (String arg : ulimitCmd) {
-          setup.add(arg);
-        }
-      }
-
+      List<String> setup = getVMSetupCmd();
       // Set up the redirection of the task's stdout and stderr streams
       // Set up the redirection of the task's stdout and stderr streams
-      File stdout = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.STDOUT);
-      File stderr = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.STDERR);
-      stdout.getParentFile().mkdirs();
-      tracker.getTaskTrackerInstrumentation().reportTaskLaunch(taskid, stdout, stderr);
-
-      Map<String, String> env = new HashMap<String, String>();
-      StringBuffer ldLibraryPath = new StringBuffer();
-      ldLibraryPath.append(workDir.toString());
-      String oldLdLibraryPath = null;
-      oldLdLibraryPath = System.getenv("LD_LIBRARY_PATH");
-      if (oldLdLibraryPath != null) {
-        ldLibraryPath.append(SYSTEM_PATH_SEPARATOR);
-        ldLibraryPath.append(oldLdLibraryPath);
-      }
-      env.put("LD_LIBRARY_PATH", ldLibraryPath.toString());
-
-      String jobTokenFile = conf.get(JobContext.JOB_TOKEN_FILE);
-      LOG.debug("putting jobToken file name into environment fn=" + jobTokenFile);
-      env.put("JOB_TOKEN_FILE", jobTokenFile);
-
-      // for the child of task jvm, set hadoop.root.logger
-      env.put("HADOOP_ROOT_LOGGER","INFO,TLA");
-      String hadoopClientOpts = System.getenv("HADOOP_CLIENT_OPTS");
-      if (hadoopClientOpts == null) {
-        hadoopClientOpts = "";
-      } else {
-        hadoopClientOpts = hadoopClientOpts + " ";
-      }
-      hadoopClientOpts = hadoopClientOpts + "-Dhadoop.tasklog.taskid=" + taskid
-                         + " -Dhadoop.tasklog.totalLogFileSize=" + logSize;
-      env.put("HADOOP_CLIENT_OPTS", "\"" + hadoopClientOpts + "\"");
+      File[] logFiles = prepareLogFiles(taskid);
+      File stdout = logFiles[0];
+      File stderr = logFiles[1];
+      tracker.getTaskTrackerInstrumentation().reportTaskLaunch(taskid, stdout,
+                 stderr);
       
       
-      // add the env variables passed by the user
-      String mapredChildEnv = conf.get("mapred.child.env");
-      if (mapredChildEnv != null && mapredChildEnv.length() > 0) {
-        String childEnvs[] = mapredChildEnv.split(",");
-        for (String cEnv : childEnvs) {
-          try {
-            String[] parts = cEnv.split("="); // split on '='
-            String value = env.get(parts[0]);
-            if (value != null) {
-              // replace $env with the child's env constructed by tt's
-              // example LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp
-              value = parts[1].replace("$" + parts[0], value);
-            } else {
-              // this key is not configured by the tt for the child .. get it 
-              // from the tt's env
-              // example PATH=$PATH:/tmp
-              value = System.getenv(parts[0]);
-              if (value != null) {
-                // the env key is present in the tt's env
-                value = parts[1].replace("$" + parts[0], value);
-              } else {
-                // the env key is note present anywhere .. simply set it
-                // example X=$X:/tmp or X=/tmp
-                value = parts[1].replace("$" + parts[0], "");
-              }
-            }
-            env.put(parts[0], value);
-          } catch (Throwable t) {
-            // set the error msg
-            errorInfo = "Invalid User environment settings : " + mapredChildEnv 
-                        + ". Failed to parse user-passed environment param."
-                        + " Expecting : env1=value1,env2=value2...";
-            LOG.warn(errorInfo);
-            throw t;
-          }
-        }
-      }
+      Map<String, String> env = new HashMap<String, String>();
+      errorInfo = getVMEnvironment(errorInfo, workDir, conf, env, taskid,
+                                   logSize);
 
 
       jvmManager.launchJvm(this, 
       jvmManager.launchJvm(this, 
           jvmManager.constructJvmEnv(setup,vargs,stdout,stderr,logSize, 
           jvmManager.constructJvmEnv(setup,vargs,stdout,stderr,logSize, 
@@ -369,7 +188,7 @@ abstract class TaskRunner extends Thread {
         LOG.fatal(t.getTaskID()+" reporting FSError", ie);
         LOG.fatal(t.getTaskID()+" reporting FSError", ie);
       }
       }
     } catch (Throwable throwable) {
     } catch (Throwable throwable) {
-      LOG.warn(t.getTaskID() + errorInfo, throwable);
+      LOG.warn(t.getTaskID() + " : " + errorInfo, throwable);
       Throwable causeThrowable = new Throwable(errorInfo, throwable);
       Throwable causeThrowable = new Throwable(errorInfo, throwable);
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
       causeThrowable.printStackTrace(new PrintStream(baos));
       causeThrowable.printStackTrace(new PrintStream(baos));
@@ -404,15 +223,343 @@ abstract class TaskRunner extends Thread {
     }
     }
   }
   }
 
 
+  /**
+   * Prepare the log files for the task
+   * 
+   * @param taskid
+   * @return an array of files. The first file is stdout, the second is stderr.
+   */
+  static File[] prepareLogFiles(TaskAttemptID taskid) {
+    File[] logFiles = new File[2];
+    logFiles[0] = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.STDOUT);
+    logFiles[1] = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.STDERR);
+    File logDir = logFiles[0].getParentFile();
+    boolean b = logDir.mkdirs();
+    if (!b) {
+      LOG.warn("mkdirs failed. Ignoring");
+    } else {
+      PermissionsHandler.setPermissions(logDir,
+          PermissionsHandler.sevenZeroZero);
+    }
+    return logFiles;
+  }
+
+  /**
+   * Write the child's configuration to the disk and set it in configuration so
+   * that the child can pick it up from there.
+   * 
+   * @param lDirAlloc
+   * @throws IOException
+   */
+  void setupChildTaskConfiguration(LocalDirAllocator lDirAlloc)
+      throws IOException {
+
+    Path localTaskFile =
+        lDirAlloc.getLocalPathForWrite(TaskTracker.getTaskConfFile(t
+            .getJobID().toString(), t.getTaskID().toString(), t
+            .isTaskCleanupTask()), conf);
+
+    // write the child's task configuration file to the local disk
+    writeLocalTaskFile(localTaskFile.toString(), conf);
+
+    // Set the final job file in the task. The child needs to know the correct
+    // path to job.xml. So set this path accordingly.
+    t.setJobFile(localTaskFile.toString());
+  }
+
+  /**
+   * @return
+   */
+  private List<String> getVMSetupCmd() {
+    String[] ulimitCmd = Shell.getUlimitMemoryCommand(conf);
+    List<String> setup = null;
+    if (ulimitCmd != null) {
+      setup = new ArrayList<String>();
+      for (String arg : ulimitCmd) {
+        setup.add(arg);
+      }
+    }
+    return setup;
+  }
+
+  /**
+   * @param taskid
+   * @param workDir
+   * @param classPaths
+   * @param logSize
+   * @return
+   * @throws IOException
+   */
+  private Vector<String> getVMArgs(TaskAttemptID taskid, File workDir,
+      List<String> classPaths, long logSize)
+      throws IOException {
+    Vector<String> vargs = new Vector<String>(8);
+    File jvm =                                  // use same jvm as parent
+      new File(new File(System.getProperty("java.home"), "bin"), "java");
+
+    vargs.add(jvm.toString());
+
+    // Add child (task) java-vm options.
+    //
+    // The following symbols if present in mapred.child.java.opts value are
+    // replaced:
+    // + @taskid@ is interpolated with value of TaskID.
+    // Other occurrences of @ will not be altered.
+    //
+    // Example with multiple arguments and substitutions, showing
+    // jvm GC logging, and start of a passwordless JVM JMX agent so can
+    // connect with jconsole and the likes to watch child memory, threads
+    // and get thread dumps.
+    //
+    //  <property>
+    //    <name>mapred.child.java.opts</name>
+    //    <value>-verbose:gc -Xloggc:/tmp/@taskid@.gc \
+    //           -Dcom.sun.management.jmxremote.authenticate=false \
+    //           -Dcom.sun.management.jmxremote.ssl=false \
+    //    </value>
+    //  </property>
+    //
+    String javaOpts = conf.get("mapred.child.java.opts", "-Xmx200m");
+    javaOpts = javaOpts.replace("@taskid@", taskid.toString());
+    String [] javaOptsSplit = javaOpts.split(" ");
+    
+    // Add java.library.path; necessary for loading native libraries.
+    //
+    // 1. To support native-hadoop library i.e. libhadoop.so, we add the 
+    //    parent processes' java.library.path to the child. 
+    // 2. We also add the 'cwd' of the task to it's java.library.path to help 
+    //    users distribute native libraries via the DistributedCache.
+    // 3. The user can also specify extra paths to be added to the 
+    //    java.library.path via mapred.child.java.opts.
+    //
+    String libraryPath = System.getProperty("java.library.path");
+    if (libraryPath == null) {
+      libraryPath = workDir.getAbsolutePath();
+    } else {
+      libraryPath += SYSTEM_PATH_SEPARATOR + workDir;
+    }
+    boolean hasUserLDPath = false;
+    for(int i=0; i<javaOptsSplit.length ;i++) { 
+      if(javaOptsSplit[i].startsWith("-Djava.library.path=")) {
+        javaOptsSplit[i] += SYSTEM_PATH_SEPARATOR + libraryPath;
+        hasUserLDPath = true;
+        break;
+      }
+    }
+    if(!hasUserLDPath) {
+      vargs.add("-Djava.library.path=" + libraryPath);
+    }
+    for (int i = 0; i < javaOptsSplit.length; i++) {
+      vargs.add(javaOptsSplit[i]);
+    }
+
+    Path childTmpDir = createChildTmpDir(workDir, conf);
+    vargs.add("-Djava.io.tmpdir=" + childTmpDir);
+
+    // Add classpath.
+    vargs.add("-classpath");
+    String classPath = StringUtils.join(SYSTEM_PATH_SEPARATOR, classPaths);
+    vargs.add(classPath);
+
+    // Setup the log4j prop
+    vargs.add("-Dhadoop.log.dir=" + 
+        new File(System.getProperty("hadoop.log.dir")
+        ).getAbsolutePath());
+    vargs.add("-Dhadoop.root.logger=INFO,TLA");
+    vargs.add("-Dhadoop.tasklog.taskid=" + taskid);
+    vargs.add("-Dhadoop.tasklog.totalLogFileSize=" + logSize);
+
+    if (conf.getProfileEnabled()) {
+      if (conf.getProfileTaskRange(t.isMapTask()
+                                   ).isIncluded(t.getPartition())) {
+        File prof = TaskLog.getTaskLogFile(taskid, TaskLog.LogName.PROFILE);
+        vargs.add(String.format(conf.getProfileParams(), prof.toString()));
+      }
+    }
+
+    // Add main class and its arguments 
+    vargs.add(Child.class.getName());  // main of Child
+    // pass umbilical address
+    InetSocketAddress address = tracker.getTaskTrackerReportAddress();
+    vargs.add(address.getAddress().getHostAddress()); 
+    vargs.add(Integer.toString(address.getPort())); 
+    vargs.add(taskid.toString());                      // pass task identifier
+    return vargs;
+  }
+
+  /**
+   * @param taskid
+   * @param workDir
+   * @return
+   * @throws IOException
+   */
+  static Path createChildTmpDir(File workDir,
+      JobConf conf)
+      throws IOException {
+
+    // add java.io.tmpdir given by mapred.child.tmp
+    String tmp = conf.get("mapred.child.tmp", "./tmp");
+    Path tmpDir = new Path(tmp);
+
+    // if temp directory path is not absolute, prepend it with workDir.
+    if (!tmpDir.isAbsolute()) {
+      tmpDir = new Path(workDir.toString(), tmp);
+
+      FileSystem localFs = FileSystem.getLocal(conf);
+      if (!localFs.mkdirs(tmpDir) && !localFs.getFileStatus(tmpDir).isDir()) {
+        throw new IOException("Mkdirs failed to create " + tmpDir.toString());
+      }
+    }
+    return tmpDir;
+  }
+
+  /**
+   */
+  private static List<String> getClassPaths(JobConf conf, File workDir,
+      URI[] archives, URI[] files)
+      throws IOException {
+    // Accumulates class paths for child.
+    List<String> classPaths = new ArrayList<String>();
+    // start with same classpath as parent process
+    appendSystemClasspaths(classPaths);
+
+    // include the user specified classpath
+    appendJobJarClasspaths(conf.getJar(), classPaths);
+    
+    // Distributed cache paths
+    appendDistributedCacheClasspaths(conf, archives, files, classPaths);
+    
+    // Include the working dir too
+    classPaths.add(workDir.toString());
+    return classPaths;
+  }
+
+  /**
+   * @param errorInfo
+   * @param workDir
+   * @param env
+   * @return
+   * @throws Throwable
+   */
+  private static String getVMEnvironment(String errorInfo, File workDir, JobConf conf,
+      Map<String, String> env, TaskAttemptID taskid, long logSize)
+      throws Throwable {
+    StringBuffer ldLibraryPath = new StringBuffer();
+    ldLibraryPath.append(workDir.toString());
+    String oldLdLibraryPath = null;
+    oldLdLibraryPath = System.getenv("LD_LIBRARY_PATH");
+    if (oldLdLibraryPath != null) {
+      ldLibraryPath.append(SYSTEM_PATH_SEPARATOR);
+      ldLibraryPath.append(oldLdLibraryPath);
+    }
+    env.put("LD_LIBRARY_PATH", ldLibraryPath.toString());
+
+    String jobTokenFile = conf.get(JobContext.JOB_TOKEN_FILE);
+    LOG.debug("putting jobToken file name into environment fn=" + jobTokenFile);
+    env.put("JOB_TOKEN_FILE", jobTokenFile);
+    
+    // for the child of task jvm, set hadoop.root.logger
+    env.put("HADOOP_ROOT_LOGGER","INFO,TLA");
+    String hadoopClientOpts = System.getenv("HADOOP_CLIENT_OPTS");
+    if (hadoopClientOpts == null) {
+      hadoopClientOpts = "";
+    } else {
+      hadoopClientOpts = hadoopClientOpts + " ";
+    }
+    hadoopClientOpts = hadoopClientOpts + "-Dhadoop.tasklog.taskid=" + taskid
+                       + " -Dhadoop.tasklog.totalLogFileSize=" + logSize;
+    env.put("HADOOP_CLIENT_OPTS", "\"" + hadoopClientOpts + "\"");
+
+    // add the env variables passed by the user
+    String mapredChildEnv = conf.get("mapred.child.env");
+    if (mapredChildEnv != null && mapredChildEnv.length() > 0) {
+      String childEnvs[] = mapredChildEnv.split(",");
+      for (String cEnv : childEnvs) {
+        try {
+          String[] parts = cEnv.split("="); // split on '='
+          String value = env.get(parts[0]);
+          if (value != null) {
+            // replace $env with the child's env constructed by tt's
+            // example LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp
+            value = parts[1].replace("$" + parts[0], value);
+          } else {
+            // this key is not configured by the tt for the child .. get it 
+            // from the tt's env
+            // example PATH=$PATH:/tmp
+            value = System.getenv(parts[0]);
+            if (value != null) {
+              // the env key is present in the tt's env
+              value = parts[1].replace("$" + parts[0], value);
+            } else {
+              // the env key is note present anywhere .. simply set it
+              // example X=$X:/tmp or X=/tmp
+              value = parts[1].replace("$" + parts[0], "");
+            }
+          }
+          env.put(parts[0], value);
+        } catch (Throwable t) {
+          // set the error msg
+          errorInfo = "Invalid User environment settings : " + mapredChildEnv 
+                      + ". Failed to parse user-passed environment param."
+                      + " Expecting : env1=value1,env2=value2...";
+          LOG.warn(errorInfo);
+          throw t;
+        }
+      }
+    }
+    return errorInfo;
+  }
+
+  /**
+   * Write the task specific job-configuration file.
+   * 
+   * @param localFs
+   * @throws IOException
+   */
+  private static void writeLocalTaskFile(String jobFile, JobConf conf)
+      throws IOException {
+    Path localTaskFile = new Path(jobFile);
+    FileSystem localFs = FileSystem.getLocal(conf);
+    localFs.delete(localTaskFile, true);
+    OutputStream out = localFs.create(localTaskFile);
+    try {
+      conf.writeXml(out);
+    } finally {
+      out.close();
+    }
+  }
+
+  /**
+   * Prepare the mapred.local.dir for the child. The child is sand-boxed now.
+   * Whenever it uses LocalDirAllocator from now on inside the child, it will
+   * only see files inside the attempt-directory. This is done in the Child's
+   * process space.
+   */
+  static void setupChildMapredLocalDirs(Task t, JobConf conf) {
+    String[] localDirs = conf.getStrings("mapred.local.dir");
+    String jobId = t.getJobID().toString();
+    String taskId = t.getTaskID().toString();
+    boolean isCleanup = t.isTaskCleanupTask();
+    StringBuffer childMapredLocalDir =
+        new StringBuffer(localDirs[0] + Path.SEPARATOR
+            + TaskTracker.getLocalTaskDir(jobId, taskId, isCleanup));
+    for (int i = 1; i < localDirs.length; i++) {
+      childMapredLocalDir.append("," + localDirs[i] + Path.SEPARATOR
+          + TaskTracker.getLocalTaskDir(jobId, taskId, isCleanup));
+    }
+    LOG.debug("mapred.local.dir for child : " + childMapredLocalDir);
+    conf.set("mapred.local.dir", childMapredLocalDir.toString());
+  }
+
   /** Creates the working directory pathname for a task attempt. */ 
   /** Creates the working directory pathname for a task attempt. */ 
   static File formWorkDir(LocalDirAllocator lDirAlloc, 
   static File formWorkDir(LocalDirAllocator lDirAlloc, 
       TaskAttemptID task, boolean isCleanup, JobConf conf) 
       TaskAttemptID task, boolean isCleanup, JobConf conf) 
       throws IOException {
       throws IOException {
-    File workDir = new File(lDirAlloc.getLocalPathToRead(
-        TaskTracker.getLocalTaskDir(task.getJobID().toString(), 
-          task.toString(), isCleanup) 
-        + Path.SEPARATOR + MRConstants.WORKDIR, conf).toString());
-    return workDir;
+    Path workDir =
+        lDirAlloc.getLocalPathToRead(TaskTracker.getTaskWorkDir(task
+            .getJobID().toString(), task.toString(), isCleanup), conf);
+
+    return new File(workDir.toString());
   }
   }
 
 
   private void setupDistributedCache(LocalDirAllocator lDirAlloc, File workDir,
   private void setupDistributedCache(LocalDirAllocator lDirAlloc, File workDir,
@@ -431,7 +578,7 @@ abstract class TaskRunner extends Thread {
           fileStatus = fileSystem.getFileStatus(
           fileStatus = fileSystem.getFileStatus(
                                     new Path(archives[i].getPath()));
                                     new Path(archives[i].getPath()));
           String cacheId = DistributedCache.makeRelative(archives[i],conf);
           String cacheId = DistributedCache.makeRelative(archives[i],conf);
-          String cachePath = TaskTracker.getCacheSubdir() + 
+          String cachePath = TaskTracker.getDistributedCacheDir() + 
                                Path.SEPARATOR + cacheId;
                                Path.SEPARATOR + cacheId;
           
           
           localPath = lDirAlloc.getLocalPathForWrite(cachePath,
           localPath = lDirAlloc.getLocalPathForWrite(cachePath,
@@ -457,7 +604,7 @@ abstract class TaskRunner extends Thread {
           fileStatus = fileSystem.getFileStatus(
           fileStatus = fileSystem.getFileStatus(
                                     new Path(files[i].getPath()));
                                     new Path(files[i].getPath()));
           String cacheId = DistributedCache.makeRelative(files[i], conf);
           String cacheId = DistributedCache.makeRelative(files[i], conf);
-          String cachePath = TaskTracker.getCacheSubdir() +
+          String cachePath = TaskTracker.getDistributedCacheDir() +
                                Path.SEPARATOR + cacheId;
                                Path.SEPARATOR + cacheId;
           
           
           localPath = lDirAlloc.getLocalPathForWrite(cachePath,
           localPath = lDirAlloc.getLocalPathForWrite(cachePath,
@@ -474,20 +621,12 @@ abstract class TaskRunner extends Thread {
         }
         }
         DistributedCache.setLocalFiles(conf, stringifyPathArray(p));
         DistributedCache.setLocalFiles(conf, stringifyPathArray(p));
       }
       }
-      Path localTaskFile = new Path(t.getJobFile());
-      FileSystem localFs = FileSystem.getLocal(conf);
-      localFs.delete(localTaskFile, true);
-      OutputStream out = localFs.create(localTaskFile);
-      try {
-        conf.writeXml(out);
-      } finally {
-        out.close();
-      }
     }
     }
   }
   }
 
 
-  private void appendDistributedCacheClasspaths(JobConf conf, URI[] archives, 
-      URI[] files, List<String> classPaths) throws IOException {
+  private static void appendDistributedCacheClasspaths(JobConf conf,
+      URI[] archives, URI[] files, List<String> classPaths)
+      throws IOException {
     // Archive paths
     // Archive paths
     Path[] archiveClasspaths = DistributedCache.getArchiveClassPaths(conf);
     Path[] archiveClasspaths = DistributedCache.getArchiveClassPaths(conf);
     if (archiveClasspaths != null && archives != null) {
     if (archiveClasspaths != null && archives != null) {
@@ -522,8 +661,9 @@ abstract class TaskRunner extends Thread {
     }
     }
   }
   }
 
 
-  private void appendSystemClasspaths(List<String> classPaths) {
-    for (String c : System.getProperty("java.class.path").split(SYSTEM_PATH_SEPARATOR)) {
+  private static void appendSystemClasspaths(List<String> classPaths) {
+    for (String c : System.getProperty("java.class.path").split(
+        SYSTEM_PATH_SEPARATOR)) {
       classPaths.add(c);
       classPaths.add(c);
     }
     }
   }
   }
@@ -605,19 +745,8 @@ abstract class TaskRunner extends Thread {
       // Do not exit even if symlinks have not been created.
       // Do not exit even if symlinks have not been created.
       LOG.warn(StringUtils.stringifyException(ie));
       LOG.warn(StringUtils.stringifyException(ie));
     }
     }
-    // add java.io.tmpdir given by mapred.child.tmp
-    String tmp = conf.get("mapred.child.tmp", "./tmp");
-    Path tmpDir = new Path(tmp);
 
 
-    // if temp directory path is not absolute
-    // prepend it with workDir.
-    if (!tmpDir.isAbsolute()) {
-      tmpDir = new Path(workDir.toString(), tmp);
-      FileSystem localFs = FileSystem.getLocal(conf);
-      if (!localFs.mkdirs(tmpDir) && !localFs.getFileStatus(tmpDir).isDir()){
-        throw new IOException("Mkdirs failed to create " + tmpDir.toString());
-      }
-    }
+    createChildTmpDir(workDir, conf);
   }
   }
 
 
   /**
   /**

+ 428 - 163
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -67,6 +67,7 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.mapred.TaskController.JobInitializationContext;
 import org.apache.hadoop.mapred.TaskLog.LogFileDetail;
 import org.apache.hadoop.mapred.TaskLog.LogFileDetail;
 import org.apache.hadoop.mapred.TaskLog.LogName;
 import org.apache.hadoop.mapred.TaskLog.LogName;
 import org.apache.hadoop.mapred.TaskStatus.Phase;
 import org.apache.hadoop.mapred.TaskStatus.Phase;
@@ -219,13 +220,19 @@ public class TaskTracker
   //for serving map output to the other nodes
   //for serving map output to the other nodes
 
 
   static Random r = new Random();
   static Random r = new Random();
-  private static final String SUBDIR = "taskTracker";
-  private static final String CACHEDIR = "archive";
-  private static final String JOBCACHE = "jobcache";
-  private static final String OUTPUT = "output";
+  static final String SUBDIR = "taskTracker";
+  private static final String DISTCACHEDIR = "distcache";
+  static final String JOBCACHE = "jobcache";
+  static final String OUTPUT = "output";
+  private static final String JARSDIR = "jars";
+  static final String LOCAL_SPLIT_FILE = "split.info";
+  static final String JOBFILE = "job.xml";
+
+  static final String JOB_LOCAL_DIR = "job.local.dir";
   static final String JOB_TOKEN_FILE="jobToken"; //localized file
   static final String JOB_TOKEN_FILE="jobToken"; //localized file
-  private JobConf originalConf;
+
   private JobConf fConf;
   private JobConf fConf;
+  private JobConf originalConf;
   private int maxMapSlots;
   private int maxMapSlots;
   private int maxReduceSlots;
   private int maxReduceSlots;
   private int failures;
   private int failures;
@@ -435,8 +442,8 @@ public class TaskTracker
     return TaskTracker.SUBDIR + Path.SEPARATOR + user;
     return TaskTracker.SUBDIR + Path.SEPARATOR + user;
   } 
   } 
 
 
-  static String getCacheSubdir() {
-    return TaskTracker.SUBDIR + Path.SEPARATOR + TaskTracker.CACHEDIR;
+  static String getDistributedCacheDir() {
+    return TaskTracker.SUBDIR + Path.SEPARATOR + TaskTracker.DISTCACHEDIR;
   }
   }
 
 
   static String getJobCacheSubdir() {
   static String getJobCacheSubdir() {
@@ -449,31 +456,66 @@ public class TaskTracker
   } 
   } 
 
 
   static String getLocalJobDir(String jobid) {
   static String getLocalJobDir(String jobid) {
-	return getJobCacheSubdir() + Path.SEPARATOR + jobid; 
+    return getJobCacheSubdir() + Path.SEPARATOR + jobid;
   }
   }
 
 
-  static String getLocalTaskDir(String jobid, String taskid) {
-	return getLocalTaskDir(jobid, taskid, false) ; 
+  static String getLocalJobConfFile(String jobid) {
+    return getLocalJobDir(jobid) + Path.SEPARATOR + TaskTracker.JOBFILE;
   }
   }
 
 
-  static String getIntermediateOutputDir(String jobid, String taskid) {
-	return getLocalTaskDir(jobid, taskid) 
-           + Path.SEPARATOR + TaskTracker.OUTPUT ; 
+  static String getTaskConfFile(String jobid, String taskid,
+      boolean isCleanupAttempt) {
+    return getLocalTaskDir(jobid, taskid, isCleanupAttempt) + Path.SEPARATOR
+        + TaskTracker.JOBFILE;
   }
   }
-  
-  static String getLocalJobTokenFile(String user, String jobid) {
-    return getLocalJobDir(user, jobid) + Path.SEPARATOR + TaskTracker.JOB_TOKEN_FILE;
+
+  static String getJobJarsDir(String jobid) {
+    return getLocalJobDir(jobid) + Path.SEPARATOR + TaskTracker.JARSDIR;
+  }
+
+  static String getJobJarFile(String jobid) {
+    return getJobJarsDir(jobid) + Path.SEPARATOR + "job.jar";
   }
   }
 
 
+  static String getJobWorkDir(String jobid) {
+    return getLocalJobDir(jobid) + Path.SEPARATOR + MRConstants.WORKDIR;
+  }
+
+  static String getLocalSplitFile(String jobid, String taskid) {
+    return TaskTracker.getLocalTaskDir(jobid, taskid) + Path.SEPARATOR
+    + TaskTracker.LOCAL_SPLIT_FILE;
+  }
+
+  static String getIntermediateOutputDir(String jobid, String taskid) {
+    return getLocalTaskDir(jobid, taskid) + Path.SEPARATOR
+        + TaskTracker.OUTPUT;
+  }
 
 
-  static String getLocalTaskDir(String jobid, 
-                                String taskid, 
-                                boolean isCleanupAttempt) {
-	String taskDir = getLocalJobDir(jobid) + Path.SEPARATOR + taskid;
-	if (isCleanupAttempt) { 
+  static String getLocalTaskDir(String jobid, String taskid) {
+    return getLocalTaskDir(jobid, taskid, false);
+  }
+
+  static String getLocalTaskDir(String jobid, String taskid,
+      boolean isCleanupAttempt) {
+    String taskDir = getLocalJobDir(jobid) + Path.SEPARATOR + taskid;
+    if (isCleanupAttempt) {
       taskDir = taskDir + TASK_CLEANUP_SUFFIX;
       taskDir = taskDir + TASK_CLEANUP_SUFFIX;
-	}
-	return taskDir;
+    }
+    return taskDir;
+  }
+
+  static String getTaskWorkDir(String jobid, String taskid,
+      boolean isCleanupAttempt) {
+    String dir =
+      getLocalJobDir(jobid) + Path.SEPARATOR + taskid;
+    if (isCleanupAttempt) {
+      dir = dir + TASK_CLEANUP_SUFFIX;
+    }
+    return dir + Path.SEPARATOR + MRConstants.WORKDIR;
+  }
+
+  static String getLocalJobTokenFile(String user, String jobid) {
+    return getLocalJobDir(user, jobid) + Path.SEPARATOR + TaskTracker.JOB_TOKEN_FILE;
   }
   }
 
 
   private void setUgi(String user, Configuration conf) {
   private void setUgi(String user, Configuration conf) {
@@ -841,92 +883,25 @@ public class TaskTracker
     Path localJarFile = null;
     Path localJarFile = null;
     Task t = tip.getTask();
     Task t = tip.getTask();
     JobID jobId = t.getJobID();
     JobID jobId = t.getJobID();
-    Path jobFile = new Path(t.getJobFile());
-    String userName = t.getUser();
-    JobConf userConf = new JobConf(getJobConf());
-    setUgi(userName, userConf);
-    FileSystem userFs = jobFile.getFileSystem(userConf);
-    // Get sizes of JobFile and JarFile
-    // sizes are -1 if they are not present.
-    FileStatus status = null;
-    long jobFileSize = -1;
-    try {
-      status = userFs.getFileStatus(jobFile);
-      jobFileSize = status.getLen();
-    } catch(FileNotFoundException fe) {
-      jobFileSize = -1;
-    }
-    Path localJobFile = lDirAlloc.getLocalPathForWrite(
-                                    getLocalJobDir(jobId.toString())
-                                    + Path.SEPARATOR + "job.xml",
-                                    jobFileSize, fConf);
+
     RunningJob rjob = addTaskToJob(jobId, tip);
     RunningJob rjob = addTaskToJob(jobId, tip);
     synchronized (rjob) {
     synchronized (rjob) {
       if (!rjob.localized) {
       if (!rjob.localized) {
-  
-        FileSystem localFs = FileSystem.getLocal(fConf);
-        // this will happen on a partial execution of localizeJob.
-        // Sometimes the job.xml gets copied but copying job.jar
-        // might throw out an exception
-        // we should clean up and then try again
-        Path jobDir = localJobFile.getParent();
-        if (localFs.exists(jobDir)){
-          localFs.delete(jobDir, true);
-          boolean b = localFs.mkdirs(jobDir);
-          if (!b)
-            throw new IOException("Not able to create job directory "
-                                  + jobDir.toString());
-        }
-        userFs.copyToLocalFile(jobFile, localJobFile);
-        JobConf localJobConf = new JobConf(localJobFile);
+        JobConf localJobConf = localizeJobFiles(t);
         
         
-        // create the 'work' directory
-        // job-specific shared directory for use as scratch space 
-        Path workDir = lDirAlloc.getLocalPathForWrite(
-                         (getLocalJobDir(jobId.toString())
-                         + Path.SEPARATOR + MRConstants.WORKDIR), fConf);
-        if (!localFs.mkdirs(workDir)) {
-          throw new IOException("Mkdirs failed to create " 
-                      + workDir.toString());
-        }
-        System.setProperty("job.local.dir", workDir.toString());
-        localJobConf.set("job.local.dir", workDir.toString());
+        // Now initialize the job via task-controller so as to set
+        // ownership/permissions of jars, job-work-dir. Note that initializeJob
+        // should be the last call after every other directory/file to be
+        // directly under the job directory is created.
+        JobInitializationContext context = new JobInitializationContext();
+        context.jobid = jobId;
+        context.user = localJobConf.getUser();
+        context.workDir = new File(localJobConf.get(JOB_LOCAL_DIR));
+        taskController.initializeJob(context);
         
         
-        // copy Jar file to the local FS and unjar it.
-        String jarFile = localJobConf.getJar();
-        long jarFileSize = -1;
-        if (jarFile != null) {
-          Path jarFilePath = new Path(jarFile);
-          try {
-            status = userFs.getFileStatus(jarFilePath);
-            jarFileSize = status.getLen();
-          } catch(FileNotFoundException fe) {
-            jarFileSize = -1;
-          }
-          // Here we check for and we check five times the size of jarFileSize
-          // to accommodate for unjarring the jar file in work directory 
-          localJarFile = new Path(lDirAlloc.getLocalPathForWrite(
-                                     getLocalJobDir(jobId.toString())
-                                     + Path.SEPARATOR + "jars",
-                                     5 * jarFileSize, fConf), "job.jar");
-          if (!localFs.mkdirs(localJarFile.getParent())) {
-            throw new IOException("Mkdirs failed to create jars directory "); 
-          }
-          userFs.copyToLocalFile(jarFilePath, localJarFile);
-          localJobConf.setJar(localJarFile.toString());
-          OutputStream out = localFs.create(localJobFile);
-          try {
-            localJobConf.writeXml(out);
-          } finally {
-            out.close();
-          }
-          // also unjar the job.jar files 
-          RunJar.unJar(new File(localJarFile.toString()),
-                       new File(localJarFile.getParent().toString()));
-        }
+        rjob.jobConf = localJobConf;  
         rjob.keepJobFiles = ((localJobConf.getKeepTaskFilesPattern() != null) ||
         rjob.keepJobFiles = ((localJobConf.getKeepTaskFilesPattern() != null) ||
                              localJobConf.getKeepFailedTaskFiles());
                              localJobConf.getKeepFailedTaskFiles());
-        rjob.jobConf = localJobConf;
         // save local copy of JobToken file
         // save local copy of JobToken file
         localizeJobTokenFile(t.getUser(), jobId, localJobConf);       
         localizeJobTokenFile(t.getUser(), jobId, localJobConf);       
         FSDataInputStream in = localFs.open(new Path(
         FSDataInputStream in = localFs.open(new Path(
@@ -936,13 +911,319 @@ public class TaskTracker
         getJobTokenSecretManager().addTokenForJob(jobId.toString(), jt);
         getJobTokenSecretManager().addTokenForJob(jobId.toString(), jt);
  
  
         rjob.localized = true;
         rjob.localized = true;
-        taskController.initializeJob(jobId);
       }
       }
     }
     }
     launchTaskForJob(tip, new JobConf(rjob.jobConf)); 
     launchTaskForJob(tip, new JobConf(rjob.jobConf)); 
   }
   }
 
 
-  private void launchTaskForJob(TaskInProgress tip, JobConf jobConf) throws IOException{
+  /**
+   * Localize the job on this tasktracker. Specifically
+   * <ul>
+   * <li>Cleanup and create job directories on all disks</li>
+   * <li>Download the job config file job.xml from the FS</li>
+   * <li>Create the job work directory and set {@link TaskTracker#JOB_LOCAL_DIR}
+   * in the configuration.
+   * <li>Download the job jar file job.jar from the FS, unjar it and set jar
+   * file in the configuration.</li>
+   * </ul>
+   *
+   * @param t task whose job has to be localized on this TT
+   * @return the modified job configuration to be used for all the tasks of this
+   *         job as a starting point.
+   * @throws IOException
+   */
+  JobConf localizeJobFiles(Task t)
+      throws IOException {
+    JobID jobId = t.getJobID();
+
+    Path jobFile = new Path(t.getJobFile());
+    String userName = t.getUser();
+    JobConf userConf = new JobConf(getJobConf());
+    setUgi(userName, userConf);
+    FileSystem userFs = jobFile.getFileSystem(userConf);
+
+    // Initialize the job directories first
+    FileSystem localFs = FileSystem.getLocal(fConf);
+    initializeJobDirs(jobId, localFs, fConf.getStrings("mapred.local.dir"));
+
+    // Download the job.xml for this job from the system FS
+    Path localJobFile =
+        localizeJobConfFile(new Path(t.getJobFile()), userFs, jobId);
+
+    JobConf localJobConf = new JobConf(localJobFile);
+
+    // create the 'job-work' directory: job-specific shared directory for use as
+    // scratch space by all tasks of the same job running on this TaskTracker.
+    Path workDir =
+        lDirAlloc.getLocalPathForWrite(getJobWorkDir(jobId.toString()),
+          fConf);
+    if (!localFs.mkdirs(workDir)) {
+      throw new IOException("Mkdirs failed to create "
+          + workDir.toString());
+    }
+    System.setProperty(JOB_LOCAL_DIR, workDir.toUri().getPath());
+    localJobConf.set(JOB_LOCAL_DIR, workDir.toUri().getPath());
+
+    // Download the job.jar for this job from the system FS
+    localizeJobJarFile(jobId, userFs, localJobConf);
+
+    return localJobConf;
+  }
+
+  static class PermissionsHandler {
+    /**
+     * Permission information useful for setting permissions for a given path.
+     * Using this, one can set all possible combinations of permissions for the
+     * owner of the file. But permissions for the group and all others can only
+     * be set together, i.e. permissions for group cannot be set different from
+     * those for others and vice versa.
+     */
+    static class PermissionsInfo {
+      public boolean readPermissions;
+      public boolean writePermissions;
+      public boolean executablePermissions;
+      public boolean readPermsOwnerOnly;
+      public boolean writePermsOwnerOnly;
+      public boolean executePermsOwnerOnly;
+
+      /**
+       * Create a permissions-info object with the given attributes
+       *
+       * @param readPerms
+       * @param writePerms
+       * @param executePerms
+       * @param readOwnerOnly
+       * @param writeOwnerOnly
+       * @param executeOwnerOnly
+       */
+      public PermissionsInfo(boolean readPerms, boolean writePerms,
+          boolean executePerms, boolean readOwnerOnly, boolean writeOwnerOnly,
+          boolean executeOwnerOnly) {
+        readPermissions = readPerms;
+        writePermissions = writePerms;
+        executablePermissions = executePerms;
+        readPermsOwnerOnly = readOwnerOnly;
+        writePermsOwnerOnly = writeOwnerOnly;
+        executePermsOwnerOnly = executeOwnerOnly;
+      }
+    }
+
+    /**
+     * Set permission on the given file path using the specified permissions
+     * information. We use java api to set permission instead of spawning chmod
+     * processes. This saves a lot of time. Using this, one can set all possible
+     * combinations of permissions for the owner of the file. But permissions
+     * for the group and all others can only be set together, i.e. permissions
+     * for group cannot be set different from those for others and vice versa.
+     *
+     * This method should satisfy the needs of most of the applications. For
+     * those it doesn't, {@link FileUtil#chmod} can be used.
+     *
+     * @param f file path
+     * @param pInfo permissions information
+     * @return true if success, false otherwise
+     */
+    static boolean setPermissions(File f, PermissionsInfo pInfo) {
+      if (pInfo == null) {
+        LOG.debug(" PermissionsInfo is null, returning.");
+        return true;
+      }
+
+      LOG.debug("Setting permission for " + f.getAbsolutePath());
+
+      boolean ret = true;
+
+      // Clear all the flags
+      ret = f.setReadable(false, false) && ret;
+      ret = f.setWritable(false, false) && ret;
+      ret = f.setExecutable(false, false) && ret;
+
+      ret = f.setReadable(pInfo.readPermissions, pInfo.readPermsOwnerOnly);
+      LOG.debug("Readable status for " + f + " set to " + ret);
+      ret =
+        f.setWritable(pInfo.writePermissions, pInfo.writePermsOwnerOnly)
+        && ret;
+      LOG.debug("Writable status for " + f + " set to " + ret);
+      ret =
+        f.setExecutable(pInfo.executablePermissions,
+            pInfo.executePermsOwnerOnly)
+            && ret;
+
+      LOG.debug("Executable status for " + f + " set to " + ret);
+      return ret;
+    }
+
+    /**
+     * Permissions rwxr_xr_x
+     */
+    static PermissionsInfo sevenFiveFive =
+      new PermissionsInfo(true, true, true, false, true, false);
+    /**
+     * Completely private permissions
+     */
+    static PermissionsInfo sevenZeroZero =
+      new PermissionsInfo(true, true, true, true, true, true);
+  }
+
+  /**
+   * Prepare the job directories for a given job. To be called by the job
+   * localization code, only if the job is not already localized.
+   *
+   * <br>
+   * Here, we set 700 permissions on the job directories created on all disks.
+   * This we do so as to avoid any misuse by other users till the time
+   * {@link TaskController#initializeJob(JobInitializationContext)} is run at a
+   * later time to set proper private permissions on the job directories. <br>
+   *
+   * @param jobId
+   * @param fs
+   * @param localDirs
+   * @throws IOException
+   */
+  private static void initializeJobDirs(JobID jobId, FileSystem fs,
+      String[] localDirs)
+  throws IOException {
+    boolean initJobDirStatus = false;
+    String jobDirPath = getLocalJobDir(jobId.toString());
+    for (String localDir : localDirs) {
+      Path jobDir = new Path(localDir, jobDirPath);
+      if (fs.exists(jobDir)) {
+        // this will happen on a partial execution of localizeJob. Sometimes
+        // copying job.xml to the local disk succeeds but copying job.jar might
+        // throw out an exception. We should clean up and then try again.
+        fs.delete(jobDir, true);
+      }
+
+      boolean jobDirStatus = fs.mkdirs(jobDir);
+      if (!jobDirStatus) {
+        LOG.warn("Not able to create job directory " + jobDir.toString());
+      }
+
+      initJobDirStatus = initJobDirStatus || jobDirStatus;
+
+      // job-dir has to be private to the TT
+      PermissionsHandler.setPermissions(new File(jobDir.toUri().getPath()),
+          PermissionsHandler.sevenZeroZero);
+    }
+
+    if (!initJobDirStatus) {
+      throw new IOException("Not able to initialize job directories "
+          + "in any of the configured local directories for job "
+          + jobId.toString());
+    }
+  }
+
+  /**
+   * Download the job configuration file from the FS.
+   *
+   * @param t Task whose job file has to be downloaded
+   * @param jobId jobid of the task
+   * @return the local file system path of the downloaded file.
+   * @throws IOException
+   */
+  private Path localizeJobConfFile(Path jobFile, FileSystem userFs, JobID jobId)
+  throws IOException {
+    // Get sizes of JobFile and JarFile
+    // sizes are -1 if they are not present.
+    FileStatus status = null;
+    long jobFileSize = -1;
+    try {
+      status = userFs.getFileStatus(jobFile);
+      jobFileSize = status.getLen();
+    } catch(FileNotFoundException fe) {
+      jobFileSize = -1;
+    }
+    Path localJobFile =
+      lDirAlloc.getLocalPathForWrite(getLocalJobConfFile(jobId.toString()),
+          jobFileSize, fConf);
+
+    // Download job.xml
+    userFs.copyToLocalFile(jobFile, localJobFile);
+    return localJobFile;
+  }
+
+  /**
+   * Download the job jar file from FS to the local file system and unjar it.
+   * Set the local jar file in the passed configuration.
+   *
+   * @param jobId
+   * @param userFs
+   * @param localJobConf
+   * @throws IOException
+   */
+  private void localizeJobJarFile(JobID jobId, FileSystem userFs,
+      JobConf localJobConf)
+  throws IOException {
+    // copy Jar file to the local FS and unjar it.
+    String jarFile = localJobConf.getJar();
+    FileStatus status = null;
+    long jarFileSize = -1;
+    if (jarFile != null) {
+      Path jarFilePath = new Path(jarFile);
+      try {
+        status = userFs.getFileStatus(jarFilePath);
+        jarFileSize = status.getLen();
+      } catch (FileNotFoundException fe) {
+        jarFileSize = -1;
+      }
+      // Here we check for and we check five times the size of jarFileSize
+      // to accommodate for unjarring the jar file in userfiles directory
+      Path localJarFile =
+        lDirAlloc.getLocalPathForWrite(getJobJarFile(jobId.toString()),
+            5 * jarFileSize, fConf);
+
+      //Download job.jar
+      userFs.copyToLocalFile(jarFilePath, localJarFile);
+
+      localJobConf.setJar(localJarFile.toString());
+
+      // Also un-jar the job.jar files. We un-jar it so that classes inside
+      // sub-directories, for e.g., lib/, classes/ are available on class-path
+      RunJar.unJar(new File(localJarFile.toString()), new File(localJarFile
+          .getParent().toString()));
+    }
+  }
+
+  /**
+   * Create taskDirs on all the disks. Otherwise, in some cases, like when
+   * LinuxTaskController is in use, child might wish to balance load across
+   * disks but cannot itself create attempt directory because of the fact that
+   * job directory is writable only by the TT.
+   *
+   * @param jobId
+   * @param attemptId
+   * @param isCleanupAttempt
+   * @param fs
+   * @param localDirs
+   * @throws IOException
+   */
+  private static void initializeAttemptDirs(String jobId, String attemptId,
+      boolean isCleanupAttempt, FileSystem fs, String[] localDirs)
+  throws IOException {
+
+    boolean initStatus = false;
+    String attemptDirPath =
+      getLocalTaskDir(jobId, attemptId, isCleanupAttempt);
+
+    for (String localDir : localDirs) {
+      Path localAttemptDir = new Path(localDir, attemptDirPath);
+
+      boolean attemptDirStatus = fs.mkdirs(localAttemptDir);
+      if (!attemptDirStatus) {
+        LOG.warn("localAttemptDir " + localAttemptDir.toString()
+            + " couldn't be created.");
+      }
+      initStatus = initStatus || attemptDirStatus;
+    }
+
+    if (!initStatus) {
+      throw new IOException("Not able to initialize attempt directories "
+          + "in any of the configured local directories for the attempt "
+          + attemptId);
+    }
+  }
+  private void launchTaskForJob(TaskInProgress tip, JobConf jobConf)
+      throws IOException{
     synchronized (tip) {
     synchronized (tip) {
       tip.setJobConf(jobConf);
       tip.setJobConf(jobConf);
       tip.launchTask();
       tip.launchTask();
@@ -1020,6 +1301,17 @@ public class TaskTracker
     }
     }
   }
   }
 
 
+  /**
+   * For testing
+   */
+  TaskTracker() {
+    server = null;
+  }
+
+  void setConf(JobConf conf) {
+    fConf = conf;
+  }
+
   /**
   /**
    * Start with the local machine name, and the default JobTracker
    * Start with the local machine name, and the default JobTracker
    */
    */
@@ -1061,13 +1353,6 @@ public class TaskTracker
     initialize();
     initialize();
   }
   }
 
 
-  /**
-   * Blank constructor. Only usable by tests.
-   */
-  TaskTracker() {
-    server = null;
-  }
-
   private void checkJettyPort(int port) throws IOException { 
   private void checkJettyPort(int port) throws IOException { 
     //See HADOOP-4744
     //See HADOOP-4744
     if (port < 0) {
     if (port < 0) {
@@ -1695,10 +1980,9 @@ public class TaskTracker
       }
       }
       
       
       MapOutputFile mapOutputFile = new MapOutputFile();
       MapOutputFile mapOutputFile = new MapOutputFile();
-      mapOutputFile.setJobId(taskId.getJobID());
       mapOutputFile.setConf(conf);
       mapOutputFile.setConf(conf);
       
       
-      Path tmp_output =  mapOutputFile.getOutputFile(taskId);
+      Path tmp_output =  mapOutputFile.getOutputFile();
       if(tmp_output == null)
       if(tmp_output == null)
         return 0;
         return 0;
       FileSystem localFS = FileSystem.getLocal(conf);
       FileSystem localFS = FileSystem.getLocal(conf);
@@ -1988,54 +2272,36 @@ public class TaskTracker
       taskTimeout = (10 * 60 * 1000);
       taskTimeout = (10 * 60 * 1000);
     }
     }
         
         
-    private void localizeTask(Task task) throws IOException{
+    void localizeTask(Task task) throws IOException{
 
 
-      Path localTaskDir = 
-        lDirAlloc.getLocalPathForWrite(
-          TaskTracker.getLocalTaskDir(task.getJobID().toString(), 
-            task.getTaskID().toString(), task.isTaskCleanupTask()), 
-          defaultJobConf );
-      
       FileSystem localFs = FileSystem.getLocal(fConf);
       FileSystem localFs = FileSystem.getLocal(fConf);
-      if (!localFs.mkdirs(localTaskDir)) {
-        throw new IOException("Mkdirs failed to create " 
-                    + localTaskDir.toString());
-      }
-
-      // create symlink for ../work if it already doesnt exist
-      String workDir = lDirAlloc.getLocalPathToRead(
-                         TaskTracker.getLocalJobDir(task.getJobID().toString())
-                         + Path.SEPARATOR  
-                         + "work", defaultJobConf).toString();
-      String link = localTaskDir.getParent().toString() 
-                      + Path.SEPARATOR + "work";
-      File flink = new File(link);
-      if (!flink.exists())
-        FileUtil.symLink(workDir, link);
-      
+
+      // create taskDirs on all the disks.
+      initializeAttemptDirs(task.getJobID().toString(), task.getTaskID()
+          .toString(), task.isTaskCleanupTask(), localFs, fConf
+          .getStrings("mapred.local.dir"));
+
       // create the working-directory of the task 
       // create the working-directory of the task 
-      Path cwd = lDirAlloc.getLocalPathForWrite(
-                   getLocalTaskDir(task.getJobID().toString(), 
-                      task.getTaskID().toString(), task.isTaskCleanupTask()) 
-                   + Path.SEPARATOR + MRConstants.WORKDIR,
-                   defaultJobConf);
+      Path cwd =
+          lDirAlloc.getLocalPathForWrite(getTaskWorkDir(task.getJobID()
+              .toString(), task.getTaskID().toString(), task
+              .isTaskCleanupTask()), defaultJobConf);
       if (!localFs.mkdirs(cwd)) {
       if (!localFs.mkdirs(cwd)) {
         throw new IOException("Mkdirs failed to create " 
         throw new IOException("Mkdirs failed to create " 
                     + cwd.toString());
                     + cwd.toString());
       }
       }
 
 
-      Path localTaskFile = new Path(localTaskDir, "job.xml");
-      task.setJobFile(localTaskFile.toString());
       localJobConf.set("mapred.local.dir",
       localJobConf.set("mapred.local.dir",
                        fConf.get("mapred.local.dir"));
                        fConf.get("mapred.local.dir"));
+
       if (fConf.get("slave.host.name") != null) {
       if (fConf.get("slave.host.name") != null) {
         localJobConf.set("slave.host.name",
         localJobConf.set("slave.host.name",
                          fConf.get("slave.host.name"));
                          fConf.get("slave.host.name"));
       }
       }
             
             
-      localJobConf.set("mapred.task.id", task.getTaskID().toString());
       keepFailedTaskFiles = localJobConf.getKeepFailedTaskFiles();
       keepFailedTaskFiles = localJobConf.getKeepFailedTaskFiles();
 
 
+      // Do the task-type specific localization
       task.localizeConfiguration(localJobConf);
       task.localizeConfiguration(localJobConf);
       
       
       List<String[]> staticResolutions = NetUtils.getAllStaticResolutions();
       List<String[]> staticResolutions = NetUtils.getAllStaticResolutions();
@@ -2071,12 +2337,6 @@ public class TaskTracker
       if (isTaskMemoryManagerEnabled()) {
       if (isTaskMemoryManagerEnabled()) {
         localJobConf.setBoolean("task.memory.mgmt.enabled", true);
         localJobConf.setBoolean("task.memory.mgmt.enabled", true);
       }
       }
-      OutputStream out = localFs.create(localTaskFile);
-      try {
-        localJobConf.writeXml(out);
-      } finally {
-        out.close();
-      }
       task.setConf(localJobConf);
       task.setConf(localJobConf);
     }
     }
         
         
@@ -2349,7 +2609,7 @@ public class TaskTracker
                                      localJobConf). toString());
                                      localJobConf). toString());
               } catch (IOException e) {
               } catch (IOException e) {
                 LOG.warn("Working Directory of the task " + task.getTaskID() +
                 LOG.warn("Working Directory of the task " + task.getTaskID() +
-                		 "doesnt exist. Caught exception " +
+                                " doesnt exist. Caught exception " +
                           StringUtils.stringifyException(e));
                           StringUtils.stringifyException(e));
               }
               }
               // Build the command  
               // Build the command  
@@ -2630,34 +2890,39 @@ public class TaskTracker
           if (localJobConf == null) {
           if (localJobConf == null) {
             return;
             return;
           }
           }
-          String taskDir = getLocalTaskDir(task.getJobID().toString(),
-                             taskId.toString(), task.isTaskCleanupTask());
+          String localTaskDir =
+              getLocalTaskDir(task.getJobID().toString(), taskId.toString(),
+                  task.isTaskCleanupTask());
+          String taskWorkDir =
+              getTaskWorkDir(task.getJobID().toString(), taskId.toString(),
+                  task.isTaskCleanupTask());
           if (needCleanup) {
           if (needCleanup) {
             if (runner != null) {
             if (runner != null) {
               //cleans up the output directory of the task (where map outputs 
               //cleans up the output directory of the task (where map outputs 
               //and reduce inputs get stored)
               //and reduce inputs get stored)
               runner.close();
               runner.close();
             }
             }
-            //We don't delete the workdir
-            //since some other task (running in the same JVM) 
-            //might be using the dir. The JVM running the tasks would clean
-            //the workdir per a task in the task process itself.
+
             if (localJobConf.getNumTasksToExecutePerJvm() == 1) {
             if (localJobConf.getNumTasksToExecutePerJvm() == 1) {
+              // No jvm reuse, remove everything
               directoryCleanupThread.addToQueue(localFs,
               directoryCleanupThread.addToQueue(localFs,
                   getLocalFiles(defaultJobConf,
                   getLocalFiles(defaultJobConf,
-                  taskDir));
+                  localTaskDir));
             }  
             }  
-            
             else {
             else {
-              directoryCleanupThread.addToQueue(localFs,
-                  getLocalFiles(defaultJobConf,
-                taskDir+"/job.xml"));
+              // Jvm reuse. We don't delete the workdir since some other task
+              // (running in the same JVM) might be using the dir. The JVM
+              // running the tasks would clean the workdir per a task in the
+              // task process itself.
+              directoryCleanupThread.addToQueue(localFs, getLocalFiles(
+                  defaultJobConf, localTaskDir + Path.SEPARATOR
+                      + TaskTracker.JOBFILE));
             }
             }
           } else {
           } else {
             if (localJobConf.getNumTasksToExecutePerJvm() == 1) {
             if (localJobConf.getNumTasksToExecutePerJvm() == 1) {
               directoryCleanupThread.addToQueue(localFs,
               directoryCleanupThread.addToQueue(localFs,
                   getLocalFiles(defaultJobConf,
                   getLocalFiles(defaultJobConf,
-                  taskDir+"/work"));
+                  taskWorkDir));
             }  
             }  
           }
           }
         } catch (Throwable ie) {
         } catch (Throwable ie) {

+ 47 - 17
src/test/org/apache/hadoop/mapred/ClusterWithLinuxTaskController.java

@@ -47,7 +47,7 @@ import junit.framework.TestCase;
  * <li>Make the built binary to setuid executable</li>
  * <li>Make the built binary to setuid executable</li>
  * <li>Execute following targets:
  * <li>Execute following targets:
  * <code>ant test -Dcompile.c++=true -Dtaskcontroller-path=<em>path to built binary</em> 
  * <code>ant test -Dcompile.c++=true -Dtaskcontroller-path=<em>path to built binary</em> 
- * -Dtaskcontroller-user=<em>user,group</em></code></li>
+ * -Dtaskcontroller-ugi=<em>user,group</em></code></li>
  * </ol>
  * </ol>
  * 
  * 
  */
  */
@@ -82,6 +82,9 @@ public class ClusterWithLinuxTaskController extends TestCase {
 
 
   private static final int NUMBER_OF_NODES = 1;
   private static final int NUMBER_OF_NODES = 1;
 
 
+  static final String TASKCONTROLLER_PATH = "taskcontroller-path";
+  static final String TASKCONTROLLER_UGI = "taskcontroller-ugi";
+
   private File configurationFile = null;
   private File configurationFile = null;
 
 
   private UserGroupInformation taskControllerUser;
   private UserGroupInformation taskControllerUser;
@@ -98,18 +101,20 @@ public class ClusterWithLinuxTaskController extends TestCase {
         MyLinuxTaskController.class.getName());
         MyLinuxTaskController.class.getName());
     mrCluster =
     mrCluster =
         new MiniMRCluster(NUMBER_OF_NODES, dfsCluster.getFileSystem().getUri()
         new MiniMRCluster(NUMBER_OF_NODES, dfsCluster.getFileSystem().getUri()
-            .toString(), 1, null, null, conf);
+            .toString(), 4, null, null, conf);
 
 
     // Get the configured taskcontroller-path
     // Get the configured taskcontroller-path
-    String path = System.getProperty("taskcontroller-path");
-    createTaskControllerConf(path);
+    String path = System.getProperty(TASKCONTROLLER_PATH);
+    configurationFile =
+        createTaskControllerConf(path, mrCluster.getTaskTrackerRunner(0)
+            .getLocalDirs());
     String execPath = path + "/task-controller";
     String execPath = path + "/task-controller";
     TaskTracker tracker = mrCluster.getTaskTrackerRunner(0).tt;
     TaskTracker tracker = mrCluster.getTaskTrackerRunner(0).tt;
     // TypeCasting the parent to our TaskController instance as we
     // TypeCasting the parent to our TaskController instance as we
     // know that that would be instance which should be present in TT.
     // know that that would be instance which should be present in TT.
     ((MyLinuxTaskController) tracker.getTaskController())
     ((MyLinuxTaskController) tracker.getTaskController())
         .setTaskControllerExe(execPath);
         .setTaskControllerExe(execPath);
-    String ugi = System.getProperty("taskcontroller-user");
+    String ugi = System.getProperty(TASKCONTROLLER_UGI);
     clusterConf = mrCluster.createJobConf();
     clusterConf = mrCluster.createJobConf();
     String[] splits = ugi.split(",");
     String[] splits = ugi.split(",");
     taskControllerUser = new UnixUserGroupInformation(splits);
     taskControllerUser = new UnixUserGroupInformation(splits);
@@ -140,21 +145,39 @@ public class ClusterWithLinuxTaskController extends TestCase {
         taskControllerUser.getGroupNames()[0]);
         taskControllerUser.getGroupNames()[0]);
   }
   }
 
 
-  private void createTaskControllerConf(String path)
+  /**
+   * Create taskcontroller.cfg.
+   * 
+   * @param path Path to the taskcontroller binary.
+   * @param localDirs
+   * @return the created conf file
+   * @throws IOException
+   */
+  static File createTaskControllerConf(String path, String[] localDirs)
       throws IOException {
       throws IOException {
     File confDirectory = new File(path, "../conf");
     File confDirectory = new File(path, "../conf");
     if (!confDirectory.exists()) {
     if (!confDirectory.exists()) {
       confDirectory.mkdirs();
       confDirectory.mkdirs();
     }
     }
-    configurationFile = new File(confDirectory, "taskcontroller.cfg");
+    File configurationFile = new File(confDirectory, "taskcontroller.cfg");
     PrintWriter writer =
     PrintWriter writer =
         new PrintWriter(new FileOutputStream(configurationFile));
         new PrintWriter(new FileOutputStream(configurationFile));
 
 
-    writer.println(String.format("mapred.local.dir=%s", mrCluster
-        .getTaskTrackerLocalDir(0)));
+    StringBuffer sb = new StringBuffer();
+    for (int i = 0; i < localDirs.length; i++) {
+      sb.append(localDirs[i]);
+      if ((i + 1) != localDirs.length) {
+        sb.append(",");
+      }
+    }
+    writer.println(String.format("mapred.local.dir=%s", sb.toString()));
+
+    writer
+        .println(String.format("hadoop.log.dir=%s", TaskLog.getBaseLogDir()));
 
 
     writer.flush();
     writer.flush();
     writer.close();
     writer.close();
+    return configurationFile;
   }
   }
 
 
   /**
   /**
@@ -162,28 +185,35 @@ public class ClusterWithLinuxTaskController extends TestCase {
    * 
    * 
    * @return boolean
    * @return boolean
    */
    */
-  protected boolean shouldRun() {
-    return isTaskExecPathPassed() && isUserPassed();
+  protected static boolean shouldRun() {
+    if (!isTaskExecPathPassed() || !isUserPassed()) {
+      LOG.info("Not running test.");
+      return false;
+    }
+    return true;
   }
   }
 
 
-  private boolean isTaskExecPathPassed() {
-    String path = System.getProperty("taskcontroller-path");
+  private static boolean isTaskExecPathPassed() {
+    String path = System.getProperty(TASKCONTROLLER_PATH);
     if (path == null || path.isEmpty()
     if (path == null || path.isEmpty()
-        || path.equals("${taskcontroller-path}")) {
+        || path.equals("${" + TASKCONTROLLER_PATH + "}")) {
+      LOG.info("Invalid taskcontroller-path : " + path); 
       return false;
       return false;
     }
     }
     return true;
     return true;
   }
   }
 
 
-  private boolean isUserPassed() {
-    String ugi = System.getProperty("taskcontroller-user");
-    if (ugi != null && !(ugi.equals("${taskcontroller-user}"))
+  private static boolean isUserPassed() {
+    String ugi = System.getProperty(TASKCONTROLLER_UGI);
+    if (ugi != null && !(ugi.equals("${" + TASKCONTROLLER_UGI + "}"))
         && !ugi.isEmpty()) {
         && !ugi.isEmpty()) {
       if (ugi.indexOf(",") > 1) {
       if (ugi.indexOf(",") > 1) {
         return true;
         return true;
       }
       }
+      LOG.info("Invalid taskcontroller-ugi : " + ugi); 
       return false;
       return false;
     }
     }
+    LOG.info("Invalid taskcontroller-ugi : " + ugi);
     return false;
     return false;
   }
   }
 
 

+ 1 - 1
src/test/org/apache/hadoop/mapred/MiniMRCluster.java

@@ -163,7 +163,7 @@ public class MiniMRCluster {
       StringBuffer localPath = new StringBuffer();
       StringBuffer localPath = new StringBuffer();
       for(int i=0; i < numDir; ++i) {
       for(int i=0; i < numDir; ++i) {
         File ttDir = new File(localDirBase, 
         File ttDir = new File(localDirBase, 
-                              Integer.toString(trackerId) + "_" + 0);
+                              Integer.toString(trackerId) + "_" + i);
         if (!ttDir.mkdirs()) {
         if (!ttDir.mkdirs()) {
           if (!ttDir.isDirectory()) {
           if (!ttDir.isDirectory()) {
             throw new IOException("Mkdirs failed to create " + ttDir);
             throw new IOException("Mkdirs failed to create " + ttDir);

+ 9 - 9
src/test/org/apache/hadoop/mapred/TestIsolationRunner.java

@@ -26,6 +26,7 @@ import junit.framework.TestCase;
 
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.mapred.lib.IdentityMapper;
 import org.apache.hadoop.mapred.lib.IdentityMapper;
@@ -99,20 +100,17 @@ public class TestIsolationRunner extends TestCase {
   
   
   private Path getAttemptJobXml(JobConf conf, JobID jobId, boolean isMap)
   private Path getAttemptJobXml(JobConf conf, JobID jobId, boolean isMap)
       throws IOException {
       throws IOException {
-    String[] localDirs = conf.getLocalDirs();
-    assertEquals(1, localDirs.length);
-    Path jobCacheDir = new Path(localDirs[0], "0_0" + Path.SEPARATOR +
-        "taskTracker" + Path.SEPARATOR + "jobcache" + Path.SEPARATOR + jobId);    
-    Path attemptDir = new Path(jobCacheDir,
-        new TaskAttemptID(new TaskID(jobId, isMap, 0), 0).toString());    
-    return new Path(attemptDir, "job.xml");
+    String taskid =
+        new TaskAttemptID(new TaskID(jobId, isMap, 0), 0).toString();
+    return new LocalDirAllocator("mapred.local.dir").getLocalPathToRead(
+        TaskTracker.getTaskConfFile(jobId.toString(), taskid, false), conf);
   }
   }
 
 
   public void testIsolationRunOfMapTask() throws 
   public void testIsolationRunOfMapTask() throws 
       IOException, InterruptedException, ClassNotFoundException {
       IOException, InterruptedException, ClassNotFoundException {
     MiniMRCluster mr = null;
     MiniMRCluster mr = null;
     try {
     try {
-      mr = new MiniMRCluster(1, "file:///", 1);
+      mr = new MiniMRCluster(1, "file:///", 4);
 
 
       // Run a job succesfully; keep task files.
       // Run a job succesfully; keep task files.
       JobConf conf = mr.createJobConf();
       JobConf conf = mr.createJobConf();
@@ -131,7 +129,9 @@ public class TestIsolationRunner extends TestCase {
       // run IsolationRunner against the map task.
       // run IsolationRunner against the map task.
       FileSystem localFs = FileSystem.getLocal(conf);
       FileSystem localFs = FileSystem.getLocal(conf);
       Path mapJobXml =
       Path mapJobXml =
-          getAttemptJobXml(conf, jobId, true).makeQualified(localFs);
+          getAttemptJobXml(
+              mr.getTaskTrackerRunner(0).getTaskTracker().getJobConf(),
+              jobId, true).makeQualified(localFs);
       assertTrue(localFs.exists(mapJobXml));
       assertTrue(localFs.exists(mapJobXml));
       
       
       new IsolationRunner().run(new String[] {
       new IsolationRunner().run(new String[] {

+ 25 - 36
src/test/org/apache/hadoop/mapred/TestJobExecutionAsDifferentUser.java

@@ -20,14 +20,11 @@ package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+import org.apache.hadoop.examples.SleepJob;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
-import java.io.DataOutputStream;
-import java.io.IOException;
 
 
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
+import org.apache.hadoop.util.ToolRunner;
 
 
 /**
 /**
  * Test a java-based mapred job with LinuxTaskController running the jobs as a
  * Test a java-based mapred job with LinuxTaskController running the jobs as a
@@ -43,42 +40,34 @@ public class TestJobExecutionAsDifferentUser extends
       return;
       return;
     }
     }
     startCluster();
     startCluster();
-    submitWordCount(getClusterConf());
-  }
-  
-  private void submitWordCount(JobConf clientConf) throws IOException {
-    Path inDir = new Path("testing/wc/input");
-    Path outDir = new Path("testing/wc/output");
-    JobConf conf = new JobConf(clientConf);
-    FileSystem fs = FileSystem.get(conf);
-    fs.delete(outDir, true);
-    if (!fs.mkdirs(inDir)) {
-      throw new IOException("Mkdirs failed to create " + inDir.toString());
-    }
+    Path inDir = new Path("input");
+    Path outDir = new Path("output");
 
 
-    DataOutputStream file = fs.create(new Path(inDir, "part-0"));
-    file.writeBytes("a b c d e f g h");
-    file.close();
+    RunningJob job;
 
 
-    conf.setJobName("wordcount");
-    conf.setInputFormat(TextInputFormat.class);
-
-    // the keys are words (strings)
-    conf.setOutputKeyClass(Text.class);
-    // the values are counts (ints)
-    conf.setOutputValueClass(IntWritable.class);
+    // Run a job with zero maps/reduces
+    job = UtilsForTests.runJob(getClusterConf(), inDir, outDir, 0, 0);
+    job.waitForCompletion();
+    assertTrue("Job failed", job.isSuccessful());
+    assertOwnerShip(outDir);
 
 
-    conf.setMapperClass(WordCount.MapClass.class);
-    conf.setCombinerClass(WordCount.Reduce.class);
-    conf.setReducerClass(WordCount.Reduce.class);
+    // Run a job with 1 map and zero reduces
+    job = UtilsForTests.runJob(getClusterConf(), inDir, outDir, 1, 0);
+    job.waitForCompletion();
+    assertTrue("Job failed", job.isSuccessful());
+    assertOwnerShip(outDir);
 
 
-    FileInputFormat.setInputPaths(conf, inDir);
-    FileOutputFormat.setOutputPath(conf, outDir);
-    conf.setNumMapTasks(1);
-    conf.setNumReduceTasks(1);
-    RunningJob rj = JobClient.runJob(conf);
-    assertTrue("Job Failed", rj.isSuccessful());
+    // Run a normal job with maps/reduces
+    job = UtilsForTests.runJob(getClusterConf(), inDir, outDir, 1, 1);
+    job.waitForCompletion();
+    assertTrue("Job failed", job.isSuccessful());
     assertOwnerShip(outDir);
     assertOwnerShip(outDir);
+
+    // Run a job with jvm reuse
+    JobConf myConf = getClusterConf();
+    myConf.set("mapred.job.reuse.jvm.num.tasks", "-1");
+    String[] args = { "-m", "6", "-r", "3", "-mt", "1000", "-rt", "1000" };
+    assertEquals(0, ToolRunner.run(myConf, new SleepJob(), args));
   }
   }
   
   
   public void testEnvironment() throws IOException {
   public void testEnvironment() throws IOException {

+ 18 - 4
src/test/org/apache/hadoop/mapred/TestKillSubProcesses.java

@@ -350,15 +350,26 @@ public class TestKillSubProcesses extends TestCase {
     if (ProcessTree.isSetsidAvailable) {
     if (ProcessTree.isSetsidAvailable) {
       FileSystem fs = FileSystem.getLocal(conf);
       FileSystem fs = FileSystem.getLocal(conf);
 
 
-      if(fs.exists(scriptDir)){
+      if (fs.exists(scriptDir)) {
         fs.delete(scriptDir, true);
         fs.delete(scriptDir, true);
       }
       }
-      // create shell script
-      Random rm = new Random();
+
+      // Create the directory and set open permissions so that the TT can
+      // access.
+      fs.mkdirs(scriptDir);
+      fs.setPermission(scriptDir, new FsPermission(FsAction.ALL, FsAction.ALL,
+          FsAction.ALL));
+
+     // create shell script
+     Random rm = new Random();
       Path scriptPath = new Path(scriptDirName, "_shellScript_" + rm.nextInt()
       Path scriptPath = new Path(scriptDirName, "_shellScript_" + rm.nextInt()
         + ".sh");
         + ".sh");
       String shellScript = scriptPath.toString();
       String shellScript = scriptPath.toString();
+
+      // Construct the script. Set umask to 0000 so that TT can access all the
+      // files.
       String script =
       String script =
+        "umask 000\n" + 
         "echo $$ > " + scriptDirName + "/childPidFile" + "$1\n" +
         "echo $$ > " + scriptDirName + "/childPidFile" + "$1\n" +
         "echo hello\n" +
         "echo hello\n" +
         "trap 'echo got SIGTERM' 15 \n" +
         "trap 'echo got SIGTERM' 15 \n" +
@@ -373,7 +384,10 @@ public class TestKillSubProcesses extends TestCase {
       file.writeBytes(script);
       file.writeBytes(script);
       file.close();
       file.close();
 
 
-      LOG.info("Calling script from map task of failjob : " + shellScript);
+      // Set executable permissions on the script.
+      new File(scriptPath.toUri().getPath()).setExecutable(true);
+
+      LOG.info("Calling script from map task : " + shellScript);
       Runtime.getRuntime()
       Runtime.getRuntime()
           .exec(shellScript + " " + numLevelsOfSubProcesses);
           .exec(shellScript + " " + numLevelsOfSubProcesses);
     
     

+ 243 - 0
src/test/org/apache/hadoop/mapred/TestLocalizationWithLinuxTaskController.java

@@ -0,0 +1,243 @@
+/**
+ * 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 java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.TaskController.JobInitializationContext;
+import org.apache.hadoop.mapred.TaskController.TaskControllerContext;
+import org.apache.hadoop.mapred.TaskTracker.TaskInProgress;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.mapred.ClusterWithLinuxTaskController.MyLinuxTaskController;
+import org.apache.hadoop.mapred.JvmManager.JvmEnv;
+
+/**
+ * Test to verify localization of a job and localization of a task on a
+ * TaskTracker when {@link LinuxTaskController} is used.
+ * 
+ */
+public class TestLocalizationWithLinuxTaskController extends
+    TestTaskTrackerLocalization {
+
+  private static final Log LOG =
+      LogFactory.getLog(TestLocalizationWithLinuxTaskController.class);
+
+  private File configFile;
+  private MyLinuxTaskController taskController;
+
+  @Override
+  protected void setUp()
+      throws Exception {
+
+    if (!ClusterWithLinuxTaskController.shouldRun()) {
+      return;
+    }
+
+    super.setUp();
+
+    taskController = new MyLinuxTaskController();
+    String path =
+        System.getProperty(ClusterWithLinuxTaskController.TASKCONTROLLER_PATH);
+    configFile =
+        ClusterWithLinuxTaskController.createTaskControllerConf(path,
+            localDirs);
+    String execPath = path + "/task-controller";
+    taskController.setTaskControllerExe(execPath);
+    taskController.setConf(trackerFConf);
+    taskController.setup();
+  }
+
+  @Override
+  protected void tearDown()
+      throws Exception {
+    if (!ClusterWithLinuxTaskController.shouldRun()) {
+      return;
+    }
+    super.tearDown();
+    if (configFile != null) {
+      configFile.delete();
+    }
+  }
+
+  /** @InheritDoc */
+  @Override
+  public void testTaskControllerSetup() {
+    // Do nothing.
+  }
+
+  /**
+   * Test job localization with {@link LinuxTaskController}. Also check the
+   * permissions and file ownership of the job related files.
+   */
+  @Override
+  public void testJobLocalization()
+      throws IOException,
+      LoginException {
+
+    if (!ClusterWithLinuxTaskController.shouldRun()) {
+      return;
+    }
+
+    // Do job localization
+    JobConf localizedJobConf = tracker.localizeJobFiles(task);
+
+    String ugi =
+        System.getProperty(ClusterWithLinuxTaskController.TASKCONTROLLER_UGI);
+    localizedJobConf.setUser(ugi.split(",")[0]);
+
+    // Now initialize the job via task-controller so as to set
+    // ownership/permissions of jars, job-work-dir
+    JobInitializationContext context = new JobInitializationContext();
+    context.jobid = jobId;
+    context.user = localizedJobConf.getUser();
+    context.workDir =
+        new File(localizedJobConf.get(TaskTracker.JOB_LOCAL_DIR));
+
+    // /////////// The method being tested
+    taskController.initializeJob(context);
+    // ///////////
+
+    UserGroupInformation taskTrackerugi =
+        UserGroupInformation.login(localizedJobConf);
+    for (String localDir : trackerFConf.getStrings("mapred.local.dir")) {
+      File jobDir =
+          new File(localDir, TaskTracker.getLocalJobDir(jobId.toString()));
+      // check the private permissions on the job directory
+      checkFilePermissions(jobDir.getAbsolutePath(), "dr-xrws---",
+          localizedJobConf.getUser(), taskTrackerugi.getGroupNames()[0]);
+    }
+
+    // check the private permissions of various directories
+    List<Path> dirs = new ArrayList<Path>();
+    Path jarsDir =
+        lDirAlloc.getLocalPathToRead(TaskTracker.getJobJarsDir(jobId
+            .toString()), trackerFConf);
+    dirs.add(jarsDir);
+    dirs.add(new Path(jarsDir, "lib"));
+    for (Path dir : dirs) {
+      checkFilePermissions(dir.toUri().getPath(), "dr-xrws---",
+          localizedJobConf.getUser(), taskTrackerugi.getGroupNames()[0]);
+    }
+
+    // job-work dir needs user writable permissions
+    Path jobWorkDir =
+        lDirAlloc.getLocalPathToRead(TaskTracker.getJobWorkDir(jobId
+            .toString()), trackerFConf);
+    checkFilePermissions(jobWorkDir.toUri().getPath(), "drwxrws---",
+        localizedJobConf.getUser(), taskTrackerugi.getGroupNames()[0]);
+
+    // check the private permissions of various files
+    List<Path> files = new ArrayList<Path>();
+    files.add(lDirAlloc.getLocalPathToRead(TaskTracker
+        .getLocalJobConfFile(jobId.toString()), trackerFConf));
+    files.add(lDirAlloc.getLocalPathToRead(TaskTracker.getJobJarFile(jobId
+        .toString()), trackerFConf));
+    files.add(new Path(jarsDir, "lib" + Path.SEPARATOR + "lib1.jar"));
+    files.add(new Path(jarsDir, "lib" + Path.SEPARATOR + "lib2.jar"));
+    for (Path file : files) {
+      checkFilePermissions(file.toUri().getPath(), "-r-xrwx---",
+          localizedJobConf.getUser(), taskTrackerugi.getGroupNames()[0]);
+    }
+  }
+
+  /**
+   * Test task localization with {@link LinuxTaskController}. Also check the
+   * permissions and file ownership of task related files.
+   */
+  @Override
+  public void testTaskLocalization()
+      throws IOException,
+      LoginException {
+
+    if (!ClusterWithLinuxTaskController.shouldRun()) {
+      return;
+    }
+
+    JobConf localizedJobConf = tracker.localizeJobFiles(task);
+    String ugi =
+        System.getProperty(ClusterWithLinuxTaskController.TASKCONTROLLER_UGI);
+    localizedJobConf.setUser(ugi.split(",")[0]);
+
+    // Now initialize the job via task-controller so as to set
+    // ownership/permissions of jars, job-work-dir
+    JobInitializationContext jobContext = new JobInitializationContext();
+    jobContext.jobid = jobId;
+    jobContext.user = localizedJobConf.getUser();
+    jobContext.workDir =
+        new File(localizedJobConf.get(TaskTracker.JOB_LOCAL_DIR));
+    taskController.initializeJob(jobContext);
+
+    TaskInProgress tip = tracker.new TaskInProgress(task, trackerFConf);
+    tip.setJobConf(localizedJobConf);
+
+    // localize the task.
+    tip.localizeTask(task);
+    TaskRunner runner = task.createRunner(tracker, tip);
+    runner.setupChildTaskConfiguration(lDirAlloc);
+    Path workDir =
+        lDirAlloc.getLocalPathToRead(TaskTracker.getTaskWorkDir(task
+            .getJobID().toString(), task.getTaskID().toString(), task
+            .isTaskCleanupTask()), trackerFConf);
+    TaskRunner.createChildTmpDir(new File(workDir.toUri().getPath()),
+        localizedJobConf);
+    File[] logFiles = TaskRunner.prepareLogFiles(task.getTaskID());
+
+    // Initialize task
+    TaskControllerContext taskContext =
+        new TaskController.TaskControllerContext();
+    taskContext.env =
+        new JvmEnv(null, null, null, null, -1, new File(localizedJobConf
+            .get(TaskTracker.JOB_LOCAL_DIR)), null, localizedJobConf);
+    taskContext.task = task;
+    // /////////// The method being tested
+    taskController.initializeTask(taskContext);
+    // ///////////
+
+    // check the private permissions of various directories
+    List<Path> dirs = new ArrayList<Path>();
+    dirs.add(lDirAlloc.getLocalPathToRead(TaskTracker.getLocalTaskDir(jobId
+        .toString(), taskId.toString()), trackerFConf));
+    dirs.add(workDir);
+    dirs.add(new Path(workDir, "tmp"));
+    dirs.add(new Path(logFiles[1].getParentFile().getAbsolutePath()));
+    UserGroupInformation taskTrackerugi =
+        UserGroupInformation.login(localizedJobConf);
+    for (Path dir : dirs) {
+      checkFilePermissions(dir.toUri().getPath(), "drwxrws---",
+          localizedJobConf.getUser(), taskTrackerugi.getGroupNames()[0]);
+    }
+
+    // check the private permissions of various files
+    List<Path> files = new ArrayList<Path>();
+    files.add(lDirAlloc.getLocalPathToRead(TaskTracker.getTaskConfFile(task
+        .getJobID().toString(), task.getTaskID().toString(), task
+        .isTaskCleanupTask()), trackerFConf));
+    for (Path file : files) {
+      checkFilePermissions(file.toUri().getPath(), "-rwxrwx---",
+          localizedJobConf.getUser(), taskTrackerugi.getGroupNames()[0]);
+    }
+  }
+}

+ 2 - 4
src/test/org/apache/hadoop/mapred/TestMapRed.java

@@ -277,14 +277,12 @@ public class TestMapRed extends TestCase implements Tool {
   private static class MyReduce extends IdentityReducer {
   private static class MyReduce extends IdentityReducer {
     private JobConf conf;
     private JobConf conf;
     private boolean compressInput;
     private boolean compressInput;
-    private TaskAttemptID taskId;
     private boolean first = true;
     private boolean first = true;
       
       
     @Override
     @Override
     public void configure(JobConf conf) {
     public void configure(JobConf conf) {
       this.conf = conf;
       this.conf = conf;
       compressInput = conf.getCompressMapOutput();
       compressInput = conf.getCompressMapOutput();
-      taskId = TaskAttemptID.forName(conf.get("mapred.task.id"));
     }
     }
       
       
     public void reduce(WritableComparable key, Iterator values,
     public void reduce(WritableComparable key, Iterator values,
@@ -292,9 +290,9 @@ public class TestMapRed extends TestCase implements Tool {
                        ) throws IOException {
                        ) throws IOException {
       if (first) {
       if (first) {
         first = false;
         first = false;
-        MapOutputFile mapOutputFile = new MapOutputFile(taskId.getJobID());
+        MapOutputFile mapOutputFile = new MapOutputFile();
         mapOutputFile.setConf(conf);
         mapOutputFile.setConf(conf);
-        Path input = mapOutputFile.getInputFile(0, taskId);
+        Path input = mapOutputFile.getInputFile(0);
         FileSystem fs = FileSystem.get(conf);
         FileSystem fs = FileSystem.get(conf);
         assertTrue("reduce input exists " + input, fs.exists(input));
         assertTrue("reduce input exists " + input, fs.exists(input));
         SequenceFile.Reader rdr = 
         SequenceFile.Reader rdr = 

+ 3 - 3
src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java

@@ -135,7 +135,7 @@ public class TestMiniMRWithDFS extends TestCase {
       int numNotDel = 0;
       int numNotDel = 0;
       File localDir = new File(mr.getTaskTrackerLocalDir(i));
       File localDir = new File(mr.getTaskTrackerLocalDir(i));
       LOG.debug("Tracker directory: " + localDir);
       LOG.debug("Tracker directory: " + localDir);
-      File trackerDir = new File(localDir, "taskTracker");
+      File trackerDir = new File(localDir, TaskTracker.SUBDIR);
       assertTrue("local dir " + localDir + " does not exist.", 
       assertTrue("local dir " + localDir + " does not exist.", 
                  localDir.isDirectory());
                  localDir.isDirectory());
       assertTrue("task tracker dir " + trackerDir + " does not exist.", 
       assertTrue("task tracker dir " + trackerDir + " does not exist.", 
@@ -150,7 +150,7 @@ public class TestMiniMRWithDFS extends TestCase {
       }
       }
       for(int fileIdx = 0; fileIdx < contents.length; ++fileIdx) {
       for(int fileIdx = 0; fileIdx < contents.length; ++fileIdx) {
         String name = contents[fileIdx];
         String name = contents[fileIdx];
-        if (!("taskTracker".equals(contents[fileIdx]))) {
+        if (!(TaskTracker.SUBDIR.equals(contents[fileIdx]))) {
           LOG.debug("Looking at " + name);
           LOG.debug("Looking at " + name);
           assertTrue("Spurious directory " + name + " found in " +
           assertTrue("Spurious directory " + name + " found in " +
                      localDir, false);
                      localDir, false);
@@ -158,7 +158,7 @@ public class TestMiniMRWithDFS extends TestCase {
       }
       }
       for (int idx = 0; idx < neededDirs.size(); ++idx) {
       for (int idx = 0; idx < neededDirs.size(); ++idx) {
         String name = neededDirs.get(idx);
         String name = neededDirs.get(idx);
-        if (new File(new File(new File(trackerDir, "jobcache"),
+        if (new File(new File(new File(trackerDir, TaskTracker.JOBCACHE),
                               jobIds[idx]), name).isDirectory()) {
                               jobIds[idx]), name).isDirectory()) {
           found[idx] = true;
           found[idx] = true;
           numNotDel++;
           numNotDel++;

+ 380 - 0
src/test/org/apache/hadoop/mapred/TestTaskTrackerLocalization.java

@@ -0,0 +1,380 @@
+/**
+ * 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 java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.jar.JarOutputStream;
+import java.util.zip.ZipEntry;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.mapred.TaskTracker.TaskInProgress;
+
+import junit.framework.TestCase;
+
+/**
+ * Test to verify localization of a job and localization of a task on a
+ * TaskTracker.
+ * 
+ */
+public class TestTaskTrackerLocalization extends TestCase {
+
+  private File TEST_ROOT_DIR;
+  private File ROOT_MAPRED_LOCAL_DIR;
+  private File HADOOP_LOG_DIR;
+
+  private int numLocalDirs = 6;
+  private static final Log LOG =
+      LogFactory.getLog(TestTaskTrackerLocalization.class);
+
+  protected TaskTracker tracker;
+  protected JobConf trackerFConf;
+  protected JobID jobId;
+  protected TaskAttemptID taskId;
+  protected Task task;
+  protected String[] localDirs;
+  protected static LocalDirAllocator lDirAlloc =
+      new LocalDirAllocator("mapred.local.dir");
+
+  @Override
+  protected void setUp()
+      throws Exception {
+    TEST_ROOT_DIR =
+        new File(System.getProperty("test.build.data", "/tmp"),
+            "testTaskTrackerLocalization");
+    if (!TEST_ROOT_DIR.exists()) {
+      TEST_ROOT_DIR.mkdirs();
+    }
+
+    ROOT_MAPRED_LOCAL_DIR = new File(TEST_ROOT_DIR, "mapred/local");
+    ROOT_MAPRED_LOCAL_DIR.mkdirs();
+
+    HADOOP_LOG_DIR = new File(TEST_ROOT_DIR, "logs");
+    HADOOP_LOG_DIR.mkdir();
+    System.setProperty("hadoop.log.dir", HADOOP_LOG_DIR.getAbsolutePath());
+
+    trackerFConf = new JobConf();
+    trackerFConf.set("fs.default.name", "file:///");
+    localDirs = new String[numLocalDirs];
+    for (int i = 0; i < numLocalDirs; i++) {
+      localDirs[i] = new File(ROOT_MAPRED_LOCAL_DIR, "0_" + i).getPath();
+    }
+    trackerFConf.setStrings("mapred.local.dir", localDirs);
+
+    // Create the job jar file
+    File jobJarFile = new File(TEST_ROOT_DIR, "jobjar-on-dfs.jar");
+    JarOutputStream jstream =
+        new JarOutputStream(new FileOutputStream(jobJarFile));
+    ZipEntry ze = new ZipEntry("lib/lib1.jar");
+    jstream.putNextEntry(ze);
+    jstream.closeEntry();
+    ze = new ZipEntry("lib/lib2.jar");
+    jstream.putNextEntry(ze);
+    jstream.closeEntry();
+    jstream.finish();
+    jstream.close();
+    trackerFConf.setJar(jobJarFile.toURI().toString());
+
+    // Create the job configuration file
+    File jobConfFile = new File(TEST_ROOT_DIR, "jobconf-on-dfs.xml");
+    FileOutputStream out = new FileOutputStream(jobConfFile);
+    trackerFConf.writeXml(out);
+    out.close();
+
+    // Set up the TaskTracker
+    tracker = new TaskTracker();
+    tracker.setConf(trackerFConf);
+    tracker.systemFS = FileSystem.getLocal(trackerFConf); // for test case
+
+    // Set up the task to be localized
+    String jtIdentifier = "200907202331";
+    jobId = new JobID(jtIdentifier, 1);
+    taskId =
+        new TaskAttemptID(jtIdentifier, jobId.getId(), true, 1, 0);
+    task =
+        new MapTask(jobConfFile.toURI().toString(), taskId, 1, null, 1);
+
+    TaskController taskController = new DefaultTaskController();
+    taskController.setConf(trackerFConf);
+    taskController.setup();
+  }
+
+  @Override
+  protected void tearDown()
+      throws Exception {
+    FileUtil.fullyDelete(TEST_ROOT_DIR);
+  }
+
+  private static String[] getFilePermissionAttrs(String path)
+      throws IOException {
+    String output = Shell.execCommand("stat", path, "-c", "%A:%U:%G");
+    return output.split(":|\n");
+  }
+
+  static void checkFilePermissions(String path, String expectedPermissions,
+      String expectedOwnerUser, String expectedOwnerGroup)
+      throws IOException {
+    String[] attrs = getFilePermissionAttrs(path);
+    assertTrue("File attrs length is not 3 but " + attrs.length,
+        attrs.length == 3);
+    assertTrue("Path " + path + " has the permissions " + attrs[0]
+        + " instead of the expected " + expectedPermissions, attrs[0]
+        .equals(expectedPermissions));
+    assertTrue("Path " + path + " is not user owned not by "
+        + expectedOwnerUser + " but by " + attrs[1], attrs[1]
+        .equals(expectedOwnerUser));
+    assertTrue("Path " + path + " is not group owned not by "
+        + expectedOwnerGroup + " but by " + attrs[2], attrs[2]
+        .equals(expectedOwnerGroup));
+  }
+
+  /**
+   * Verify the task-controller's setup functionality
+   * 
+   * @throws IOException
+   * @throws LoginException
+   */
+  public void testTaskControllerSetup()
+      throws IOException,
+      LoginException {
+    // Task-controller is already set up in the test's setup method. Now verify.
+    UserGroupInformation ugi = UserGroupInformation.login(new JobConf());
+    for (String localDir : localDirs) {
+
+      // Verify the local-dir itself.
+      File lDir = new File(localDir);
+      assertTrue("localDir " + lDir + " doesn't exists!", lDir.exists());
+      checkFilePermissions(lDir.getAbsolutePath(), "drwxr-xr-x", ugi
+          .getUserName(), ugi.getGroupNames()[0]);
+
+      // Verify the distributed cache dir.
+      File distributedCacheDir =
+          new File(localDir, TaskTracker.getDistributedCacheDir());
+      assertTrue("distributed cache dir " + distributedCacheDir
+          + " doesn't exists!", distributedCacheDir.exists());
+      checkFilePermissions(distributedCacheDir.getAbsolutePath(),
+          "drwxr-xr-x", ugi.getUserName(), ugi.getGroupNames()[0]);
+
+      // Verify the job cache dir.
+      File jobCacheDir = new File(localDir, TaskTracker.getJobCacheSubdir());
+      assertTrue("jobCacheDir " + jobCacheDir + " doesn't exists!",
+          jobCacheDir.exists());
+      checkFilePermissions(jobCacheDir.getAbsolutePath(), "drwxr-xr-x", ugi
+          .getUserName(), ugi.getGroupNames()[0]);
+    }
+
+    // Verify the pemissions on the userlogs dir
+    File taskLog = TaskLog.getUserLogDir();
+    checkFilePermissions(taskLog.getAbsolutePath(), "drwxr-xr-x", ugi
+        .getUserName(), ugi.getGroupNames()[0]);
+  }
+
+  /**
+   * Test job localization on a TT. Tests localization of job.xml, job.jar and
+   * corresponding setting of configuration.
+   * 
+   * @throws IOException
+   * @throws LoginException
+   */
+  public void testJobLocalization()
+      throws IOException,
+      LoginException {
+
+    // /////////// The main method being tested
+    JobConf localizedJobConf = tracker.localizeJobFiles(task);
+    // ///////////
+
+    // Check the directory structure
+    for (String dir : localDirs) {
+
+      File localDir = new File(dir);
+      assertTrue("mapred.local.dir " + localDir + " isn'task created!",
+          localDir.exists());
+
+      File taskTrackerSubDir = new File(localDir, TaskTracker.SUBDIR);
+      assertTrue("taskTracker sub-dir in the local-dir " + localDir
+          + "is not created!", taskTrackerSubDir.exists());
+
+      File jobCache = new File(taskTrackerSubDir, TaskTracker.JOBCACHE);
+      assertTrue("jobcache in the taskTrackerSubdir " + taskTrackerSubDir
+          + " isn'task created!", jobCache.exists());
+
+      File jobDir = new File(jobCache, jobId.toString());
+      assertTrue("job-dir in " + jobCache + " isn'task created!", jobDir
+          .exists());
+
+      // check the private permissions on the job directory
+      UserGroupInformation ugi = UserGroupInformation.login(localizedJobConf);
+      checkFilePermissions(jobDir.getAbsolutePath(), "drwx------", ugi
+          .getUserName(), ugi.getGroupNames()[0]);
+    }
+
+    // check the localization of job.xml
+    LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
+
+    assertTrue("job.xml is not localized on this TaskTracker!!", lDirAlloc
+        .getLocalPathToRead(TaskTracker.getLocalJobConfFile(jobId.toString()),
+            trackerFConf) != null);
+
+    // check the localization of job.jar
+    Path jarFileLocalized =
+        lDirAlloc.getLocalPathToRead(TaskTracker.getJobJarFile(jobId
+            .toString()), trackerFConf);
+    assertTrue("job.jar is not localized on this TaskTracker!!",
+        jarFileLocalized != null);
+    assertTrue("lib/lib1.jar is not unjarred on this TaskTracker!!", new File(
+        jarFileLocalized.getParent() + Path.SEPARATOR + "lib/lib1.jar")
+        .exists());
+    assertTrue("lib/lib2.jar is not unjarred on this TaskTracker!!", new File(
+        jarFileLocalized.getParent() + Path.SEPARATOR + "lib/lib2.jar")
+        .exists());
+
+    // check the creation of job work directory
+    assertTrue("job-work dir is not created on this TaskTracker!!", lDirAlloc
+        .getLocalPathToRead(TaskTracker.getJobWorkDir(jobId.toString()),
+            trackerFConf) != null);
+
+    // Check the setting of job.local.dir and job.jar which will eventually be
+    // used by the user's task
+    boolean jobLocalDirFlag = false, mapredJarFlag = false;
+    String localizedJobLocalDir =
+        localizedJobConf.get(TaskTracker.JOB_LOCAL_DIR);
+    String localizedJobJar = localizedJobConf.getJar();
+    for (String localDir : localizedJobConf.getStrings("mapred.local.dir")) {
+      if (localizedJobLocalDir.equals(localDir + Path.SEPARATOR
+          + TaskTracker.getJobWorkDir(jobId.toString()))) {
+        jobLocalDirFlag = true;
+      }
+      if (localizedJobJar.equals(localDir + Path.SEPARATOR
+          + TaskTracker.getJobJarFile(jobId.toString()))) {
+        mapredJarFlag = true;
+      }
+    }
+    assertTrue(TaskTracker.JOB_LOCAL_DIR
+        + " is not set properly to the target users directory : "
+        + localizedJobLocalDir, jobLocalDirFlag);
+    assertTrue(
+        "mapred.jar is not set properly to the target users directory : "
+            + localizedJobJar, mapredJarFlag);
+  }
+
+  /**
+   * Test task localization on a TT.
+   * 
+   * @throws IOException
+   * @throws LoginException
+   */
+  public void testTaskLocalization()
+      throws IOException,
+      LoginException {
+
+    JobConf localizedJobConf = tracker.localizeJobFiles(task);
+
+    TaskInProgress tip = tracker.new TaskInProgress(task, trackerFConf);
+    tip.setJobConf(localizedJobConf);
+
+    // ////////// The central method being tested
+    tip.localizeTask(task);
+    // //////////
+
+    // check the functionality of localizeTask
+    for (String dir : trackerFConf.getStrings("mapred.local.dir")) {
+      assertTrue("attempt-dir in localDir " + dir + " is not created!!",
+          new File(dir, TaskTracker.getLocalTaskDir(jobId.toString(), taskId
+              .toString())).exists());
+    }
+
+    Path workDir =
+        lDirAlloc.getLocalPathToRead(TaskTracker.getTaskWorkDir(task
+            .getJobID().toString(), task.getTaskID().toString(), task
+            .isTaskCleanupTask()), trackerFConf);
+    assertTrue("atttempt work dir for " + taskId.toString()
+        + " is not created in any of the configured dirs!!", workDir != null);
+
+    TaskRunner runner = task.createRunner(tracker, tip);
+
+    // /////// Few more methods being tested
+    runner.setupChildTaskConfiguration(lDirAlloc);
+    TaskRunner.createChildTmpDir(new File(workDir.toUri().getPath()),
+        localizedJobConf);
+    File[] logFiles = TaskRunner.prepareLogFiles(task.getTaskID());
+    // ///////
+
+    // Make sure the task-conf file is created
+    Path localTaskFile =
+        lDirAlloc.getLocalPathToRead(TaskTracker.getTaskConfFile(task
+            .getJobID().toString(), task.getTaskID().toString(), task
+            .isTaskCleanupTask()), trackerFConf);
+    assertTrue("Task conf file " + localTaskFile.toString()
+        + " is not created!!", new File(localTaskFile.toUri().getPath())
+        .exists());
+
+    // /////// One more method being tested. This happens in child space.
+    JobConf localizedTaskConf = new JobConf(localTaskFile);
+    TaskRunner.setupChildMapredLocalDirs(task, localizedTaskConf);
+    // ///////
+
+    // Make sure that the mapred.local.dir is sandboxed
+    for (String childMapredLocalDir : localizedTaskConf
+        .getStrings("mapred.local.dir")) {
+      assertTrue("Local dir " + childMapredLocalDir + " is not sandboxed !!",
+          childMapredLocalDir.endsWith(TaskTracker.getLocalTaskDir(jobId
+              .toString(), taskId.toString(), false)));
+    }
+
+    // Make sure task task.getJobFile is changed and pointed correctly.
+    assertTrue(task.getJobFile().endsWith(
+        TaskTracker
+            .getTaskConfFile(jobId.toString(), taskId.toString(), false)));
+
+    // Make sure that the tmp directories are created
+    assertTrue("tmp dir is not created in workDir "
+        + workDir.toUri().getPath(),
+        new File(workDir.toUri().getPath(), "tmp").exists());
+
+    // Make sure that the log are setup properly
+    File logDir =
+        new File(HADOOP_LOG_DIR, TaskLog.USERLOGS_DIR_NAME + Path.SEPARATOR
+            + task.getTaskID().toString());
+    assertTrue("task's log dir " + logDir.toString() + " doesn't exist!",
+        logDir.exists());
+    UserGroupInformation ugi = UserGroupInformation.login(localizedJobConf);
+    checkFilePermissions(logDir.getAbsolutePath(), "drwx------", ugi
+        .getUserName(), ugi.getGroupNames()[0]);
+
+    File expectedStdout = new File(logDir, TaskLog.LogName.STDOUT.toString());
+    assertTrue("stdout log file is improper. Expected : "
+        + expectedStdout.toString() + " Observed : " + logFiles[0].toString(),
+        expectedStdout.toString().equals(logFiles[0].toString()));
+    File expectedStderr =
+        new File(logDir, Path.SEPARATOR + TaskLog.LogName.STDERR.toString());
+    assertTrue("stderr log file is improper. Expected : "
+        + expectedStderr.toString() + " Observed : " + logFiles[1].toString(),
+        expectedStderr.toString().equals(logFiles[1].toString()));
+  }
+}