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 aa...@apache.org on 2021/06/05 22:29:56 UTC

[hadoop] branch branch-3.3 updated: Fix container-executor

This is an automated email from the ASF dual-hosted git repository.

aajisaka pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new 8ca57fc  Fix container-executor
8ca57fc is described below

commit 8ca57fc9f567b37d82d286feb77d35e670fc635e
Author: Hideyuki Furue <hf...@yahoo-corp.jp>
AuthorDate: Wed Jun 2 19:56:38 2021 +0900

    Fix container-executor
    
    Signed-off-by: Akira Ajisaka <aa...@apache.org>
    (cherry picked from commit 9c7b8cf54ea88833d54fc71a9612c448dc0eb78d)
---
 .../container-executor/impl/container-executor.c   | 130 ++++++++++++++++-----
 .../src/main/native/container-executor/impl/main.c |   4 +-
 .../container-executor/impl/runc/runc_reap.c       |   6 +-
 .../src/main/native/container-executor/impl/util.c |  10 +-
 .../container-executor/impl/utils/docker-util.c    |   4 +-
 .../container-executor/impl/utils/string-utils.c   |   1 +
 .../test/test-container-executor.c                 |  41 +++++++
 7 files changed, 158 insertions(+), 38 deletions(-)

diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
index d69acf3..ab03d39 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
@@ -49,6 +49,10 @@
 #include <getopt.h>
 #include <sys/param.h>
 #include <termios.h>
+#ifdef __linux
+#include <sys/vfs.h>
+#include <linux/magic.h>
+#endif
 
 #ifndef HAVE_FCHMODAT
 #include "compat/fchmodat.h"
@@ -231,6 +235,19 @@ static int write_pid_to_cgroup_as_root(const char* cgroup_file, pid_t pid) {
     goto cleanup;
   }
 
