Prechádzať zdrojové kódy

YARN-8207. Docker container launch use popen have risk of shell expansion. Contributed by Eric Yang.

Jason Lowe 7 rokov pred
rodič
commit
a2ea756420

+ 64 - 54
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c

@@ -20,6 +20,7 @@
 #include "container-executor.h"
 #include "utils/docker-util.h"
 #include "utils/path-utils.h"
+#include "utils/string-utils.h"
 #include "util.h"
 #include "config.h"
 
@@ -737,7 +738,6 @@ static int create_container_directories(const char* user, const char *app_id,
     result = OUT_OF_MEMORY;
   } else {
     sprintf(combined_name, "%s/%s", app_id, container_id);
-
     char* const* log_dir_ptr;
     for(log_dir_ptr = log_dir; *log_dir_ptr != NULL; ++log_dir_ptr) {
       char *container_log_dir = get_app_log_directory(*log_dir_ptr, combined_name);
@@ -753,10 +753,10 @@ static int create_container_directories(const char* user, const char *app_id,
         free(combined_name);
         return OUT_OF_MEMORY;
       } else if (mkdirs(container_log_dir, perms) != 0) {
-    	free(container_log_dir);
+        free(container_log_dir);
       } else {
-    	result = 0;
-    	free(container_log_dir);
+        free(container_log_dir);
+        result = 0;
       }
     }
     free(combined_name);
@@ -799,7 +799,7 @@ static struct passwd* get_user_info(const char* user) {
         string_size, &result) != 0) {
     free(buffer);
     fprintf(LOGFILE, "Can't get user information %s - %s\n", user,
-	    strerror(errno));
+           strerror(errno));
     return NULL;
   }
   return result;
