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 as...@apache.org on 2017/08/24 19:36:16 UTC

[23/50] [abbrv] hadoop git commit: YARN-6852. Native code changes to support isolate GPU devices by using CGroups. (wangda)

YARN-6852. Native code changes to support isolate GPU devices by using CGroups. (wangda)

Change-Id: I4869cc4d8ad539539ccba4bea5a178cacdb741ab


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/436c2638
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/436c2638
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/436c2638

Branch: refs/heads/YARN-5972
Commit: 436c2638f9ca1fb8de6a630cb5e91d956ac75216
Parents: 8991f0b
Author: Wangda Tan <wa...@apache.org>
Authored: Fri Aug 18 18:26:36 2017 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Fri Aug 18 18:26:36 2017 -0700

----------------------------------------------------------------------
 .../src/CMakeLists.txt                          |  12 +-
 .../impl/container-executor.c                   |  10 +-
 .../impl/container-executor.h                   |   2 +
 .../main/native/container-executor/impl/main.c  |  13 +-
 .../impl/modules/cgroups/cgroups-operations.c   | 161 +++++++++++++
 .../impl/modules/cgroups/cgroups-operations.h   |  55 +++++
 .../impl/modules/common/constants.h             |  29 +++
 .../impl/modules/common/module-configs.c        |  41 ++++
 .../impl/modules/common/module-configs.h        |  33 +++
 .../impl/modules/gpu/gpu-module.c               | 229 +++++++++++++++++++
 .../impl/modules/gpu/gpu-module.h               |  45 ++++
 .../container-executor/impl/utils/path-utils.c  |  52 +++++
 .../container-executor/impl/utils/path-utils.h  |  35 +++
 .../impl/utils/string-utils.c                   | 106 +++++++--
 .../impl/utils/string-utils.h                   |   7 +-
 .../test/modules/cgroups/test-cgroups-module.cc | 121 ++++++++++
 .../test/modules/gpu/test-gpu-module.cc         | 203 ++++++++++++++++
 .../test/test-container-executor-common.h       |  36 +++
 .../test/test-container-executor.c              |  23 +-
 .../native/container-executor/test/test_main.cc |  11 +-
 .../test/utils/test-path-utils.cc               |  67 ++++++
 .../test/utils/test-string-utils.cc             |  93 ++++++++
 22 files changed, 1338 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
