You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by id...@apache.org on 2014/05/21 01:02:39 UTC

[4/5] git commit: Make execute in MesosContainerizer async signal safe.

Make execute in MesosContainerizer async signal safe.

Test MesosContainerizer redirects task stdout and stderr.

Review: https://reviews.apache.org/r/20958


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

Branch: refs/heads/master
Commit: b669ee97da5b5ab15751fe41e5dc50180d7d3bce
Parents: b8cbeac
Author: Ian Downes <id...@twitter.com>
Authored: Wed Apr 30 15:29:58 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue May 20 15:26:15 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos_containerizer.cpp | 175 ++++++++++++-------
 src/tests/containerizer_tests.cpp               |  51 ++++++
 2 files changed, 161 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b669ee97/src/slave/containerizer/mesos_containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos_containerizer.cpp b/src/slave/containerizer/mesos_containerizer.cpp
index 2a4816e..48ae06d 100644
--- a/src/slave/containerizer/mesos_containerizer.cpp
+++ b/src/slave/containerizer/mesos_containerizer.cpp
@@ -28,6 +28,8 @@
 #include <stout/os.hpp>
 #include <stout/unreachable.hpp>
 
+#include <stout/os/execenv.hpp>
+
 #include "slave/paths.hpp"
 #include "slave/slave.hpp"
 
@@ -371,49 +373,48 @@ Future<Nothing> MesosContainerizerProcess::_recover(
 }
 
 
-// This function is executed by the forked child and should be
+// This function is executed by the forked child and must remain
 // async-signal-safe.
