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 eb...@apache.org on 2019/12/06 18:46:50 UTC

[hadoop] branch trunk updated: YARN-9561. Add C changes for the new RuncContainerRuntime. Contributed by Eric Badger

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

ebadger pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 76bb297  YARN-9561. Add C changes for the new RuncContainerRuntime. Contributed by Eric Badger
76bb297 is described below

commit 76bb297db9543eb66eee9d9306b7172378efdbb5
Author: Eric Badger <eb...@verizonmedia.com>
AuthorDate: Fri Dec 6 18:33:58 2019 +0000

    YARN-9561. Add C changes for the new RuncContainerRuntime. Contributed by Eric Badger
---
 LICENSE.txt                                        |  23 ++
 .../src/CMakeLists.txt                             |  31 +-
 .../container-executor/impl/container-executor.c   | 157 +++++++++-
 .../container-executor/impl/container-executor.h   |  29 +-
 .../src/main/native/container-executor/impl/main.c | 133 +++++----
 .../src/main/native/container-executor/impl/util.c |   6 +
 .../src/main/native/container-executor/impl/util.h |   5 +-
 .../container-executor/impl/utils/string-utils.c   | 172 +++++++++++
 .../container-executor/impl/utils/string-utils.h   |  67 +++++
 .../native/container-executor/test/test_main.cc    |  74 ++++-
 .../test/utils/test-string-utils.cc                | 316 +++++++++++++++------
 11 files changed, 858 insertions(+), 155 deletions(-)

diff --git a/LICENSE.txt b/LICENSE.txt
index d0d5746..c8e90f2 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -256,3 +256,26 @@ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/st
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js
 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/TERMINAL
+
+=======
+For hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/cJSON.[ch]:
+
+Copyright (c) 2009-2017 Dave Gamble and cJSON contributors
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
index 7e8b19f..677429b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
@@ -16,12 +16,15 @@
 
 cmake_minimum_required(VERSION 3.1 FATAL_ERROR)
 
-list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/../../../../../hadoop-common-project/hadoop-common)
+set(HADOOP_COMMON_PATH ${CMAKE_SOURCE_DIR}/../../../../../hadoop-common-project/hadoop-common)
+list(APPEND CMAKE_MODULE_PATH ${HADOOP_COMMON_PATH})
 include(HadoopCommon)
 
 # Set gtest path
 set(GTEST_SRC_DIR ${CMAKE_SOURCE_DIR}/../../../../../hadoop-common-project/hadoop-common/src/main/native/gtest)
 
+set(HADOOP_COMMON_SEC_PATH ${HADOOP_COMMON_PATH}/src/main/native/src/org/apache/hadoop/security)
+
 # determine if container-executor.conf.dir is an absolute
 # path in case the OS we're compiling on doesn't have
 # a hook in get_executable. We'll use this define
@@ -115,6 +118,7 @@ include_directories(
     main/native/container-executor
     main/native/container-executor/impl
     main/native/oom-listener/impl
+    ${HADOOP_COMMON_SEC_PATH}
 )
 # add gtest as system library to suppress gcc warnings
 include_directories(SYSTEM ${GTEST_SRC_DIR}/include)