----------------------------------------------------------------------
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 100d7ca..07c29bf 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
@@ -101,6 +101,10 @@ add_library(container
     main/native/container-executor/impl/container-executor.c
     main/native/container-executor/impl/get_executable.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
+    main/native/container-executor/impl/modules/common/module-configs.c
+    main/native/container-executor/impl/modules/gpu/gpu-module.c
 )
 
 add_executable(container-executor
@@ -113,12 +117,14 @@ target_link_libraries(container-executor
 
 output_directory(container-executor target/usr/local/bin)
 
+# Test cases
 add_executable(test-container-executor
     main/native/container-executor/test/test-container-executor.c
 )
 target_link_libraries(test-container-executor
     container ${EXTRA_LIBS}
 )
+
 output_directory(test-container-executor target/usr/local/bin)
 
 # unit tests for container executor
@@ -126,6 +132,10 @@ add_executable(cetest
         main/native/container-executor/impl/util.c
         main/native/container-executor/test/test_configuration.cc
         main/native/container-executor/test/test_main.cc
+        main/native/container-executor/test/utils/test-string-utils.cc
+        main/native/container-executor/test/utils/test-path-utils.cc
+        main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc
+        main/native/container-executor/test/modules/gpu/test-gpu-module.cc
         main/native/container-executor/test/test_util.cc)
-target_link_libraries(cetest gtest)
+target_link_libraries(cetest gtest container)
 output_directory(cetest test)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.c
----------------------------------------------------------------------
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 7361808..560ec18 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
@@ -1384,7 +1384,6 @@ char* sanitize_docker_command(const char *line) {
 }
 
 char* parse_docker_command_file(const char* command_file) {
-
   size_t len = 0;
   char *line = NULL;
   ssize_t read;
@@ -2443,3 +2442,12 @@ int traffic_control_read_state(char *command_file) {
 int traffic_control_read_stats(char *command_file) {
   return run_traffic_control(TC_READ_STATS_OPTS, command_file);
 }
+
+/**
+ * FIXME: (wangda) it's better to move executor_cfg out of container-executor.c
+ * Now initialize of executor_cfg and data structures are stored inside
+ * container-executor which is not a good design.
+ */
+struct configuration* get_cfg() {
+  return &CFG;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/container-executor.h
----------------------------------------------------------------------
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 ea8b5e3..aa38abf 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
@@ -274,3 +274,5 @@ int execute_regex_match(const char *regex_str, const char *input);
  * Return 0 on success.
  */
 int validate_docker_image_name(const char *image_name);
+
+struct configuration* get_cfg();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/main.c
----------------------------------------------------------------------
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 b2187c9..a05dc78 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
@@ -20,6 +20,8 @@
 #include "configuration.h"
 #include "container-executor.h"
 #include "util.h"
+#include "modules/gpu/gpu-module.h"
+#include "modules/cgroups/cgroups-operations.h"
 
 #include <errno.h>
 #include <grp.h>
@@ -253,6 +255,14 @@ static int validate_arguments(int argc, char **argv , int *operation) {
     return INVALID_ARGUMENT_NUMBER;
   }
 
+  /*
+   * Check if it is a known module, if yes, redirect to module
+   */
+  if (strcmp("--module-gpu", argv[1]) == 0) {
+    return handle_gpu_request(&update_cgroups_parameters, "gpu", argc - 1,
+           &argv[1]);
+  }
+
   if (strcmp("--checksetup", argv[1]) == 0) {
     *operation = CHECK_SETUP;
     return 0;
@@ -332,6 +342,7 @@ static int validate_arguments(int argc, char **argv , int *operation) {
         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 */
@@ -522,7 +533,7 @@ int main(int argc, char **argv) {
   open_log_files();
   assert_valid_setup(argv[0]);
 
-  int operation;
+  int operation = -1;
   int ret = validate_arguments(argc, argv, &operation);
 
   if (ret != 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.c
new file mode 100644
index 0000000..b234109
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.c
@@ -0,0 +1,161 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "configuration.h"
+#include "container-executor.h"
+#include "utils/string-utils.h"
+#include "utils/path-utils.h"
+#include "modules/common/module-configs.h"
+#include "modules/common/constants.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "util.h"
+
+#include <string.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <sys/stat.h>
+
+#define MAX_PATH_LEN 4096
+
+static const struct section* cgroup_cfg_section = NULL;
+
+void reload_cgroups_configuration() {
+  cgroup_cfg_section = get_configuration_section(CGROUPS_SECTION_NAME, get_cfg());
+}
+
+char* get_cgroups_path_to_write(
+    const char* hierarchy_name,
+    const char* param_name,
+    const char* group_id) {
+  int failed = 0;
+  char* buffer = NULL;
+  const char* cgroups_root = get_section_value(CGROUPS_ROOT_KEY,
+     cgroup_cfg_section);
+  const char* yarn_hierarchy_name = get_section_value(
+     CGROUPS_YARN_HIERARCHY_KEY, cgroup_cfg_section);
+
+  // Make sure it is defined.
+  if (!cgroups_root || cgroups_root[0] == 0) {
+    fprintf(ERRORFILE, "%s is not defined in container-executor.cfg\n",
+      CGROUPS_ROOT_KEY);
+    failed = 1;
+    goto cleanup;
+  }
+
+  // Make sure it is defined.
+  if (!yarn_hierarchy_name || yarn_hierarchy_name[0] == 0) {
+    fprintf(ERRORFILE, "%s is not defined in container-executor.cfg\n",
+      CGROUPS_YARN_HIERARCHY_KEY);
+    failed = 1;
+    goto cleanup;
+  }
+
+  buffer = malloc(MAX_PATH_LEN + 1);
+  if (!buffer) {
+    fprintf(ERRORFILE, "Failed to allocate memory for output path.\n");
+    failed = 1;
+    goto cleanup;
+  }
+
+  // Make a path.
+  // CGroups path should not be too long.
+  if (snprintf(buffer, MAX_PATH_LEN, "%s/%s/%s/%s/%s.%s",
+    cgroups_root, hierarchy_name, yarn_hierarchy_name,
+    group_id, hierarchy_name, param_name) < 0) {
+    fprintf(ERRORFILE, "Failed to print output path.\n");
+    failed = 1;
+    goto cleanup;
+  }
+
+cleanup:
+  if (failed) {
+    if (buffer) {
+      free(buffer);
+    }
+    return NULL;
+  }
+  return buffer;
+}
+
+int update_cgroups_parameters(
+   const char* hierarchy_name,
+   const char* param_name,
+   const char* group_id,
+   const char* value) {
+#ifndef __linux
+  fprintf(ERRORFILE, "Failed to update cgroups parameters, not supported\n");
+  return -1;
+#endif
+  int failure = 0;
+
+  if (!cgroup_cfg_section) {
+    reload_cgroups_configuration();
+  }
+
+  char* full_path = get_cgroups_path_to_write(hierarchy_name, param_name,
+    group_id);
+
+  if (!full_path) {
+    fprintf(ERRORFILE,
+      "Failed to get cgroups path to write, it should be a configuration issue");
+    failure = 1;
+    goto cleanup;
+  }
+
+  if (!verify_path_safety(full_path)) {
+    failure = 1;
+    goto cleanup;
+  }
+
+  // Make sure file exists
+  struct stat sb;
+  if (stat(full_path, &sb) != 0) {
+    fprintf(ERRORFILE, "CGroups: Could not find file to write, %s", full_path);
+    failure = 1;
+    goto cleanup;
+  }
+
+  fprintf(ERRORFILE, "CGroups: Updating cgroups, path=%s, value=%s",
+    full_path, value);
+
+  // Write values to file
+  FILE *f;
+  f = fopen(full_path, "a");
+  if (!f) {
+    fprintf(ERRORFILE, "CGroups: Failed to open cgroups file, %s", full_path);
+    failure = 1;
+    goto cleanup;
+  }
+  if (fprintf(f, "%s", value) < 0) {
+    fprintf(ERRORFILE, "CGroups: Failed to write cgroups file, %s", full_path);
+    fclose(f);
+    failure = 1;
+    goto cleanup;
+  }
+  if (fclose(f) != 0) {
+    fprintf(ERRORFILE, "CGroups: Failed to close cgroups file, %s", full_path);
+    failure = 1;
+    goto cleanup;
+  }
+
+cleanup:
+  if (full_path) {
+    free(full_path);
+  }
+  return -failure;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.h
new file mode 100644
index 0000000..cf80bcf
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/cgroups/cgroups-operations.h
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _CGROUPS_OPERATIONS_H_
+#define _CGROUPS_OPERATIONS_H_
+
+#define CGROUPS_SECTION_NAME "cgroups"
+#define CGROUPS_ROOT_KEY "root"
+#define CGROUPS_YARN_HIERARCHY_KEY "yarn-hierarchy"
+
+/**
+ * Handle update CGroups parameter update requests:
+ * - hierarchy_name: e.g. devices / cpu,cpuacct
+ * - param_name: e.g. deny
+ * - group_id: e.g. container_x_y
+ * - value: e.g. "a *:* rwm"
+ *
+ * return 0 if succeeded
+ */
+int update_cgroups_parameters(
+   const char* hierarchy_name,
+   const char* param_name,
+   const char* group_id,
+   const char* value);
+
+ /**
+  * Get CGroups path to update. Visible for testing.
+  * Return 0 if succeeded
+  */
+ char* get_cgroups_path_to_write(
+    const char* hierarchy_name,
+    const char* param_name,
+    const char* group_id);
+
+ /**
+  * Reload config from filesystem, visible for testing.
+  */
+ void reload_cgroups_configuration();
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/constants.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/constants.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/constants.h
new file mode 100644
index 0000000..5c8c4e9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/constants.h
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/* FreeBSD protects the getline() prototype. See getline(3) for more */
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _MODULES_COMMON_CONSTANTS_H_
+#define _MODULES_COMMON_CONSTANTS_H_
+
+#define CONFIGS_MODULES_PREFIX "yarn.container-executor.modules."
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.c
new file mode 100644
index 0000000..f0c6d16
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.c
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "util.h"
+#include "configuration.h"
+#include "container-executor.h"
+#include "modules/common/constants.h"
+
+#include <string.h>
+#include <stdio.h>
+#include <stdlib.h>
+
+#define ENABLED_CONFIG_KEY "module.enabled"
+
+int module_enabled(const struct section* section_cfg, const char* module_name) {
+  char* enabled_str = get_section_value(ENABLED_CONFIG_KEY, section_cfg);
+  int enabled = 0;
+  if (enabled_str && 0 == strcmp(enabled_str, "true")) {
+    enabled = 1;
+  } else {
+    fprintf(LOGFILE, "Module %s is disabled\n", module_name);
+  }
+
+  free(enabled_str);
+  return enabled;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.h
new file mode 100644
index 0000000..d58c618
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/common/module-configs.h
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _MODULES_COMMON_MODULE_CONFIGS_H_
+#define _MODULES_COMMON_MODULE_CONFIGS_H_
+
+
+/**
+ * check if module enabled given name of module.
+ * return 0 if disabled
+ */
+int module_enabled(const struct section* section_cfg, const char* module_name);
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c
new file mode 100644
index 0000000..f96645d
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.c
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "configuration.h"
+#include "container-executor.h"
+#include "utils/string-utils.h"
+#include "modules/gpu/gpu-module.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "modules/common/module-configs.h"
+#include "modules/common/constants.h"
+#include "util.h"
+
+#include <stdio.h>
+#include <string.h>
+#include <stdlib.h>
+#include <getopt.h>
+#include <unistd.h>
+
+#define EXCLUDED_GPUS_OPTION "excluded_gpus"
+#define CONTAINER_ID_OPTION "container_id"
+#define DEFAULT_NVIDIA_MAJOR_NUMBER 195
+#define MAX_CONTAINER_ID_LEN 128
+
+static const struct section* cfg_section;
+
+static int internal_handle_gpu_request(
+    update_cgroups_parameters_func update_cgroups_parameters_func_p,
+    size_t n_minor_devices_to_block, int minor_devices[],
+    const char* container_id) {
+  char* allowed_minor_numbers_str = NULL;
+  int* allowed_minor_numbers = NULL;
+  size_t n_allowed_minor_numbers = 0;
+  int return_code = 0;
+
+  if (n_minor_devices_to_block == 0) {
+    // no device to block, just return;
+    return 0;
+  }
+
+  // Get major device number from cfg, if not set, major number of (Nvidia)
+  // will be the default value.
+  int major_device_number;
+  char* major_number_str = get_section_value(GPU_MAJOR_NUMBER_CONFIG_KEY,
+     cfg_section);
+  if (!major_number_str || 0 == major_number_str[0]) {
+    // Default major number of Nvidia devices
+    major_device_number = DEFAULT_NVIDIA_MAJOR_NUMBER;
+  } else {
+    major_device_number = strtol(major_number_str, NULL, 0);
+  }
+
+  // Get allowed minor device numbers from cfg, if not set, means all minor
+  // devices can be used by YARN
+  allowed_minor_numbers_str = get_section_value(
+      GPU_ALLOWED_DEVICES_MINOR_NUMBERS,
+      cfg_section);
+  if (!allowed_minor_numbers_str || 0 == allowed_minor_numbers_str[0]) {
+    allowed_minor_numbers = NULL;
+  } else {
+    int rc = get_numbers_split_by_comma(allowed_minor_numbers_str,
+                                        &allowed_minor_numbers,
+                                        &n_allowed_minor_numbers);
+    if (0 != rc) {
+      fprintf(ERRORFILE,
+          "Failed to get allowed minor device numbers from cfg, value=%s\n",
+          allowed_minor_numbers_str);
+      return_code = -1;
+      goto cleanup;
+    }
+
+    // Make sure we're trying to black devices allowed in config
+    for (int i = 0; i < n_minor_devices_to_block; i++) {
+      int found = 0;
+      for (int j = 0; j < n_allowed_minor_numbers; j++) {
+        if (minor_devices[i] == allowed_minor_numbers[j]) {
+          found = 1;
+          break;
+        }
+      }
+
+      if (!found) {
+        fprintf(ERRORFILE,
+          "Trying to blacklist device with minor-number=%d which is not on allowed list\n",
+          minor_devices[i]);
+        return_code = -1;
+        goto cleanup;
+      }
+    }
+  }
+
+  // Use cgroup helpers to blacklist devices
+  for (int i = 0; i < n_minor_devices_to_block; i++) {
+    char param_value[128];
+    memset(param_value, 0, sizeof(param_value));
+    snprintf(param_value, sizeof(param_value), "c %d:%d rwm",
+             major_device_number, i);
+
+    int rc = update_cgroups_parameters_func_p("devices", "deny",
+      container_id, param_value);
+
+    if (0 != rc) {
+      fprintf(ERRORFILE, "CGroups: Failed to update cgroups\n");
+      return_code = -1;
+      goto cleanup;
+    }
+  }
+
+cleanup:
+  if (major_number_str) {
+    free(major_number_str);
+  }
+  if (allowed_minor_numbers) {
+    free(allowed_minor_numbers);
+  }
+  if (allowed_minor_numbers_str) {
+    free(allowed_minor_numbers_str);
+  }
+
+  return return_code;
+}
+
+void reload_gpu_configuration() {
+  cfg_section = get_configuration_section(GPU_MODULE_SECTION_NAME, get_cfg());
+}
+
+/*
+ * Format of GPU request commandline:
+ *
+ * c-e gpu --excluded_gpus 0,1,3 --container_id container_x_y
+ */
+int handle_gpu_request(update_cgroups_parameters_func func,
+    const char* module_name, int module_argc, char** module_argv) {
+  if (!cfg_section) {
+    reload_gpu_configuration();
+  }
+
+  if (!module_enabled(cfg_section, GPU_MODULE_SECTION_NAME)) {
+    fprintf(ERRORFILE,
+      "Please make sure gpu module is enabled before using it.\n");
+    return -1;
+  }
+
+  static struct option long_options[] = {
+    {EXCLUDED_GPUS_OPTION, required_argument, 0, 'e' },
+    {CONTAINER_ID_OPTION, required_argument, 0, 'c' },
+    {0, 0, 0, 0}
+  };
+
+  int rc = 0;
+  int c = 0;
+  int option_index = 0;
+
+  int* minor_devices = NULL;
+  char container_id[MAX_CONTAINER_ID_LEN];
+  memset(container_id, 0, sizeof(container_id));
+  size_t n_minor_devices_to_block = 0;
+  int failed = 0;
+
+  optind = 1;
+  while((c = getopt_long(module_argc, module_argv, "e:c:",
+                         long_options, &option_index)) != -1) {
+    switch(c) {
+      case 'e':
+        rc = get_numbers_split_by_comma(optarg, &minor_devices,
+          &n_minor_devices_to_block);
+        if (0 != rc) {
+          fprintf(ERRORFILE,
+            "Failed to get minor devices number from command line, value=%s\n",
+            optarg);
+          failed = 1;
+          goto cleanup;
+        }
+        break;
+      case 'c':
+        if (!validate_container_id(optarg)) {
+          fprintf(ERRORFILE,
+            "Specified container_id=%s is invalid\n", optarg);
+          failed = 1;
+          goto cleanup;
+        }
+        strncpy(container_id, optarg, MAX_CONTAINER_ID_LEN);
+        break;
+      default:
+        fprintf(ERRORFILE,
+          "Unknown option in gpu command character %d %c, optionindex = %d\n",
+          c, c, optind);
+        failed = 1;
+        goto cleanup;
+    }
+  }
+
+  if (0 == container_id[0]) {
+    fprintf(ERRORFILE,
+      "[%s] --container_id must be specified.\n", __func__);
+    failed = 1;
+    goto cleanup;
+  }
+
+  if (!minor_devices) {
+     // Minor devices is null, skip following call.
+     fprintf(ERRORFILE, "is not specified, skip cgroups call.\n");
+     goto cleanup;
+  }
+
+  failed = internal_handle_gpu_request(func, n_minor_devices_to_block,
+         minor_devices,
+         container_id);
+
+cleanup:
+  if (minor_devices) {
+    free(minor_devices);
+  }
+  return failed;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.h
new file mode 100644
index 0000000..59d4c7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/modules/gpu/gpu-module.h
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _MODULES_GPU_GPU_MUDULE_H_
+#define _MODULES_GPU_GPU_MUDULE_H_
+
+#define GPU_MAJOR_NUMBER_CONFIG_KEY "gpu.major-device-number"
+#define GPU_ALLOWED_DEVICES_MINOR_NUMBERS "gpu.allowed-device-minor-numbers"
+#define GPU_MODULE_SECTION_NAME "gpu"
+
+// For unit test stubbing
+typedef int (*update_cgroups_parameters_func)(const char*, const char*,
+   const char*, const char*);
+
+/**
+ * Handle gpu requests
+ */
+int handle_gpu_request(update_cgroups_parameters_func func,
+   const char* module_name, int module_argc, char** module_argv);
+
+/**
+ * Reload config from filesystem, visible for testing.
+ */
+void reload_gpu_configuration();
+
+#endif
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.c
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.c b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.c
new file mode 100644
index 0000000..dea656b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.c
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "util.h"
+
+#include <strings.h>
+#include <string.h>
+#include <stdio.h>
+#include <stdlib.h>
+
+int verify_path_safety(const char* path) {
+  if (!path || path[0] == 0) {
+    return 1;
+  }
+
+  char* dup = strdup(path);
+  if (!dup) {
+    fprintf(ERRORFILE, "%s: Failed to allocate memory for path.\n", __func__);
+    return 0;
+  }
+
+  char* p = strtok(dup, "/");
+  int succeeded = 1;
+
+  while (p != NULL) {
+    if (0 == strcmp(p, "..")) {
+      fprintf(ERRORFILE, "%s: Path included \"..\", path=%s.\n", __func__, path);
+      succeeded = 0;
+      break;
+    }
+
+    p = strtok(NULL, "/");
+  }
+  free(dup);
+
+  return succeeded;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.h
new file mode 100644
index 0000000..a42f936
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/path-utils.h
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifdef __FreeBSD__
+#define _WITH_GETLINE
+#endif
+
+#ifndef _UTILS_PATH_UTILS_H_
+#define _UTILS_PATH_UTILS_H_
+
+/*
+ * Verify if a given path is safe or not. For example, we don't want a path
+ * include ".." which can do things like:
+ * - "/cgroups/cpu,cpuacct/container/../../../etc/passwd"
+ *
+ * return false/true
+ */
+int verify_path_safety(const char* path);
+
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.c
----------------------------------------------------------------------
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 063df7e..d19c084 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
@@ -15,7 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+#include "util.h"
 
+#include <limits.h>
+#include <errno.h>
 #include <strings.h>
 #include <string.h>
 #include <stdio.h>
@@ -26,60 +29,131 @@
  * return true/false
  */
 static int all_numbers(char* input) {
-  if (0 == strlen(input)) {
-    return 0;
-  }
-
-  for (int i = 0; i < strlen(input); i++) {
-    if (input[i] < '0' || input[i] > '9') {
+  for (; input[0] != 0; input++) {
+    if (input[0] < '0' || input[0] > '9') {
       return 0;
     }
   }
   return 1;
 }
 
+int get_numbers_split_by_comma(const char* input, int** numbers,
+                               size_t* ret_n_numbers) {
+  size_t allocation_size = 1;
+  int i = 0;
+  while (input[i] != 0) {
+    if (input[i] == ',') {
+      allocation_size++;
+    }
+    i++;
+  }
+
+  (*numbers) = malloc(sizeof(int) * allocation_size);
+  if (!(*numbers)) {
+    fprintf(ERRORFILE, "Failed to allocating memory for *numbers: %s\n",
+            __func__);
+    exit(OUT_OF_MEMORY);
+  }
+  memset(*numbers, 0, sizeof(int) * allocation_size);
+
+  char* input_cpy = strdup(input);
+  if (!input_cpy) {
+    fprintf(ERRORFILE, "Failed to allocating memory for input_cpy: %s\n",
+            __func__);
+    exit(OUT_OF_MEMORY);
+  }
+
+  char* p = strtok(input_cpy, ",");
+  int idx = 0;
+  size_t n_numbers = 0;
+  while (p != NULL) {
+    char *temp;
+    long n = strtol(p, &temp, 0);
+    // According to answer:
+    // https://stackoverflow.com/questions/14176123/correct-usage-of-strtol
+    // We need to properly check errno and overflows
+    if (temp == p || *temp != '\0' ||
+        ((n == LONG_MIN || n == LONG_MAX) && errno == ERANGE)) {
+      fprintf(stderr,
+              "Could not convert '%s' to long and leftover string is: '%s'\n",
+              p, temp);
+      free(input_cpy);
+      return -1;
+    }
+
+    n_numbers++;
+    (*numbers)[idx] = n;
+    p = strtok(NULL, ",");
+    idx++;
+  }
+
+  free(input_cpy);
+  *ret_n_numbers = n_numbers;
+
+  return 0;
+}
+
 int validate_container_id(const char* input) {
+  int is_container_id = 1;
+
   /*
    * Two different forms of container_id
    * container_e17_1410901177871_0001_01_000005
    * container_1410901177871_0001_01_000005
    */
+  if (!input) {
+    return 0;
+  }
+
   char* input_cpy = strdup(input);
+  if (!input_cpy) {
+    return 0;
+  }
+
   char* p = strtok(input_cpy, "_");
   int idx = 0;
   while (p != NULL) {
     if (0 == idx) {
       if (0 != strcmp("container", p)) {
-        return 0;
+        is_container_id = 0;
+        goto cleanup;
       }
     } else if (1 == idx) {
       // this could be e[n][n], or [n][n]...
       if (!all_numbers(p)) {
-        if (strlen(p) == 0) {
-          return 0;
+        if (p[0] == 0) {
+          is_container_id = 0;
+          goto cleanup;
         }
         if (p[0] != 'e') {
-          return 0;
+          is_container_id = 0;
+          goto cleanup;
         }
         if (!all_numbers(p + 1)) {
-          return 0;
+          is_container_id = 0;
+          goto cleanup;
         }
       }
     } else {
       // otherwise, should be all numbers
       if (!all_numbers(p)) {
-        return 0;
+        is_container_id = 0;
+        goto cleanup;
       }
     }
 
     p = strtok(NULL, "_");
     idx++;
   }
-  free(input_cpy);
+
+cleanup:
+  if (input_cpy) {
+    free(input_cpy);
+  }
 
   // We should have [5,6] elements split by '_'
   if (idx > 6 || idx < 5) {
-    return 0;
+    is_container_id = 0;
   }
-  return 1;
-}
\ No newline at end of file
+  return is_container_id;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/utils/string-utils.h
----------------------------------------------------------------------
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 0a41ad1..c095eb6 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
@@ -29,4 +29,9 @@
  */
 int validate_container_id(const char* input);
 
-#endif
\ No newline at end of file
+/*
+ * return 0 if succeeded
+ */
+int get_numbers_split_by_comma(const char* input, int** numbers, size_t* n_numbers);
+
+#endif

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc
new file mode 100644
index 0000000..8ffbe88
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/cgroups/test-cgroups-module.cc
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <errno.h>
+#include <fcntl.h>
+#include <inttypes.h>
+#include <signal.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <sys/wait.h>
+#include <unistd.h>
+
+#include <gtest/gtest.h>
+#include <sstream>
+
+extern "C" {
+#include "configuration.h"
+#include "container-executor.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "test/test-container-executor-common.h"
+#include "util.h"
+}
+
+namespace ContainerExecutor {
+
+class TestCGroupsModule : public ::testing::Test {
+protected:
+  virtual void SetUp() {
+    if (mkdirs(TEST_ROOT, 0755) != 0) {
+      fprintf(ERRORFILE, "Failed to mkdir TEST_ROOT: %s\n", TEST_ROOT);
+      exit(1);
+    }
+    LOGFILE = stdout;
+    ERRORFILE = stderr;
+  }
+
+  virtual void TearDown() {}
+};
+
+TEST_F(TestCGroupsModule, test_cgroups_get_path_without_define_root) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_cgroups_get_path_without_root.cfg";
+  FILE *file = fopen(filename, "w");
+  if (file == NULL) {
+    printf("FAIL: Could not open configuration file: %s\n", filename);
+    exit(1);
+  }
+  fprintf(file, "[cgroups]\n");
+  fprintf(file, "yarn-hierarchy=yarn\n");
+  fclose(file);
+
+  // Read config file
+  read_executor_config(filename);
+  reload_cgroups_configuration();
+
+  char* path = get_cgroups_path_to_write("devices", "deny", "container_1");
+
+  ASSERT_TRUE(NULL == path) << "Should fail.\n";
+}
+
+TEST_F(TestCGroupsModule, test_cgroups_get_path_without_define_yarn_hierarchy) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_cgroups_get_path_without_root.cfg";
+  FILE *file = fopen(filename, "w");
+
+  ASSERT_TRUE(file) << "FAIL: Could not open configuration file: " << filename
+                    << "\n";
+  fprintf(file, "[cgroups]\n");
+  fprintf(file, "root=/sys/fs/cgroups\n");
+  fclose(file);
+
+  // Read config file
+  read_executor_config(filename);
+  reload_cgroups_configuration();
+  char* path = get_cgroups_path_to_write("devices", "deny", "container_1");
+
+  ASSERT_TRUE(NULL == path) << "Should fail.\n";
+}
+
+TEST_F(TestCGroupsModule, test_cgroups_get_path_succeeded) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_cgroups_get_path.cfg";
+  FILE *file = fopen(filename, "w");
+
+  ASSERT_TRUE(file) << "FAIL: Could not open configuration file\n";
+  fprintf(file, "[cgroups]\n");
+  fprintf(file, "root=/sys/fs/cgroups \n");
+  fprintf(file, "yarn-hierarchy=yarn \n");
+  fclose(file);
+
+  // Read config file
+  read_executor_config(filename);
+  reload_cgroups_configuration();
+
+  char* path = get_cgroups_path_to_write("devices", "deny", "container_1");
+  ASSERT_TRUE(NULL != path) << "Should success.\n";
+
+  const char *EXPECTED =
+      "/sys/fs/cgroups/devices/yarn/container_1/devices.deny";
+
+  ASSERT_STREQ(EXPECTED, path)
+      << "Return cgroup-path-to-write is not expected\n";
+}
+} // namespace ContainerExecutor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc
new file mode 100644
index 0000000..7e41fb4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/modules/gpu/test-gpu-module.cc
@@ -0,0 +1,203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <vector>
+
+#include <errno.h>
+#include <fcntl.h>
+#include <inttypes.h>
+#include <signal.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <sys/stat.h>
+#include <sys/wait.h>
+#include <unistd.h>
+
+#include <gtest/gtest.h>
+#include <sstream>
+
+extern "C" {
+#include "configuration.h"
+#include "container-executor.h"
+#include "modules/cgroups/cgroups-operations.h"
+#include "modules/gpu/gpu-module.h"
+#include "test/test-container-executor-common.h"
+#include "util.h"
+}
+
+namespace ContainerExecutor {
+
+class TestGpuModule : public ::testing::Test {
+protected:
+  virtual void SetUp() {
+    if (mkdirs(TEST_ROOT, 0755) != 0) {
+      fprintf(ERRORFILE, "Failed to mkdir TEST_ROOT: %s\n", TEST_ROOT);
+      exit(1);
+    }
+    LOGFILE = stdout;
+    ERRORFILE = stderr;
+  }
+
+  virtual void TearDown() {
+
+  }
+};
+
+static std::vector<const char*> cgroups_parameters_invoked;
+
+static int mock_update_cgroups_parameters(
+   const char* controller_name,
+   const char* param_name,
+   const char* group_id,
+   const char* value) {
+  char* buf = (char*) malloc(128);
+  strcpy(buf, controller_name);
+  cgroups_parameters_invoked.push_back(buf);
+
+  buf = (char*) malloc(128);
+  strcpy(buf, param_name);
+  cgroups_parameters_invoked.push_back(buf);
+
+  buf = (char*) malloc(128);
+  strcpy(buf, group_id);
+  cgroups_parameters_invoked.push_back(buf);
+
+  buf = (char*) malloc(128);
+  strcpy(buf, value);
+  cgroups_parameters_invoked.push_back(buf);
+  return 0;
+}
+
+static void verify_param_updated_to_cgroups(
+    int argc, const char** argv) {
+  ASSERT_EQ(argc, cgroups_parameters_invoked.size());
+
+  int offset = 0;
+  while (offset < argc) {
+    ASSERT_STREQ(argv[offset], cgroups_parameters_invoked[offset]);
+    offset++;
+  }
+}
+
+static void write_and_load_gpu_module_to_cfg(const char* cfg_filepath, int enabled) {
+  FILE *file = fopen(cfg_filepath, "w");
+  if (file == NULL) {
+    printf("FAIL: Could not open configuration file: %s\n", cfg_filepath);
+    exit(1);
+  }
+  fprintf(file, "[gpu]\n");
+  if (enabled) {
+    fprintf(file, "module.enabled=true\n");
+  } else {
+    fprintf(file, "module.enabled=false\n");
+  }
+  fclose(file);
+
+  // Read config file
+  read_executor_config(cfg_filepath);
+  reload_gpu_configuration();
+}
+
+static void test_gpu_module_enabled_disabled(int enabled) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_cgroups_module_enabled_disabled.cfg";
+  write_and_load_gpu_module_to_cfg(filename, enabled);
+
+  char* argv[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1",
+                   (char*) "--container_id",
+                   (char*) "container_1498064906505_0001_01_000001" };
+
+  int rc = handle_gpu_request(&mock_update_cgroups_parameters,
+              "gpu", 5, argv);
+
+  int EXPECTED_RC;
+  if (enabled) {
+    EXPECTED_RC = 0;
+  } else {
+    EXPECTED_RC = -1;
+  }
+  ASSERT_EQ(EXPECTED_RC, rc);
+}
+
+TEST_F(TestGpuModule, test_verify_gpu_module_calls_cgroup_parameter) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_verify_gpu_module_calls_cgroup_parameter.cfg";
+  write_and_load_gpu_module_to_cfg(filename, 1);
+
+  char* container_id = (char*) "container_1498064906505_0001_01_000001";
+  char* argv[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1",
+                   (char*) "--container_id",
+                   container_id };
+
+  /* Test case 1: block 2 devices */
+  cgroups_parameters_invoked.clear();
+  int rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 5, argv);
+  ASSERT_EQ(0, rc) << "Should success.\n";
+
+  // Verify cgroups parameters
+  const char* expected_cgroups_argv[] = { "devices", "deny", container_id, "c 195:0 rwm",
+    "devices", "deny", container_id, "c 195:1 rwm"};
+  verify_param_updated_to_cgroups(8, expected_cgroups_argv);
+
+  /* Test case 2: block 0 devices */
+  cgroups_parameters_invoked.clear();
+  char* argv_1[] = { (char*) "--module-gpu", (char*) "--container_id", container_id };
+  rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 3, argv_1);
+  ASSERT_EQ(0, rc) << "Should success.\n";
+
+  // Verify cgroups parameters
+  verify_param_updated_to_cgroups(0, NULL);
+}
+
+TEST_F(TestGpuModule, test_illegal_cli_parameters) {
+  // Write config file.
+  const char *filename = TEST_ROOT "/test_illegal_cli_parameters.cfg";
+  write_and_load_gpu_module_to_cfg(filename, 1);
+
+  // Illegal container id - 1
+  char* argv[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1",
+                   (char*) "--container_id", (char*) "xxxx" };
+  int rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 5, argv);
+  ASSERT_NE(0, rc) << "Should fail.\n";
+
+  // Illegal container id - 2
+  char* argv_1[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1",
+                   (char*) "--container_id", (char*) "container_1" };
+  rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 5, argv_1);
+  ASSERT_NE(0, rc) << "Should fail.\n";
+
+  // Illegal container id - 3
+  char* argv_2[] = { (char*) "--module-gpu", (char*) "--excluded_gpus", (char*) "0,1" };
+  rc = handle_gpu_request(&mock_update_cgroups_parameters,
+     "gpu", 3, argv_2);
+  ASSERT_NE(0, rc) << "Should fail.\n";
+}
+
+TEST_F(TestGpuModule, test_gpu_module_disabled) {
+  test_gpu_module_enabled_disabled(0);
+}
+
+TEST_F(TestGpuModule, test_gpu_module_enabled) {
+  test_gpu_module_enabled_disabled(1);
+}
+} // namespace ContainerExecutor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor-common.h
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor-common.h b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor-common.h
new file mode 100644
index 0000000..d353625
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor-common.h
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+ #ifdef __APPLE__
+ #include <CoreFoundation/CFString.h>
+ #include <CoreFoundation/CFPreferences.h>
+
+ #define TMPDIR "/private/tmp"
+ #define RELTMPDIR "../.."
+ #else
+ #define RELTMPDIR ".."
+ #define TMPDIR "/tmp"
+ #endif
+
+ #define TEST_ROOT TMPDIR "/test-container-executor"
+
+ #define DONT_TOUCH_FILE "dont-touch-me"
+ #define NM_LOCAL_DIRS       TEST_ROOT "/local-1%" TEST_ROOT "/local-2%" \
+                TEST_ROOT "/local-3%" TEST_ROOT "/local-4%" TEST_ROOT "/local-5"
+ #define NM_LOG_DIRS         TEST_ROOT "/logs/userlogs"
+ #define ARRAY_SIZE 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test-container-executor.c
----------------------------------------------------------------------
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 3cfefa0..64ee717 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
@@ -19,6 +19,7 @@
 #include "container-executor.h"
 #include "utils/string-utils.h"
 #include "util.h"
+#include "test/test-container-executor-common.h"
 
 #include <inttypes.h>
 #include <errno.h>
@@ -31,25 +32,6 @@
 #include <sys/stat.h>
 #include <sys/wait.h>
 
-#ifdef __APPLE__
-#include <CoreFoundation/CFString.h>
-#include <CoreFoundation/CFPreferences.h>
-
-#define TMPDIR "/private/tmp"
-#define RELTMPDIR "../.."
-#else
-#define RELTMPDIR ".."
-#define TMPDIR "/tmp"
-#endif
-
-#define TEST_ROOT TMPDIR "/test-container-executor"
-
-#define DONT_TOUCH_FILE "dont-touch-me"
-#define NM_LOCAL_DIRS       TEST_ROOT "/local-1%" TEST_ROOT "/local-2%" \
-               TEST_ROOT "/local-3%" TEST_ROOT "/local-4%" TEST_ROOT "/local-5"
-#define NM_LOG_DIRS         TEST_ROOT "/logs/userlogs"
-#define ARRAY_SIZE 1000
-
 static char* username = NULL;
 static char* yarn_username = NULL;
 static char** local_dirs = NULL;
@@ -1486,10 +1468,7 @@ int main(int argc, char **argv) {
   test_check_user(1);
 #endif
 
-  run("rm -fr " TEST_ROOT);
-
   test_trim_function();
-
   printf("\nFinished tests\n");
 
   free(current_username);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/test_main.cc
----------------------------------------------------------------------
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 d59a3f2..44c9b1b 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
@@ -20,10 +20,13 @@
 #include <main/native/container-executor/impl/util.h>
 #include <cstdio>
 
-FILE* ERRORFILE = stderr;
-FILE* LOGFILE = stdout;
+extern "C" {
+#include "util.h"
+}
 
 int main(int argc, char **argv) {
-    testing::InitGoogleTest(&argc, argv);
-    return RUN_ALL_TESTS();
+  ERRORFILE = stderr;
+  LOGFILE = stdout;
+  testing::InitGoogleTest(&argc, argv);
+  return RUN_ALL_TESTS();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-path-utils.cc
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-path-utils.cc b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-path-utils.cc
new file mode 100644
index 0000000..a24c0c7
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-path-utils.cc
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+ #include <errno.h>
+ #include <fcntl.h>
+ #include <inttypes.h>
+ #include <signal.h>
+ #include <stdio.h>
+ #include <stdlib.h>
+ #include <string.h>
+ #include <sys/stat.h>
+ #include <sys/wait.h>
+ #include <unistd.h>
+
+ #include <gtest/gtest.h>
+ #include <sstream>
+
+ extern "C" {
+ #include "utils/path-utils.h"
+ }
+
+ namespace ContainerExecutor {
+
+ class TestPathUtils : public ::testing::Test {
+ protected:
+   virtual void SetUp() {
+
+   }
+
+   virtual void TearDown() {
+
+   }
+ };
+
+ TEST_F(TestPathUtils, test_path_safety) {
+   const char* input = "./../abc/";
+   int flag = verify_path_safety(input);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_FALSE(flag) << "Should failed\n";
+
+   input = "abc/./cde";
+   flag = verify_path_safety(input);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_TRUE(flag) << "Should succeeded\n";
+
+   input = "/etc/abc/cde/./x/./y";
+   flag = verify_path_safety(input);
+   std::cout << "Testing input=" << input << "\n";
+   ASSERT_TRUE(flag) << "Should succeeded\n";
+}
+
+} // namespace ContainerExecutor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/436c2638/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..037816a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/test/utils/test-string-utils.cc
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+ #include <errno.h>
+ #include <fcntl.h>
+ #include <inttypes.h>
+ #include <signal.h>
+ #include <stdio.h>
+ #include <stdlib.h>
+ #include <string.h>
+ #include <sys/stat.h>
+ #include <sys/wait.h>
+ #include <unistd.h>
+
+ #include <gtest/gtest.h>
+ #include <sstream>
+
+ extern "C" {
+ #include "utils/string-utils.h"
+ }
+
+ 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]);
+
+   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]);
+
+   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);
+
+   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);
+
+   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";
+
+   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";
+}
+
+} // namespace ContainerExecutor
\ No newline at end of file


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