+  // statfs
+  struct statfs buf;
+  if (statfs(cgroup_file, &buf) == -1) {
+    fprintf(LOGFILE, "Can't statfs file %s as node manager - %s\n", cgroup_file,
+           strerror(errno));
+    rc = -1;
+    goto cleanup;
+  } else if (buf.f_type != CGROUP_SUPER_MAGIC) {
+    fprintf(LOGFILE, "Pid file %s is not located on cgroup filesystem\n", cgroup_file);
+    rc = -1;
+    goto cleanup;
+  }
+
   // open
   int cgroup_fd = open(cgroup_file, O_WRONLY | O_APPEND, 0);
   if (cgroup_fd == -1) {
@@ -524,27 +541,16 @@ int is_runc_support_enabled() {
 
 /**
  * Utility function to concatenate argB to argA using the concat_pattern.
+ * For historical reasons, redundant argument numArgs exists.
  */
 char *concatenate(char *concat_pattern, char *return_path_name,
                   int numArgs, ...) {
   va_list ap;
   va_start(ap, numArgs);
-  int strlen_args = 0;
-  char *arg = NULL;
-  int j;
-  for (j = 0; j < numArgs; j++) {
-    arg = va_arg(ap, char*);
-    if (arg == NULL) {
-      fprintf(LOGFILE, "One of the arguments passed for %s is null.\n",
-          return_path_name);
-      return NULL;
-    }
-    strlen_args += strlen(arg);
-  }
+  int str_len = vsnprintf(NULL, 0, concat_pattern, ap) + 1;
   va_end(ap);
 
   char *return_path = NULL;
-  int str_len = strlen(concat_pattern) + strlen_args + 1;
 
   return_path = (char *) malloc(str_len);
   if (return_path == NULL) {
@@ -785,6 +791,10 @@ static int create_container_directories(const char* user, const char *app_id,
     if (container_dir == NULL) {
       return OUT_OF_MEMORY;
     }
+    if (strstr(container_dir, "..") != 0) {
+      fprintf(LOGFILE, "Unsupported container directory path detected.\n");
+      return COULD_NOT_CREATE_WORK_DIRECTORIES;
+    }
     if (mkdirs(container_dir, perms) == 0) {
       result = 0;
     }
@@ -812,19 +822,26 @@ static int create_container_directories(const char* user, const char *app_id,
       char *container_log_dir = get_app_log_directory(*log_dir_ptr, combined_name);
       int check = check_nm_local_dir(nm_uid, *log_dir_ptr);
       if (check != 0) {
-        container_log_dir = NULL;
-      }
-      if (strstr(container_log_dir, "..") != 0) {
-        fprintf(LOGFILE, "Unsupported container log directory path detected.\n");
-        container_log_dir = NULL;
+        free(container_log_dir);
+        free(combined_name);
+        return COULD_NOT_CREATE_APP_LOG_DIRECTORIES;
       }
       if (container_log_dir == NULL) {
         free(combined_name);
         return OUT_OF_MEMORY;
+      }
+      if (strstr(container_log_dir, "..") != 0) {
+        fprintf(LOGFILE, "Unsupported container log directory path detected.\n");
+        free(container_log_dir);
+        free(combined_name);
+        return COULD_NOT_CREATE_APP_LOG_DIRECTORIES;
       } else if (mkdirs(container_log_dir, logdir_perms) != 0) {
         free(container_log_dir);
       } else {
         result = 0;
+        if (chosen_container_log_dir != NULL) {
+          free(chosen_container_log_dir);
+        }
         chosen_container_log_dir = strdup(container_log_dir);
         free(container_log_dir);
       }
@@ -845,6 +862,12 @@ static int create_container_directories(const char* user, const char *app_id,
     return OUT_OF_MEMORY;
   }
 
+  if (strstr(tmp_dir, "..") != 0 || strstr(private_tmp_dir, "..") != 0 || strstr(private_var_tmp_dir, "..") != 0) {
+    fprintf(ERRORFILE, "Unsupported tmp directory path detected.\n");
+    result = COULD_NOT_CREATE_TMP_DIRECTORIES;
+    goto cleanup;
+  }
+
   if (mkdirs(tmp_dir, perms) != 0) {
     fprintf(ERRORFILE, "Could not create tmp_dir: %s\n", tmp_dir);
     result = COULD_NOT_CREATE_TMP_DIRECTORIES;
@@ -857,7 +880,7 @@ static int create_container_directories(const char* user, const char *app_id,
     goto cleanup;
   }
 
-  // clear group sticky bit on private_tmp_dir
+  // clear setgid bit on private_tmp_dir
   if (chmod(private_tmp_dir, perms) != 0) {
     fprintf(ERRORFILE, "Could not chmod private_tmp_dir: %s\n", private_tmp_dir);
     result = COULD_NOT_CREATE_TMP_DIRECTORIES;
@@ -870,7 +893,7 @@ static int create_container_directories(const char* user, const char *app_id,
     goto cleanup;
   }
 
-  // clear group sticky bit on private_tmp_dir
+  // clear setgid bit on private_tmp_dir
   if (chmod(private_var_tmp_dir, perms) != 0) {
     fprintf(ERRORFILE, "Could not chmod private_var_tmp_dir: %s\n", private_var_tmp_dir);
     result = COULD_NOT_CREATE_TMP_DIRECTORIES;
@@ -1053,7 +1076,7 @@ static int change_owner(const char* path, uid_t user, gid_t group) {
  * return non-0 on failure
  */
 int create_directory_for_user(const char* path) {
-  // set 2750 permissions and group sticky bit
+  // set 750 permissions and setgid bit
   mode_t permissions = S_IRWXU | S_IRGRP | S_IXGRP | S_ISGID;
   uid_t user = geteuid();
   gid_t group = getegid();
@@ -1066,13 +1089,13 @@ int create_directory_for_user(const char* path) {
 
   if (ret == 0) {
     if (0 == mkdir(path, permissions) || EEXIST == errno) {
-      // need to reassert the group sticky bit
+      // need to reassert the setgid bit
       if (change_owner(path, user, nm_gid) != 0) {
         fprintf(LOGFILE, "Failed to chown %s to %d:%d: %s\n", path, user, nm_gid,
             strerror(errno));
         ret = -1;
       } else if (chmod(path, permissions) != 0) {
-        fprintf(LOGFILE, "Can't chmod %s to add the sticky bit - %s\n",
+        fprintf(LOGFILE, "Can't chmod %s to add the setgid bit - %s\n",
                 path, strerror(errno));
         ret = -1;
       }
@@ -1212,6 +1235,11 @@ int initialize_user(const char *user, char* const* local_dirs) {
       fprintf(LOGFILE, "Couldn't get userdir directory for %s.\n", user);
       failed = 1;
       break;
+    // Avoid possible wrong validation. Username can contain double dots.
+    } else if (strstr(user_dir, "/../") != 0) {
+      fprintf(LOGFILE, "Unsupported userdir directory path detected.\n");
+      failed = 1;
+      break;
     }
     if (create_directory_for_user(user_dir) != 0) {
       failed = 1;
@@ -1233,6 +1261,9 @@ int create_log_dirs(const char *app_id, char * const * log_dirs) {
     }
     if (app_log_dir == NULL) {
       // try the next one
+    } else if (strstr(app_log_dir, "..") != 0) {
+      fprintf(LOGFILE, "Unsupported app-log directory path detected.\n");
+      free(app_log_dir);
     } else if (create_directory_for_user(app_log_dir) != 0) {
       free(app_log_dir);
       return -1;
@@ -1301,7 +1332,11 @@ int create_container_log_dirs(const char *container_id, const char *app_id,
     }
 
     int result = check_nm_local_dir(nm_uid, *log_root);
-    if (result != 0 && container_log_dir != NULL) {
+    if (result != 0) {
+      free(container_log_dir);
+      container_log_dir = NULL;
+      continue;
+    } else if (strstr(container_log_dir, "..") != 0) {
       fprintf(LOGFILE, "Unsupported container log directory path (%s) detected.\n",
               container_log_dir);
       free(container_log_dir);
@@ -1346,6 +1381,9 @@ static char *create_app_dirs(const char *user,
     char *app_dir = get_app_directory(*nm_root, user, app_id);
     if (app_dir == NULL) {
       // try the next one
+    } else if (strstr(app_dir, "..") != 0) {
+      fprintf(LOGFILE, "Unsupported app directory path detected.\n");
+      free(app_dir);
     } else if (mkdirs(app_dir, permissions) != 0) {
       free(app_dir);
     } else if (primary_app_dir == NULL) {
@@ -1412,7 +1450,7 @@ int initialize_app(const char *user, const char *app_id,
 
   char *nmPrivate_credentials_file_copy = strdup(nmPrivate_credentials_file);
   // TODO: FIXME. The user's copy of creds should go to a path selected by
-  // localDirAllocatoir
+  // localDirAllocator
   char *cred_file_name = concatenate("%s/%s", "cred file", 2,
 				   primary_app_dir, basename(nmPrivate_credentials_file_copy));
   if (cred_file_name == NULL) {
@@ -1742,14 +1780,14 @@ int create_script_paths(const char *work_dir,
   int exit_code = -1;
 
   *script_file_dest = get_container_launcher_file(work_dir);
-  if (script_file_dest == NULL) {
+  if (*script_file_dest == NULL) {
     exit_code = OUT_OF_MEMORY;
     fprintf(ERRORFILE, "Could not create script_file_dest\n");
     return exit_code;
   }
 
   *cred_file_dest = get_container_credentials_file(work_dir);
-  if (NULL == cred_file_dest) {
+  if (NULL == *cred_file_dest) {
     exit_code = OUT_OF_MEMORY;
     fprintf(ERRORFILE, "Could not create cred_file_dest\n");
     return exit_code;
@@ -1757,13 +1795,13 @@ int create_script_paths(const char *work_dir,
 
   if (https == 1) {
     *keystore_file_dest = get_container_keystore_file(work_dir);
-    if (NULL == keystore_file_dest) {
+    if (NULL == *keystore_file_dest) {
       exit_code = OUT_OF_MEMORY;
       fprintf(ERRORFILE, "Could not create keystore_file_dest\n");
       return exit_code;
     }
     *truststore_file_dest = get_container_truststore_file(work_dir);
-    if (NULL == truststore_file_dest) {
+    if (NULL == *truststore_file_dest) {
       exit_code = OUT_OF_MEMORY;
       fprintf(ERRORFILE, "Could not create truststore_file_dest\n");
       return exit_code;
@@ -1917,6 +1955,12 @@ int create_user_filecache_dirs(const char * user, char* const* local_dirs) {
       rc = INITIALIZE_USER_FAILED;
       break;
     }
+    if (strstr(filecache_dir, "..") != 0) {
+      fprintf(LOGFILE, "Unsupported filecache directory path detected.\n");
+      free(filecache_dir);
+      rc = INITIALIZE_USER_FAILED;
+      break;
+    }
     if (0 != mkdir(filecache_dir, permissions) && EEXIST != errno) {
       fprintf(LOGFILE, "Failed to create directory %s - %s\n", filecache_dir,
               strerror(errno));
@@ -1941,6 +1985,12 @@ int create_yarn_sysfs(const char* user, const char *app_id,
       return OUT_OF_MEMORY;
     }
     char *yarn_sysfs_dir = make_string("%s/%s", container_dir, "sysfs");
+    if (strstr(yarn_sysfs_dir, "..") != 0) {
+      fprintf(LOGFILE, "Unsupported yarn sysfs directory path detected.\n");
+      free(yarn_sysfs_dir);
+      free(container_dir);
+      return OUT_OF_MEMORY;
+    }
     if (mkdir(yarn_sysfs_dir, perms) == 0) {
       result = 0;
     }
@@ -2062,7 +2112,6 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
   if (exit_code != 0) {
     fprintf(ERRORFILE, "Could not create user yarn sysfs directory\n");
     exit(-1);
-    goto cleanup;
   }
 
   docker_command = construct_docker_command(command_file);
@@ -2096,6 +2145,12 @@ int launch_docker_container_as_user(const char * user, const char *app_id,
 
   docker_command_with_binary = flatten(docker_command);
 
+  if (docker_command_with_binary == NULL) {
+    fprintf (ERRORFILE, "Could not flatten docker command.\n");
+    exit_code = UNABLE_TO_EXECUTE_CONTAINER_SCRIPT;
+    goto cleanup;
+  }
+
   // Launch container
   pid_t child_pid = fork();
   if (child_pid == -1) {
@@ -2800,6 +2855,7 @@ int list_as_user(const char *target_dir) {
             strerror(errno));
         ret = -1;
       }
+      closedir(dir);
     } else {
       fprintf(LOGFILE, "Could not open directory %s - %s\n", target_dir,
           strerror(errno));
@@ -2857,8 +2913,10 @@ int is_empty(char *target_dir) {
       continue;
     }
     fprintf(LOGFILE, "Directory is not empty %s\n", target_dir);
+    closedir(dir);
     return 0;
   }
+  closedir(dir);
   return 1;
 }
 
@@ -2886,7 +2944,7 @@ int mount_cgroup(const char *pair, const char *hierarchy) {
     goto cleanup;
   }
   if (hierarchy == NULL || strstr(hierarchy, "..") != NULL) {
-    fprintf(LOGFILE, "Unsupported cgroup hierarhy path detected.\n");
+    fprintf(LOGFILE, "Unsupported cgroup hierarchy path detected.\n");
     result = INVALID_COMMAND_PROVIDED;
     goto cleanup;
   }
@@ -2907,8 +2965,13 @@ int mount_cgroup(const char *pair, const char *hierarchy) {
       result = INVALID_COMMAND_PROVIDED;
       goto cleanup;
     }
+    if (strlen(mount_path) + strlen(hierarchy) + 2 > EXECUTOR_PATH_MAX) {
+      fprintf(LOGFILE, "cgroup hierarchy path is too long.\n");
+      result = INVALID_COMMAND_PROVIDED;
+      goto cleanup;
+    }
     if (mount("none", mount_path, "cgroup", 0, controller) == 0) {
-      char *buf = stpncpy(hier_path, mount_path, strlen(mount_path));
+      char *buf = stpncpy(hier_path, mount_path, EXECUTOR_PATH_MAX);
       *buf++ = '/';
       snprintf(buf, EXECUTOR_PATH_MAX - (buf - hier_path), "%s", hierarchy);
 
@@ -3083,6 +3146,9 @@ char* flatten(char **args) {
     total = total + strlen(args[i]) + 1;
   }
   char *buffer = (char *) malloc(total * sizeof(char));
+  if (buffer == NULL) {
+    return NULL;
+  }
   char *to = NULL;
   to = buffer;
   for (int i = 0; args[i] != NULL; i++) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
index ff59b96..1b91e8a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
@@ -68,8 +68,8 @@ static void display_usage(FILE *stream) {
   fprintf(stream,
       "       container-executor <user> <yarn-user> <command> <command-args>\n"
       "       where command and command-args: \n" \
-      "            initialize container:  %2d appid tokens nm-local-dirs "
-      "nm-log-dirs cmd app...\n"
+      "            initialize container:  %2d appid containerid tokens nm-local-dirs "
+      "nm-log-dirs cmd...\n"
       "            launch container:      %2d appid containerid workdir "
       "container-script tokens http-option pidfile nm-local-dirs nm-log-dirs resources ",
       INITIALIZE_CONTAINER, LAUNCH_CONTAINER);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/runc/runc_reap.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/runc/runc_reap.c
index b67c604..ff5329b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/runc/runc_reap.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/runc/runc_reap.c
@@ -446,6 +446,7 @@ static struct mntent* get_layer_mounts(size_t* num_entries_out,
   size_t num_entries = 0;
   size_t entries_capacity = num_entries_per_alloc;
   struct mntent* entries = malloc(sizeof(*entries) * entries_capacity);
+  struct mntent* new_entries;
   if (entries == NULL) {
     fputs("Unable to allocate memory\n", ERRORFILE);
     goto fail;
@@ -484,11 +485,12 @@ static struct mntent* get_layer_mounts(size_t* num_entries_out,
 
     if (num_entries == entries_capacity) {
       entries_capacity += num_entries_per_alloc;
-      entries = realloc(entries, sizeof(*entries) * entries_capacity);
-      if (entries == NULL) {
+      new_entries = realloc(entries, sizeof(*entries) * entries_capacity);
+      if (new_entries == NULL) {
         fputs("Unable to allocate memory\n", ERRORFILE);
         goto fail;
       }
+      entries = new_entries;
     }
 
     if (!copy_mntent(entries + num_entries, me)) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c
index 9567ccc..c8ee7b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.c
@@ -25,6 +25,7 @@
 
 char** split_delimiter(char *value, const char *delim) {
   char **return_values = NULL;
+  char **new_return_values;
   char *temp_tok = NULL;
   char *tempstr = NULL;
   int size = 0;
@@ -60,8 +61,15 @@ char** split_delimiter(char *value, const char *delim) {
       // Make sure returned values has enough space for the trailing NULL.
       if (size >= return_values_size - 1) {
         return_values_size += per_alloc_size;
-        return_values = (char **) realloc(return_values,(sizeof(char *) *
+        new_return_values = (char **) realloc(return_values,(sizeof(char *) *
           return_values_size));
+        if (!new_return_values) {
+          fprintf(ERRORFILE, "Reallocation error for return_values in %s.\n",
+                  __func__);
+          failed = 1;
+          goto cleanup;
+        }
+        return_values = new_return_values;
 
         // Make sure new added memory are filled with NULL
         for (int i = size; i < return_values_size; i++) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
index 8bc66b3..b81468a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/docker-util.c
@@ -1385,7 +1385,9 @@ static char* get_docker_mount_options_string(mount_options *options) {
       return NULL;
     }
 
-    idx += sprintf(options_string, "%s", options->opts[0]);
+    if (options->num_opts > 0) {
+        idx += sprintf(options_string, "%s", options->opts[0]);
+    }
     for (i = 1; i < options->num_opts; i++) {
         idx += sprintf(options_string + idx, ",%s", options->opts[i]);
     }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
index 62d54a9..e9e733d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
@@ -181,6 +181,7 @@ char *make_string(const char *fmt, ...) {
     int ret = vsnprintf(buf, buflen, fmt, vargs);
     va_end(vargs);
     if (ret < 0) {
+      free(buf);
       buf = NULL;
     }
   }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
index f75a5ea..f209ea5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
@@ -1322,6 +1322,46 @@ void test_trim_function() {
   free(trimmed);
 }
 
+/**
+ * This test is used to verify that concatenate() works correctly
+ */
+void test_concatenate() {
+  char *concatenate(char *concat_pattern, char *return_path_name, int numArgs, ...);
+  printf("\nTesting concatenate function\n");
+
+  // numArgs: 0
+  char *expected1 = "fixed1";
+  char *actual1 = concatenate("fixed1", "test1", 0);
+  if (actual1 == NULL || strcmp(actual1, expected1) != 0) {
+    printf("FAIL: concatenate: test1: expected %s got %s\n", expected1, actual1);
+    exit(1);
+  }
+
+  // numArgs: 1
+  char *expected2 = "fixed1/var1";
+  char *actual2 = concatenate("fixed1/%s", "test2", 1, "var1");
+  if (actual2 == NULL || strcmp(actual2, expected2) != 0) {
+    printf("FAIL: concatenate: test2: expected %s got %s\n", expected2, actual2);
+    exit(1);
+  }
+
+  // numArgs: 2
+  char *expected3 = "fixed1/var1/fixed2/var2";
+  char *actual3 = concatenate("fixed1/%s/fixed2/%s", "test3", 2, "var1", "var2");
+  if (actual3 == NULL || strcmp(actual3, expected3) != 0) {
+    printf("FAIL: concatenate: test3: expected %s got %s\n", expected3, actual3);
+    exit(1);
+  }
+
+  // concat_pattern with field width
+  char *expected4 = "[x         ]";
+  char *actual4 = concatenate("[%-10s]", "test4", 1, "x");
+  if (actual4 == NULL || strcmp(actual4, expected4) != 0) {
+    printf("FAIL: concatenate: test4: expected %s got %s\n", expected4, actual4);
+    exit(1);
+  }
+}
+
 int is_empty(char *name);
 
 void test_is_empty() {
@@ -1762,6 +1802,7 @@ int main(int argc, char **argv) {
 #endif
 
   test_trim_function();
+  test_concatenate();
   printf("\nFinished tests\n");
 
   printf("\nAttempting to clean up from the run\n");

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org