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:36 UTC

[1/5] git commit: Add os::getgid() and os::getuid() using reentrant version of getpwnam. Also, updated os::user() to use reentrant version of getpwuid.

Repository: mesos
Updated Branches:
  refs/heads/master dc72e2965 -> f51f33dc2


Add os::getgid() and os::getuid() using reentrant version of getpwnam.
Also, updated os::user() to use reentrant version of getpwuid.

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


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

Branch: refs/heads/master
Commit: c647c4cf32fade82dd87fd7551d14e92c7be9a60
Parents: dc72e29
Author: Ian Downes <id...@twitter.com>
Authored: Wed Apr 30 15:21:10 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue May 20 15:17:35 2014 -0700

----------------------------------------------------------------------
 .../3rdparty/stout/include/stout/os.hpp         | 152 +++++++++++++++++--
 .../3rdparty/stout/tests/os_tests.cpp           |  27 ++++
 2 files changed, 165 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c647c4cf/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
index 3875661..94d8b4c 100644
--- a/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os.hpp
@@ -634,22 +634,114 @@ inline bool chdir(const std::string& directory)
 }
 
 
-inline bool su(const std::string& user)
+inline Result<uid_t> getuid(const Option<std::string>& user = None())
 {
-  passwd* passwd;
-  if ((passwd = ::getpwnam(user.c_str())) == NULL) {
-    PLOG(ERROR) << "Failed to get user information for '"
-                << user << "', getpwnam";
-    return false;
+  if (user.isNone()) {
+    return ::getuid();
+  }
+
+  struct passwd passwd;
+  struct passwd* result = NULL;
+
+  int size = sysconf(_SC_GETPW_R_SIZE_MAX);
+  if (size == -1) {
+    // Initial value for buffer size.
+    size = 1024;
+  }
+
+  while (true) {
+    char* buffer = new char[size];
+
+    if (getpwnam_r(user.get().c_str(), &passwd, buffer, size, &result) == 0) {
+      // getpwnam_r will return 0 but set result == NULL if the user
+      // is not found.
+      if (result == NULL) {
+        delete[] buffer;
+        return None();
+      }
+
+      uid_t uid = passwd.pw_uid;
+      delete[] buffer;
+      return uid;
+    } else {
+      if (errno != ERANGE) {
+        delete[] buffer;
+        return ErrnoError("Failed to get username information");
+      }
+      // getpwnam_r set ERANGE so try again with a larger buffer.
+      size *= 2;
+      delete[] buffer;
+    }
+  }
+
+  return UNREACHABLE();
+}
+
+
+inline Result<gid_t> getgid(const Option<std::string>& user = None())
+{
+  if (user.isNone()) {
+    return ::getgid();
+  }
+
+  struct passwd passwd;
+  struct passwd* result = NULL;
+
+  int size = sysconf(_SC_GETPW_R_SIZE_MAX);
+  if (size == -1) {
+    // Initial value for buffer size.
+    size = 1024;
+  }
+
+  while (true) {
+    char* buffer = new char[size];
+
+    if (getpwnam_r(user.get().c_str(), &passwd, buffer, size, &result) == 0) {
+      // getpwnam_r will return 0 but set result == NULL if the user
+      // is not found.
+      if (result == NULL) {
+        delete[] buffer;
+        return None();
+      }
+
+      gid_t gid = passwd.pw_gid;
+      delete[] buffer;
+      return gid;
+    } else {
+      if (errno != ERANGE) {
+        delete[] buffer;
+        return ErrnoError("Failed to get username information");
+      }
+      // getpwnam_r set ERANGE so try again with a larger buffer.
+      size *= 2;
+      delete[] buffer;
+    }
   }
 
-  if (::setgid(passwd->pw_gid) < 0) {
-    PLOG(ERROR) << "Failed to set group id, setgid";
+  return UNREACHABLE();
+}
+
+
+// TODO(idownes): Refactor to return a Try and to not log internally.
+inline bool su(const std::string& user)
+{
+  Result<gid_t> gid = os::getgid(user);
+  if (gid.isError() || gid.isNone()) {
+    LOG(ERROR) << "Failed to set gid: "
+               << (gid.isError() ? gid.error() : "unknown user");
+    return false;
+  } else if (::setgid(gid.get())) {
+    PLOG(ERROR) << "Failed to setgid";
     return false;
   }
 
-  if (::setuid(passwd->pw_uid) < 0) {
-    PLOG(ERROR) << "Failed to set user id, setuid";
+  Result<uid_t> uid = os::getuid(user);
+  if (uid.isError() || uid.isNone()) {
+    LOG(ERROR) << "Failed to set uid: "
+               << (uid.isError() ? uid.error() : "unknown user");
+    return false;
+  } else if (::setuid(uid.get())) {
+    PLOG(ERROR) << "Failed to setuid";
     return false;
   }
 
@@ -720,14 +812,46 @@ inline Try<std::list<std::string> > find(
 }
 
 
+// TODO(idownes): Refactor to return a Result<string>, returning
+// None() and ErrnoError as appropriate rather than LOG(FATAL).
 inline std::string user()
 {
-  passwd* passwd;
-  if ((passwd = getpwuid(getuid())) == NULL) {
-    LOG(FATAL) << "Failed to get username information";
+  int size = sysconf(_SC_GETPW_R_SIZE_MAX);
+  if (size == -1) {
+    // Initial value for buffer size.
+    size = 1024;
+  }
+
+  struct passwd passwd;
+  struct passwd* result = NULL;
+
+  while (true) {
+    char* buffer = new char[size];
+
+    if (getpwuid_r(::getuid(), &passwd, buffer, size, &result) == 0) {
+      // getpwuid_r will return 0 but set result == NULL if the uid is
+      // not found.
+      if (result == NULL) {
+        delete[] buffer;
+        LOG(FATAL) << "Failed to find username for uid " << ::getuid();
+      }
+
+      std::string user(passwd.pw_name);
+      delete[] buffer;
+      return user;
+    } else {
+      if (errno != ERANGE) {
+        delete[] buffer;
+        PLOG(FATAL) << "Failed to get username information";
+      }
+
+      // getpwuid_r set ERANGE so try again with a larger buffer.
+      size *= 2;
+      delete[] buffer;
+    }
   }
 
-  return passwd->pw_name;
+  return UNREACHABLE();
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/c647c4cf/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
index 3e51135..76563c2 100644
--- a/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
+++ b/3rdparty/libprocess/3rdparty/stout/tests/os_tests.cpp
@@ -11,6 +11,7 @@
 #include <cstdlib> // For rand.
 #include <list>
 #include <set>
+#include <sstream>
 #include <string>
 
 #include <stout/duration.hpp>
@@ -707,3 +708,29 @@ TEST_F(OsTest, ProcessExists)
 
   EXPECT_FALSE(os::exists(pid));
 }
+
+
+TEST_F(OsTest, user)
+{
+  std::ostringstream user_;
+  EXPECT_SOME_EQ(0, os::shell(&user_ , "id -un"));
+  EXPECT_EQ(strings::trim(user_.str()), os::user());
+
+  std::ostringstream uid_;
+  EXPECT_SOME_EQ(0, os::shell(&uid_, "id -u"));
+  Try<uid_t> uid = numify<uid_t>(strings::trim(uid_.str()));
+  ASSERT_SOME(uid);
+  EXPECT_SOME_EQ(uid.get(), os::getuid(os::user()));
+
+  std::ostringstream gid_;
+  EXPECT_SOME_EQ(0, os::shell(&gid_, "id -g"));
+  Try<gid_t> gid = numify<gid_t>(strings::trim(gid_.str()));
+  ASSERT_SOME(gid);
+  EXPECT_SOME_EQ(gid.get(), os::getgid(os::user()));
+
+  EXPECT_NONE(os::getuid(UUID::random().toString()));
+  EXPECT_NONE(os::getgid(UUID::random().toString()));
+
+  EXPECT_TRUE(os::su(os::user()));
+  EXPECT_FALSE(os::su(UUID::random().toString()));
+}


[5/5] git commit: Cleaned up some style issues in MesosContainerizer

Posted by id...@apache.org.
Cleaned up some style issues in MesosContainerizer


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

Branch: refs/heads/master
Commit: f51f33dc2a2be13f06057d6b60e4cb4f449be1db
Parents: b669ee9
Author: Ian Downes <id...@twitter.com>
Authored: Tue May 20 15:15:55 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue May 20 15:26:25 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos_containerizer.cpp | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f51f33dc/src/slave/containerizer/mesos_containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos_containerizer.cpp b/src/slave/containerizer/mesos_containerizer.cpp
index 48ae06d..9084082 100644
--- a/src/slave/containerizer/mesos_containerizer.cpp
+++ b/src/slave/containerizer/mesos_containerizer.cpp
@@ -393,7 +393,8 @@ int execute(
   // Do a blocking read on the pipe until the parent signals us to continue.
   char dummy;
   ssize_t length;
-  while ((length = read(pipeRead, &dummy, sizeof(dummy))) == -1 && errno == EINTR);
+  while ((length = read(pipeRead, &dummy, sizeof(dummy))) == -1 &&
+         errno == EINTR);
 
   if (length != sizeof(dummy)) {
     close(pipeRead);
@@ -853,7 +854,8 @@ Future<Nothing> MesosContainerizerProcess::exec(
   // writing to the pipe.
   char dummy;
   ssize_t length;
-  while ((length = write(pipeWrite, &dummy, sizeof(dummy))) == -1 && errno == EINTR);
+  while ((length = write(pipeWrite, &dummy, sizeof(dummy))) == -1 &&
+         errno == EINTR);
 
   if (length != sizeof(dummy)) {
     return Failure("Failed to synchronize child process: " +


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

Posted by id...@apache.org.
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();
+}


[2/5] git commit: Add ExecEnv for use with execle to safely exec.

Posted by id...@apache.org.
Add ExecEnv for use with execle to safely exec.

execle() should be used with envp to set a child's environment variables
in an async safe manner.

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


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

Branch: refs/heads/master
Commit: 2482788c4f01bd0df1574d8689ba265f5b6c7322
Parents: c647c4c
Author: Ian Downes <id...@twitter.com>
Authored: Wed Apr 30 15:22:44 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue May 20 15:25:23 2014 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/3rdparty/stout/Makefile.am  |  1 +
 .../3rdparty/stout/include/stout/os/execenv.hpp | 98 ++++++++++++++++++++
 2 files changed, 99 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2482788c/3rdparty/libprocess/3rdparty/stout/Makefile.am
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/Makefile.am b/3rdparty/libprocess/3rdparty/stout/Makefile.am
index a9269ad..ea3049f 100644
--- a/3rdparty/libprocess/3rdparty/stout/Makefile.am
+++ b/3rdparty/libprocess/3rdparty/stout/Makefile.am
@@ -13,6 +13,7 @@ EXTRA_DIST =					\
   include/stout/duration.hpp			\
   include/stout/dynamiclibrary.hpp		\
   include/stout/error.hpp			\
+  include/stout/execenv.hpp			\
   include/stout/exit.hpp			\
   include/stout/fatal.hpp			\
   include/stout/flags.hpp			\

http://git-wip-us.apache.org/repos/asf/mesos/blob/2482788c/3rdparty/libprocess/3rdparty/stout/include/stout/os/execenv.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/3rdparty/stout/include/stout/os/execenv.hpp b/3rdparty/libprocess/3rdparty/stout/include/stout/os/execenv.hpp
new file mode 100644
index 0000000..1dd6c90
--- /dev/null
+++ b/3rdparty/libprocess/3rdparty/stout/include/stout/os/execenv.hpp
@@ -0,0 +1,98 @@
+/**
+ * Licensed 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 __STOUT_OS_ENVP_HPP__
+#define __STOUT_OS_ENVP_HPP__
+
+#include <map>
+
+#include <stout/hashmap.hpp>
+#include <stout/os.hpp>
+
+namespace os {
+
+// Used to build an environment suitable for execle()
+class ExecEnv
+{
+public:
+  explicit ExecEnv(const std::map<std::string, std::string>& environment);
+  ~ExecEnv();
+
+  ExecEnv(const ExecEnv&);
+
+  char** operator () () const { return envp; }
+
+private:
+  // Not default constructable and not assignable.
+  ExecEnv();
+  ExecEnv& operator = (const ExecEnv&);
+
+  char** envp;
+  size_t size;
+};
+
+
+inline ExecEnv::ExecEnv(const std::map<std::string, std::string>& _environment)
+  : envp(NULL),
+    size(0)
+{
+  // Merge passed environment with OS environment, overriding where necessary.
+  hashmap<std::string, std::string> environment = os::environment();
+
+  foreachpair (const std::string& key, const std::string& value, _environment) {
+    environment[key] = value;
+  }
+
+  size = environment.size();
+
+  // Convert environment to internal representation.
+  // Add 1 to the size for a NULL terminator.
+  envp = new char*[size + 1];
+  int index = 0;
+  foreachpair (const std::string& key, const std::string& value, environment) {
+    std::string entry = key + "=" + value;
+    envp[index] = new char[entry.size() + 1];
+    strncpy(envp[index], entry.c_str(), entry.size() + 1);
+    ++index;
+  }
+
+  envp[index] = NULL;
+}
+
+
+inline ExecEnv::~ExecEnv()
+{
+  for (size_t i = 0; i < size; ++i) {
+    delete[] envp[i];
+  }
+  delete[] envp;
+  envp = NULL;
+}
+
+
+inline ExecEnv::ExecEnv(const ExecEnv& other)
+{
+  size = other.size;
+
+  envp = new char*[size + 1];
+  for (size_t i = 0; i < size; ++i) {
+    envp[i] = new char[strlen(other.envp[i]) + 1];
+    strncpy(envp[i], other.envp[i], strlen(other.envp[i]) + 1);
+  }
+
+  envp[size] = NULL;
+}
+
+}  // namespace os {
+
+#endif // __STOUT_OS_ENVP_HPP__


[3/5] git commit: Update Subprocess to use ExecEnv from stout.

Posted by id...@apache.org.
Update Subprocess to use ExecEnv from stout.

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


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

Branch: refs/heads/master
Commit: b8cbeac60f6eaf46c471c606f07466e9b0d0d572
Parents: 2482788
Author: Ian Downes <id...@twitter.com>
Authored: Wed Apr 30 15:23:25 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Tue May 20 15:25:32 2014 -0700

----------------------------------------------------------------------
 3rdparty/libprocess/src/subprocess.cpp | 61 ++---------------------------
 1 file changed, 3 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b8cbeac6/3rdparty/libprocess/src/subprocess.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/libprocess/src/subprocess.cpp b/3rdparty/libprocess/src/subprocess.cpp
index 27a4355..9f8f37f 100644
--- a/3rdparty/libprocess/src/subprocess.cpp
+++ b/3rdparty/libprocess/src/subprocess.cpp
@@ -18,6 +18,8 @@
 #include <stout/os.hpp>
 #include <stout/try.hpp>
 
+#include <stout/os/execenv.hpp>
+
 using std::map;
 using std::string;
 
@@ -42,63 +44,6 @@ void cleanup(
   delete promise;
 }
 
-// Used to build the environment passed to the subproces.
-class Envp
-{
-public:
-  explicit Envp(const map<string, string>& environment);
-  ~Envp();
-
-  char** operator () () const { return envp; }
-
-private:
-  // Not default constructable, not copyable, not assignable.
-  Envp();
-  Envp(const Envp&);
-  Envp& operator = (const Envp&);
-
-  char** envp;
-  size_t size;
-};
-
-
-Envp::Envp(const map<string, string>& _environment)
-  : envp(NULL),
-    size(0)
-{
-  // Merge passed environment with OS environment, overriding where necessary.
-  hashmap<string, string> environment = os::environment();
-
-  foreachpair (const string& key, const string& value, _environment) {
-    environment[key] = value;
-  }
-
-  size = environment.size();
-
-  // Convert environment to internal representation.
-  // Add 1 to the size for a NULL terminator.
-  envp = new char*[size + 1];
-  int index = 0;
-  foreachpair (const string& key, const string& value, environment) {
-    string entry = key + "=" + value;
-    envp[index] = new char[entry.size() + 1];
-    strncpy(envp[index], entry.c_str(), entry.size() + 1);
-    ++index;
-  }
-
-  envp[index] = NULL;
-}
-
-
-Envp::~Envp()
-{
-  for (size_t i = 0; i < size; ++i) {
-    delete[] envp[i];
-  }
-  delete[] envp;
-  envp = NULL;
-}
-
 }  // namespace internal {
 
 
@@ -133,7 +78,7 @@ Try<Subprocess> subprocess(
   // TODO(tillt): Consider optimizing this to not pass an empty map
   // into the constructor or even further to use execl instead of
   // execle once we have no user supplied environment.
-  internal::Envp envp(environment.get(map<string, string>()));
+  os::ExecEnv envp(environment.get(map<string, string>()));
 
   pid_t pid;
   if ((pid = fork()) == -1) {