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

svn commit: r1076893 [3/3] - in /hadoop/common/branches/branch-0.20-security-patches: ./ conf/ src/c++/task-controller/

Added: hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/configure.ac
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/c%2B%2B/task-controller/configure.ac?rev=1076893&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/configure.ac (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/configure.ac Thu Mar  3 23:50:57 2011
@@ -0,0 +1,61 @@
+#                                               -*- Autoconf -*-
+# Process this file with autoconf to produce a configure script.
+
+#
+# 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.
+#
+
+AC_PREREQ(2.59)
+AC_INIT([task-controller],[0.1])
+
+#changing default prefix value to empty string, so that binary does not
+#gets installed within system
+AC_PREFIX_DEFAULT(.)
+
+#add new argument called -with-confdir
+AC_ARG_WITH(confdir,[--with-confdir path to hadoop conf dir])
+AC_CONFIG_SRCDIR([task-controller.h])
+AC_CONFIG_HEADER([configuration.h])
+
+# Checks for programs.
+AC_PROG_CC
+
+# Checks for libraries.
+
+# Checks for header files.
+AC_HEADER_STDC
+AC_CHECK_HEADERS([stdlib.h string.h unistd.h])
+
+#check for HADOOP_CONF_DIR
+
+
+if test "$with_confdir" != ""
+then
+AC_DEFINE_UNQUOTED(HADOOP_CONF_DIR,"$with_confdir")
+fi
+# Checks for typedefs, structures, and compiler characteristics.
+AC_C_CONST
+AC_TYPE_PID_T
+
+# Checks for library functions.
+AC_FUNC_MALLOC
+AC_FUNC_REALLOC
+AC_CHECK_FUNCS([strerror])
+
+AC_CONFIG_FILES([Makefile])
+AC_OUTPUT
+

Added: hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/main.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/c%2B%2B/task-controller/main.c?rev=1076893&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/main.c (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/main.c Thu Mar  3 23:50:57 2011
@@ -0,0 +1,116 @@
+/**
+ * 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"
+
+
+int main(int argc, char **argv) {
+  int command;
+  int next_option = 0;
+  const char * job_id = NULL;
+  const char * task_id = NULL;
+  const char * tt_root = NULL;
+  int exit_code = 0;
+  const char* const short_options = "l:";
+  const struct option long_options[] = { { "log", 1, NULL, 'l' }, { NULL, 0,
+      NULL, 0 } };
+
+  const char* log_file = NULL;
+
+  // when we support additional commands without ttroot, this check
+  // may become command specific.
+  if (argc < 6) {
+    display_usage(stderr);
+    return INVALID_ARGUMENT_NUMBER;
+  }
+
+#ifndef HADOOP_CONF_DIR
+  hadoop_conf_dir = (char *) malloc (sizeof(char) *
+      (strlen(argv[0]) - strlen(EXEC_PATTERN)) + 1);
+  strncpy(hadoop_conf_dir,argv[0],(strlen(argv[0]) - strlen(EXEC_PATTERN)));
+  hadoop_conf_dir[(strlen(argv[0]) - strlen(EXEC_PATTERN))] = '\0';
+#endif
+
+  do {
+    next_option = getopt_long(argc, argv, short_options, long_options, NULL);
+    switch (next_option) {
+    case 'l':
+      log_file = optarg;
+    default:
+      break;
+    }
+  } 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;
+      }
+    }
+  }
+  //checks done for user name
+  //checks done if the user is root or not.
+  if (argv[optind] == NULL) {
+    fprintf(LOGFILE, "Invalid user name \n");
+    return INVALID_USER_NAME;
+  }
+  if (get_user_details(argv[optind]) != 0) {
+    return INVALID_USER_NAME;
+  }
+  //implicit conversion to int instead of __gid_t and __uid_t
+  if (user_detail->pw_gid == 0 || user_detail->pw_uid == 0) {
+    fprintf(LOGFILE, "Cannot run tasks as super user\n");
+    return SUPER_USER_NOT_ALLOWED_TO_RUN_TASKS;
+  }
+  optind = optind + 1;
+  command = atoi(argv[optind++]);
+  job_id = argv[optind++];
+  task_id = argv[optind++];
+
+#ifdef DEBUG
+  fprintf(LOGFILE, "main : command provided %d\n",command);
+  fprintf(LOGFILE, "main : user is %s\n", user_detail->pw_name);
+  fprintf(LOGFILE, "main : job id %s \n", job_id);
+  fprintf(LOGFILE, "main : task id %s \n", task_id);
+#endif
+  switch (command) {
+  case RUN_TASK:
+    tt_root = argv[optind];
+    exit_code
+        = run_task_as_user(user_detail->pw_name, job_id, task_id, tt_root);
+    break;
+  case KILL_TASK:
+    tt_root = argv[optind];
+    exit_code = kill_user_task(user_detail->pw_name, job_id, task_id, tt_root);
+    break;
+  default:
+    exit_code = INVALID_COMMAND_PROVIDED;
+  }
+  fflush(LOGFILE);
+  fclose(LOGFILE);
+  return exit_code;
+}

Added: hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/task-controller.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/c%2B%2B/task-controller/task-controller.c?rev=1076893&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/task-controller.c (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/task-controller.c Thu Mar  3 23:50:57 2011
@@ -0,0 +1,463 @@
+/**
+ * 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"
+
+//struct to store the user details
+struct passwd *user_detail = NULL;
+
+//LOGFILE
+FILE *LOGFILE;
+
+//hadoop temp dir root which is configured in secure configuration
+const char *mapred_local_dir;
+
+//placeholder for global cleanup operations
+void cleanup() {
+  free_configurations();
+}
+
+//change the user to passed user for executing/killing tasks
+int change_user(const char * user) {
+  if (get_user_details(user) < 0) {
+    return -1;
+  }
+#ifdef DEBUG
+  fprintf(LOGFILE,"change_user : setting user as %s ", user_detail->pw_name);
+#endif
+  errno = 0;
+  setgid(user_detail->pw_gid);
+  if (errno != 0) {
+    fprintf(LOGFILE, "unable to setgid : %s\n", strerror(errno));
+    cleanup();
+    return SETUID_OPER_FAILED;
+  }
+
+  setegid(user_detail->pw_gid);
+  if (errno != 0) {
+    fprintf(LOGFILE, "unable to setegid : %s\n", strerror(errno));
+    cleanup();
+    return SETUID_OPER_FAILED;
+  }
+
+  setuid(user_detail->pw_uid);
+  if (errno != 0) {
+    fprintf(LOGFILE, "unable to setuid : %s\n", strerror(errno));
+    cleanup();
+    return SETUID_OPER_FAILED;
+  }
+
+  seteuid(user_detail->pw_uid);
+  if (errno != 0) {
+    fprintf(LOGFILE, "unable to seteuid : %s\n", strerror(errno));
+    cleanup();
+    return SETUID_OPER_FAILED;
+  }
+  return 0;
+}
+
+//Function to set the hadoop.temp.dir key from configuration.
+//would return -1 if the configuration is not proper.
+
+int get_mapred_local_dir() {
+
+  if (mapred_local_dir == NULL) {
+    mapred_local_dir = get_value(TT_SYS_DIR_KEY);
+  }
+
+  //after the call it should not be null
+  if (mapred_local_dir == NULL) {
+    return -1;
+  } else {
+    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 *token;
+  int found = -1;
+
+  if (tt_root == NULL) {
+    return -1;
+  }
+
+  if (mapred_local_dir == NULL) {
+    if (get_mapred_local_dir() < 0) {
+      fprintf(LOGFILE, "invalid hadoop config\n");
+      return -1;
+    }
+  }
+
+  token = strtok((char *) mapred_local_dir, ",");
+  if (token == NULL && mapred_local_dir != NULL) {
+#ifdef DEBUG
+    fprintf(LOGFILE,"Single hadoop.tmp.dir configured");
+#endif
+    token = (char *)mapred_local_dir;
+  }
+
+  while (1) {
+    if (strcmp(tt_root, token) == 0) {
+      found = 0;
+      break;
+    }
+    token = strtok(NULL, ",");
+    if (token == NULL) {
+      break;
+    }
+  }
+
+  return found;
+
+}
+
+/*
+ *d function which would return .pid file path which is used while running
+ * and killing of the tasks by the user.
+ *
+ * check TT_SYS_DIR for pattern
+ */
+void get_pid_path(const char * jobid, const char * taskid, const char *tt_root,
+    char ** pid_path) {
+
+  int str_len = strlen(TT_SYS_DIR) + strlen(jobid) + strlen(taskid) + strlen(
+      tt_root);
+  *pid_path = NULL;
+
+  if (mapred_local_dir == NULL) {
+    if (get_mapred_local_dir() < 0) {
+      return;
+    }
+  }
+
+  *pid_path = (char *) malloc(sizeof(char) * (str_len + 1));
+
+  if (*pid_path == NULL) {
+    fprintf(LOGFILE, "unable to allocate memory for pid path\n");
+    return;
+  }
+  memset(*pid_path,'\0',str_len+1);
+  snprintf(*pid_path, str_len, TT_SYS_DIR, tt_root, jobid, taskid);
+#ifdef DEBUG
+  fprintf(LOGFILE, "get_pid_path : pid path = %s\n", *pid_path);
+  fflush(LOGFILE);
+#endif
+
+}
+
+/*
+ * function to provide path to the task file which is created by the tt
+ *
+ *Check TT_LOCAL_TASK_SCRIPT_PATTERN for pattern
+ */
+void get_task_file_path(const char * jobid, const char * taskid,
+    const char * tt_root, char **task_script_path) {
+  *task_script_path = NULL;
+  int str_len = strlen(TT_LOCAL_TASK_SCRIPT_PATTERN) + strlen(jobid) + (strlen(
+      taskid)) + strlen(tt_root);
+
+  if (mapred_local_dir == NULL) {
+    if (get_mapred_local_dir() < 0) {
+      return;
+    }
+  }
+
+  *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");
+    return;
+  }
+
+  memset(*task_script_path,'\0',str_len+1);
+  snprintf(*task_script_path, str_len, TT_LOCAL_TASK_SCRIPT_PATTERN, tt_root,
+      jobid, taskid);
+#ifdef DEBUG
+  fprintf(LOGFILE, "get_task_file_path : task script path = %s\n", *task_script_path);
+  fflush(LOGFILE);
+#endif
+
+}
+
+//end of private functions
+void display_usage(FILE *stream) {
+  fprintf(stream,
+      "Usage: task-controller [-l logile] user command command-args\n");
+}
+
+//function used to populate and user_details structure.
+
+int get_user_details(const char *user) {
+  if (user_detail == NULL) {
+    user_detail = getpwnam(user);
+    if (user_detail == NULL) {
+      fprintf(LOGFILE, "Invalid user\n");
+      return -1;
+    }
+  }
+  return 0;
+}
+
+/*
+ *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
+ *
+ *Then gets the path to which the task has to write its pid from
+ *get_pid_path.
+ *
+ * THen writes its pid into the file.
+ *
+ * Then changes the permission of the pid file into 777
+ *
+ * Then 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.
+ *
+ */
+
+int run_task_as_user(const char * user, const char *jobid, const char *taskid,
+    const char *tt_root) {
+  char *task_script_path = NULL;
+  char *pid_path = NULL;
+  char *task_script = NULL;
+  FILE *file_handle = NULL;
+  int exit_code = 0;
+  int i = 0;
+
+#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);
+#endif
+
+  if (check_tt_root(tt_root) < 0) {
+    fprintf(LOGFILE, "invalid tt root passed %s\n", tt_root);
+    cleanup();
+    return INVALID_TT_ROOT;
+  }
+
+  get_pid_path(jobid, taskid, tt_root, &pid_path);
+
+  if (pid_path == NULL) {
+    fprintf(LOGFILE, "Invalid task-pid path provided");
+    cleanup();
+    return INVALID_PID_PATH;
+  }
+
+  errno = 0;
+  file_handle = fopen(pid_path, "w");
+
+  if (file_handle == NULL) {
+    fprintf(LOGFILE, "Error opening task-pid file %s :%s\n", pid_path,
+        strerror(errno));
+    exit_code = UNABLE_TO_OPEN_PID_FILE_WRITE_MODE;
+    goto cleanup;
+  }
+
+  errno = 0;
+  if (fprintf(file_handle, "%d\n", getpid()) < 0) {
+    fprintf(LOGFILE, "Error writing to task-pid file :%s\n", strerror(errno));
+    exit_code = UNABLE_TO_WRITE_TO_PID_FILE;
+    goto cleanup;
+  }
+
+  fflush(file_handle);
+  fclose(file_handle);
+  //change the permissions of the file
+  errno = 0;
+  //setting permission to 777
+
+  if (chmod(pid_path, S_IREAD | S_IEXEC | S_IWRITE | S_IROTH | S_IWOTH
+      | S_IXOTH | S_IRGRP | S_IWGRP | S_IXGRP) < 0) {
+    fprintf(LOGFILE, "Error changing permission of %s task-pid file : %s",
+        pid_path, strerror(errno));
+    errno = 0;
+    if (remove(pid_path) < 0) {
+      fprintf(LOGFILE, "Error deleting %s task-pid file : %s", pid_path,
+          strerror(errno));
+      exit_code = UNABLE_TO_CHANGE_PERMISSION_AND_DELETE_PID_FILE;
+    } else {
+      exit_code = UNABLE_TO_CHANGE_PERMISSION_OF_PID_FILE;
+    }
+    goto cleanup;
+  }
+#ifdef DEBUG
+  fprintf(LOGFILE,"changing file ownership\n");
+  fprintf(LOGFILE, "run_task_as_user : uid id %d \n", getuid());
+  fprintf(LOGFILE, "run_task_as_user : gid id %d \n", getgid());
+#endif
+  //change the owner ship of the file to the launching user.
+  if(chown(pid_path, getuid(), getgid()) <0 ) {
+    fprintf(LOGFILE, "Error changing ownership of %s task-pid file : %s",
+        pid_path, strerror(errno));
+    if(remove(pid_path) < 0) {
+      fprintf(LOGFILE, "Error deleting %s task-pid file : %s", pid_path,
+          strerror(errno));
+      exit_code = UNABLE_TO_CHANGE_OWNERSHIP_OF_PID_FILE_AND_DELETE_PID_FILE;
+    } else {
+      exit_code = UNABLE_TO_CHANGE_OWNERSHIP_OF_PID_FILE;
+    }
+    goto cleanup;
+  }
+
+
+  //free pid_t path which is allocated
+  free(pid_path);
+
+  //change the user
+  fcloseall();
+  fclose(LOGFILE);
+  umask(0);
+  if (change_user(user) != 0) {
+    cleanup();
+    return SETUID_OPER_FAILED;
+  }
+  //change set the launching process as the session leader.
+  if(setsid() < 0) {
+    fprintf(LOGFILE,"Set sid failed %s\n", strerror(errno));
+    cleanup();
+    return SETSID_FAILED;
+  }
+
+  get_task_file_path(jobid, taskid, tt_root, &task_script_path);
+
+  if (task_script_path == NULL) {
+    fprintf(LOGFILE, "Unable to locate task script");
+    cleanup();
+    return INVALID_TASK_SCRIPT_PATH;
+  }
+  errno = 0;
+  cleanup();
+  execlp(task_script_path, task_script_path, NULL);
+  if (errno != 0) {
+    fprintf(LOGFILE, "Error execing script %s", strerror(errno));
+    free(task_script_path);
+    exit_code = UNABLE_TO_EXECUTE_TASK_SCRIPT;
+  }
+
+  return exit_code;
+
+cleanup:
+  if (pid_path != NULL) {
+    free(pid_path);
+  }
+  if (task_script_path != NULL) {
+    free(task_script_path);
+  }
+  if (file_handle != NULL) {
+    fclose(file_handle);
+  }
+  // free configurations
+  cleanup();
+  return exit_code;
+}
+/**
+ * Function used to terminate a task launched by the user.
+ *
+ * The function first checks if the passed tt-root is found in
+ * configured hadoop.temp.dir (which is a list of tt_roots).
+ *
+ * Then gets the task-pid path using function get_pid_path.
+ *
+ * reads the task-pid from the file which is mentioned by get_pid_path
+ *
+ * kills the task by sending SIGTERM to that particular process.
+ *
+ */
+
+int kill_user_task(const char *user, const char *jobid, const char *taskid,
+    const char *tt_root) {
+  int pid = 0;
+  int i = 0;
+  char *pid_path = NULL;
+  FILE *file_handle = NULL;
+  const char *sleep_interval_char;
+  int sleep_interval = 0;
+#ifdef DEBUG
+  fprintf(LOGFILE,"kill_user_task : Job id : %s \n", jobid);
+  fprintf(LOGFILE,"kill_user_task : task id : %s \n", taskid);
+  fprintf(LOGFILE,"kill_user_task : tt_root : %s \n", tt_root);
+  fflush(LOGFILE);
+#endif
+  if (check_tt_root(tt_root) < 0) {
+    fprintf(LOGFILE, "invalid tt root specified");
+    cleanup();
+    return INVALID_TT_ROOT;
+  }
+  get_pid_path(jobid, taskid, tt_root, &pid_path);
+  if (pid_path == NULL) {
+    cleanup();
+    return INVALID_PID_PATH;
+  }
+#ifdef DEBUG
+  fprintf(LOGFILE,"kill_user_task : task-pid path :%s \n",pid_path);
+  fflush(LOGFILE);
+#endif
+  file_handle = fopen(pid_path, "r");
+  if (file_handle == NULL) {
+    fprintf(LOGFILE, "unable to open task-pid file :%s \n", pid_path);
+    free(pid_path);
+    cleanup();
+    return UNABLE_TO_OPEN_PID_FILE_READ_MODE;
+  }
+  fscanf(file_handle, "%d", &pid);
+  fclose(file_handle);
+  free(pid_path);
+  if (pid == 0) {
+    fprintf(LOGFILE, "Unable to read task-pid from path: %s \n", pid_path);
+    cleanup();
+    return UNABLE_TO_READ_PID;
+  }
+  if (change_user(user) != 0) {
+    cleanup();
+    return SETUID_OPER_FAILED;
+  }
+  //kill the entire session.
+  if (kill(-pid, SIGTERM) < 0) {
+    fprintf(LOGFILE, "%s\n", strerror(errno));
+    cleanup();
+    return UNABLE_TO_KILL_TASK;
+  }
+  //get configured interval time.
+  sleep_interval_char = get_value("mapred.tasktracker.tasks.sleeptime-before-sigkill");
+  if(sleep_interval_char != NULL) {
+    sleep_interval = atoi(sleep_interval_char);
+  }
+  if(sleep_interval == 0) {
+    sleep_interval = 5;
+  }
+  //sleep for configured interval.
+  sleep(sleep_interval);
+  //check pid exists
+  if(kill(-pid,0) == 0) {
+    //if pid present then sigkill it
+    if(kill(-pid, SIGKILL) <0) {
+      //ignore no such pid present.
+      if(errno != ESRCH) {
+        //log error ,exit unclean
+        fprintf(LOGFILE,"%s\n",strerror(errno));
+        cleanup();
+        return UNABLE_TO_KILL_TASK;
+      }
+    }
+  }
+  cleanup();
+  return 0;
+}

Added: hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/task-controller.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/c%2B%2B/task-controller/task-controller.h?rev=1076893&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/task-controller.h (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/c++/task-controller/task-controller.h Thu Mar  3 23:50:57 2011
@@ -0,0 +1,93 @@
+/**
+ * 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 <stdio.h>
+#include <stdlib.h>
+#include <stdarg.h>
+#include <string.h>
+#include <errno.h>
+#include <unistd.h>
+#include <sys/types.h>
+#include <pwd.h>
+#include <assert.h>
+#include <getopt.h>
+#include <sys/stat.h>
+#include <sys/signal.h>
+#include <getopt.h>
+#include "configuration.h"
+
+//command definitions
+enum command {
+  RUN_TASK,
+  KILL_TASK
+};
+
+enum errorcodes {
+  INVALID_ARGUMENT_NUMBER = 1,
+  INVALID_USER_NAME,
+  INVALID_COMMAND_PROVIDED,
+  SUPER_USER_NOT_ALLOWED_TO_RUN_TASKS,
+  OUT_OF_MEMORY,
+  INVALID_TT_ROOT,
+  INVALID_PID_PATH,
+  UNABLE_TO_OPEN_PID_FILE_WRITE_MODE,
+  UNABLE_TO_OPEN_PID_FILE_READ_MODE,
+  UNABLE_TO_WRITE_TO_PID_FILE,
+  UNABLE_TO_CHANGE_PERMISSION_OF_PID_FILE,
+  UNABLE_TO_CHANGE_PERMISSION_AND_DELETE_PID_FILE,
+  UNABLE_TO_CHANGE_OWNERSHIP_OF_PID_FILE,
+  UNABLE_TO_CHANGE_OWNERSHIP_OF_PID_FILE_AND_DELETE_PID_FILE,
+  SETUID_OPER_FAILED,
+  INVALID_TASK_SCRIPT_PATH,
+  UNABLE_TO_EXECUTE_TASK_SCRIPT,
+  UNABLE_TO_READ_PID,
+  UNABLE_TO_KILL_TASK,
+  UNABLE_TO_FIND_PARENT_PID_FILE,
+  TASK_CONTROLLER_SPAWNED_BY_INVALID_PARENT_PROCESS,
+  UNABLE_TO_READ_PARENT_PID,
+  SETSID_FAILED
+};
+
+
+#define TT_PID_PATTERN "%s/hadoop-%s-tasktracker.pid"
+
+#define TT_LOCAL_TASK_SCRIPT_PATTERN "%s/taskTracker/jobcache/%s/%s/taskjvm.sh"
+
+#define TT_SYS_DIR "%s/taskTracker/jobcache/%s/%s/.pid"
+
+#define TT_SYS_DIR_KEY "mapred.local.dir"
+
+#define MAX_ITEMS 10
+
+#ifndef HADOOP_CONF_DIR
+  #define EXEC_PATTERN "/bin/task-controller"
+  extern char * hadoop_conf_dir;
+#endif
+
+extern struct passwd *user_detail;
+
+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 verify_parent();
+
+int kill_user_task(const char *user, const char *jobid, const char *taskid, const char *tt_root);
+
+int get_user_details(const char *user);