@@ -129,6 +133,7 @@ add_library(container
     main/native/container-executor/impl/configuration.c
     main/native/container-executor/impl/container-executor.c
     main/native/container-executor/impl/get_executable.c
+    main/native/container-executor/impl/utils/file-utils.c
     main/native/container-executor/impl/utils/string-utils.c
     main/native/container-executor/impl/utils/path-utils.c
     main/native/container-executor/impl/modules/cgroups/cgroups-operations.c
@@ -138,6 +143,14 @@ add_library(container
     main/native/container-executor/impl/modules/devices/devices-module.c
     main/native/container-executor/impl/utils/docker-util.c
     main/native/container-executor/impl/utils/mount-utils.c
+    main/native/container-executor/impl/utils/cJSON/cJSON.c
+    main/native/container-executor/impl/runc/runc.c
+    main/native/container-executor/impl/runc/runc_base_ctx.c
+    main/native/container-executor/impl/runc/runc_launch_cmd.c
+    main/native/container-executor/impl/runc/runc_reap.c
+    main/native/container-executor/impl/runc/runc_write_config.c
+    ${HADOOP_COMMON_SEC_PATH}/hadoop_user_info.c
+    ${HADOOP_COMMON_SEC_PATH}/hadoop_group_info.c
 )
 
 add_executable(container-executor
@@ -146,6 +159,7 @@ add_executable(container-executor
 
 target_link_libraries(container-executor
     container
+    crypto
 )
 
 output_directory(container-executor target/usr/local/bin)
@@ -155,7 +169,9 @@ add_executable(test-container-executor
     main/native/container-executor/test/test-container-executor.c
 )
 target_link_libraries(test-container-executor
-    container ${EXTRA_LIBS}
+    container
+    ${EXTRA_LIBS}
+    crypto
 )
 
 output_directory(test-container-executor target/usr/local/bin)
@@ -173,8 +189,15 @@ add_executable(cetest
         main/native/container-executor/test/modules/fpga/test-fpga-module.cc
         main/native/container-executor/test/modules/devices/test-devices-module.cc
         main/native/container-executor/test/test_util.cc
-        main/native/container-executor/test/utils/test_docker_util.cc)
-target_link_libraries(cetest gtest container)
+        main/native/container-executor/test/utils/test_docker_util.cc
+        main/native/container-executor/test/utils/test_runc_util.cc
+)
+target_link_libraries(cetest
+        gtest
+        container
+        crypto
+)
+
 output_directory(cetest test)
 
 # CGroup OOM listener
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 72e0cf1..3de7365 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
@@ -21,6 +21,7 @@
 #include "utils/docker-util.h"
 #include "utils/path-utils.h"
 #include "utils/string-utils.h"
+#include "runc/runc.h"
 #include "util.h"
 #include "config.h"
 
@@ -78,6 +79,7 @@ static const int DEFAULT_DOCKER_SUPPORT_ENABLED = 0;
 static const int DEFAULT_TC_SUPPORT_ENABLED = 0;
 static const int DEFAULT_MOUNT_CGROUP_SUPPORT_ENABLED = 0;
 static const int DEFAULT_YARN_SYSFS_SUPPORT_ENABLED = 0;
+static const int DEFAULT_RUNC_SUPPORT_ENABLED = 0;
 
 static const char* PROC_PATH = "/proc";
 
@@ -191,7 +193,7 @@ int check_executor_permissions(char *executable_file) {
 /**
  * Change the effective user id to limit damage.
  */
-static int change_effective_user(uid_t user, gid_t group) {
+int change_effective_user(uid_t user, gid_t group) {
   if (geteuid() == user) {
     return 0;
   }
@@ -211,6 +213,10 @@ static int change_effective_user(uid_t user, gid_t group) {
   return 0;
 }
 
+int change_effective_user_to_nm() {
+  return change_effective_user(nm_uid, nm_gid);
+}
+
 #ifdef __linux
 /**
  * Write the pid of the current process to the cgroup file.
@@ -408,7 +414,7 @@ static int wait_and_get_exit_code(pid_t pid) {
  * the exit code file.
  * Returns the exit code of the container process.
  */
-static int wait_and_write_exit_code(pid_t pid, const char* exit_code_file) {
+int wait_and_write_exit_code(pid_t pid, const char* exit_code_file) {
   int exit_code = -1;
 
   exit_code = wait_and_get_exit_code(pid);
@@ -510,6 +516,12 @@ int is_yarn_sysfs_support_enabled() {
                             DEFAULT_YARN_SYSFS_SUPPORT_ENABLED, &executor_cfg);
 }
 
+int is_runc_support_enabled() {
+  return is_feature_enabled(RUNC_SUPPORT_ENABLED_KEY,
+                            DEFAULT_RUNC_SUPPORT_ENABLED, &executor_cfg)
+      || runc_module_enabled(&CFG);
+}
+
 /**
  * Utility function to concatenate argB to argA using the concat_pattern.
  */
@@ -643,6 +655,20 @@ char *get_tmp_directory(const char *work_dir) {
 }
 
 /**
+ * Get the private /tmp directory under the working directory
+ */
+char *get_privatetmp_directory(const char *work_dir) {
+  return concatenate("%s/%s", "private /tmp dir", 2, work_dir, ROOT_TMP_DIR);
+}
+
+/**
+ * Get the private /tmp directory under the working directory
+ */
+char *get_private_var_tmp_directory(const char *work_dir) {
+  return concatenate("%s/%s", "private /var/tmp dir", 2, work_dir, ROOT_VAR_TMP_DIR);
+}
+
+/**
  * Ensure that the given path and all of the parent directories are created
  * with the desired permissions.
  */
@@ -810,17 +836,51 @@ static int create_container_directories(const char* user, const char *app_id,
     return result;
   }
 
-  result = COULD_NOT_CREATE_TMP_DIRECTORIES;
   // also make the tmp directory
   char *tmp_dir = get_tmp_directory(work_dir);
+  char *private_tmp_dir = get_privatetmp_directory(work_dir);
+  char *private_var_tmp_dir = get_private_var_tmp_directory(work_dir);
 
-  if (tmp_dir == NULL) {
+  if (tmp_dir == NULL || private_tmp_dir == NULL || private_var_tmp_dir == NULL) {
     return OUT_OF_MEMORY;
   }
-  if (mkdirs(tmp_dir, perms) == 0) {
-    result = 0;
+
+  if (mkdirs(tmp_dir, perms) != 0) {
+    fprintf(ERRORFILE, "Could not create tmp_dir: %s\n", tmp_dir);
+    result = COULD_NOT_CREATE_TMP_DIRECTORIES;
+    goto cleanup;
+  }
+
+  if (mkdirs(private_tmp_dir, perms) != 0) {
+    fprintf(ERRORFILE, "Could not create private_tmp_dir: %s\n", private_tmp_dir);
+    result = COULD_NOT_CREATE_TMP_DIRECTORIES;
+    goto cleanup;
+  }
+
+  // clear group sticky 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;
+    goto cleanup;
+  }
+
+  if (mkdirs(private_var_tmp_dir, perms) != 0) {
+    fprintf(ERRORFILE, "Could not create private_var_tmp_dir: %s\n", private_var_tmp_dir);
+    result = COULD_NOT_CREATE_TMP_DIRECTORIES;
+    goto cleanup;
+  }
+
+  // clear group sticky 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;
+    goto cleanup;
   }
+
+cleanup:
   free(tmp_dir);
+  free(private_tmp_dir);
+  free(private_var_tmp_dir);
 
   return result;
 }
@@ -1052,6 +1112,36 @@ static int open_file_as_nm(const char* filename) {
 }
 
 /**
+ * Check the pidfile as the node manager. File should not exist.
+ * Returns 0 on file doesn't exist and -1 on file does exist.
+ */
+int check_pidfile_as_nm(const char* pidfile) {
+  int result = 0;
+  uid_t user = geteuid();
+  gid_t group = getegid();
+  if (change_effective_user(nm_uid, nm_gid) != 0) {
+    return -1;
+  }
+
+  struct stat statbuf;
+  if (stat(pidfile, &statbuf) == 0) {
+    fprintf(ERRORFILE, "pid file already exists: %s\n", pidfile);
+    result = -1;
+  }
+
+  if (errno != ENOENT) {
+    fprintf(ERRORFILE, "Error accessing %s : %s\n", pidfile,
+            strerror(errno));
+    result = -1;
+  }
+
+  if (change_effective_user(user, group)) {
+    result = -1;
+  }
+  return result;
+}
+
+/**
  * Copy a file from a fd to a given filename.
  * The new file must not exist and it is created with permissions perm.
  * The input stream is closed.
@@ -1863,6 +1953,61 @@ int create_yarn_sysfs(const char* user, const char *app_id,
   return result;
 }
 
+int setup_container_paths(const char* user, const char* app_id,
+    const char *container_id, const char *work_dir, const char *script_name,
+    const char *cred_file, int https, const char *keystore_file, const char *truststore_file,
+    char* const* local_dirs, char* const* log_dirs) {
+  char *script_file_dest = NULL;
+  char *cred_file_dest = NULL;
+  char *keystore_file_dest = NULL;
+  char *truststore_file_dest = NULL;
+  int container_file_source = -1;
+  int cred_file_source = -1;
+  int keystore_file_source = -1;
+  int truststore_file_source = -1;
+
+  int result = initialize_user(user, local_dirs);
+  if (result != 0) {
+    return result;
+  }
+
+  int rc = create_script_paths(
+    work_dir, script_name, cred_file, https, keystore_file, truststore_file, &script_file_dest, &cred_file_dest,
+    &keystore_file_dest, &truststore_file_dest, &container_file_source, &cred_file_source, &keystore_file_source, &truststore_file_source);
+
+  if (rc != 0) {
+    fputs("Could not create script path\n", ERRORFILE);
+    goto cleanup;
+  }
+
+  rc = create_log_dirs(app_id, log_dirs);
+  if (rc != 0) {
+    fputs("Could not create log files and directories\n", ERRORFILE);
+    goto cleanup;
+  }
+
+  rc = create_local_dirs(user, app_id, container_id,
+    work_dir, script_name, cred_file, https, keystore_file, truststore_file, local_dirs, log_dirs,
+    1, script_file_dest, cred_file_dest, keystore_file_dest, truststore_file_dest,
+    container_file_source, cred_file_source, keystore_file_source, truststore_file_source);
+
+  if (rc != 0) {
+    fputs("Could not create local files and directories\n", ERRORFILE);
+    goto cleanup;
+  }
+
+  rc = create_yarn_sysfs(user, app_id, container_id, work_dir, local_dirs);
+  if (rc != 0) {
+    fputs("Could not create user yarn sysfs directory\n", ERRORFILE);
+    goto cleanup;
+  }
+
+cleanup:
+  free(script_file_dest);
+  free(cred_file_dest);
+  return rc;
+}
+
 int launch_docker_container_as_user(const char * user, const char *app_id,
                               const char *container_id, const char *work_dir,
                               const char *script_name, const char *cred_file,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
index 757bd16..8219a67 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
@@ -52,7 +52,9 @@ enum operations {
   REMOVE_DOCKER_CONTAINER = 13,
   INSPECT_DOCKER_CONTAINER = 14,
   RUN_AS_USER_SYNC_YARN_SYSFS = 15,
-  EXEC_CONTAINER = 16
+  EXEC_CONTAINER = 16,
+  RUN_RUNC_CONTAINER = 17,
+  REAP_RUNC_LAYER_MOUNTS = 18
 };
 
 #define NM_GROUP_KEY "yarn.nodemanager.linux-container-executor.group"
@@ -72,10 +74,14 @@ enum operations {
 #define TC_SUPPORT_ENABLED_KEY "feature.tc.enabled"
 #define MOUNT_CGROUP_SUPPORT_ENABLED_KEY "feature.mount-cgroup.enabled"
 #define YARN_SYSFS_SUPPORT_ENABLED_KEY "feature.yarn.sysfs.enabled"
+#define RUNC_SUPPORT_ENABLED_KEY "feature.runc.enabled"
 #define TMP_DIR "tmp"
+#define ROOT_TMP_DIR "private_slash_tmp"
+#define ROOT_VAR_TMP_DIR "private_var_slash_tmp"
 #define COMMAND_FILE_SECTION "command-execution"
 
 extern struct passwd *user_detail;
+extern struct section executor_cfg;
 
 //function used to load the configurations present in the secure config
 void read_executor_config(const char* file_name);
@@ -175,6 +181,9 @@ int delete_as_user(const char *user,
 // assumed to be an absolute path.
 int list_as_user(const char *target_dir);
 
+// Check the pidfile as the node manager. File should not exist.
+int check_pidfile_as_nm(const char* filename);
+
 // set the uid and gid of the node manager.  This is used when doing some
 // priviledged operations for setting the effective uid and gid.
 void set_nm_uid(uid_t user, gid_t group);
@@ -244,6 +253,10 @@ int create_directory_for_user(const char* path);
 
 int change_user(uid_t user, gid_t group);
 
+int change_effective_user(uid_t user, gid_t group);
+
+int change_effective_user_to_nm();
+
 int mount_cgroup(const char *pair, const char *hierarchy);
 
 int check_dir(const char* npath, mode_t st_mode, mode_t desired,
@@ -255,6 +268,14 @@ int create_validate_dir(const char* npath, mode_t perm, const char* path,
 /** Check if a feature is enabled in the specified configuration. */
 int is_feature_enabled(const char* feature_key, int default_value,
                               struct section *cfg);
+char* get_exit_code_file(const char* pid_file);
+
+int wait_and_write_exit_code(pid_t pid, const char* exit_code_file);
+
+int setup_container_paths(const char* user, const char* app_id,
+    const char *container_id, const char* work_dir, const char* script_path,
+    const char *cred_path, int https, const char *keystore_file, const char *truststore_file,
+    char * const* local_dirs, char* const* log_dirs);
 
 /** Check if tc (traffic control) support is enabled in configuration. */
 int is_tc_support_enabled();
@@ -341,3 +362,9 @@ int remove_docker_container(char **argv, int argc);
  * Check if terminal feature is enabled
  */
 int is_terminal_support_enabled();
+
+
+/**
+ * Check if runC feature is enabled
+ */
+int is_runc_support_enabled();
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 ce3e21e..01c054a 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
@@ -26,6 +26,8 @@
 #include "modules/cgroups/cgroups-operations.h"
 #include "modules/devices/devices-module.h"
 #include "utils/string-utils.h"
+#include "runc/runc.h"
+#include "runc/runc_reap.h"
 
 #include <errno.h>
 #include <grp.h>
@@ -35,45 +37,33 @@
 #include <signal.h>
 
 static void display_usage(FILE *stream) {
+  const char* disabled = "[DISABLED]";
+  const char* enabled  = "      ";
+
+  fputs("Usage: container-executor --checksetup\n"
+        "       container-executor --mount-cgroups <hierarchy> "
+        "<controller=path>...\n", stream);
+
+  const char* de = is_tc_support_enabled() ? enabled : disabled;
   fprintf(stream,
-    "Usage: container-executor --checksetup\n"
-    "       container-executor --mount-cgroups <hierarchy> "
-    "<controller=path>\n" );
+      "%s container-executor --tc-modify-state <command-file>\n"
+      "%s container-executor --tc-read-state <command-file>\n"
+      "%s container-executor --tc-read-stats <command-file>\n",
+      de, de, de);
 
-  if(is_tc_support_enabled()) {
-    fprintf(stream,
-      "       container-executor --tc-modify-state <command-file>\n"
-      "       container-executor --tc-read-state <command-file>\n"
-      "       container-executor --tc-read-stats <command-file>\n" );
-  } else {
-    fprintf(stream,
-      "[DISABLED] container-executor --tc-modify-state <command-file>\n"
-      "[DISABLED] container-executor --tc-read-state <command-file>\n"
-      "[DISABLED] container-executor --tc-read-stats <command-file>\n");
-  }
+  de = is_terminal_support_enabled() ? enabled : disabled;
+  fprintf(stream, "%s container-executor --exec-container <command-file>\n", de);
 
-  if(is_docker_support_enabled()) {
-    fprintf(stream,
-      "       container-executor --run-docker <command-file>\n"
-      "       container-executor --remove-docker-container [hierarchy] "
-      "<container_id>\n"
-      "       container-executor --inspect-docker-container <container_id>\n");
-  } else {
-    fprintf(stream,
-      "[DISABLED] container-executor --run-docker <command-file>\n"
-      "[DISABLED] container-executor --remove-docker-container [hierarchy] "
-      "<container_id>\n"
-      "[DISABLED] container-executor --inspect-docker-container "
-      "<format> ... <container_id>\n");
-  }
+  de = is_docker_support_enabled() ? enabled : disabled;
+  fprintf(stream, "%s container-executor --run-docker <command-file>\n", de);
+  fprintf(stream, "%s container-executor --remove-docker-container [hierarchy] <container_id>\n", de);
+  fprintf(stream, "%s container-executor --inspect-docker-container <container_id>\n", de);
 
-  if (is_terminal_support_enabled()) {
-    fprintf(stream,
-      "       container-executor --exec-container <command-file>\n");
-  } else {
-    fprintf(stream,
-      "[DISABLED] container-executor --exec-container <command-file>\n");
-  }
+  de = is_runc_support_enabled() ? enabled : disabled;
+  fprintf(stream,
+      "%s container-executor --run-runc-container <command-file>\n", de);
+  fprintf(stream,
+      "%s container-executor --reap-runc-layer-mounts <retain-count>\n", de);
 
   fprintf(stream,
       "       container-executor <user> <yarn-user> <command> <command-args>\n"
@@ -85,27 +75,21 @@ static void display_usage(FILE *stream) {
       INITIALIZE_CONTAINER, LAUNCH_CONTAINER);
 
   if(is_tc_support_enabled()) {
-    fprintf(stream, "optional-tc-command-file\n");
+    fputs("optional-tc-command-file\n", stream);
   } else {
-    fprintf(stream, "\n");
+    fputs("\n", stream);
   }
 
-  if(is_docker_support_enabled()) {
-    fprintf(stream,
-      "            launch docker container:      %2d appid containerid workdir "
-      "container-script tokens pidfile nm-local-dirs nm-log-dirs "
-      "docker-command-file resources ", LAUNCH_DOCKER_CONTAINER);
-  } else {
-    fprintf(stream,
-      "[DISABLED]  launch docker container:      %2d appid containerid workdir "
+  de = is_docker_support_enabled() ? enabled : disabled;
+  fprintf(stream,
+      "%11s launch docker container:      %2d appid containerid workdir "
       "container-script tokens pidfile nm-local-dirs nm-log-dirs "
-      "docker-command-file resources ", LAUNCH_DOCKER_CONTAINER);
-  }
+      "docker-command-file resources ", de, LAUNCH_DOCKER_CONTAINER);
 
   if(is_tc_support_enabled()) {
-    fprintf(stream, "optional-tc-command-file\n");
+    fputs("optional-tc-command-file\n", stream);
   } else {
-    fprintf(stream, "\n");
+    fputs("\n", stream);
   }
 
   fprintf(stream,
@@ -244,7 +228,7 @@ static void display_feature_disabled_message(const char* name) {
     fprintf(ERRORFILE, "Feature disabled: %s\n", name);
 }
 
-/* Use to store parsed input parmeters for various operations */
+/* Use to store parsed input parameters for various operations */
 static struct {
   char *cgroups_hierarchy;
   char *traffic_control_command_file;
@@ -267,6 +251,7 @@ static struct {
   const char *target_dir;
   int container_pid;
   int signal;
+  int runc_layer_count;
   const char *command_file;
 } cmd_input;
 
@@ -435,6 +420,44 @@ static int validate_arguments(int argc, char **argv , int *operation) {
     }
   }
 
+  if (strcmp("--run-runc-container", argv[1]) == 0) {
+    if (is_runc_support_enabled()) {
+      if (argc != 3) {
+        display_usage(stdout);
+        return INVALID_ARGUMENT_NUMBER;
+      }
+      optind++;
+      cmd_input.command_file = argv[optind++];
+      *operation = RUN_RUNC_CONTAINER;
+      return 0;
+    } else {
+      display_feature_disabled_message("runc");
+      return FEATURE_DISABLED;
+    }
+  }
+
+  if (strcmp("--reap-runc-layer-mounts", argv[1]) == 0) {
+    if (is_runc_support_enabled()) {
+      if (argc != 3) {
+        display_usage(stdout);
+        return INVALID_ARGUMENT_NUMBER;
+      }
+      optind++;
+      const char* valstr = argv[optind++];
+      if (sscanf(valstr, "%d", &cmd_input.runc_layer_count) != 1
+          || cmd_input.runc_layer_count < 0) {
+        fprintf(ERRORFILE, "Bad runc layer count: %s\n", valstr);
+        return INVALID_COMMAND_PROVIDED;
+      }
+      *operation = REAP_RUNC_LAYER_MOUNTS;
+      return 0;
+    } else {
+      display_feature_disabled_message("runc");
+      return FEATURE_DISABLED;
+    }
+  }
+
+
   /* Now we have to validate 'run as user' operations that don't use
     a 'long option' - we should fix this at some point. The validation/argument
     parsing here is extensive enough that it done in a separate function */
@@ -786,6 +809,16 @@ int main(int argc, char **argv) {
       exit_code = FEATURE_DISABLED;
     }
     break;
+  case RUN_RUNC_CONTAINER:
+    exit_code = run_runc_container(cmd_input.command_file);
+    break;
+  case REAP_RUNC_LAYER_MOUNTS:
+    exit_code = reap_runc_layer_mounts(cmd_input.runc_layer_count);
+    break;
+  default:
+    fprintf(ERRORFILE, "Unexpected operation code: %d\n", operation);
+    exit_code = INVALID_COMMAND_PROVIDED;
+    break;
   }
 
   if (exit_code) {
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 c0b73d3..9567ccc 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
@@ -323,6 +323,12 @@ const char *get_error_message(const int error_code) {
         return "Invalid docker runtime";
       case DOCKER_SERVICE_MODE_DISABLED:
         return "Docker service mode disabled";
+      case ERROR_RUNC_SETUP_FAILED:
+        return "runC setup failed";
+      case ERROR_RUNC_RUN_FAILED:
+        return "runC run failed";
+      case ERROR_RUNC_REAP_LAYER_MOUNTS_FAILED:
+        return "runC reap layer mounts failed";
       default:
         return "Unknown error code";
     }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
index dcc00a9..b984a23 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/util.h
@@ -100,7 +100,10 @@ enum errorcodes {
   INVALID_DOCKER_IMAGE_TRUST = 72,
   INVALID_DOCKER_TMPFS_MOUNT = 73,
   INVALID_DOCKER_RUNTIME = 74,
-  DOCKER_SERVICE_MODE_DISABLED = 75
+  DOCKER_SERVICE_MODE_DISABLED = 75,
+  ERROR_RUNC_SETUP_FAILED = 76,
+  ERROR_RUNC_RUN_FAILED = 77,
+  ERROR_RUNC_REAP_LAYER_MOUNTS_FAILED = 78
 };
 
 /* Macros for min/max. */
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 68857a9..62d54a9 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
@@ -22,11 +22,14 @@
 #include <dirent.h>
 #include <limits.h>
 #include <errno.h>
+#include <stdarg.h>
 #include <strings.h>
 #include <string.h>
 #include <stdlib.h>
 #include <stdarg.h>
 
+#include "string-utils.h"
+
 /*
  * if all chars in the input str are numbers
  * return true/false
@@ -189,3 +192,172 @@ int str_ends_with(const char *s, const char *suffix) {
     size_t suffix_len = strlen(suffix);
     return suffix_len <= slen && !strcmp(s + slen - suffix_len, suffix);
 }
+
+/* Returns the corresponding hexadecimal character for a nibble. */
+static char nibble_to_hex(unsigned char nib) {
+  return nib < 10 ? '0' + nib : 'a' + nib - 10;
+}
+
+/**
+ * Converts a sequence of bytes into a hexadecimal string.
+ *
+ * Returns a pointer to the allocated string on success or NULL on error.
+ */
+char* to_hexstring(unsigned char* bytes, unsigned int len) {
+  char* hexstr = malloc(len * 2 + 1);
+  if (hexstr == NULL) {
+    return NULL;
+  }
+  unsigned char* src = bytes;
+  char* dest = hexstr;
+  for (unsigned int i = 0; i < len; ++i) {
+    unsigned char val = *src++;
+    *dest++ = nibble_to_hex((val >> 4) & 0xF);
+    *dest++ = nibble_to_hex(val & 0xF);
+  }
+  *dest = '\0';
+  return hexstr;
+}
+
+/**
+ * Initialize an uninitialized strbuf with the specified initial capacity.
+ *
+ * Returns true on success or false if memory could not be allocated.
+ */
+bool strbuf_init(strbuf* sb, size_t initial_capacity) {
+  memset(sb, 0, sizeof(*sb));
+  char* new_buffer = malloc(initial_capacity);
+  if (new_buffer == NULL) {
+    return false;
+  }
+  sb->buffer = new_buffer;
+  sb->capacity = initial_capacity;
+  sb->length = 0;
+  return true;
+}
+
+/**
+ * Allocate and initialize a strbuf with the specified initial capacity.
+ *
+ * Returns a pointer to the allocated and initialized strbuf or NULL on error.
+ */
+strbuf* strbuf_alloc(size_t initial_capacity) {
+  strbuf* sb = malloc(sizeof(*sb));
+  if (sb != NULL) {
+    if (!strbuf_init(sb, initial_capacity)) {
+      free(sb);
+      sb = NULL;
+    }
+  }
+  return sb;
+}
+
+/**
+ * Detach the underlying character buffer from a string buffer.
+ *
+ * Returns the heap-allocated, NULL-terminated character buffer.
+ * NOTE: The caller is responsible for freeing the result.
+ */
+char* strbuf_detach_buffer(strbuf* sb) {
+  char* result = NULL;
+  if (sb != NULL) {
+    result = sb->buffer;
+    sb->buffer = NULL;
+    sb->length = 0;
+    sb->capacity = 0;
+  }
+  return result;
+}
+
+/**
+ * Release memory associated with a strbuf but not the strbuf structure itself.
+ * Useful for stack-allocated strbuf objects or structures that embed a strbuf.
+ * Use strbuf_free for heap-allocated string buffers.
+ */
+void strbuf_destroy(strbuf* sb) {
+  if (sb != NULL) {
+    free(sb->buffer);
+    sb->buffer = NULL;
+    sb->capacity = 0;
+    sb->length = 0;
+  }
+}
+
+/**
+ * Free a strbuf and all memory associated with it.
+ */
+void strbuf_free(strbuf* sb) {
+  if (sb != NULL) {
+    strbuf_destroy(sb);
+    free(sb);
+  }
+}
+
+/**
+ * Resize a strbuf to the specified new capacity.
+ *
+ * Returns true on success or false if there was an error.
+ */
+bool strbuf_realloc(strbuf* sb, size_t new_capacity) {
+  if (new_capacity < sb->length + 1) {
+    // New capacity would result in a truncation of the existing string.
+    return false;
+  }
+
+  char* new_buffer = realloc(sb->buffer, new_capacity);
+  if (!new_buffer) {
+    return false;
+  }
+
+  sb->buffer = new_buffer;
+  sb->capacity = new_capacity;
+  return true;
+}
+
+/**
+ * Append a formatted string to the current contents of a strbuf.
+ *
+ * Returns true on success or false if there was an error.
+ */
+bool strbuf_append_fmt(strbuf* sb, size_t realloc_extra,
+    const char* fmt, ...) {
+  if (sb->length > sb->capacity) {
+    return false;
+  }
+
+  if (sb->length == sb->capacity) {
+    size_t incr = (realloc_extra == 0) ? 1024 : realloc_extra;
+    if (!strbuf_realloc(sb, sb->capacity + incr)) {
+      return false;
+    }
+  }
+
+  size_t remain = sb->capacity - sb->length;
+  va_list vargs;
+  va_start(vargs, fmt);
+  int needed = vsnprintf(sb->buffer + sb->length, remain, fmt, vargs);
+  va_end(vargs);
+  if (needed == -1) {
+    return false;
+  }
+
+  needed += 1;  // vsnprintf result does NOT include terminating NUL
+  if (needed > remain) {
+    // result was truncated so need to realloc and reprint
+    size_t new_size = sb->length + needed + realloc_extra;
+    if (!strbuf_realloc(sb, new_size)) {
+      return false;
+    }
+    remain = sb->capacity - sb->length;
+    va_start(vargs, fmt);
+    needed = vsnprintf(sb->buffer + sb->length, remain, fmt, vargs);
+    va_end(vargs);
+    if (needed == -1) {
+      return false;
+    }
+    needed += 1;  // vsnprintf result does NOT include terminating NUL
+  }
+
+  sb->length += needed - 1;  // length does not include terminating NUL
+  return true;
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
index 995cdf3..25c3a82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
@@ -23,6 +23,16 @@
 #ifndef _UTILS_STRING_UTILS_H_
 #define _UTILS_STRING_UTILS_H_
 
+#include <stdbool.h>
+#include <stddef.h>
+
+typedef struct strbuf_struct {
+  char* buffer;               // points to beginning of the string
+  size_t length;              // strlen of buffer (sans trailing NUL)
+  size_t capacity;            // capacity of the buffer
+} strbuf;
+
+
 /*
  * Get numbers split by comma from a input string
  * return false/true
@@ -44,4 +54,61 @@ char *make_string(const char *fmt, ...);
  * return 1 if succeeded
  */
 int str_ends_with(const char *s, const char *suffix);
+
+/**
+ * Converts a sequence of bytes into a hexadecimal string.
+ *
+ * Returns a pointer to the allocated string on success or NULL on error.
+ */
+char* to_hexstring(unsigned char* bytes, unsigned int len);
+
+/**
+ * Allocate and initialize a strbuf with the specified initial capacity.
+ *
+ * Returns a pointer to the allocated and initialized strbuf or NULL on error.
+ */
+strbuf* strbuf_alloc(size_t initial_capacity);
+
+/**
+ * Initialize an uninitialized strbuf with the specified initial capacity.
+ *
+ * Returns true on success or false if memory could not be allocated.
+ */
+bool strbuf_init(strbuf* sb, size_t initial_capacity);
+
+/**
+ * Resize a strbuf to the specified new capacity.
+ *
+ * Returns true on success or false if there was an error.
+ */
+bool strbuf_realloc(strbuf* sb, size_t new_capacity);
+
+/**
+ * Detach the underlying character buffer from a string buffer.
+ *
+ * Returns the heap-allocated, NULL-terminated character buffer.
+ * NOTE: The caller is responsible for freeing the result.
+ */
+char* strbuf_detach_buffer(strbuf* sb);
+
+/**
+ * Releases the memory underneath a string buffer but does NOT free the
+ * strbuf structure itself. This is particularly useful for stack-allocated
+ * strbuf objects or structures that embed a strbuf structure.
+ * strbuf_free should be used for heap-allocated string buffers.
+ */
+void strbuf_destroy(strbuf* sb);
+
+/**
+ * Free a strbuf and all memory associated with it.
+ */
+void strbuf_free(strbuf* sb);
+
+/**
+ * Append a formatted string to the current contents of a strbuf.
+ *
+ * Returns true on success or false if there was an error.
+ */
+bool strbuf_append_fmt(strbuf* sb, size_t realloc_extra, const char* fmt, ...);
+
 #endif
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc
index 44c9b1b..91fc3bf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc
@@ -17,16 +17,86 @@
  */
 
 #include <gtest/gtest.h>
-#include <main/native/container-executor/impl/util.h>
 #include <cstdio>
+#include <pwd.h>
 
 extern "C" {
 #include "util.h"
+#include "container-executor.h"
+}
+
+#define TMPDIR "/tmp"
+#define TEST_ROOT TMPDIR "/test-container-executor"
+
+int write_config_file(const char *file_name, int banned) {
+  FILE *file;
+  file = fopen(file_name, "w");
+  if (file == NULL) {
+    printf("Failed to open %s.\n", file_name);
+    return EXIT_FAILURE;
+  }
+  if (banned != 0) {
+    fprintf(file, "banned.users=bannedUser\n");
+    fprintf(file, "min.user.id=500\n");
+  } else {
+    fprintf(file, "min.user.id=0\n");
+  }
+  fprintf(file, "allowed.system.users=allowedUser,daemon\n");
+  fprintf(file, "feature.yarn.sysfs.enabled=1\n");
+  fclose(file);
+  return 0;
 }
 
 int main(int argc, char **argv) {
   ERRORFILE = stderr;
   LOGFILE = stdout;
+
+  printf("\nMaking test dir\n");
+  if (mkdirs(TEST_ROOT, 0755) != 0) {
+    exit(1);
+  }
+  if (chmod(TEST_ROOT, 0755) != 0) {    // in case of umask
+    exit(1);
+  }
+
+  // We need a valid config before the test really starts for the check_user
+  // and set_user calls
+  printf("\nCreating test.cfg\n");
+  if (write_config_file(TEST_ROOT "/test.cfg", 1) != 0) {
+    exit(1);
+  }
+  printf("\nLoading test.cfg\n");
+  read_executor_config(TEST_ROOT "/test.cfg");
+
+  printf("\nDetermining user details\n");
+  char* username = strdup(getpwuid(getuid())->pw_name);
+  struct passwd *username_info = check_user(username);
+  printf("\nSetting NM UID\n");
+  set_nm_uid(username_info->pw_uid, username_info->pw_gid);
+
+  // Make sure that username owns all the files now
+  printf("\nEnsuring ownership of test dir\n");
+  if (chown(TEST_ROOT, username_info->pw_uid, username_info->pw_gid) != 0) {
+    exit(1);
+  }
+  if (chown(TEST_ROOT "/test.cfg",
+        username_info->pw_uid, username_info->pw_gid) != 0) {
+    exit(1);
+  }
+
+  printf("\nSetting effective user\n");
+  if (set_user(username)) {
+    exit(1);
+  }
+
   testing::InitGoogleTest(&argc, argv);
-  return RUN_ALL_TESTS();
+  int rc = RUN_ALL_TESTS();
+
+  printf("Attempting to clean up from any previous runs\n");
+  // clean up any junk from previous run
+  if (system("chmod -R u=rwx " TEST_ROOT "; rm -fr " TEST_ROOT)) {
+    exit(1);
+
+  return rc;
+  }
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc
index 138e32a..a70a704 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc
@@ -29,6 +29,7 @@
 
  #include <gtest/gtest.h>
  #include <sstream>
+ #include <openssl/evp.h>
 
  extern "C" {
  #include "utils/string-utils.h"
@@ -36,97 +37,230 @@
 
  namespace ContainerExecutor {
 
- class TestStringUtils : public ::testing::Test {
- protected:
-   virtual void SetUp() {
-
-   }
-
-   virtual void TearDown() {
-
-   }
- };
-
- TEST_F(TestStringUtils, test_get_numbers_split_by_comma) {
-   const char* input = ",1,2,3,-1,,1,,0,";
-   int* numbers;
-   size_t n_numbers;
-   int rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
-
-   std::cout << "Testing input=" << input << "\n";
-   ASSERT_EQ(0, rc) << "Should succeeded\n";
-   ASSERT_EQ(6, n_numbers);
-   ASSERT_EQ(1, numbers[0]);
-   ASSERT_EQ(-1, numbers[3]);
-   ASSERT_EQ(0, numbers[5]);
-   free(numbers);
-
-   input = "3";
-   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
-   std::cout << "Testing input=" << input << "\n";
-   ASSERT_EQ(0, rc) << "Should succeeded\n";
-   ASSERT_EQ(1, n_numbers);
-   ASSERT_EQ(3, numbers[0]);
-   free(numbers);
-
-   input = "";
-   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
-   std::cout << "Testing input=" << input << "\n";
-   ASSERT_EQ(0, rc) << "Should succeeded\n";
-   ASSERT_EQ(0, n_numbers);
-   free(numbers);
-
-   input = ",,";
-   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
-   std::cout << "Testing input=" << input << "\n";
-   ASSERT_EQ(0, rc) << "Should succeeded\n";
-   ASSERT_EQ(0, n_numbers);
-   free(numbers);
-
-   input = "1,2,aa,bb";
-   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
-   std::cout << "Testing input=" << input << "\n";
-   ASSERT_TRUE(0 != rc) << "Should failed\n";
-   free(numbers);
-
-   input = "1,2,3,-12312312312312312312321311231231231";
-   rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
-   std::cout << "Testing input=" << input << "\n";
-   ASSERT_TRUE(0 != rc) << "Should failed\n";
-   free(numbers);
- }
+    class TestStringUtils : public ::testing::Test {
+    protected:
+      virtual void SetUp() {
+
+      }
+
+      virtual void TearDown() {
+
+      }
+    };
+
+    TEST_F(TestStringUtils, test_get_numbers_split_by_comma) {
+      const char* input = ",1,2,3,-1,,1,,0,";
+      int* numbers;
+      size_t n_numbers;
+      int rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+
+      std::cout << "Testing input=" << input << "\n";
+      ASSERT_EQ(0, rc) << "Should succeeded\n";
+      ASSERT_EQ(6, n_numbers);
+      ASSERT_EQ(1, numbers[0]);
+      ASSERT_EQ(-1, numbers[3]);
+      ASSERT_EQ(0, numbers[5]);
+      free(numbers);
+
+      input = "3";
+      rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+      std::cout << "Testing input=" << input << "\n";
+      ASSERT_EQ(0, rc) << "Should succeeded\n";
+      ASSERT_EQ(1, n_numbers);
+      ASSERT_EQ(3, numbers[0]);
+      free(numbers);
+
+      input = "";
+      rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+      std::cout << "Testing input=" << input << "\n";
+      ASSERT_EQ(0, rc) << "Should succeeded\n";
+      ASSERT_EQ(0, n_numbers);
+      free(numbers);
+
+      input = ",,";
+      rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+      std::cout << "Testing input=" << input << "\n";
+      ASSERT_EQ(0, rc) << "Should succeeded\n";
+      ASSERT_EQ(0, n_numbers);
+      free(numbers);
+
+      input = "1,2,aa,bb";
+      rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+      std::cout << "Testing input=" << input << "\n";
+      ASSERT_TRUE(0 != rc) << "Should failed\n";
+      free(numbers);
+
+      input = "1,2,3,-12312312312312312312321311231231231";
+      rc = get_numbers_split_by_comma(input, &numbers, &n_numbers);
+      std::cout << "Testing input=" << input << "\n";
+      ASSERT_TRUE(0 != rc) << "Should failed\n";
+      free(numbers);
+    }
+
+    TEST_F(TestStringUtils, test_validate_container_id) {
+
+      const char *good_input[] = {
+          "container_e134_1499953498516_50875_01_000007",
+          "container_1499953498516_50875_01_000007",
+          "container_e1_12312_11111_02_000001"
+      };
+
+      const char *bad_input[] = {
+          "CONTAINER",
+          "container_e1_12312_11111_02_000001 | /tmp/file"
+              "container_e1_12312_11111_02_000001 || # /tmp/file",
+          "container_e1_12312_11111_02_000001 # /tmp/file",
+          "container_e1_12312_11111_02_000001' || touch /tmp/file #",
+          "ubuntu || touch /tmp/file #",
+          "''''''''"
+      };
+
+      int good_input_size = sizeof(good_input) / sizeof(char *);
+      int i = 0;
+      for (i = 0; i < good_input_size; i++) {
+        int op = validate_container_id(good_input[i]);
+        ASSERT_EQ(1, op);
+      }
+
+      int bad_input_size = sizeof(bad_input) / sizeof(char *);
+      int j = 0;
+      for (j = 0; j < bad_input_size; j++) {
+        int op = validate_container_id(bad_input[j]);
+        ASSERT_EQ(0, op);
+      }
+    }
+
+    TEST_F(TestStringUtils, test_to_hexstring) {
+      const char* input = "hello";
+      char* digest = NULL;
+      unsigned char raw_digest[EVP_MAX_MD_SIZE];
+      unsigned int raw_digest_len = 0;
+      int rc = 0;
+
+      EVP_MD_CTX* mdctx = EVP_MD_CTX_create();
+      ASSERT_NE(nullptr, mdctx) << "Unable to create EVP MD context\n";
+
+      rc = EVP_DigestInit_ex(mdctx, EVP_sha256(), NULL);
+      ASSERT_EQ(1, rc) << "Unable to initialize SHA256 digester\n";
+
+      rc = EVP_DigestFinal_ex(mdctx, raw_digest, &raw_digest_len);
+      ASSERT_EQ(1, rc) << "Unable to compute digest\n";
+
+      rc = EVP_DigestUpdate(mdctx, input, strlen(input));
+      ASSERT_EQ(1, rc) << "Unable to compute digest\n";
+
+      digest = to_hexstring(raw_digest, raw_digest_len);
+
+      ASSERT_STREQ("e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855",
+                   digest) << "Digest is not equal to expected hash\n";
+
+      EVP_MD_CTX_destroy(mdctx);
+      free(digest);
+    }
+
+    TEST_F(TestStringUtils, test_strbuf_on_stack) {
+      const int sb_incr = 16;
+      strbuf sb;
+      bool rc;
+
+      rc = strbuf_init(&sb, sb_incr);
+      ASSERT_EQ(true, rc) << "Unable to init strbuf\n";
+
+      rc = strbuf_append_fmt(&sb, sb_incr, "%s%s%s", "hello", "foo", "bar");
+      ASSERT_EQ(true, rc) << "Unable to append format to strbuf\n";
+
+      ASSERT_STREQ("hellofoobar", sb.buffer);
+
+      rc = strbuf_append_fmt(&sb, sb_incr, "%s%s%s", "some longer strings",
+          " that will cause the strbuf", " to have to realloc");
+      ASSERT_EQ(true, rc) << "Unable to append format to strbuf\n";
+
+      ASSERT_STREQ("hellofoobarsome longer strings that will cause the strbuf to have to realloc", sb.buffer);
+
+      strbuf_destroy(&sb);
+    }
+
+    TEST_F(TestStringUtils, test_strbuf_in_heap) {
+      const int sb_incr = 16;
+      strbuf *sb = NULL;
+      bool rc;
+
+      sb = strbuf_alloc(sb_incr);
+      ASSERT_NE(nullptr, sb) << "Unable to init strbuf\n";
+
+      rc = strbuf_append_fmt(sb, sb_incr, "%s%s%s", "hello", "foo", "bar");
+      ASSERT_EQ(true, rc) << "Unable to append format to strbuf";
+
+      ASSERT_STREQ("hellofoobar", sb->buffer);
+
+      rc = strbuf_append_fmt(sb, sb_incr, "%s%s%s", "some longer strings",
+          " that will cause the strbuf", " to have to realloc");
+      ASSERT_EQ(true, rc) << "Unable to append format to strbuf\n";
+
+      ASSERT_STREQ("hellofoobarsome longer strings that will cause the strbuf to have to realloc", sb->buffer);
+
+      strbuf_free(sb);
+    }
+
+    TEST_F(TestStringUtils, test_strbuf_detach) {
+      const int sb_incr = 16;
+      strbuf sb;
+      char *buf;
+      bool rc;
+
+      rc = strbuf_init(&sb, sb_incr);
+      ASSERT_EQ(true, rc) << "Unable to init strbuf\n";
+
+      rc = strbuf_append_fmt(&sb, sb_incr, "%s%s%s", "hello", "foo", "bar");
+      ASSERT_EQ(true, rc) << "Unable to append format to strbuf\n";
+
+      ASSERT_STREQ("hellofoobar", sb.buffer);
+
+      rc = strbuf_append_fmt(&sb, sb_incr, "%s%s%s", "some longer strings",
+          " that will cause the strbuf", " to have to realloc");
+      ASSERT_EQ(true, rc) << "Unable to append format to strbuf\n";
+
+      ASSERT_STREQ("hellofoobarsome longer strings that will cause the strbuf to have to realloc", sb.buffer);
+
+      buf = strbuf_detach_buffer(&sb);
+      ASSERT_NE(nullptr, buf) << "Unable to detach char buf from strbuf\n";
+
+
+      rc = strbuf_append_fmt(&sb, sb_incr, "%s%s%s", "Buffer detached",
+          " so this should allocate", " a new buffer in strbuf");
+      ASSERT_EQ(true, rc) << "Unable to append format to strbuf\n";
+
+      ASSERT_STREQ("Buffer detached so this should allocate a new buffer in strbuf", sb.buffer);
+      ASSERT_STREQ("hellofoobarsome longer strings that will cause the strbuf to have to realloc", buf);
+
+      free(buf);
+      strbuf_destroy(&sb);
+    }
+
+    TEST_F(TestStringUtils, test_strbuf_realloc) {
+      const int sb_incr = 5;
+      strbuf sb;
+      char buf[] = "1234567890";
+      bool rc;
+
+      int len = strlen(buf);
+
+      rc = strbuf_init(&sb, sb_incr);
+      ASSERT_EQ(true, rc) << "Unable to init strbuf\n";
+      ASSERT_NE(nullptr, sb.buffer) << "Unable to init strbuf buffer\n";
+      ASSERT_EQ(5, sb.capacity) << "Unable to init strbuf capacity\n";
+      ASSERT_EQ(0, sb.length) << "Unable to init strbuf length\n";
+
+      rc = strbuf_append_fmt(&sb, sb_incr, "%s", buf);
+      ASSERT_EQ(true, rc) << "Unable to append format to strbuf\n";
+      ASSERT_NE(nullptr, sb.buffer) << "Unable to append strbuf buffer\n";
+      ASSERT_EQ(len + sb_incr + 1, sb.capacity) << "Unable to update strbuf capacity\n";
+      ASSERT_EQ(len, sb.length) << "Unable to update strbuf length\n";
+
+      rc = strbuf_realloc(&sb, 10);
+      ASSERT_EQ(false, rc) << "realloc to smaller capacity succeeded and has truncated existing string\n";
 
-   TEST_F(TestStringUtils, test_validate_container_id) {
-
-     const char *good_input[] = {
-         "container_e134_1499953498516_50875_01_000007",
-         "container_1499953498516_50875_01_000007",
-         "container_e1_12312_11111_02_000001"
-     };
-
-     const char *bad_input[] = {
-         "CONTAINER",
-         "container_e1_12312_11111_02_000001 | /tmp/file"
-             "container_e1_12312_11111_02_000001 || # /tmp/file",
-         "container_e1_12312_11111_02_000001 # /tmp/file",
-         "container_e1_12312_11111_02_000001' || touch /tmp/file #",
-         "ubuntu || touch /tmp/file #",
-         "''''''''"
-     };
-
-     int good_input_size = sizeof(good_input) / sizeof(char *);
-     int i = 0;
-     for (i = 0; i < good_input_size; i++) {
-       int op = validate_container_id(good_input[i]);
-       ASSERT_EQ(1, op);
-     }
-
-     int bad_input_size = sizeof(bad_input) / sizeof(char *);
-     int j = 0;
-     for (j = 0; j < bad_input_size; j++) {
-       int op = validate_container_id(bad_input[j]);
-       ASSERT_EQ(0, op);
-     }
-   }
+      strbuf_destroy(&sb);
+    }
 
 } // namespace ContainerExecutor


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