You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2016/11/29 19:41:20 UTC

[1/9] mesos git commit: Made MesosContainerizer launch helper to take ContainerLaunchInfo.

Repository: mesos
Updated Branches:
  refs/heads/master d536a135d -> 8d27b8cbf


Made MesosContainerizer launch helper to take ContainerLaunchInfo.

Prior to this patch, the launch helper takes various flags from
MesosContainerizer to launch the container. This makes it very hard to
add more parameters to the launch helper. This patch simplifies this
by passing 'ContainerLaunchInfo' instead. 'ContainerLaunchInfo' is
also the protobuf message returned by isolators during 'prepare()'.
This makes it very easy to merge them and send it to the launch
helper. More importantly, this makes it very easy to add more
parameters to the launch helper in the future.

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


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

Branch: refs/heads/master
Commit: 8d27b8cbfcd00e84061a9a26a3e50116c34f04db
Parents: 93eacdb
Author: Jie Yu <yu...@gmail.com>
Authored: Sat Nov 26 23:32:26 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 29 11:40:50 2016 -0800

----------------------------------------------------------------------
 include/mesos/slave/containerizer.proto         |  14 +-
 src/launcher/posix/executor.cpp                 |  38 +-
 src/slave/containerizer/mesos/containerizer.cpp | 390 ++++++++-----------
 .../mesos/isolators/docker/volume/isolator.cpp  |   2 +-
 .../mesos/isolators/filesystem/linux.cpp        |   4 +-
 .../mesos/isolators/filesystem/shared.cpp       |   2 +-
 .../mesos/isolators/namespaces/pid.cpp          |   4 +-
 .../mesos/isolators/network/cni/cni.cpp         |  15 +-
 .../mesos/isolators/network/port_mapping.cpp    |   3 +-
 src/slave/containerizer/mesos/launch.cpp        | 292 +++++++-------
 src/slave/containerizer/mesos/launch.hpp        |  14 +-
 src/tests/containerizer/port_mapping_tests.cpp  |  26 +-
 12 files changed, 369 insertions(+), 435 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/include/mesos/slave/containerizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/slave/containerizer.proto b/include/mesos/slave/containerizer.proto