-// TODO(idownes): Several functions used here are not actually
-// async-signal-safe:
-// 1) os::close, os::chown and os::chdir concatenate strings on error
-// 2) os::setenv uses ::setenv that is not listed as safe
-// 3) freopen is not listed as safe
-// These can all be corrected.
 int execute(
     const CommandInfo& command,
     const string& directory,
-    const Option<string>& user,
-    const map<string, string>& env,
+    const os::ExecEnv& envp,
+    uid_t uid,
+    gid_t gid,
     bool redirectIO,
     int pipeRead,
     int pipeWrite,
     const list<Option<CommandInfo> >& commands)
 {
-  // TODO(idownes): There are several stout/os functions used here that are not
-  // async-signal safe. Replace with plain system calls.
+  if (close(pipeWrite) != 0) {
+    ABORT("Failed to close pipe[1]");
+  }
 
   // Do a blocking read on the pipe until the parent signals us to continue.
-  os::close(pipeWrite);
-  int buf;
-  ssize_t len;
-  while ((len = read(pipeRead, &buf, sizeof(buf))) == -1 && errno == EINTR);
+  char dummy;
+  ssize_t length;
+  while ((length = read(pipeRead, &dummy, sizeof(dummy))) == -1 && errno == EINTR);
 
-  if (len != sizeof(buf)) {
-    os::close(pipeRead);
+  if (length != sizeof(dummy)) {
+    close(pipeRead);
     ABORT("Failed to synchronize with parent");
   }
-  os::close(pipeRead);
 
-  // Chown the work directory if a user is provided.
-  if (user.isSome()) {
-    Try<Nothing> chown = os::chown(user.get(), directory);
-    if (chown.isError()) {
-      ABORT("Failed to chown work directory");
-    }
+  if (close(pipeRead) != 0) {
+    ABORT("Failed to close pipe[0]");
+  }
+
+  // Change gid and uid.
+  if (setgid(gid) != 0) {
+    ABORT("Failed to set gid");
+  }
+
+  if (setuid(uid) != 0) {
+    ABORT("Failed to set uid");
   }
 
   // Enter working directory.
-  if (os::chdir(directory) < 0) {
+  if (chdir(directory.c_str()) != 0) {
     ABORT("Failed to chdir into work directory");
   }
 
@@ -423,52 +424,58 @@ int execute(
   // stdout and redirecting, we instead always output to stderr /
   // stdout. Also tee'ing their output into the work directory files
   // when redirection is desired.
-  // TODO(idownes): freopen is not async-signal-safe. Could use dup2 and open
-  // directly.
   if (redirectIO) {
-    if (freopen("stdout", "a", stdout) == NULL) {
-      ABORT("freopen failed");
+    int fd;
+    while ((fd = open("stdout", O_CREAT | O_WRONLY | O_APPEND)) == -1 &&
+           errno == EINTR);
+    if (fd == -1) {
+      ABORT("Failed to open stdout");
     }
-    if (freopen("stderr", "a", stderr) == NULL) {
-      ABORT("freopen failed");
+
+    int status;
+    while ((status = dup2(fd, STDOUT_FILENO)) == -1 && errno == EINTR);
+    if (status == -1) {
+      ABORT("Failed to dup2 for stdout");
+    }
+
+    if (close(fd) == -1) {
+      ABORT("Failed to close stdout fd");
+    }
+
+    while ((fd = open("stderr", O_CREAT | O_WRONLY | O_APPEND)) == -1 &&
+           errno == EINTR);
+    if (fd == -1) {
+      ABORT("Failed to open stderr");
+    }
+
+    while ((status = dup2(fd, STDERR_FILENO)) == -1 && errno == EINTR);
+    if (status == -1) {
+      ABORT("Failed to dup2 for stderr");
+    }
+
+    if (close(fd) == -1) {
+      ABORT("Failed to close stderr fd");
     }
   }
 
-  // Run additional preparation commands. These are run as the same user as the
-  // slave. This code is run in the forked child and must be async-signal-safe.
+  // Run additional preparation commands. These are run as the same user and
+  // with the environment as the slave.
+  // NOTE: os::system() is async-signal-safe.
   foreach (const Option<CommandInfo>& command, commands) {
     if (command.isSome()) {
       // Block until the command completes.
       int status = os::system(command.get().value());
 
       if (!WIFEXITED(status) || (WEXITSTATUS(status) != 0)) {
-        ABORT("Command ",
+        ABORT("Command '",
               command.get().value().c_str(),
-              " failed to execute successfully");
+              "' failed to execute successfully");
       }
     }
   }
 
-  // Change user if provided.
-  if (user.isSome() && !os::su(user.get())) {
-    ABORT("Failed to change user");
-  }
-
-  // First set up any additional environment variables.
-  // TODO(idownes): setenv is not async-signal-safe. Environment variables
-  // could instead be set using execle.
-  foreachpair (const string& key, const string& value, env) {
-    os::setenv(key, value);
-  }
-
-  // Then set up environment variables from CommandInfo.
-  foreach(const Environment::Variable& variable,
-      command.environment().variables()) {
-    os::setenv(variable.name(), variable.value());
-  }
-
-  // Execute the command (via '/bin/sh -c command').
-  execl("/bin/sh", "sh", "-c", command.value().c_str(), (char*) NULL);
+  // Execute the command (via '/bin/sh -c command') with its environment.
+  execle("/bin/sh", "sh", "-c", command.value().c_str(), (char*) NULL, envp());
 
   // If we get here, the execv call failed.
   ABORT("Failed to execute command");
@@ -582,6 +589,8 @@ Future<list<Option<CommandInfo> > > MesosContainerizerProcess::prepare(
 
 Future<Nothing> _fetch(
     const ContainerID& containerId,
+    const string& directory,
+    const Option<string>& user,
     const Option<int>& status)
 {
   if (status.isNone() || (status.get() != 0)) {
@@ -590,6 +599,14 @@ Future<Nothing> _fetch(
                    (status.isNone() ? "none" : stringify(status.get())));
   }
 
+  // Chown the work directory if a user is provided.
+  if (user.isSome()) {
+    Try<Nothing> chown = os::chown(user.get(), directory);
+    if (chown.isError()) {
+      return Failure("Failed to chown work directory");
+    }
+  }
+
   return Nothing();
 }
 
@@ -687,7 +704,7 @@ Future<Nothing> MesosContainerizerProcess::fetch(
     .onAny(lambda::bind(&os::close, err.get()));
 
   return fetcher.get().status()
-    .then(lambda::bind(&_fetch, containerId, lambda::_1));
+    .then(lambda::bind(&_fetch, containerId, directory, user, lambda::_1));
 }
 
 
@@ -701,8 +718,8 @@ Future<Nothing> MesosContainerizerProcess::_launch(
     bool checkpoint,
     const list<Option<CommandInfo> >& commands)
 {
-  // Prepare additional environment variables for the executor.
-  const map<string, string>& env = executorEnvironment(
+  // Prepare environment variables for the executor.
+  map<string, string> env = executorEnvironment(
       executorInfo,
       directory,
       slaveId,
@@ -710,18 +727,46 @@ Future<Nothing> MesosContainerizerProcess::_launch(
       checkpoint,
       flags.recovery_timeout);
 
+  // Include any enviroment variables from CommandInfo.
+  foreach (const Environment::Variable& variable,
+           executorInfo.command().environment().variables()) {
+    env[variable.name()] = variable.value();
+  }
+
+  // Construct a representation of the environment suitable for
+  // passing to execle in the child. We construct it here because it
+  // is not async-signal-safe.
+  os::ExecEnv envp(env);
+
   // Use a pipe to block the child until it's been isolated.
   int pipes[2];
-  // We assume this should not fail under reasonable conditions so we use CHECK.
+  // We assume this should not fail under reasonable conditions so we
+  // use CHECK.
   CHECK(pipe(pipes) == 0);
 
+  // Determine the uid and gid for the child now because getpwnam is
+  // not async signal safe.
+  Result<uid_t> uid = os::getuid(user);
+  if (uid.isError() || uid.isNone()) {
+    return Failure("Invalid user: " + (uid.isError() ? uid.error()
+                                                     : "nonexistent"));
+  }
+
+  Result<gid_t> gid = os::getgid(user);
+  if (gid.isError() || gid.isNone()) {
+    return Failure("Invalid user: " + (gid.isError() ? gid.error()
+                                                     : "nonexistent"));
+  }
+
+
   // Prepare a function for the forked child to exec() the executor.
   lambda::function<int()> childFunction = lambda::bind(
       &execute,
       executorInfo.command(),
       directory,
-      user,
-      env,
+      envp,
+      uid.get(),
+      gid.get(),
       !local,
       pipes[0],
       pipes[1],
@@ -806,11 +851,11 @@ Future<Nothing> MesosContainerizerProcess::exec(
 
   // Now that we've contained the child we can signal it to continue by
   // writing to the pipe.
-  int buf;
-  ssize_t len;
-  while ((len = write(pipeWrite, &buf, sizeof(buf))) == -1 && errno == EINTR);
+  char dummy;
+  ssize_t length;
+  while ((length = write(pipeWrite, &dummy, sizeof(dummy))) == -1 && errno == EINTR);
 
-  if (len != sizeof(buf)) {
+  if (length != sizeof(dummy)) {
     return Failure("Failed to synchronize child process: " +
                    string(strerror(errno)));
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/b669ee97/src/tests/containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer_tests.cpp b/src/tests/containerizer_tests.cpp
index 9d20853..2f4888d 100644
--- a/src/tests/containerizer_tests.cpp
+++ b/src/tests/containerizer_tests.cpp
@@ -437,3 +437,54 @@ TEST_F(MesosContainerizerIsolatorPreparationTest, MultipleScripts)
 
   delete containerizer.get();
 }
+
+
+class MesosContainerizerExecuteTest : public tests::TemporaryDirectoryTest {};
+
+TEST_F(MesosContainerizerExecuteTest, IoRedirection)
+{
+  string directory = os::getcwd(); // We're inside a temporary sandbox.
+
+  slave::Flags flags;
+  flags.launcher_dir = path::join(tests::flags.build_dir, "src");
+
+  // Use local=false so std{err,out} are redirected to files.
+  Try<MesosContainerizer*> containerizer =
+    MesosContainerizer::create(flags, false);
+  ASSERT_SOME(containerizer);
+
+  ContainerID containerId;
+  containerId.set_value("test_container");
+
+  string errMsg = "this is stderr";
+  string outMsg = "this is stderr";
+  string command =
+    "(echo -n '" + errMsg + "' 1>&2) && echo -n '" + outMsg + "'";
+
+  process::Future<Nothing> launch = containerizer.get()->launch(
+      containerId,
+      CREATE_EXECUTOR_INFO("executor", command),
+      directory,
+      None(),
+      SlaveID(),
+      process::PID<Slave>(),
+      false);
+
+  // Wait for the launch to complete.
+  AWAIT_READY(launch);
+
+  // Wait on the container.
+  process::Future<containerizer::Termination> wait =
+    containerizer.get()->wait(containerId);
+  AWAIT_READY(wait);
+
+  // Check the executor exited correctly.
+  EXPECT_TRUE(wait.get().has_status());
+  EXPECT_EQ(0, wait.get().status());
+
+  // Check that std{err, out} was redirected.
+  EXPECT_SOME_EQ(errMsg, os::read(path::join(directory, "stderr")));
+  EXPECT_SOME_EQ(outMsg, os::read(path::join(directory, "stdout")));
+
+  delete containerizer.get();
+}