@@ -1094,7 +1094,6 @@ int initialize_user(const char *user, char* const* local_dirs) {
 }
 
 int create_log_dirs(const char *app_id, char * const * log_dirs) {
-
   char* const* log_root;
   char *any_one_app_log_dir = NULL;
   for(log_root=log_dirs; *log_root != NULL; ++log_root) {
@@ -1275,11 +1274,9 @@ int initialize_app(const char *user, const char *app_id,
   return -1;
 }
 
-char *construct_docker_command(const char *command_file) {
+char **construct_docker_command(const char *command_file) {
   int ret = 0;
-  size_t command_size = MIN(sysconf(_SC_ARG_MAX), 128*1024);
-  char *buffer = alloc_and_clear_memory(command_size, sizeof(char));
-
+  struct args buffer = ARGS_INITIAL_VALUE;
   uid_t user = geteuid();
   gid_t group = getegid();
   if (change_effective_user(nm_uid, nm_gid) != 0) {
@@ -1287,8 +1284,7 @@ char *construct_docker_command(const char *command_file) {
     fflush(ERRORFILE);
     exit(SETUID_OPER_FAILED);
   }
-
-  ret = get_docker_command(command_file, &CFG, buffer, command_size);
+  ret = get_docker_command(command_file, &CFG, &buffer);
   if (ret != 0) {
     fprintf(ERRORFILE, "Error constructing docker command, docker error code=%d, error message='%s'\n", ret,
             get_docker_error_message(ret));
@@ -1302,31 +1298,24 @@ char *construct_docker_command(const char *command_file) {
     exit(SETUID_OPER_FAILED);
   }
 
-  return buffer;
+  char** copy = extract_execv_args(&buffer);
+  return copy;
 }
 
 int run_docker(const char *command_file) {
-  char* docker_command = construct_docker_command(command_file);
+  char **args = construct_docker_command(command_file);
   char* docker_binary = get_docker_binary(&CFG);
-  size_t command_size = MIN(sysconf(_SC_ARG_MAX), 128*1024);
-
-  char* docker_command_with_binary = alloc_and_clear_memory(command_size, sizeof(char));
-  snprintf(docker_command_with_binary, command_size, "%s %s", docker_binary, docker_command);
-  fprintf(LOGFILE, "Invoking '%s'\n", docker_command_with_binary);
-  char **args = split_delimiter(docker_command_with_binary, " ");
-
   int exit_code = -1;
   if (execvp(docker_binary, args) != 0) {
     fprintf(ERRORFILE, "Couldn't execute the container launch with args %s - %s",
               docker_binary, strerror(errno));
-      fflush(LOGFILE);
-      fflush(ERRORFILE);
-      free(docker_binary);
-      free(args);
-      free(docker_command_with_binary);
-      free(docker_command);
-      exit_code = DOCKER_RUN_FAILED;
+    fflush(LOGFILE);
+    fflush(ERRORFILE);
+    free(docker_binary);
+    free_values(args);
+    exit_code = DOCKER_RUN_FAILED;
   } else {
+    free_values(args);
     exit_code = 0;
   }
   return exit_code;
@@ -1452,7 +1441,7 @@ int create_local_dirs(const char * user, const char *app_id,
 
   // Copy script file with permissions 700
   if (copy_file(container_file_source, script_name, script_file_dest,S_IRWXU) != 0) {
-    fprintf(ERRORFILE, "Could not create copy file %d %s\n", container_file_source, script_file_dest);
+    fprintf(ERRORFILE, "Could not create copy file %s %s (%d)\n", script_name, script_file_dest, container_file_source);
     fflush(ERRORFILE);
     exit_code = COULD_NOT_CREATE_SCRIPT_COPY;
     goto cleanup;
@@ -1513,25 +1502,15 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
   char *cred_file_dest = NULL;
   char *exit_code_file = NULL;
   char *docker_command_with_binary = NULL;
-  char *docker_wait_command = NULL;
   char *docker_inspect_command = NULL;
-  char *docker_rm_command = NULL;
   char *docker_inspect_exitcode_command = NULL;
   int container_file_source =-1;
   int cred_file_source = -1;
 
-  size_t command_size = MIN(sysconf(_SC_ARG_MAX), 128*1024);
-
-  docker_command_with_binary = (char *) alloc_and_clear_memory(command_size, sizeof(char));
-  docker_wait_command = (char *) alloc_and_clear_memory(command_size, sizeof(char));
-  docker_inspect_command = (char *) alloc_and_clear_memory(command_size, sizeof(char));
-  docker_rm_command = (char *) alloc_and_clear_memory(command_size, sizeof(char));
-  docker_inspect_exitcode_command = (char *) alloc_and_clear_memory(command_size, sizeof(char));
-
   gid_t user_gid = getegid();
   uid_t prev_uid = geteuid();
 
-  char *docker_command = NULL;
+  char **docker_command = NULL;
   char *docker_binary = NULL;
 
   fprintf(LOGFILE, "Creating script paths...\n");
@@ -1581,21 +1560,31 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
     goto cleanup;
   }
 
-  snprintf(docker_command_with_binary, command_size, "%s %s", docker_binary, docker_command);
+  docker_command_with_binary = flatten(docker_command);
 
-  fprintf(LOGFILE, "Launching docker container...\n");
-  fprintf(LOGFILE, "Docker run command: %s\n", docker_command_with_binary);
-  FILE* start_docker = popen(docker_command_with_binary, "r");
-  if (WEXITSTATUS(pclose (start_docker)) != 0)
-  {
+  // Launch container
+  pid_t child_pid = fork();
+  if (child_pid == -1) {
     fprintf (ERRORFILE,
-     "Could not invoke docker %s.\n", docker_command_with_binary);
+      "Could not invoke docker %s.\n", docker_command_with_binary);
     fflush(ERRORFILE);
     exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
     goto cleanup;
   }
 
-  snprintf(docker_inspect_command, command_size,
+  if (child_pid == 0) {
+    execvp(docker_binary, docker_command);
+    fprintf(ERRORFILE, "failed to execute docker command! error: %s\n", strerror(errno));
+    return UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
+  } else {
+    exit_code = wait_and_get_exit_code(child_pid);
+    if (exit_code != 0) {
+      exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
+      goto cleanup;
+    }
+  }
+
+  docker_inspect_command = make_string(
     "%s inspect --format {{.State.Pid}} %s",
     docker_binary, container_id);
 
@@ -1643,10 +1632,10 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
     }
 
     fprintf(LOGFILE, "Waiting for docker container to finish.\n");
+
+    // wait for pid to finish
 #ifdef __linux
-    size_t command_size = MIN(sysconf(_SC_ARG_MAX), 128*1024);
-    char* proc_pid_path = alloc_and_clear_memory(command_size, sizeof(char));
-    snprintf(proc_pid_path, command_size, "%s/%d", PROC_PATH, pid);
+    char* proc_pid_path = make_string("%s/%d", PROC_PATH, pid);
     while (dir_exists(proc_pid_path) == 0) {
       sleep(1);
     }
@@ -1661,7 +1650,8 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
 #endif
   }
 
-  sprintf(docker_inspect_exitcode_command,
+  // discover container exit code
+  docker_inspect_exitcode_command = make_string(
     "%s inspect --format {{.State.ExitCode}} %s",
   docker_binary, container_id);
   fprintf(LOGFILE, "Obtaining the exit code...\n");
@@ -1713,9 +1703,8 @@ cleanup:
   free(script_file_dest);
   free(cred_file_dest);
   free(docker_command_with_binary);
-  free(docker_wait_command);
   free(docker_inspect_command);
-  free(docker_rm_command);
+  free_values(docker_command);
   return exit_code;
 }
 
@@ -2404,3 +2393,24 @@ int traffic_control_read_stats(char *command_file) {
 struct configuration* get_cfg() {
   return &CFG;
 }
+
+/**
+ * Flatten docker launch command
+ */
+char* flatten(char **args) {
+  size_t total = 1;
+  for (int i = 0; args[i] != NULL; i++) {
+    total = total + strlen(args[i]) + 1;
+  }
+  char *buffer = (char *) malloc(total * sizeof(char));
+  char *to = NULL;
+  to = buffer;
+  for (int i = 0; args[i] != NULL; i++) {
+    to = stpcpy(to, args[i]);
+    to = stpcpy(to, " ");
+  }
+  *to = '\0';
+  return buffer;
+}
+
+

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h

@@ -284,3 +284,8 @@ int execute_regex_match(const char *regex_str, const char *input);
 int validate_docker_image_name(const char *image_name);
 
 struct configuration* get_cfg();
+
+/**
+ * Flatten docker launch command
+ */
+char* flatten(char **args);

Rozdielové dáta súboru neboli zobrazené, pretože súbor je príliš veľký
+ 238 - 267
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c


+ 27 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.h

@@ -33,7 +33,13 @@
 #define DOCKER_KILL_COMMAND "kill"
 #define DOCKER_VOLUME_COMMAND "volume"
 #define DOCKER_START_COMMAND "start"
+#define DOCKER_ARG_MAX 1024
+#define ARGS_INITIAL_VALUE { 0 };
 
+typedef struct args {
+    int length;
+    char *data[DOCKER_ARG_MAX];
+} args;
 
 enum docker_error_codes {
     INVALID_COMMAND_FILE = 1,
@@ -77,7 +83,7 @@ char *get_docker_binary(const struct configuration *conf);
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_command(const char* command_file, const struct configuration* conf, char *out, const size_t outlen);
+int get_docker_command(const char* command_file, const struct configuration* conf, args *args);
 
 /**
  * Get the Docker inspect command line string. The function will verify that the params file is meant for the
@@ -88,7 +94,7 @@ int get_docker_command(const char* command_file, const struct configuration* con
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_inspect_command(const char* command_file, const struct configuration* conf, char *out, const size_t outlen);
+int get_docker_inspect_command(const char* command_file, const struct configuration* conf, args *args);
 
 /**
  * Get the Docker load command line string. The function will verify that the params file is meant for the load command.
@@ -98,7 +104,7 @@ int get_docker_inspect_command(const char* command_file, const struct configurat
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_load_command(const char* command_file, const struct configuration* conf, char *out, const size_t outlen);
+int get_docker_load_command(const char* command_file, const struct configuration* conf, args *args);
 
 /**
  * Get the Docker pull command line string. The function will verify that the params file is meant for the pull command.
@@ -108,7 +114,7 @@ int get_docker_load_command(const char* command_file, const struct configuration
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_pull_command(const char* command_file, const struct configuration* conf, char *out, const size_t outlen);
+int get_docker_pull_command(const char* command_file, const struct configuration* conf, args *args);
 
 /**
  * Get the Docker rm command line string. The function will verify that the params file is meant for the rm command.
@@ -118,7 +124,7 @@ int get_docker_pull_command(const char* command_file, const struct configuration
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_rm_command(const char* command_file, const struct configuration* conf, char *out, const size_t outlen);
+int get_docker_rm_command(const char* command_file, const struct configuration* conf, args *args);
 
 /**
  * Get the Docker run command line string. The function will verify that the params file is meant for the run command.
@@ -128,7 +134,7 @@ int get_docker_rm_command(const char* command_file, const struct configuration*
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_run_command(const char* command_file, const struct configuration* conf, char *out, const size_t outlen);
+int get_docker_run_command(const char* command_file, const struct configuration* conf, args *args);
 
 /**
  * Get the Docker stop command line string. The function will verify that the params file is meant for the stop command.
@@ -138,7 +144,7 @@ int get_docker_run_command(const char* command_file, const struct configuration*
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_stop_command(const char* command_file, const struct configuration* conf, char *out, const size_t outlen);
+int get_docker_stop_command(const char* command_file, const struct configuration* conf, args *args);
 
 /**
  * Get the Docker kill command line string. The function will verify that the params file is meant for the kill command.
@@ -148,7 +154,7 @@ int get_docker_stop_command(const char* command_file, const struct configuration
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_kill_command(const char* command_file, const struct configuration* conf, char *out, const size_t outlen);
+int get_docker_kill_command(const char* command_file, const struct configuration* conf, args *args);
 
 /**
  * Get the Docker volume command line string. The function will verify that the
@@ -159,8 +165,7 @@ int get_docker_kill_command(const char* command_file, const struct configuration
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_volume_command(const char *command_file, const struct configuration *conf, char *out,
-                               const size_t outlen);
+int get_docker_volume_command(const char *command_file, const struct configuration *conf, args *args);
 
 /**
  * Get the Docker start command line string. The function will verify that the params file is meant for the start command.
@@ -170,7 +175,7 @@ int get_docker_volume_command(const char *command_file, const struct configurati
  * @param outlen Size of the output buffer
  * @return Return code with 0 indicating success and non-zero codes indicating error
  */
-int get_docker_start_command(const char* command_file, const struct configuration* conf, char *out, const size_t outlen);
+int get_docker_start_command(const char* command_file, const struct configuration* conf, args *args);
 
 /**
  * Give an error message for the supplied error code
@@ -186,4 +191,15 @@ const char *get_docker_error_message(const int error_code);
  */
 int docker_module_enabled(const struct configuration *conf);
 
+/**
+ * Helper function to reset args data structure.
+ * @param args Pointer reference to args data structure
+ */
+void reset_args(args *args);
+
+/**
+ * Extract execv args from args data structure.
+ * @param args Pointer reference to args data structure
+ */
+char** extract_execv_args(args *args);
 #endif

+ 24 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c

@@ -22,6 +22,7 @@
 #include <strings.h>
 #include <string.h>
 #include <stdlib.h>
+#include <stdarg.h>
 
 /*
  * if all chars in the input str are numbers
@@ -156,3 +157,26 @@ cleanup:
   }
   return is_container_id;
 }
+
+/*
+ * Format string utility.
+ */
+char *make_string(const char *fmt, ...) {
+  va_list vargs;
+  va_start(vargs, fmt);
+  size_t buflen = vsnprintf(NULL, 0, fmt, vargs) + 1;
+  va_end(vargs);
+  if (buflen <= 0) {
+    return NULL;
+  }
+  char* buf = malloc(buflen);
+  if (buf != NULL) {
+    va_start(vargs, fmt);
+    int ret = vsnprintf(buf, buflen, fmt, vargs);
+    va_end(vargs);
+    if (ret < 0) {
+      buf = NULL;
+    }
+  }
+  return buf;
+}

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h

@@ -34,4 +34,8 @@ int validate_container_id(const char* input);
  */
 int get_numbers_split_by_comma(const char* input, int** numbers, size_t* n_numbers);
 
+/*
+ * String format utility
+ */
+char *make_string(const char *fmt, ...);
 #endif

Rozdielové dáta súboru neboli zobrazené, pretože súbor je príliš veľký
+ 226 - 188
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test_docker_util.cc


Niektoré súbory nie sú zobrazené, pretože je v týchto rozdielových dátach zmenené mnoho súborov