index 2621522..cdb125a 100644
--- a/include/mesos/slave/containerizer.proto
+++ b/include/mesos/slave/containerizer.proto
@@ -161,19 +161,19 @@ message ContainerLaunchInfo {
   repeated CommandInfo pre_exec_commands = 1;
   optional Environment environment = 2;
 
-  // The root filesystem for the container.
-  //
-  // This field is not used since 0.28, and it is deprecated
-  // since 1.0.1.
-  optional string rootfs = 3 [deprecated = true]; // Since 1.0.1.
+  // (Linux only) The root filesystem for the container.
+  optional string rootfs = 3;
 
   // (Linux only) The namespaces a nested container
   // should enter in its parent before cloning.
-  optional uint32 enter_namespaces = 9 [default = 0];
+  repeated int32 enter_namespaces = 9;
 
   // (Linux only) The namespaces that should be cloned for the container.
   // The namespaces are created while launching the executor.
-  optional uint32 clone_namespaces = 4 [default = 0];
+  repeated int32 clone_namespaces = 4;
+
+  // (POSIX only) The user to launch the command as.
+  optional string user = 10;
 
   // If specified, it'll become the launch command for the custom
   // executor, or the launch command for the user task in the case of

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/launcher/posix/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.cpp b/src/launcher/posix/executor.cpp
index da0081c..a29b31c 100644
--- a/src/launcher/posix/executor.cpp
+++ b/src/launcher/posix/executor.cpp
@@ -22,6 +22,8 @@
 #include <stout/protobuf.hpp>
 #include <stout/strings.hpp>
 
+#include <mesos/slave/containerizer.hpp>
+
 #include "launcher/posix/executor.hpp"
 
 #ifdef __linux__
@@ -46,6 +48,8 @@ using std::vector;
 using mesos::internal::slave::MESOS_CONTAINERIZER;
 using mesos::internal::slave::MesosContainerizerLaunch;
 
+using mesos::slave::ContainerLaunchInfo;
+
 namespace mesos {
 namespace internal {
 
@@ -61,17 +65,15 @@ pid_t launchTaskPosix(
   // Prepare the flags to pass to the launch process.
   MesosContainerizerLaunch::Flags launchFlags;
 
+  ContainerLaunchInfo launchInfo;
+  launchInfo.mutable_command()->CopyFrom(command);
+
   if (rootfs.isSome()) {
     // The command executor is responsible for chrooting into the
     // root filesystem and changing the user before exec-ing the
     // user process.
 #ifdef __linux__
-    Result<string> _user = os::user();
-    if (_user.isError()) {
-      ABORT("Failed to get current user: " + _user.error());
-    } else if (_user.isNone()) {
-      ABORT("Current username is not found");
-    } else if (_user.get() != "root") {
+    if (geteuid() != 0) {
       ABORT("The command executor requires root with rootfs");
     }
 
@@ -82,15 +84,14 @@ pid_t launchTaskPosix(
 #else
     ABORT("Not expecting root volume with non-linux platform");
 #endif // __linux__
-  }
 
-  launchFlags.command = JSON::protobuf(command);
+    launchInfo.set_rootfs(rootfs.get());
 
-  if (rootfs.isSome()) {
     CHECK_SOME(sandboxDirectory);
-    launchFlags.working_directory = workingDirectory.isSome()
-      ? workingDirectory
-      : sandboxDirectory;
+
+    launchInfo.set_working_directory(workingDirectory.isSome()
+      ? workingDirectory.get()
+      : sandboxDirectory.get());
 
     // TODO(jieyu): If the task has a rootfs, the executor itself will
     // be running as root. Its sandbox is owned by root as well. In
@@ -108,12 +109,15 @@ pid_t launchTaskPosix(
     }
   }
 
-  launchFlags.rootfs = rootfs;
-  launchFlags.user = user;
+  if (user.isSome()) {
+    launchInfo.set_user(user.get());
+  }
 
-#ifdef __linux__
-  launchFlags.capabilities = capabilities;
-#endif // __linux__
+  if (capabilities.isSome()) {
+    launchInfo.mutable_capabilities()->CopyFrom(capabilities.get());
+  }
+
+  launchFlags.launch_info = JSON::protobuf(launchInfo);
 
   string commandString = strings::format(
       "%s %s %s",

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 0e42ed7..4a03f97 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1191,7 +1191,7 @@ Future<Nothing> MesosContainerizerProcess::fetch(
 
 Future<bool> MesosContainerizerProcess::_launch(
     const ContainerID& containerId,
-    const map<string, string>& _environment,
+    const map<string, string>& environment,
     const SlaveID& slaveId,
     bool checkpoint)
 {
@@ -1206,172 +1206,174 @@ Future<bool> MesosContainerizerProcess::_launch(
   }
 
   CHECK_EQ(container->state, PREPARING);
-
-  // The environment for the launched command.
-  JSON::Object environment;
-  foreachpair (const string& key, const string& value, _environment) {
-    environment.values[key] = value;
-  }
-
-  // TODO(klueska): Remove the check below once we have a good way of
-  // setting the sandbox directory for DEBUG containers.
-  if (!container->config.has_container_class() ||
-       container->config.container_class() != ContainerClass::DEBUG) {
-    // TODO(jieyu): Consider moving this to 'executorEnvironment' and
-    // consolidating with docker containerizer.
-    //
-    // NOTE: For the command executor case, although it uses the host
-    // filesystem for itself, we still set 'MESOS_SANDBOX' according to
-    // the root filesystem of the task (if specified). Command executor
-    // itself does not use this environment variable.
-    environment.values["MESOS_SANDBOX"] = container->config.has_rootfs()
-      ? flags.sandbox_directory
-      : container->config.directory();
-  }
-
-  // NOTE: Command task is a special case. Even if the container
-  // config has a root filesystem, the executor container still uses
-  // the host filesystem.
-  Option<string> rootfs;
-  if (!container->config.has_task_info() &&
-      container->config.has_rootfs()) {
-    rootfs = container->config.rootfs();
-  }
-
-  Option<CommandInfo> launchCommand;
-  Option<string> workingDirectory;
-  JSON::Array preExecCommands;
-  Option<CapabilityInfo> capabilities;
-  Option<RLimitInfo> rlimits;
-
-  Option<int> enterNamespaces = None();
-  Option<int> cloneNamespaces = None();
-
   CHECK_READY(container->launchInfos);
 
-  foreach (const Option<ContainerLaunchInfo>& launchInfo,
+  ContainerLaunchInfo launchInfo;
+
+  foreach (const Option<ContainerLaunchInfo>& isolatorLaunchInfo,
            container->launchInfos.get()) {
-    if (launchInfo.isNone()) {
+    if (isolatorLaunchInfo.isNone()) {
       continue;
     }
 
-    if (launchInfo->has_environment()) {
-      foreach (const Environment::Variable& variable,
-               launchInfo->environment().variables()) {
-        const string& name = variable.name();
-        const string& value = variable.value();
-
-        if (environment.values.count(name) > 0) {
-          VLOG(1) << "Overwriting environment variable '"
-                  << name << "', original: '"
-                  << environment.values[name] << "', new: '"
-                  << value << "', for container "
-                  << containerId;
-        }
-
-        environment.values[name] = value;
-      }
+    // Merge isolator launch infos. Perform necessary sanity checks to
+    // make sure launch infos returned by isolators do not conflict.
+    if (isolatorLaunchInfo->has_rootfs()) {
+      return Failure("Isolator should not specify rootfs");
     }
 
-    if (launchInfo->has_command()) {
-      // NOTE: 'command' from 'launchInfo' will be merged. It is
-      // isolators' responsibility to make sure that the merged
-      // command is a valid command.
-      if (launchCommand.isSome()) {
-        VLOG(1) << "Merging launch commands '" << launchCommand.get()
-                << "' and '" << launchInfo->command()
-                << "' from two different isolators";
-
-        launchCommand->MergeFrom(launchInfo->command());
-      } else {
-        launchCommand = launchInfo->command();
-      }
+    if (isolatorLaunchInfo->has_user()) {
+      return Failure("Isolator should not specify user");
     }
 
-    if (launchInfo->has_working_directory()) {
-      if (workingDirectory.isSome()) {
-        return Failure(
-            "At most one working directory can be returned from isolators");
-      } else {
-        workingDirectory = launchInfo->working_directory();
-      }
+    // NOTE: 'command' from 'isolatorLaunchInfo' will be merged. It
+    // is isolators' responsibility to make sure that the merged
+    // command is a valid command.
+    if (isolatorLaunchInfo->has_command() &&
+        launchInfo.has_command()) {
+      VLOG(1) << "Merging launch commands '" << launchInfo.command()
+              << "' and '" << isolatorLaunchInfo->command()
+              << "' from two different isolators";
     }
 
-    foreach (const CommandInfo& command, launchInfo->pre_exec_commands()) {
-      preExecCommands.values.emplace_back(JSON::protobuf(command));
+    if (isolatorLaunchInfo->has_working_directory() &&
+        launchInfo.has_working_directory()) {
+      return Failure("Multiple isolators specify working directory");
     }
 
-    if (launchInfo->has_enter_namespaces()) {
-      if (enterNamespaces.isNone()) {
-        enterNamespaces = 0;
-      }
-      enterNamespaces = enterNamespaces.get() | launchInfo->enter_namespaces();
+    if (isolatorLaunchInfo->has_capabilities() &&
+        launchInfo.has_capabilities()) {
+      return Failure("Multiple isolators specify capabilities");
     }
 
-    if (launchInfo->has_clone_namespaces()) {
-      if (cloneNamespaces.isNone()) {
-        cloneNamespaces = 0;
-      }
-      cloneNamespaces = cloneNamespaces.get() | launchInfo->clone_namespaces();
+    if (isolatorLaunchInfo->has_rlimits() &&
+        launchInfo.has_rlimits()) {
+      return Failure("Multiple isolators specify rlimits");
     }
 
-    if (launchInfo->has_capabilities()) {
-      if (capabilities.isSome()) {
-        return Failure(
-            "At most one capabilities set can be returned from isolators");
-      } else {
-        capabilities = launchInfo->capabilities();
-      }
-    }
+    launchInfo.MergeFrom(isolatorLaunchInfo.get());
+  }
 
-    if (launchInfo->has_rlimits()) {
-      if (rlimits.isSome()) {
-        return Failure(
-            "At most one rlimits set can be returned from isolators");
-      } else {
-        rlimits = launchInfo->rlimits();
-      }
-    }
+  // Remove duplicated entries in enter and clone namespaces.
+  set<int> enterNamespaces(
+      launchInfo.enter_namespaces().begin(),
+      launchInfo.enter_namespaces().end());
+
+  set<int> cloneNamespaces(
+      launchInfo.clone_namespaces().begin(),
+      launchInfo.clone_namespaces().end());
+
+  launchInfo.clear_enter_namespaces();
+  launchInfo.clear_clone_namespaces();
+
+  foreach (int ns, enterNamespaces) {
+    launchInfo.add_enter_namespaces(ns);
+  }
+
+  foreach (int ns, cloneNamespaces) {
+    launchInfo.add_clone_namespaces(ns);
   }
 
   // Determine the launch command for the container.
-  if (launchCommand.isNone()) {
-    launchCommand = container->config.command_info();
+  if (!launchInfo.has_command()) {
+    launchInfo.mutable_command()->CopyFrom(container->config.command_info());
   }
 
   // For the command executor case, we should add the rootfs flag to
   // the launch command of the command executor.
   // TODO(jieyu): Remove this once we no longer support the old style
   // command task (i.e., that uses mesos-execute).
+  // TODO(jieyu): Consider move this to filesystem isolator.
   if (container->config.has_task_info() &&
       container->config.has_rootfs()) {
-    CHECK_SOME(launchCommand);
-    launchCommand->add_arguments(
+    launchInfo.mutable_command()->add_arguments(
         "--rootfs=" + container->config.rootfs());
   }
 
-  // TODO(jieyu): 'uris', 'environment' and 'user' in 'launchCommand'
-  // will be ignored. In fact, the above fields should be moved to
-  // TaskInfo or ExecutorInfo, instead of putting them in CommandInfo.
-  launchCommand->clear_uris();
-  launchCommand->clear_environment();
-  launchCommand->clear_user();
-
-  // Include any enviroment variables from CommandInfo.
-  foreach (const Environment::Variable& variable,
-           container->config.command_info().environment().variables()) {
-    const string& name = variable.name();
-    const string& value = variable.value();
-
-    if (environment.values.count(name) > 0) {
-      VLOG(1) << "Overwriting environment variable '"
-              << name << "', original: '"
-              << environment.values[name] << "', new: '"
-              << value << "', for container "
-              << containerId;
+  // TODO(jieyu): 'uris', 'environment' and 'user' in the launch
+  // command will be ignored. 'environment' and 'user' are set
+  // explicitly in 'ContainerLaunchInfo'. In fact, the above fields
+  // should be moved to TaskInfo or ExecutorInfo, instead of putting
+  // them in CommandInfo.
+  launchInfo.mutable_command()->clear_uris();
+  launchInfo.mutable_command()->clear_environment();
+  launchInfo.mutable_command()->clear_user();
+
+  // Determine the environment for the command to be launched.
+  // NOTE: We always set the environment in 'launchInfo' so that the
+  // container does not inherit agent environment variables.
+  Environment* containerEnvironment = launchInfo.mutable_environment();
+
+  foreachpair (const string& key, const string& value, environment) {
+    Environment::Variable* variable = containerEnvironment->add_variables();
+    variable->set_name(key);
+    variable->set_value(value);
+  }
+
+  // TODO(klueska): Remove the check below once we have a good way of
+  // setting the sandbox directory for DEBUG containers.
+  if (!container->config.has_container_class() ||
+       container->config.container_class() != ContainerClass::DEBUG) {
+    // TODO(jieyu): Consider moving this to filesystem isolator.
+    //
+    // NOTE: For the command executor case, although it uses the host
+    // filesystem for itself, we still set 'MESOS_SANDBOX' according to
+    // the root filesystem of the task (if specified). Command executor
+    // itself does not use this environment variable.
+    Environment::Variable* variable = containerEnvironment->add_variables();
+    variable->set_name("MESOS_SANDBOX");
+    variable->set_value(container->config.has_rootfs()
+      ? flags.sandbox_directory
+      : container->config.directory());
+  }
+
+  // Include any user specified environment variables.
+  if (container->config.command_info().has_environment()) {
+    containerEnvironment->MergeFrom(
+        container->config.command_info().environment());
+  }
+
+  // Determine the rootfs for the container to be launched.
+  //
+  // NOTE: Command task is a special case. Even if the container
+  // config has a root filesystem, the executor container still uses
+  // the host filesystem.
+  if (!container->config.has_task_info() &&
+       container->config.has_rootfs()) {
+    launchInfo.set_rootfs(container->config.rootfs());
+  }
+
+  // Determine the working directory for the container.
+  if (launchInfo.has_rootfs()) {
+    if (!launchInfo.has_working_directory()) {
+      launchInfo.set_working_directory(flags.sandbox_directory);
+    }
+  } else {
+    // NOTE: If the container shares the host filesystem, we should
+    // not allow them to 'cd' into an arbitrary directory because
+    // that'll create security issues.
+    if (launchInfo.has_working_directory()) {
+      LOG(WARNING) << "Ignore the working directory '"
+                   << launchInfo.working_directory() << "' specified in "
+                   << "the container launch info for container "
+                   << containerId << " since the container is using the "
+                   << "host filesystem";
     }
 
-    environment.values[name] = value;
+    launchInfo.set_working_directory(container->config.directory());
+  }
+
+  // TODO(klueska): Debug containers should set their working
+  // directory to their sandbox directory (once we know how to set
+  // that properly).
+  if (container->config.has_container_class() &&
+      container->config.container_class() == ContainerClass::DEBUG) {
+    launchInfo.clear_working_directory();
+  }
+
+  // Determine the user to launch the container as.
+  if (container->config.has_user()) {
+    launchInfo.set_user(container->config.user());
   }
 
   // Determine the 'ExecutorInfo' for the io switchboard. If launching
@@ -1388,20 +1390,18 @@ Future<bool> MesosContainerizerProcess::_launch(
     // parent container.
     CHECK(containerId.has_parent());
     const ContainerID& rootContainerId = getRootContainerId(containerId);
+
     CHECK(containers_.contains(rootContainerId));
     CHECK(containers_[rootContainerId]->config.has_executor_info());
     executorInfo = containers_[rootContainerId]->config.executor_info();
   }
 
-  Option<string> user;
-  if (container->config.has_user()) {
-    user = container->config.user();
-  }
-
   return ioSwitchboard->prepare(
       executorInfo,
       container->config.directory(),
-      user)
+      container->config.has_user()
+        ? Option<string>(container->config.user())
+        : None())
     .then(defer(
         self(),
         [=](const IOSwitchboard::SubprocessInfo& subprocessInfo)
@@ -1427,79 +1427,12 @@ Future<bool> MesosContainerizerProcess::_launch(
     // Prepare the flags to pass to the launch process.
     MesosContainerizerLaunch::Flags launchFlags;
 
-    launchFlags.command = JSON::protobuf(launchCommand.get());
-    launchFlags.environment = environment;
-
-    if (rootfs.isNone()) {
-      // NOTE: If the executor shares the host filesystem, we should
-      // not allow them to 'cd' into an arbitrary directory because
-      // that'll create security issues.
-      if (workingDirectory.isSome()) {
-        LOG(WARNING) << "Ignore working directory '" << workingDirectory.get()
-                     << "' specified in container launch info for container "
-                     << containerId << " since the executor is using the "
-                     << "host filesystem";
-      }
-
-      // TODO(klueska): Debug containers should set their working
-      // directory to their sandbox directory (once we know how to set
-      // that properly).
-      if (container->config.has_container_class() &&
-          container->config.container_class() == ContainerClass::DEBUG) {
-        launchFlags.working_directory = None();
-      } else {
-        launchFlags.working_directory = container->config.directory();
-      }
-    } else {
-      launchFlags.working_directory = workingDirectory.isSome()
-        ? workingDirectory
-        : flags.sandbox_directory;
-    }
-
-#ifdef __linux__
-    // TODO(bbannier): For the case where the user requested
-    // capabilities, but no capabilities isolation was configured for
-    // the agent, the master should reject the task.
-    launchFlags.capabilities = capabilities;
-#endif // __linux__
-
-#ifdef __WINDOWS__
-    if (rootfs.isSome()) {
-      return Failure(
-          "`chroot` is not supported on Windows, but the executor "
-          "specifies a root filesystem.");
-    }
-
-    if (container->config.has_user()) {
-      return Failure(
-          "`su` is not supported on Windows, but the executor "
-          "specifies a user.");
-    }
-#else
-    launchFlags.rootfs = rootfs;
-
-    if (container->config.has_user()) {
-      launchFlags.user = container->config.user();
-    }
-
-    // TODO(bbannier): For the case where the user requested
-    // rlimits, but no rlimits isolation was configured for
-    // the agent, the master should reject the task.
-    launchFlags.rlimits = rlimits;
-#endif // __WINDOWS__
+    launchFlags.launch_info = JSON::protobuf(launchInfo);
 
 #ifndef __WINDOWS__
     launchFlags.pipe_read = pipes[0];
     launchFlags.pipe_write = pipes[1];
-#else
-    // NOTE: On windows we need to pass `Handle`s between processes, as fds
-    // are not unique across processes.
-    launchFlags.pipe_read = os::fd_to_handle(pipes[0]);
-    launchFlags.pipe_write = os::fd_to_handle(pipes[1]);
-#endif // __WINDOWS
-    launchFlags.pre_exec_commands = preExecCommands;
 
-#ifndef __WINDOWS__
     // Set the `runtime_directory` launcher flag so that the launch
     // helper knows where to checkpoint the status of the container
     // once it exits.
@@ -1509,22 +1442,45 @@ Future<bool> MesosContainerizerProcess::_launch(
     CHECK(os::exists(runtimePath));
 
     launchFlags.runtime_directory = runtimePath;
-#endif // __WINDOWS__
+#else
+    // NOTE: On windows we need to pass `Handle`s between processes, as fds
+    // are not unique across processes.
+    launchFlags.pipe_read = os::fd_to_handle(pipes[0]);
+    launchFlags.pipe_write = os::fd_to_handle(pipes[1]);
+#endif // __WINDOWS
 
+    VLOG(1) << "Launching '" << MESOS_CONTAINERIZER << "' with flags '"
+            << launchFlags << "'";
+
+    Option<int> _enterNamespaces;
+    Option<int> _cloneNamespaces;
+
+    foreach (int ns, enterNamespaces) {
+      _enterNamespaces = _enterNamespaces.isSome()
+        ? _enterNamespaces.get() | ns
+        : ns;
+    }
+
+    foreach (int ns, cloneNamespaces) {
+      _cloneNamespaces = _cloneNamespaces.isSome()
+        ? _cloneNamespaces.get() | ns
+        : ns;
+    }
+
+#ifdef __linux__
     // For now we need to special case entering a parent container's
     // mount namespace. We do this to ensure that we have access to
     // the binary we launch with `launcher->fork()`.
     //
     // TODO(klueska): Remove this special case once we pull
     // the container's `init` process out of its container.
-    Option<int> _enterNamespaces = enterNamespaces;
-
-#ifdef __linux__
-    if (enterNamespaces.isSome() && (enterNamespaces.get() & CLONE_NEWNS)) {
+    if (_enterNamespaces.isSome() && (_enterNamespaces.get() & CLONE_NEWNS)) {
       CHECK(containerId.has_parent());
+
       if (!containers_.contains(containerId.parent())) {
         return Failure("Unknown parent container");
       }
+
       if (containers_.at(containerId.parent())->pid.isNone()) {
         return Failure("Unknown parent container pid");
       }
@@ -1533,18 +1489,16 @@ Future<bool> MesosContainerizerProcess::_launch(
 
       Try<pid_t> mountNamespaceTarget = getMountNamespaceTarget(parentPid);
       if (mountNamespaceTarget.isError()) {
-        return Failure("Cannot get target mount namespace from process"
-                       " '" + stringify(parentPid) + "'");
+        return Failure(
+            "Cannot get target mount namespace from "
+            "process " + stringify(parentPid));
       }
 
       launchFlags.namespace_mnt_target = mountNamespaceTarget.get();
-      _enterNamespaces = enterNamespaces.get() & ~CLONE_NEWNS;
+      _enterNamespaces = _enterNamespaces.get() & ~CLONE_NEWNS;
     }
 #endif // __linux__
 
-    VLOG(1) << "Launching '" << MESOS_CONTAINERIZER << "' with flags '"
-            << launchFlags << "'";
-
     // Passing the launch flags via environment variables to the
     // launch helper due to the sensitivity of those flags. Otherwise
     // the launch flags would have been visible through commands like
@@ -1577,7 +1531,7 @@ Future<bool> MesosContainerizerProcess::_launch(
         // 'enterNamespaces' will be ignored by PosixLauncher.
         _enterNamespaces,
         // 'cloneNamespaces' will be ignored by PosixLauncher.
-        cloneNamespaces);
+        _cloneNamespaces);
 
     if (forked.isError()) {
       return Failure("Failed to fork: " + forked.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/slave/containerizer/mesos/isolators/docker/volume/isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/docker/volume/isolator.cpp b/src/slave/containerizer/mesos/isolators/docker/volume/isolator.cpp
index 6f0d6b0..5b630c1 100644
--- a/src/slave/containerizer/mesos/isolators/docker/volume/isolator.cpp
+++ b/src/slave/containerizer/mesos/isolators/docker/volume/isolator.cpp
@@ -467,7 +467,7 @@ Future<Option<ContainerLaunchInfo>> DockerVolumeIsolatorProcess::_prepare(
     const list<Future<string>>& futures)
 {
   ContainerLaunchInfo launchInfo;
-  launchInfo.set_clone_namespaces(CLONE_NEWNS);
+  launchInfo.add_clone_namespaces(CLONE_NEWNS);
 
   vector<string> messages;
   vector<string> sources;

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp b/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
index 8f024d0..74ab1fd 100644
--- a/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
+++ b/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
@@ -327,7 +327,7 @@ Future<Option<ContainerLaunchInfo>> LinuxFilesystemIsolatorProcess::prepare(
     }
 
     ContainerLaunchInfo launchInfo;
-    launchInfo.set_enter_namespaces(CLONE_NEWNS);
+    launchInfo.add_enter_namespaces(CLONE_NEWNS);
     return launchInfo;
   }
 
@@ -347,7 +347,7 @@ Future<Option<ContainerLaunchInfo>> LinuxFilesystemIsolatorProcess::prepare(
       executorInfo)));
 
   ContainerLaunchInfo launchInfo;
-  launchInfo.set_clone_namespaces(CLONE_NEWNS);
+  launchInfo.add_clone_namespaces(CLONE_NEWNS);
 
   // Prepare the commands that will be run in the container's mount
   // namespace right after forking the executor process. We use these

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/slave/containerizer/mesos/isolators/filesystem/shared.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/filesystem/shared.cpp b/src/slave/containerizer/mesos/isolators/filesystem/shared.cpp
index d16b42f..dad5a95 100644
--- a/src/slave/containerizer/mesos/isolators/filesystem/shared.cpp
+++ b/src/slave/containerizer/mesos/isolators/filesystem/shared.cpp
@@ -101,7 +101,7 @@ Future<Option<ContainerLaunchInfo>> SharedFilesystemIsolatorProcess::prepare(
   containerPaths.insert(containerConfig.directory());
 
   ContainerLaunchInfo launchInfo;
-  launchInfo.set_clone_namespaces(CLONE_NEWNS);
+  launchInfo.add_clone_namespaces(CLONE_NEWNS);
 
   foreach (const Volume& volume, executorInfo.container().volumes()) {
     // Because the filesystem is shared we require the container path

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/slave/containerizer/mesos/isolators/namespaces/pid.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/namespaces/pid.cpp b/src/slave/containerizer/mesos/isolators/namespaces/pid.cpp
index 5bf5ffb..f1dfc9f 100644
--- a/src/slave/containerizer/mesos/isolators/namespaces/pid.cpp
+++ b/src/slave/containerizer/mesos/isolators/namespaces/pid.cpp
@@ -84,7 +84,7 @@ Future<Option<ContainerLaunchInfo>> NamespacesPidIsolatorProcess::prepare(
   if (containerId.has_parent()) {
     // If we are a nested container, then we want to enter our
     // parent's pid namespace before cloning a new one.
-    launchInfo.set_enter_namespaces(CLONE_NEWPID);
+    launchInfo.add_enter_namespaces(CLONE_NEWPID);
 
     // However, if we are a nested container in the `DEBUG` class,
     // then we don't want to clone a new PID namespace at all, so we
@@ -95,7 +95,7 @@ Future<Option<ContainerLaunchInfo>> NamespacesPidIsolatorProcess::prepare(
     }
   }
 
-  launchInfo.set_clone_namespaces(CLONE_NEWPID);
+  launchInfo.add_clone_namespaces(CLONE_NEWPID);
 
   // Mount /proc with standard options for the container's pid
   // namespace to show the container's pids (and other /proc files),

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
index 3b2d4db..ddb4d33 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -674,22 +674,23 @@ Future<Option<ContainerLaunchInfo>> NetworkCniIsolatorProcess::prepare(
         // NOTE: There is an implicit assumption here that when used
         // for testing, '__MESOS_TEST__' is the only network the
         // container is going to join.
-        launchInfo.set_clone_namespaces(CLONE_NEWNS | CLONE_NEWUTS);
+        launchInfo.add_clone_namespaces(CLONE_NEWNS);
+        launchInfo.add_clone_namespaces(CLONE_NEWUTS);
       } else {
-        launchInfo.set_clone_namespaces(
-            CLONE_NEWNET |
-            CLONE_NEWNS |
-            CLONE_NEWUTS);
+        launchInfo.add_clone_namespaces(CLONE_NEWNET);
+        launchInfo.add_clone_namespaces(CLONE_NEWNS);
+        launchInfo.add_clone_namespaces(CLONE_NEWUTS);
       }
     } else {
       // This is a nested container. This shares the parent's network
       // and UTS namespace. For non-DEBUG containers it also needs a
       // new mount namespace.
-      launchInfo.set_enter_namespaces(CLONE_NEWNET | CLONE_NEWUTS);
+      launchInfo.add_enter_namespaces(CLONE_NEWNET);
+      launchInfo.add_enter_namespaces(CLONE_NEWUTS);
 
       if (!containerConfig.has_container_class() ||
           containerConfig.container_class() != ContainerClass::DEBUG) {
-        launchInfo.set_clone_namespaces(CLONE_NEWNS);
+        launchInfo.add_clone_namespaces(CLONE_NEWNS);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp b/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
index a994adf..fba79b5 100644
--- a/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
@@ -2522,7 +2522,8 @@ Future<Option<ContainerLaunchInfo>> PortMappingIsolatorProcess::prepare(
   // other isolators, we need to set mount sharing accordingly for
   // PORT_MAPPING_BIND_MOUNT_ROOT to avoid races described in
   // MESOS-1558. So we turn on mount namespace here for consistency.
-  launchInfo.set_clone_namespaces(CLONE_NEWNET | CLONE_NEWNS);
+  launchInfo.add_clone_namespaces(CLONE_NEWNET);
+  launchInfo.add_clone_namespaces(CLONE_NEWNS);
 
   return launchInfo;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index 320e427..d78ca4d 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -34,6 +34,8 @@
 #include <mesos/mesos.hpp>
 #include <mesos/type_utils.hpp>
 
+#include <mesos/slave/containerizer.hpp>
+
 #include "common/parse.hpp"
 
 #ifdef __linux__
@@ -62,6 +64,8 @@ using mesos::internal::capabilities::Capability;
 using mesos::internal::capabilities::ProcessCapabilities;
 #endif // __linux__
 
+using mesos::slave::ContainerLaunchInfo;
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -71,54 +75,9 @@ const string MesosContainerizerLaunch::NAME = "launch";
 
 MesosContainerizerLaunch::Flags::Flags()
 {
-  add(&Flags::command,
-      "command",
-      "The command to execute.");
-
-  add(&Flags::environment,
-      "environment",
-      "The environment variables for the command.");
-
-  add(&Flags::working_directory,
-      "working_directory",
-      "The working directory for the command. It has to be an absolute path \n"
-      "w.r.t. the root filesystem used for the command.");
-
-#ifndef __WINDOWS__
-  add(&Flags::runtime_directory,
-      "runtime_directory",
-      "The runtime directory for the container (used for checkpointing)");
-
-  add(&Flags::rootfs,
-      "rootfs",
-      "Absolute path to the container root filesystem. The command will be \n"
-      "interpreted relative to this path");
-
-  add(&Flags::user,
-      "user",
-      "The user to change to.");
-
-  add(&Flags::rlimits,
-      "rlimits",
-      "Resource limits for the launched process. This might require\n"
-      "additional priviledges if the new limits exceed the system\n"
-      "limits\n"
-      "Example with limits for CPU time, and unlimited size of created files:\n"
-      "{\n"
-      "  \"rlimits\":[\n"
-      "    {\n"
-      "      \"type\":\"RLMT_CPU\",\n"
-      "      \"soft\":\"1\",\n"
-      "      \"hard\":\"1\",\n"
-      "    },\n"
-      "    {\n"
-      "      \"type\":\"RLMT_FSIZE\"\n"
-      "    }\n"
-      "  ]\n"
-      "}"
-
-      );
-#endif // __WINDOWS__
+  add(&Flags::launch_info,
+      "launch_info",
+      "");
 
   add(&Flags::pipe_read,
       "pipe_read",
@@ -136,16 +95,13 @@ MesosContainerizerLaunch::Flags::Flags()
       "properly in the subprocess. It's used to synchronize with the \n"
       "parent process. If not specified, no synchronization will happen.");
 
-  add(&Flags::pre_exec_commands,
-      "pre_exec_commands",
-      "The additional preparation commands to execute before\n"
-      "executing the command.");
+#ifndef __WINDOWS__
+  add(&Flags::runtime_directory,
+      "runtime_directory",
+      "The runtime directory for the container (used for checkpointing)");
+#endif // __WINDOWS__
 
 #ifdef __linux__
-  add(&Flags::capabilities,
-      "capabilities",
-      "Capabilities the command can use.");
-
   add(&Flags::namespace_mnt_target,
       "namespace_mnt_target",
       "The target 'pid' of the process whose mount namespace we'd like\n"
@@ -318,44 +274,49 @@ int MesosContainerizerLaunch::execute()
   }
 #endif // __WINDOWS__
 
-  // Check command line flags.
-  if (flags.command.isNone()) {
-    cerr << "Flag --command is not specified" << endl;
+  if (flags.launch_info.isNone()) {
+    cerr << "Flag --launch_info is not specified" << endl;
     exitWithStatus(EXIT_FAILURE);
   }
 
-  bool controlPipeSpecified =
-    flags.pipe_read.isSome() && flags.pipe_write.isSome();
+  Try<ContainerLaunchInfo> _launchInfo =
+    ::protobuf::parse<ContainerLaunchInfo>(flags.launch_info.get());
 
-  if ((flags.pipe_read.isSome() && flags.pipe_write.isNone()) ||
-      (flags.pipe_read.isNone() && flags.pipe_write.isSome())) {
-    cerr << "Flag --pipe_read and --pipe_write should either be "
-         << "both set or both not set" << endl;
+  if (_launchInfo.isError()) {
+    cerr << "Failed to parse launch info: " << _launchInfo.error() << endl;
     exitWithStatus(EXIT_FAILURE);
   }
 
-  // Parse the command.
-  Try<CommandInfo> command =
-    ::protobuf::parse<CommandInfo>(flags.command.get());
+  ContainerLaunchInfo launchInfo = _launchInfo.get();
 
-  if (command.isError()) {
-    cerr << "Failed to parse the command: " << command.error() << endl;
+  if (!launchInfo.has_command()) {
+    cerr << "Launch command is not specified" << endl;
     exitWithStatus(EXIT_FAILURE);
   }
 
   // Validate the command.
-  if (command.get().shell()) {
-    if (!command.get().has_value()) {
+  if (launchInfo.command().shell()) {
+    if (!launchInfo.command().has_value()) {
       cerr << "Shell command is not specified" << endl;
       exitWithStatus(EXIT_FAILURE);
     }
   } else {
-    if (!command.get().has_value()) {
+    if (!launchInfo.command().has_value()) {
       cerr << "Executable path is not specified" << endl;
       exitWithStatus(EXIT_FAILURE);
     }
   }
 
+  if ((flags.pipe_read.isSome() && flags.pipe_write.isNone()) ||
+      (flags.pipe_read.isNone() && flags.pipe_write.isSome())) {
+    cerr << "Flag --pipe_read and --pipe_write should either be "
+         << "both set or both not set" << endl;
+    exitWithStatus(EXIT_FAILURE);
+  }
+
+  bool controlPipeSpecified =
+    flags.pipe_read.isSome() && flags.pipe_write.isSome();
+
   if (controlPipeSpecified) {
     int pipe[2] = { flags.pipe_read.get(), flags.pipe_write.get() };
 
@@ -422,54 +383,40 @@ int MesosContainerizerLaunch::execute()
 
   // Run additional preparation commands. These are run as the same
   // user and with the environment as the agent.
-  if (flags.pre_exec_commands.isSome()) {
-    // TODO(jieyu): Use JSON::Array if we have generic parse support.
-    JSON::Array array = flags.pre_exec_commands.get();
-    foreach (const JSON::Value& value, array.values) {
-      if (!value.is<JSON::Object>()) {
-        cerr << "Invalid JSON format for flag --commands" << endl;
-        exitWithStatus(EXIT_FAILURE);
-      }
-
-      Try<CommandInfo> parse = ::protobuf::parse<CommandInfo>(value);
-      if (parse.isError()) {
-        cerr << "Failed to parse a preparation command: "
-             << parse.error() << endl;
-        exitWithStatus(EXIT_FAILURE);
-      }
-
-      if (!parse.get().has_value()) {
-        cerr << "The 'value' of a preparation command is not specified" << endl;
-        exitWithStatus(EXIT_FAILURE);
-      }
-
-      cout << "Executing pre-exec command '" << value << "'" << endl;
+  foreach (const CommandInfo& command, launchInfo.pre_exec_commands()) {
+    if (!command.has_value()) {
+      cerr << "The 'value' of a preparation command is not specified" << endl;
+      exitWithStatus(EXIT_FAILURE);
+    }
 
-      int status = 0;
+    cout << "Executing pre-exec command '"
+         << JSON::protobuf(command) << "'" << endl;
 
-      if (parse->shell()) {
-        // Execute the command using the system shell.
-        status = os::system(parse->value());
-      } else {
-        // Directly spawn all non-shell commands to prohibit users
-        // from injecting arbitrary shell commands in the arguments.
-        vector<string> args;
-        foreach (const string& arg, parse->arguments()) {
-          args.push_back(arg);
-        }
+    int status = 0;
 
-        status = os::spawn(parse->value(), args);
+    if (command.shell()) {
+      // Execute the command using the system shell.
+      status = os::system(command.value());
+    } else {
+      // Directly spawn all non-shell commands to prohibit users
+      // from injecting arbitrary shell commands in the arguments.
+      vector<string> args;
+      foreach (const string& arg, command.arguments()) {
+        args.push_back(arg);
       }
 
-      if (!WIFEXITED(status) || (WEXITSTATUS(status) != 0)) {
-        cerr << "Failed to execute pre-exec command '" << value << "'" << endl;
-        exitWithStatus(EXIT_FAILURE);
-      }
+      status = os::spawn(command.value(), args);
+    }
+
+    if (!WIFEXITED(status) || (WEXITSTATUS(status) != 0)) {
+      cerr << "Failed to execute pre-exec command '"
+           << JSON::protobuf(command) << "'" << endl;
+      exitWithStatus(EXIT_FAILURE);
     }
   }
 
 #ifndef __WINDOWS__
-  // NOTE: If 'flags.user' is set, we will get the uid, gid, and the
+  // NOTE: If 'user' is set, we will get the uid, gid, and the
   // supplementary group ids associated with the specified user before
   // changing the filesystem root. This is because after changing the
   // filesystem root, the current process might no longer have access
@@ -483,10 +430,10 @@ int MesosContainerizerLaunch::execute()
   // user namespace and container capabilities is available for
   // mesos container.
 
-  if (flags.user.isSome()) {
-    Result<uid_t> _uid = os::getuid(flags.user.get());
+  if (launchInfo.has_user()) {
+    Result<uid_t> _uid = os::getuid(launchInfo.user());
     if (!_uid.isSome()) {
-      cerr << "Failed to get the uid of user '" << flags.user.get() << "': "
+      cerr << "Failed to get the uid of user '" << launchInfo.user() << "': "
            << (_uid.isError() ? _uid.error() : "not found") << endl;
       exitWithStatus(EXIT_FAILURE);
     }
@@ -494,17 +441,17 @@ int MesosContainerizerLaunch::execute()
     // No need to change user/groups if the specified user is the same
     // as that of the current process.
     if (_uid.get() != os::getuid().get()) {
-      Result<gid_t> _gid = os::getgid(flags.user.get());
+      Result<gid_t> _gid = os::getgid(launchInfo.user());
       if (!_gid.isSome()) {
-        cerr << "Failed to get the gid of user '" << flags.user.get() << "': "
+        cerr << "Failed to get the gid of user '" << launchInfo.user() << "': "
              << (_gid.isError() ? _gid.error() : "not found") << endl;
         exitWithStatus(EXIT_FAILURE);
       }
 
-      Try<vector<gid_t>> _gids = os::getgrouplist(flags.user.get());
+      Try<vector<gid_t>> _gids = os::getgrouplist(launchInfo.user());
       if (_gids.isError()) {
         cerr << "Failed to get the supplementary gids of user '"
-             << flags.user.get() << "': "
+             << launchInfo.user() << "': "
              << (_gids.isError() ? _gids.error() : "not found") << endl;
         exitWithStatus(EXIT_FAILURE);
       }
@@ -514,13 +461,18 @@ int MesosContainerizerLaunch::execute()
       gids = _gids.get();
     }
   }
+#else
+  if (launchInfo.has_user()) {
+    cerr << "Switching user is not supported on Windows" << endl;
+    exitWithStatus(EXIT_FAILURE);
+  }
 #endif // __WINDOWS__
 
 #ifdef __linux__
   // Initialize capabilities support if necessary.
   Try<Capabilities> capabilitiesManager = Error("Not initialized");
 
-  if (flags.capabilities.isSome()) {
+  if (launchInfo.has_capabilities()) {
     capabilitiesManager = Capabilities::create();
     if (capabilitiesManager.isError()) {
       cerr << "Failed to initialize capabilities support: "
@@ -538,21 +490,20 @@ int MesosContainerizerLaunch::execute()
       }
     }
   }
-#endif // __linux__
-
-#ifdef __WINDOWS__
-  // Not supported on Windows.
-  const Option<string> rootfs = None();
 #else
-  const Option<string> rootfs = flags.rootfs;
-#endif // __WINDOWS__
+  if (launchInfo.has_capabilities()) {
+    cerr << "Capabilities are not supported on non Linux system" << endl;
+    exitWithStatus(EXIT_FAILURE);
+  }
+#endif // __linux__
 
+#ifndef __WINDOWS__
   // Change root to a new root, if provided.
-  if (rootfs.isSome()) {
-    cout << "Changing root to " << rootfs.get() << endl;
+  if (launchInfo.has_rootfs()) {
+    cout << "Changing root to " << launchInfo.rootfs() << endl;
 
     // Verify that rootfs is an absolute path.
-    Result<string> realpath = os::realpath(rootfs.get());
+    Result<string> realpath = os::realpath(launchInfo.rootfs());
     if (realpath.isError()) {
       cerr << "Failed to determine if rootfs is an absolute path: "
            << realpath.error() << endl;
@@ -560,29 +511,35 @@ int MesosContainerizerLaunch::execute()
     } else if (realpath.isNone()) {
       cerr << "Rootfs path does not exist" << endl;
       exitWithStatus(EXIT_FAILURE);
-    } else if (realpath.get() != rootfs.get()) {
+    } else if (realpath.get() != launchInfo.rootfs()) {
       cerr << "Rootfs path is not an absolute path" << endl;
       exitWithStatus(EXIT_FAILURE);
     }
 
 #ifdef __linux__
-    Try<Nothing> chroot = fs::chroot::enter(rootfs.get());
-#elif defined(__WINDOWS__)
-    Try<Nothing> chroot = Error("`chroot` not supported on Windows");
-#else // For any other platform we'll just use POSIX chroot.
-    Try<Nothing> chroot = os::chroot(rootfs.get());
+    Try<Nothing> chroot = fs::chroot::enter(launchInfo.rootfs());
+#else
+    // For any other platform we'll just use POSIX chroot.
+    Try<Nothing> chroot = os::chroot(launchInfo.rootfs());
 #endif // __linux__
+
     if (chroot.isError()) {
-      cerr << "Failed to enter chroot '" << rootfs.get()
+      cerr << "Failed to enter chroot '" << launchInfo.rootfs()
            << "': " << chroot.error();
       exitWithStatus(EXIT_FAILURE);
     }
   }
+#else
+  if (launchInfo.has_rootfs()) {
+    cerr << "Changing rootfs is not supported on Windows" << endl;
+    exitWithStatus(EXIT_FAILURE);
+  }
+#endif // __WINDOWS__
 
 #ifndef __WINDOWS__
   // Setting resource limits for the process.
-  if (flags.rlimits.isSome()) {
-    foreach (const RLimitInfo::RLimit& limit, flags.rlimits->rlimits()) {
+  if (launchInfo.has_rlimits()) {
+    foreach (const RLimitInfo::RLimit& limit, launchInfo.rlimits().rlimits()) {
       Try<Nothing> set = rlimits::set(limit);
       if (set.isError()) {
         cerr << "Failed to set rlimit: " << set.error() << endl;
@@ -590,7 +547,14 @@ int MesosContainerizerLaunch::execute()
       }
     }
   }
+#else
+  if (launchInfo.has_rlimits()) {
+    cerr << "Rlimits are not supported on Windows" << endl;
+    exitWithStatus(EXIT_FAILURE);
+  }
+#endif // __WINDOWS__
 
+#ifndef __WINDOWS__
   // Change user if provided. Note that we do that after executing the
   // preparation commands so that those commands will be run with the
   // same privilege as the mesos-agent.
@@ -619,7 +583,7 @@ int MesosContainerizerLaunch::execute()
 #endif // __WINDOWS__
 
 #ifdef __linux__
-  if (flags.capabilities.isSome()) {
+  if (launchInfo.has_capabilities()) {
     Try<ProcessCapabilities> capabilities = capabilitiesManager->get();
     if (capabilities.isError()) {
       cerr << "Failed to get capabilities for the current process: "
@@ -640,7 +604,7 @@ int MesosContainerizerLaunch::execute()
     }
 
     // Set up requested capabilities.
-    set<Capability> target = capabilities::convert(flags.capabilities.get());
+    set<Capability> target = capabilities::convert(launchInfo.capabilities());
 
     capabilities->set(capabilities::EFFECTIVE, target);
     capabilities->set(capabilities::PERMITTED, target);
@@ -655,35 +619,53 @@ int MesosContainerizerLaunch::execute()
   }
 #endif // __linux__
 
-  if (flags.working_directory.isSome()) {
-    Try<Nothing> chdir = os::chdir(flags.working_directory.get());
+  if (launchInfo.has_working_directory()) {
+    Try<Nothing> chdir = os::chdir(launchInfo.working_directory());
     if (chdir.isError()) {
       cerr << "Failed to chdir into current working directory "
-           << "'" << flags.working_directory.get() << "': "
+           << "'" << launchInfo.working_directory() << "': "
            << chdir.error() << endl;
       exitWithStatus(EXIT_FAILURE);
     }
   }
 
   // Prepare the executable and the argument list for the child.
-  string executable(command->shell()
+  string executable(launchInfo.command().shell()
     ? os::Shell::name
-    : command->value().c_str());
+    : launchInfo.command().value().c_str());
 
-  os::raw::Argv argv(command->shell()
-    ? vector<string>({os::Shell::arg0, os::Shell::arg1, command->value()})
-    : vector<string>(command->arguments().begin(), command->arguments().end()));
+  os::raw::Argv argv(launchInfo.command().shell()
+    ? vector<string>({
+          os::Shell::arg0,
+          os::Shell::arg1,
+          launchInfo.command().value()})
+    : vector<string>(
+          launchInfo.command().arguments().begin(),
+          launchInfo.command().arguments().end()));
 
   // Prepare the environment for the child. If 'environment' is not
   // specified, inherit the environment of the current process.
   Option<os::raw::Envp> envp;
-  if (flags.environment.isSome()) {
-    JSON::Object environment = flags.environment.get();
+  if (launchInfo.has_environment()) {
+    hashmap<string, string> environment;
+
+    foreach (const Environment::Variable& variable,
+             launchInfo.environment().variables()) {
+      const string& name = variable.name();
+      const string& value = variable.value();
+
+      if (environment.contains(name)) {
+        cout << "Overwriting environment variable '" << name
+             << "', original: '" << environment[name]
+             << "', new: '" << value << "'" << endl;
+      }
+
+      environment[name] = value;
+    }
 
-    Result<JSON::String> path = flags.environment->find<JSON::String>("PATH");
-    if (path.isNone()) {
-      environment.values["PATH"] =
-          "/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin";
+    if (!environment.contains("PATH")) {
+      environment["PATH"] =
+        "/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin";
     }
 
     envp = os::raw::Envp(environment);

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/slave/containerizer/mesos/launch.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.hpp b/src/slave/containerizer/mesos/launch.hpp
index d9729d7..5bba139 100644
--- a/src/slave/containerizer/mesos/launch.hpp
+++ b/src/slave/containerizer/mesos/launch.hpp
@@ -25,6 +25,7 @@
 
 #include <mesos/mesos.hpp>
 
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -38,20 +39,13 @@ public:
   {
     Flags();
 
-    Option<JSON::Object> command;
-    Option<JSON::Object> environment;
-    Option<std::string> working_directory;
+    Option<JSON::Object> launch_info;
+    Option<int> pipe_read;
+    Option<int> pipe_write;
 #ifndef __WINDOWS__
     Option<std::string> runtime_directory;
-    Option<std::string> rootfs;
-    Option<std::string> user;
-    Option<RLimitInfo> rlimits;
 #endif // __WINDOWS__
-    Option<int> pipe_read;
-    Option<int> pipe_write;
-    Option<JSON::Array> pre_exec_commands;
 #ifdef __linux__
-    Option<CapabilityInfo> capabilities;
     Option<pid_t> namespace_mnt_target;
     bool unshare_namespace_mnt;
 #endif // __linux__

http://git-wip-us.apache.org/repos/asf/mesos/blob/8d27b8cb/src/tests/containerizer/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/port_mapping_tests.cpp b/src/tests/containerizer/port_mapping_tests.cpp
index 79b0a60..207742f 100644
--- a/src/tests/containerizer/port_mapping_tests.cpp
+++ b/src/tests/containerizer/port_mapping_tests.cpp
@@ -307,7 +307,7 @@ protected:
       int pipes[2],
       const ContainerID& containerId,
       const string& command,
-      const Option<ContainerLaunchInfo>& launchInfo)
+      const Option<ContainerLaunchInfo>& isolatorLaunchInfo)
   {
     CommandInfo commandInfo;
     commandInfo.set_value(command);
@@ -315,24 +315,22 @@ protected:
     // The flags to pass to the helper process.
     MesosContainerizerLaunch::Flags launchFlags;
 
-    launchFlags.command = JSON::protobuf(commandInfo);
-    launchFlags.working_directory = os::getcwd();
-
-    CHECK_SOME(os::user());
-    launchFlags.user = os::user().get();
-
     launchFlags.pipe_read = pipes[0];
     launchFlags.pipe_write = pipes[1];
 
-    if (launchInfo->pre_exec_commands().size() != 1) {
-      return Error("No valid commands inside ContainerLaunchInfo.");
-    }
+    ContainerLaunchInfo launchInfo;
+    launchInfo.mutable_command()->CopyFrom(commandInfo);
+    launchInfo.set_working_directory(os::getcwd());
 
-    JSON::Array preExecCommands;
-    preExecCommands.values.push_back(JSON::protobuf(
-        launchInfo->pre_exec_commands(0)));
+    CHECK_SOME(os::user());
+    launchInfo.set_user(os::user().get());
+
+    if (isolatorLaunchInfo.isSome()) {
+      launchInfo.mutable_pre_exec_commands()->MergeFrom(
+          isolatorLaunchInfo->pre_exec_commands());
+    }
 
-    launchFlags.pre_exec_commands = preExecCommands;
+    launchFlags.launch_info = JSON::protobuf(launchInfo);
 
     vector<string> argv(2);
     argv[0] = MESOS_CONTAINERIZER;


[9/9] mesos git commit: Removed two disabled shared filesystem isolator tests.

Posted by ji...@apache.org.
Removed two disabled shared filesystem isolator tests.

Shared filesystem isolator has been deprecated for a while, and those
two tests are disabled all the time. This patch removes them.

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


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

Branch: refs/heads/master
Commit: fbbe2a111b895e53970867c175290d3ef72223fd
Parents: 9d6cbf0
Author: Jie Yu <yu...@gmail.com>
Authored: Sat Nov 26 20:46:20 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 29 11:40:50 2016 -0800

----------------------------------------------------------------------
 src/tests/containerizer/isolator_tests.cpp | 273 ++----------------------
 1 file changed, 13 insertions(+), 260 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fbbe2a11/src/tests/containerizer/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 9766aaf..da46278 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -16,293 +16,45 @@
 
 #include <unistd.h>
 
-#include <functional>
-#include <iostream>
 #include <string>
-#include <vector>
-
-#include <gmock/gmock.h>
-
-#include <mesos/resources.hpp>
-
-#include <mesos/module/isolator.hpp>
-
-#include <mesos/slave/isolator.hpp>
 
 #include <process/future.hpp>
-#include <process/io.hpp>
 #include <process/owned.hpp>
-#include <process/reap.hpp>
 
-#include <stout/abort.hpp>
-#include <stout/duration.hpp>
 #include <stout/gtest.hpp>
-#include <stout/hashset.hpp>
 #include <stout/os.hpp>
 #include <stout/path.hpp>
+#include <stout/stringify.hpp>
+#include <stout/strings.hpp>
+#include <stout/uuid.hpp>
 
-#ifdef __linux__
-#include "linux/ns.hpp"
-#endif // __linux__
-
-#include "master/master.hpp"
-
-#include "slave/flags.hpp"
-#include "slave/gc.hpp"
-#include "slave/slave.hpp"
+#include <mesos/mesos.hpp>
 
 #ifdef __linux__
-#include "slave/containerizer/mesos/isolators/filesystem/shared.hpp"
-#endif // __linux__
-#include "slave/containerizer/mesos/isolators/posix.hpp"
+#include "linux/ns.hpp"
+#endif
 
-#include "slave/containerizer/mesos/launcher.hpp"
-#ifdef __linux__
 #include "slave/containerizer/fetcher.hpp"
+
 #include "slave/containerizer/mesos/containerizer.hpp"
-#include "slave/containerizer/mesos/launch.hpp"
-#include "slave/containerizer/mesos/linux_launcher.hpp"
-#endif // __linux__
 
-#include "tests/flags.hpp"
 #include "tests/mesos.hpp"
-#include "tests/module.hpp"
-#include "tests/utils.hpp"
 
-using namespace process;
+using std::string;
+
+using process::Future;
+using process::Owned;
 
-using mesos::internal::master::Master;
-#ifdef __linux__
 using mesos::internal::slave::Fetcher;
-using mesos::internal::slave::LinuxLauncher;
-using mesos::internal::slave::SharedFilesystemIsolatorProcess;
-#endif // __linux__
-using mesos::internal::slave::Launcher;
 using mesos::internal::slave::MesosContainerizer;
-using mesos::internal::slave::PosixLauncher;
-using mesos::internal::slave::Slave;
 
-using mesos::master::detector::MasterDetector;
-
-using mesos::slave::ContainerConfig;
-using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerTermination;
-using mesos::slave::Isolator;
-
-using std::ostringstream;
-using std::set;
-using std::string;
-using std::vector;
 
 namespace mesos {
 namespace internal {
 namespace tests {
 
 #ifdef __linux__
-class SharedFilesystemIsolatorTest : public MesosTest {};
-
-
-// Test that a container can create a private view of a system
-// directory (/var/tmp). Check that a file written by a process inside
-// the container doesn't appear on the host filesystem but does appear
-// under the container's work directory.
-// This test is disabled since we're planning to remove the shared
-// filesystem isolator and this test is not working on other distros
-// such as CentOS 7.1
-// TODO(tnachen): Remove this test when shared filesystem isolator
-// is removed.
-TEST_F(SharedFilesystemIsolatorTest, DISABLED_ROOT_RelativeVolume)
-{
-  slave::Flags flags = CreateSlaveFlags();
-  flags.isolation = "filesystem/shared";
-
-  Try<Isolator*> _isolator = SharedFilesystemIsolatorProcess::create(flags);
-  ASSERT_SOME(_isolator);
-  Owned<Isolator> isolator(_isolator.get());
-
-  Try<Launcher*> _launcher = LinuxLauncher::create(flags);
-  ASSERT_SOME(_launcher);
-  Owned<Launcher> launcher(_launcher.get());
-
-  // Use /var/tmp so we don't mask the work directory (under /tmp).
-  const string containerPath = "/var/tmp";
-  ASSERT_TRUE(os::stat::isdir(containerPath));
-
-  // Use a host path relative to the container work directory.
-  const string hostPath = strings::remove(containerPath, "/", strings::PREFIX);
-
-  ContainerInfo containerInfo;
-  containerInfo.set_type(ContainerInfo::MESOS);
-  containerInfo.add_volumes()->CopyFrom(
-      createVolumeFromHostPath(containerPath, hostPath, Volume::RW));
-
-  ExecutorInfo executorInfo;
-  executorInfo.mutable_container()->CopyFrom(containerInfo);
-
-  ContainerID containerId;
-  containerId.set_value(UUID::random().toString());
-
-  ContainerConfig containerConfig;
-  containerConfig.mutable_executor_info()->CopyFrom(executorInfo);
-  containerConfig.set_directory(flags.work_dir);
-
-  Future<Option<ContainerLaunchInfo>> prepare =
-    isolator->prepare(
-        containerId,
-        containerConfig);
-
-  AWAIT_READY(prepare);
-  ASSERT_SOME(prepare.get());
-  ASSERT_EQ(1, prepare.get().get().pre_exec_commands().size());
-  EXPECT_TRUE(prepare.get().get().has_clone_namespaces());
-
-  // The test will touch a file in container path.
-  const string file = path::join(containerPath, UUID::random().toString());
-  ASSERT_FALSE(os::exists(file));
-
-  // Manually run the isolator's preparation command first, then touch
-  // the file.
-  vector<string> args;
-  args.push_back("sh");
-  args.push_back("-x");
-  args.push_back("-c");
-  args.push_back(
-      prepare.get().get().pre_exec_commands(0).value() + " && touch " + file);
-
-  Try<pid_t> pid = launcher->fork(
-      containerId,
-      "sh",
-      args,
-      Subprocess::FD(STDIN_FILENO),
-      Subprocess::FD(STDOUT_FILENO),
-      Subprocess::FD(STDERR_FILENO),
-      nullptr,
-      None(),
-      None(),
-      prepare.get().get().clone_namespaces());
-  ASSERT_SOME(pid);
-
-  // Set up the reaper to wait on the forked child.
-  Future<Option<int>> status = process::reap(pid.get());
-
-  AWAIT_READY(status);
-  EXPECT_SOME_EQ(0, status.get());
-
-  // Check the correct hierarchy was created under the container work
-  // directory.
-  string dir = "/";
-  foreach (const string& subdir, strings::tokenize(containerPath, "/")) {
-    dir = path::join(dir, subdir);
-
-    struct stat hostStat;
-    EXPECT_EQ(0, ::stat(dir.c_str(), &hostStat));
-
-    struct stat containerStat;
-    EXPECT_EQ(0,
-              ::stat(path::join(flags.work_dir, dir).c_str(), &containerStat));
-
-    EXPECT_EQ(hostStat.st_mode, containerStat.st_mode);
-    EXPECT_EQ(hostStat.st_uid, containerStat.st_uid);
-    EXPECT_EQ(hostStat.st_gid, containerStat.st_gid);
-  }
-
-  // Check it did *not* create a file in the host namespace.
-  EXPECT_FALSE(os::exists(file));
-
-  // Check it did create the file under the container's work directory
-  // on the host.
-  EXPECT_TRUE(os::exists(path::join(flags.work_dir, file)));
-}
-
-
-// This test is disabled since we're planning to remove the shared
-// filesystem isolator and this test is not working on other distros
-// such as CentOS 7.1
-// TODO(tnachen): Remove this test when shared filesystem isolator
-// is removed.
-TEST_F(SharedFilesystemIsolatorTest, DISABLED_ROOT_AbsoluteVolume)
-{
-  slave::Flags flags = CreateSlaveFlags();
-  flags.isolation = "filesystem/shared";
-
-  Try<Isolator*> _isolator = SharedFilesystemIsolatorProcess::create(flags);
-  ASSERT_SOME(_isolator);
-  Owned<Isolator> isolator(_isolator.get());
-
-  Try<Launcher*> _launcher = LinuxLauncher::create(flags);
-  ASSERT_SOME(_launcher);
-  Owned<Launcher> launcher(_launcher.get());
-
-  // We'll mount the absolute test work directory as /var/tmp in the
-  // container.
-  const string hostPath = flags.work_dir;
-  const string containerPath = "/var/tmp";
-
-  ContainerInfo containerInfo;
-  containerInfo.set_type(ContainerInfo::MESOS);
-  containerInfo.add_volumes()->CopyFrom(
-      createVolumeFromHostPath(containerPath, hostPath, Volume::RW));
-
-  ExecutorInfo executorInfo;
-  executorInfo.mutable_container()->CopyFrom(containerInfo);
-
-  ContainerID containerId;
-  containerId.set_value(UUID::random().toString());
-
-  ContainerConfig containerConfig;
-  containerConfig.mutable_executor_info()->CopyFrom(executorInfo);
-  containerConfig.set_directory(flags.work_dir);
-
-  Future<Option<ContainerLaunchInfo>> prepare =
-    isolator->prepare(
-        containerId,
-        containerConfig);
-
-  AWAIT_READY(prepare);
-  ASSERT_SOME(prepare.get());
-  ASSERT_EQ(1, prepare.get().get().pre_exec_commands().size());
-  EXPECT_TRUE(prepare.get().get().has_clone_namespaces());
-
-  // Test the volume mounting by touching a file in the container's
-  // /tmp, which should then be in flags.work_dir.
-  const string filename = UUID::random().toString();
-  ASSERT_FALSE(os::exists(path::join(containerPath, filename)));
-
-  vector<string> args;
-  args.push_back("sh");
-  args.push_back("-x");
-  args.push_back("-c");
-  args.push_back(prepare.get().get().pre_exec_commands(0).value() +
-                 " && touch " +
-                 path::join(containerPath, filename));
-
-  Try<pid_t> pid = launcher->fork(
-      containerId,
-      "sh",
-      args,
-      Subprocess::FD(STDIN_FILENO),
-      Subprocess::FD(STDOUT_FILENO),
-      Subprocess::FD(STDERR_FILENO),
-      nullptr,
-      None(),
-      None(),
-      prepare.get().get().clone_namespaces());
-  ASSERT_SOME(pid);
-
-  // Set up the reaper to wait on the forked child.
-  Future<Option<int>> status = process::reap(pid.get());
-
-  AWAIT_READY(status);
-  EXPECT_SOME_EQ(0, status.get());
-
-  // Check the file was created in flags.work_dir.
-  EXPECT_TRUE(os::exists(path::join(hostPath, filename)));
-
-  // Check it didn't get created in the host's view of containerPath.
-  EXPECT_FALSE(os::exists(path::join(containerPath, filename)));
-}
-
-
 class NamespacesPidIsolatorTest : public MesosTest {};
 
 
@@ -328,7 +80,7 @@ TEST_F(NamespacesPidIsolatorTest, ROOT_PidNamespace)
   const string command =
     "stat -c %i /proc/self/ns/pid > ns && (cat /proc/1/comm > init)";
 
-  process::Future<bool> launch = containerizer->launch(
+  Future<bool> launch = containerizer->launch(
       containerId,
       None(),
       createExecutorInfo("executor", command),
@@ -337,6 +89,7 @@ TEST_F(NamespacesPidIsolatorTest, ROOT_PidNamespace)
       SlaveID(),
       std::map<string, string>(),
       false);
+
   AWAIT_READY(launch);
   ASSERT_TRUE(launch.get());
 


[2/9] mesos git commit: Removed MesosContainerizerLaunchTest.

Posted by ji...@apache.org.
Removed MesosContainerizerLaunchTest.

This test is not necessary as we already test that in linux filesystem
isolator tests. This patch removes it.

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


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

Branch: refs/heads/master
Commit: 418433ddc1effd0d8f75d9fc426c8a4cb6c70f96
Parents: fbbe2a1
Author: Jie Yu <yu...@gmail.com>
Authored: Sat Nov 26 21:02:32 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 29 11:40:50 2016 -0800

----------------------------------------------------------------------
 src/Makefile.am                          |   1 -
 src/tests/containerizer/launch_tests.cpp | 136 --------------------------
 2 files changed, 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/418433dd/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 85eda53..d1cc101 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -2244,7 +2244,6 @@ mesos_tests_SOURCES +=						\
   tests/containerizer/docker_volume_isolator_tests.cpp		\
   tests/containerizer/linux_filesystem_isolator_tests.cpp	\
   tests/containerizer/fs_tests.cpp				\
-  tests/containerizer/launch_tests.cpp				\
   tests/containerizer/memory_pressure_tests.cpp			\
   tests/containerizer/nested_mesos_containerizer_tests.cpp	\
   tests/containerizer/ns_tests.cpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/418433dd/src/tests/containerizer/launch_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/launch_tests.cpp b/src/tests/containerizer/launch_tests.cpp
deleted file mode 100644
index 680c147..0000000
--- a/src/tests/containerizer/launch_tests.cpp
+++ /dev/null
@@ -1,136 +0,0 @@
-// 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 <string>
-#include <vector>
-
-#include <gmock/gmock.h>
-
-#include <stout/gtest.hpp>
-#include <stout/none.hpp>
-#include <stout/option.hpp>
-#include <stout/os.hpp>
-#include <stout/try.hpp>
-
-#include <stout/tests/utils.hpp>
-
-#include <process/gtest.hpp>
-#include <process/reap.hpp>
-#include <process/subprocess.hpp>
-
-#include "slave/containerizer/mesos/launch.hpp"
-
-#include "tests/flags.hpp"
-#include "tests/utils.hpp"
-
-#include "tests/containerizer/rootfs.hpp"
-
-using namespace process;
-
-using std::string;
-using std::vector;
-
-namespace mesos {
-namespace internal {
-namespace tests {
-
-// TODO(jieyu): Move this test to mesos_containerizer_tests.cpp once
-// we have a filesystem isolator that supports changing rootfs.
-
-class MesosContainerizerLaunchTest : public TemporaryDirectoryTest
-{
-public:
-  Try<Subprocess> run(
-      const string& _command,
-      const Option<string>& rootfs = None())
-  {
-    slave::MesosContainerizerLaunch::Flags launchFlags;
-
-    CommandInfo command;
-    command.set_value(_command);
-
-    launchFlags.command = JSON::protobuf(command);
-    launchFlags.working_directory = "/tmp";
-    launchFlags.pipe_read = open("/dev/zero", O_RDONLY);
-    launchFlags.pipe_write = open("/dev/null", O_WRONLY);
-    launchFlags.rootfs = rootfs;
-
-    vector<string> argv(2);
-    argv[0] = "mesos-containerizer";
-    argv[1] = slave::MesosContainerizerLaunch::NAME;
-
-    Try<Subprocess> s = subprocess(
-        path::join(getLauncherDir(), "mesos-containerizer"),
-        argv,
-        Subprocess::PATH("/dev/null"),
-        Subprocess::FD(STDOUT_FILENO),
-        Subprocess::FD(STDERR_FILENO),
-        &launchFlags,
-        None(),
-        [](const lambda::function<int()>& child) {
-          return os::clone(child, CLONE_NEWNS | SIGCHLD);
-        });
-
-    close(launchFlags.pipe_read.get());
-    close(launchFlags.pipe_write.get());
-
-    return s;
-  }
-};
-
-
-TEST_F(MesosContainerizerLaunchTest, ROOT_ChangeRootfs)
-{
-  Try<Owned<Rootfs>> rootfs =
-    LinuxRootfs::create(path::join(os::getcwd(), "rootfs"));
-
-  ASSERT_SOME(rootfs);
-
-  // Add /usr/bin/stat into the rootfs.
-  ASSERT_SOME(rootfs.get()->add("/usr/bin/stat"));
-
-  Clock::pause();
-
-  Try<Subprocess> s = run(
-      "/usr/bin/stat -c %i / >" + path::join("/", "stat.output"),
-      rootfs.get()->root);
-
-  ASSERT_SOME(s);
-
-  // Advance time until the internal reaper reaps the subprocess.
-  while (s.get().status().isPending()) {
-    Clock::advance(process::MAX_REAP_INTERVAL());
-    Clock::settle();
-  }
-
-  AWAIT_ASSERT_WEXITSTATUS_EQ(0, s.get().status());
-
-  // Check the rootfs has a different root by comparing the inodes.
-  Try<ino_t> self = os::stat::inode("/");
-  ASSERT_SOME(self);
-
-  Try<string> read = os::read(path::join(rootfs.get()->root, "stat.output"));
-  ASSERT_SOME(read);
-
-  Try<ino_t> other = numify<ino_t>(strings::trim(read.get()));
-  ASSERT_SOME(other);
-
-  EXPECT_NE(self.get(), other.get());
-}
-
-} // namespace tests {
-} // namespace internal {
-} // namespace mesos {


[4/9] mesos git commit: Removed an unused function in launch helper.

Posted by ji...@apache.org.
Removed an unused function in launch helper.

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


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

Branch: refs/heads/master
Commit: 0b6ac2f34dec26e80edc2c5590dd33d565b97d8f
Parents: d536a13
Author: Jie Yu <yu...@gmail.com>
Authored: Wed Nov 23 17:19:21 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 29 11:40:50 2016 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/launch.hpp | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0b6ac2f3/src/slave/containerizer/mesos/launch.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.hpp b/src/slave/containerizer/mesos/launch.hpp
index 155e3c5..d9729d7 100644
--- a/src/slave/containerizer/mesos/launch.hpp
+++ b/src/slave/containerizer/mesos/launch.hpp
@@ -57,8 +57,6 @@ public:
 #endif // __linux__
   };
 
-  static Flags prepare();
-
   MesosContainerizerLaunch() : Subcommand(NAME) {}
 
   Flags flags;


[6/9] mesos git commit: Fixed an indentation problem in Subcommand.

Posted by ji...@apache.org.
Fixed an indentation problem in Subcommand.

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


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

Branch: refs/heads/master
Commit: afab55a3579aa04ad81db00c52389255ae27fb48
Parents: 0b6ac2f
Author: Jie Yu <yu...@gmail.com>
Authored: Thu Nov 24 00:08:51 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 29 11:40:50 2016 -0800

----------------------------------------------------------------------
 3rdparty/stout/include/stout/subcommand.hpp | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/afab55a3/3rdparty/stout/include/stout/subcommand.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/subcommand.hpp b/3rdparty/stout/include/stout/subcommand.hpp
index 930e3c9..e2fc89c 100644
--- a/3rdparty/stout/include/stout/subcommand.hpp
+++ b/3rdparty/stout/include/stout/subcommand.hpp
@@ -97,10 +97,10 @@ private:
       const std::vector<Subcommand*>& subcommands);
 
   static int dispatch(
-    const Option<std::string>& prefix,
-    int argc,
-    char** argv,
-    const std::vector<Subcommand*>& subcommands);
+      const Option<std::string>& prefix,
+      int argc,
+      char** argv,
+      const std::vector<Subcommand*>& subcommands);
 
   // The name of this subcommand.
   std::string name_;


[3/9] mesos git commit: Added support for building environment variables from flags.

Posted by ji...@apache.org.
Added support for building environment variables from flags.

This patch added a method to FlagsBase to build a map of environment
variables from the flags. This simplifies the logic when the caller
wants to pass flags as environment variables to the subprocess.

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


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

Branch: refs/heads/master
Commit: c06c697693a5bc78fd210a10b0df1b2ab2780a18
Parents: afab55a
Author: Jie Yu <yu...@gmail.com>
Authored: Thu Nov 24 00:11:02 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 29 11:40:50 2016 -0800

----------------------------------------------------------------------
 3rdparty/stout/include/stout/flags/flags.hpp | 24 +++++++++++++++++++++++
 3rdparty/stout/tests/flags_tests.cpp         | 23 ++++++++++++++++++++++
 2 files changed, 47 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c06c6976/3rdparty/stout/include/stout/flags/flags.hpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/include/stout/flags/flags.hpp b/3rdparty/stout/include/stout/flags/flags.hpp
index 06aac13..ecadaf8 100644
--- a/3rdparty/stout/include/stout/flags/flags.hpp
+++ b/3rdparty/stout/include/stout/flags/flags.hpp
@@ -298,6 +298,10 @@ public:
   std::map<std::string, Option<std::string>> extract(
       const std::string& prefix) const;
 
+  // Build environment variables from the flags.
+  std::map<std::string, std::string> buildEnvironment(
+      const Option<std::string>& prefix = None()) const;
+
 protected:
   // The program's name, extracted from argv[0] by default;
   // declared 'protected' so that derived classes can alter this
@@ -538,6 +542,26 @@ inline std::map<std::string, Option<std::string>> FlagsBase::extract(
 }
 
 
+inline std::map<std::string, std::string> FlagsBase::buildEnvironment(
+    const Option<std::string>& prefix) const
+{
+  std::map<std::string, std::string> result;
+
+  foreachvalue (const Flag& flag, flags_) {
+    Option<std::string> value = flag.stringify(*this);
+    if (value.isSome()) {
+      const std::string key = prefix.isSome()
+        ? prefix.get() + strings::upper(flag.effective_name().value)
+        : strings::upper(flag.effective_name().value);
+
+      result[key] = value.get();
+    }
+  }
+
+  return result;
+}
+
+
 inline Try<Warnings> FlagsBase::load(const std::string& prefix)
 {
   return load(extract(prefix));

http://git-wip-us.apache.org/repos/asf/mesos/blob/c06c6976/3rdparty/stout/tests/flags_tests.cpp
----------------------------------------------------------------------
diff --git a/3rdparty/stout/tests/flags_tests.cpp b/3rdparty/stout/tests/flags_tests.cpp
index 9ee538b..e2681f8 100644
--- a/3rdparty/stout/tests/flags_tests.cpp
+++ b/3rdparty/stout/tests/flags_tests.cpp
@@ -890,6 +890,29 @@ TEST(FlagsTest, OptionalMessage)
 }
 
 
+TEST(FlagsTest, BuildEnvironment)
+{
+  TestFlagsBase flags;
+  flags.name4 = true;
+
+  map<string, string> environment = flags.buildEnvironment("PREFIX_");
+
+  ASSERT_EQ(1u, environment.count("PREFIX_NAME1"));
+  EXPECT_EQ("ben folds", environment["PREFIX_NAME1"]);
+
+  ASSERT_EQ(1u, environment.count("PREFIX_NAME2"));
+  EXPECT_EQ("42", environment["PREFIX_NAME2"]);
+
+  ASSERT_EQ(1u, environment.count("PREFIX_NAME3"));
+  EXPECT_EQ("false", environment["PREFIX_NAME3"]);
+
+  ASSERT_EQ(1u, environment.count("PREFIX_NAME4"));
+  EXPECT_EQ("true", environment["PREFIX_NAME4"]);
+
+  EXPECT_EQ(0u, environment.count("PREFIX_NAME5"));
+}
+
+
 TEST(FlagsTest, Duration)
 {
   class TestFlags : public virtual FlagsBase


[8/9] mesos git commit: Moved the default PATH setting to containerizer.

Posted by ji...@apache.org.
Moved the default PATH setting to containerizer.

Currently, it is set as part of the executor environment. This is not
ideal because isolators might choose to set environment variables for
the contianer. We should set the default PATH in the last step (right
before launch) if it does not exist.

Mesos containerizer launch helper already does default PATH setting
right before launching the container. This patch did the same in
Docker containerizer, and remove the default PATH setting when
generating the executor environment.

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


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

Branch: refs/heads/master
Commit: 93eacdb16d2622f862522e51a60ab1b91d1d66da
Parents: 418433d
Author: Jie Yu <yu...@gmail.com>
Authored: Sat Nov 26 22:44:49 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 29 11:40:50 2016 -0800

----------------------------------------------------------------------
 src/slave/containerizer/docker.cpp |  5 +++++
 src/slave/slave.cpp                | 10 ----------
 2 files changed, 5 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/93eacdb1/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 5ce4667..28b487a 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -1449,6 +1449,11 @@ Future<pid_t> DockerContainerizerProcess::launchExecutorProcess(
     environment["GLOG_v"] = glog.get();
   }
 
+  if (environment.count("PATH") == 0) {
+    environment["PATH"] =
+      "/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin";
+  }
+
   vector<string> argv;
   argv.push_back("mesos-docker-executor");
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/93eacdb1/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 1273b0f..7eb4503 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -6957,16 +6957,6 @@ map<string, string> executorEnvironment(
     }
   }
 
-#ifndef __WINDOWS__
-  // Include a default $PATH if there isn't.
-  // On Windows, we ensure the path is set by checking the system environment.
-  // See `createProcessEnvironment` in `process/windows/subprocess.hpp`.
-  if (environment.count("PATH") == 0) {
-    environment["PATH"] =
-      "/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin";
-  }
-#endif // __WINDOWS__
-
   // Set LIBPROCESS_PORT so that we bind to a random free port (since
   // this might have been set via --port option). We do this before
   // the environment variables below in case it is included.


[5/9] mesos git commit: Moved a VLOG to a proper place in MesosContainerizer.

Posted by ji...@apache.org.
Moved a VLOG to a proper place in MesosContainerizer.

We should print the VLOG after we calculate the launch flags.

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


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

Branch: refs/heads/master
Commit: 08ff8763e5e43761e08b1297b16eed6fef4a4fc7
Parents: c06c697
Author: Jie Yu <yu...@gmail.com>
Authored: Thu Nov 24 22:41:03 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 29 11:40:50 2016 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/08ff8763/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 9b33495..72d55e7 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1521,9 +1521,6 @@ Future<bool> MesosContainerizerProcess::_launch(
     launchFlags.runtime_directory = runtimePath;
 #endif // __WINDOWS__
 
-    VLOG(1) << "Launching '" << MESOS_CONTAINERIZER << "' with flags '"
-            << launchFlags << "'";
-
     // For now we need to special case entering a parent container's
     // mount namespace. We do this to ensure that we have access to
     // the binary we launch with `launcher->fork()`.
@@ -1555,6 +1552,9 @@ Future<bool> MesosContainerizerProcess::_launch(
     }
 #endif // __linux__
 
+    VLOG(1) << "Launching '" << MESOS_CONTAINERIZER << "' with flags '"
+            << launchFlags << "'";
+
     // Fork the child using launcher.
     vector<string> argv(2);
     argv[0] = MESOS_CONTAINERIZER;


[7/9] mesos git commit: Used environment to pass flags to launch helper.

Posted by ji...@apache.org.
Used environment to pass flags to launch helper.

Instead of doing that just for command environment variables, this
patch does it consistently by using environment variables to pass
launch helper flags. This will be more secure.

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


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

Branch: refs/heads/master
Commit: 9d6cbf0b32f1760a07fff0c4409fa815ba5d9634
Parents: 08ff876
Author: Jie Yu <yu...@gmail.com>
Authored: Thu Nov 24 23:53:45 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Nov 29 11:40:50 2016 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 29 ++++++++++++--------
 1 file changed, 17 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/9d6cbf0b/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 72d55e7..0e42ed7 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1428,17 +1428,7 @@ Future<bool> MesosContainerizerProcess::_launch(
     MesosContainerizerLaunch::Flags launchFlags;
 
     launchFlags.command = JSON::protobuf(launchCommand.get());
-
-    // The launch helper should inherit the agent's environment.
-    map<string, string> launchEnvironment = os::environment();
-
-    // Passing the command environment via an environment variable
-    // to the launch helper instead of a flag due to the sensitivity
-    // of environment variables. Otherwise the command environment
-    // would have been visible through commands like `ps` which are
-    // not protected from unprivileged users on the host.
-    launchEnvironment["MESOS_CONTAINERIZER_ENVIRONMENT"] =
-      stringify(environment);
+    launchFlags.environment = environment;
 
     if (rootfs.isNone()) {
       // NOTE: If the executor shares the host filesystem, we should
@@ -1555,6 +1545,21 @@ Future<bool> MesosContainerizerProcess::_launch(
     VLOG(1) << "Launching '" << MESOS_CONTAINERIZER << "' with flags '"
             << launchFlags << "'";
 
+    // Passing the launch flags via environment variables to the
+    // launch helper due to the sensitivity of those flags. Otherwise
+    // the launch flags would have been visible through commands like
+    // `ps` which are not protected from unprivileged users on the
+    // host.
+    map<string, string> launchFlagsEnvironment =
+      launchFlags.buildEnvironment("MESOS_CONTAINERIZER_");
+
+    // The launch helper should inherit the agent's environment.
+    map<string, string> launchEnvironment = os::environment();
+
+    launchEnvironment.insert(
+        launchFlagsEnvironment.begin(),
+        launchFlagsEnvironment.end());
+
     // Fork the child using launcher.
     vector<string> argv(2);
     argv[0] = MESOS_CONTAINERIZER;
@@ -1567,7 +1572,7 @@ Future<bool> MesosContainerizerProcess::_launch(
         Subprocess::IO(subprocessInfo.in),
         Subprocess::IO(subprocessInfo.out),
         Subprocess::IO(subprocessInfo.err),
-        &launchFlags,
+        nullptr,
         launchEnvironment,
         // 'enterNamespaces' will be ignored by PosixLauncher.
         _enterNamespaces,