You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2014/08/14 19:28:44 UTC

[1/2] git commit: Added explicit DockerInfo within ContainerInfo.

Repository: mesos
Updated Branches:
  refs/heads/master f105ed772 -> 5165a4a54


Added explicit DockerInfo within ContainerInfo.

Added new DockerInfo to explicitly capture Docker options, and allow
command URIs to be fetched and mapped into sandbox, which gets
bind-mounted into the container.

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


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

Branch: refs/heads/master
Commit: 2057e3fa37f880b52d766feb5ed33a0209f218bc
Parents: f105ed7
Author: Timothy Chen <tn...@apache.org>
Authored: Thu Aug 14 09:58:11 2014 -0700
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Aug 14 09:58:11 2014 -0700

----------------------------------------------------------------------
 include/mesos/mesos.proto                       |  14 +
 src/docker/docker.cpp                           |  73 ++++-
 src/docker/docker.hpp                           |   6 +-
 src/slave/containerizer/containerizer.cpp       |  36 +++
 src/slave/containerizer/containerizer.hpp       |   8 +
 src/slave/containerizer/docker.cpp              | 308 ++++++++++++++-----
 src/slave/containerizer/mesos/containerizer.cpp |  35 ---
 src/slave/flags.hpp                             |   7 +
 src/tests/docker_containerizer_tests.cpp        |  80 ++++-
 src/tests/docker_tests.cpp                      |  33 +-
 10 files changed, 469 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index adc8fab..dea51f9 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -263,6 +263,10 @@ message ExecutorInfo {
   required ExecutorID executor_id = 1;
   optional FrameworkID framework_id = 8; // TODO(benh): Make this required.
   required CommandInfo command = 7;
+  // Executor provided with a container will launch the container
+  // with the executor's CommandInfo and we expect the container to
+  // act as a Mesos executor.
+  optional ContainerInfo container = 11;
   repeated Resource resources = 5;
   optional string name = 9;
 
@@ -578,6 +582,9 @@ message TaskInfo {
   repeated Resource resources = 4;
   optional ExecutorInfo executor = 5;
   optional CommandInfo command = 7;
+  // Task provided with a container will launch the container as part
+  // of this task paired with the task's CommandInfo.
+  optional ContainerInfo container = 9;
   optional bytes data = 6;
   // A health check for the task (currently in *alpha* and initial
   // support will only be for TaskInfo's that have a CommandInfo).
@@ -845,6 +852,13 @@ message ContainerInfo {
     DOCKER = 1;
   }
 
+  message DockerInfo {
+    // The docker image that is going to be passed to the registry.
+    required string image = 1;
+  }
+
   required Type type = 1;
   repeated Volume volumes = 2;
+
+  optional DockerInfo docker = 3;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/src/docker/docker.cpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.cpp b/src/docker/docker.cpp
index 1cba381..71dbb13 100644
--- a/src/docker/docker.cpp
+++ b/src/docker/docker.cpp
@@ -39,6 +39,8 @@
 #include "slave/containerizer/isolators/cgroups/cpushare.hpp"
 #include "slave/containerizer/isolators/cgroups/mem.hpp"
 
+using namespace mesos;
+
 using namespace mesos::internal::slave;
 
 using namespace process;
@@ -209,12 +211,20 @@ Try<Docker::Container> Docker::Container::create(const JSON::Object& json)
 
 
 Future<Nothing> Docker::run(
-    const string& image,
-    const string& command,
+    const ContainerInfo& containerInfo,
+    const CommandInfo& commandInfo,
     const string& name,
-    const Option<mesos::Resources>& resources,
+    const string& sandboxDirectory,
+    const string& mappedDirectory,
+    const Option<Resources>& resources,
     const Option<map<string, string> >& env) const
 {
+  if (!containerInfo.has_docker()) {
+    return Failure("No docker info found in container info");
+  }
+
+  const ContainerInfo::DockerInfo& dockerInfo = containerInfo.docker();
+
   string cmd = path + " run -d";
 
   if (resources.isSome()) {
@@ -243,15 +253,66 @@ Future<Nothing> Docker::run(
     }
   }
 
-  cmd += " --net=host --name=" + name + " " + image + " " + command;
+  foreach (const Environment::Variable& variable,
+           commandInfo.environment().variables()) {
+    // TODO(tnachen): Use subprocess with args instead once we can
+    // handle splitting command string into args.
+    string key = strings::replace(variable.name(), "\"", "\\\"");
+    string value = strings::replace(variable.value(), "\"", "\\\"");
+    cmd += " -e \"" + key + "=" + value + "\"";
+  }
+
+  cmd += " -e \"MESOS_SANDBOX=" + mappedDirectory + "\"";
+
+  foreach (const Volume& volume, containerInfo.volumes()) {
+    string volumeConfig = volume.container_path();
+    if (volume.has_host_path()) {
+      volumeConfig = volume.host_path() + ":" + volumeConfig;
+      if (volume.has_mode()) {
+        switch (volume.mode()) {
+          case Volume::RW: volumeConfig += ":rw"; break;
+          case Volume::RO: volumeConfig += ":ro"; break;
+          default: return Failure("Unsupported volume mode");
+        }
+      }
+    } else if (volume.has_mode() && !volume.has_host_path()) {
+      return Failure("Host path is required with mode");
+    }
+
+    cmd += " -v=" + volumeConfig;
+  }
+
+  // Mapping sandbox directory into the contianer mapped directory.
+  cmd += " -v=" + sandboxDirectory + ":" + mappedDirectory;
+
+  const string& image = dockerInfo.image();
+
+  // TODO(tnachen): Support more network options other than host
+  // networking that docker provides (ie: BRIDGE). We currently
+  // require host networking since if the docker container is
+  // expected to be an executor it needs to be able to communicate
+  // with the slave by the slave's PID. There can be more future work
+  // to allow a bridge to connect but this is not yet implemented.
+  cmd += " --net=host --name=" + name + " " + image + " " +
+         commandInfo.value();
 
   VLOG(1) << "Running " << cmd;
 
+  map<string, string> environment;
+
+  // Currently the Docker CLI picks up dockerconfig by looking for
+  // the config file in the $HOME directory. If one of the URIs
+  // provided is a docker config file we want docker to be able to
+  // pick it up from the sandbox directory where we store all the
+  // URI downloads.
+  environment["HOME"] = sandboxDirectory;
+
   Try<Subprocess> s = subprocess(
       cmd,
       Subprocess::PATH("/dev/null"),
-      Subprocess::PATH("/dev/null"),
-      Subprocess::PIPE());
+      Subprocess::PIPE(),
+      Subprocess::PIPE(),
+      environment);
 
   if (s.isError()) {
     return Failure(s.error());

http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/src/docker/docker.hpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.hpp b/src/docker/docker.hpp
index 98b2d60..3270c91 100644
--- a/src/docker/docker.hpp
+++ b/src/docker/docker.hpp
@@ -65,9 +65,11 @@ public:
 
   // Performs 'docker run IMAGE'.
   process::Future<Nothing> run(
-      const std::string& image,
-      const std::string& command,
+      const mesos::ContainerInfo& containerInfo,
+      const mesos::CommandInfo& commandInfo,
       const std::string& name,
+      const std::string& sandboxDirectory,
+      const std::string& mappedDirectory,
       const Option<mesos::Resources>& resources = None(),
       const Option<std::map<std::string, std::string> >& env = None()) const;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index c91ba38..0a262ee 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -269,6 +269,42 @@ map<string, string> executorEnvironment(
   return env;
 }
 
+
+// Helper method to build the environment map used to launch fetcher.
+map<string, string> fetcherEnvironment(
+    const CommandInfo& commandInfo,
+    const std::string& directory,
+    const Option<std::string>& user,
+    const Flags& flags)
+{
+  // Prepare the environment variables to pass to mesos-fetcher.
+  string uris = "";
+  foreach (const CommandInfo::URI& uri, commandInfo.uris()) {
+    uris += uri.value() + "+" +
+    (uri.has_executable() && uri.executable() ? "1" : "0") +
+    (uri.extract() ? "X" : "N");
+    uris += " ";
+  }
+  // Remove extra space at the end.
+  uris = strings::trim(uris);
+
+  map<string, string> environment;
+  environment["MESOS_EXECUTOR_URIS"] = uris;
+  environment["MESOS_WORK_DIRECTORY"] = directory;
+  if (user.isSome()) {
+    environment["MESOS_USER"] = user.get();
+  }
+  if (!flags.frameworks_home.empty()) {
+    environment["MESOS_FRAMEWORKS_HOME"] = flags.frameworks_home;
+  }
+  if (!flags.hadoop_home.empty()) {
+    environment["HADOOP_HOME"] = flags.hadoop_home;
+  }
+
+  return environment;
+}
+
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/src/slave/containerizer/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.hpp b/src/slave/containerizer/containerizer.hpp
index 02754cd..8a66412 100644
--- a/src/slave/containerizer/containerizer.hpp
+++ b/src/slave/containerizer/containerizer.hpp
@@ -134,6 +134,14 @@ std::map<std::string, std::string> executorEnvironment(
     bool checkpoint,
     const Duration& recoveryTimeout);
 
+
+std::map<std::string, std::string> fetcherEnvironment(
+    const CommandInfo& commandInfo,
+    const std::string& directory,
+    const Option<std::string>& user,
+    const Flags& flags);
+
+
 } // namespace slave {
 } // namespace internal {
 } // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/src/slave/containerizer/docker.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/docker.cpp b/src/slave/containerizer/docker.cpp
index 904cdd3..c18023c 100644
--- a/src/slave/containerizer/docker.cpp
+++ b/src/slave/containerizer/docker.cpp
@@ -21,6 +21,7 @@
 #include <string>
 
 #include <process/defer.hpp>
+#include <process/io.hpp>
 #include <process/reap.hpp>
 #include <process/subprocess.hpp>
 
@@ -28,6 +29,8 @@
 #include <stout/hashset.hpp>
 #include <stout/os.hpp>
 
+#include "common/status_utils.hpp"
+
 #include "docker/docker.hpp"
 
 #ifdef __linux__
@@ -117,6 +120,19 @@ public:
   virtual process::Future<hashset<ContainerID> > containers();
 
 private:
+  process::Future<Nothing> fetch(
+      const ContainerID& containerId,
+      const CommandInfo& commandInfo,
+      const std::string& directory);
+
+  process::Future<Nothing> _fetch(
+      const ContainerID& containerId,
+      const Option<int>& status);
+
+  process::Future<Nothing> _fetchFailed(
+      const ContainerID& containerId,
+      const std::string& failure);
+
   // Continuations and helpers.
   process::Future<Nothing> _recover(
       const std::list<Docker::Container>& containers);
@@ -133,12 +149,29 @@ private:
   process::Future<bool> _launch(
       const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
+      const string& directory,
       const SlaveID& slaveId,
       const PID<Slave>& slavePid,
       bool checkpoint);
 
   process::Future<bool> __launch(
       const ContainerID& containerId,
+      const TaskInfo& taskInfo,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const SlaveID& slaveId,
+      const PID<Slave>& slavePid,
+      bool checkpoint);
+
+  process::Future<bool> __launch(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const SlaveID& slaveId,
+      const PID<Slave>& slavePid,
+      bool checkpoint);
+
+  process::Future<bool> ___launch(
+      const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
       const SlaveID& slaveId,
       const PID<Slave>& slavePid,
@@ -195,8 +228,7 @@ private:
 
 // Parse the ContainerID from a Docker container and return None if
 // the container was not launched from Mesos.
-Option<ContainerID> parse(
-    const Docker::Container& container)
+Option<ContainerID> parse(const Docker::Container& container)
 {
   Option<string> name = None();
 
@@ -218,8 +250,7 @@ Option<ContainerID> parse(
 }
 
 
-Try<DockerContainerizer*> DockerContainerizer::create(
-    const Flags& flags)
+Try<DockerContainerizer*> DockerContainerizer::create(const Flags& flags)
 {
   Try<Docker> docker = Docker::create(flags.docker);
   if (docker.isError()) {
@@ -247,6 +278,85 @@ DockerContainerizer::~DockerContainerizer()
 }
 
 
+Future<Nothing> DockerContainerizerProcess::fetch(
+    const ContainerID& containerId,
+    const CommandInfo& commandInfo,
+    const string& directory)
+{
+  if (commandInfo.uris().size() == 0) {
+    return Nothing();
+  }
+
+  Result<string> realpath = os::realpath(
+      path::join(flags.launcher_dir, "mesos-fetcher"));
+  if (!realpath.isSome()) {
+    LOG(ERROR) << "Failed to determine the canonical path "
+               << "for the mesos-fetcher '"
+               << path::join(flags.launcher_dir, "mesos-fetcher")
+               << "': "
+               << (realpath.isError() ? realpath.error() :
+                   "No such file or directory");
+    return Failure("Could not fetch URIs: failed to find mesos-fetcher");
+  }
+
+  map<string, string> fetcherEnv = fetcherEnvironment(
+      commandInfo,
+      directory,
+      None(),
+      flags);
+
+  VLOG(1) << "Starting to fetch URIs for container: " << containerId
+          << ", directory: " << directory;
+
+  Try<Subprocess> fetcher = subprocess(
+      realpath.get(),
+      Subprocess::PATH("/dev/null"),
+      Subprocess::PATH(path::join(directory, "stdout")),
+      Subprocess::PATH(path::join(directory, "stderr")),
+      fetcherEnv);
+
+  if (fetcher.isError()) {
+    return Failure("Failed to execute mesos-fetcher: " + fetcher.error());
+  }
+
+  const Future<Option<int> >& status = fetcher.get().status();
+
+  return status
+    .onFailed(defer(self(), &Self::_fetchFailed, containerId, lambda::_1))
+    .then(defer(self(), &Self::_fetch, containerId, lambda::_1));
+}
+
+
+Future<Nothing> DockerContainerizerProcess::_fetchFailed(
+    const ContainerID& containerId,
+    const string& failure)
+{
+  containerizer::Termination termination;
+  termination.set_message("Fetch failed for container: " + failure);
+  promises[containerId]->set(termination);
+  promises.erase(containerId);
+  return Nothing();
+}
+
+
+Future<Nothing> DockerContainerizerProcess::_fetch(
+    const ContainerID& containerId,
+    const Option<int>& status)
+{
+  if (!status.isSome()) {
+    return _fetchFailed(
+        containerId,
+        "No status available from fetch");
+  } else if (status.get() != 0) {
+    return _fetchFailed(
+        containerId,
+        "Fetch failed with status " + WSTRINGIFY(status.get()));
+  }
+
+  return Nothing();
+}
+
+
 Future<Nothing> DockerContainerizer::recover(
     const Option<SlaveState>& state)
 {
@@ -509,63 +619,36 @@ Future<bool> DockerContainerizerProcess::launch(
     return Failure("Container already started");
   }
 
-  CommandInfo command = executorInfo.command();
-
-  if (!command.has_container()) {
+  if (!executorInfo.has_container()) {
     LOG(INFO) << "No container info found, skipping launch";
     return false;
   }
 
-  string image = command.container().image();
+  ContainerInfo containerInfo = executorInfo.container();
 
-  // Check if we should try and launch this command.
-  if (!strings::startsWith(image, "docker:///")) {
-    LOG(INFO) << "No docker image found, skipping launch";
+  if (containerInfo.type() != ContainerInfo::DOCKER) {
+    LOG(INFO) << "Skipping non-docker container";
     return false;
   }
 
-  Owned<Promise<containerizer::Termination> > promise(
-    new Promise<containerizer::Termination>());
-
-  promises.put(containerId, promise);
-
   LOG(INFO) << "Starting container '" << containerId
             << "' for executor '" << executorInfo.executor_id()
             << "' and framework '" << executorInfo.framework_id() << "'";
 
-  // Extract the Docker image.
-  image = strings::remove(image, "docker:///", strings::PREFIX);
-
-  // Construct the Docker container name.
-  string name = containerName(containerId);
-
-  map<string, string> env = executorEnvironment(
-      executorInfo,
-      directory,
-      slaveId,
-      slavePid,
-      checkpoint,
-      flags.recovery_timeout);
-
-  // Include any environment variables from CommandInfo.
-  foreach (const Environment::Variable& variable,
-           command.environment().variables()) {
-    env[variable.name()] = variable.value();
-  }
+  Owned<Promise<containerizer::Termination> > promise(
+    new Promise<containerizer::Termination>());
 
-  Resources resources = executorInfo.resources();
+  promises.put(containerId, promise);
 
-  // Start a docker container then launch the executor (but destroy
-  // the Docker container if launching the executor failed).
-  return docker.run(image, command.value(), name, resources, env)
+  return fetch(containerId, executorInfo.command(), directory)
     .then(defer(self(),
-               &Self::_launch,
-               containerId,
-               executorInfo,
-               slaveId,
-               slavePid,
-               checkpoint))
-    .onFailed(defer(self(), &Self::destroy, containerId, false));
+                &Self::_launch,
+                containerId,
+                executorInfo,
+                directory,
+                slaveId,
+                slavePid,
+                checkpoint));
 }
 
 
@@ -583,46 +666,78 @@ Future<bool> DockerContainerizerProcess::launch(
     return Failure("Container already started");
   }
 
-  if (!taskInfo.has_command()) {
-    LOG(WARNING) << "Not expecting call without command info";
+  if (!taskInfo.has_container()) {
+    LOG(INFO) << "No container info found, skipping launch";
     return false;
   }
 
-  const CommandInfo& command = taskInfo.command();
+  ContainerInfo containerInfo = executorInfo.container();
 
-  // Check if we should try and launch this command.
-  if (!command.has_container() ||
-      !strings::startsWith(command.container().image(), "docker:///")) {
-    LOG(INFO) << "No container info or container image is not docker image, "
-              << "skipping launch";
+  if (containerInfo.type() != ContainerInfo::DOCKER) {
+    LOG(INFO) << "Skipping non-docker container";
     return false;
   }
 
+  LOG(INFO) << "Starting container '" << containerId
+            << "' for task '" << taskInfo.task_id()
+            << "' (and executor '" << executorInfo.executor_id()
+            << "') of framework '" << executorInfo.framework_id() << "'";
+
   Owned<Promise<containerizer::Termination> > promise(
-      new Promise<containerizer::Termination>());
+    new Promise<containerizer::Termination>());
 
   promises.put(containerId, promise);
 
+  const CommandInfo& command = taskInfo.command();
+
+  return fetch(containerId, command, directory)
+    .then(defer(self(),
+                &Self::_launch,
+                containerId,
+                taskInfo,
+                executorInfo,
+                directory,
+                slaveId,
+                slavePid,
+                checkpoint));
+}
+
+
+Future<bool> DockerContainerizerProcess::_launch(
+    const ContainerID& containerId,
+    const TaskInfo& taskInfo,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const SlaveID& slaveId,
+    const PID<Slave>& slavePid,
+    bool checkpoint)
+{
   // Store the resources for usage().
   resources.put(containerId, taskInfo.resources());
 
-  LOG(INFO) << "Starting container '" << containerId
-            << "' for task '" << taskInfo.task_id()
-            << "' (and executor '" << executorInfo.executor_id()
-            << "') of framework '" << executorInfo.framework_id() << "'";
+  const ContainerInfo& container = taskInfo.container();
 
   // Extract the Docker image.
-  string image = command.container().image();
-  image = strings::remove(image, "docker:///", strings::PREFIX);
+  string image = container.docker().image();
 
   // Construct the Docker container name.
   string name = containerName(containerId);
 
-  // Start a docker container then launch the executor (but destroy
-  // the Docker container if launching the executor failed).
-  return docker.run(image, command.value(), name, taskInfo.resources())
+  const CommandInfo& command = taskInfo.command();
+
+  // Start a docker container and docker logs then launch the executor
+  // (but destroy the Docker container if launching the executor
+  // failed). Docker logs will automatically exit once the container
+  // is stopped.
+  return docker.run(
+      container,
+      command,
+      name,
+      directory,
+      flags.docker_sandbox_directory,
+      taskInfo.resources())
     .then(defer(self(),
-                &Self::_launch,
+                &Self::__launch,
                 containerId,
                 taskInfo,
                 executorInfo,
@@ -634,7 +749,7 @@ Future<bool> DockerContainerizerProcess::launch(
 }
 
 
-Future<bool> DockerContainerizerProcess::_launch(
+Future<bool> DockerContainerizerProcess::__launch(
     const ContainerID& containerId,
     const TaskInfo& taskInfo,
     const ExecutorInfo& executorInfo,
@@ -652,7 +767,7 @@ Future<bool> DockerContainerizerProcess::_launch(
       checkpoint,
       flags.recovery_timeout);
 
-  // Include any enviroment variables from CommandInfo.
+  // Include any enviroment variables from ExecutorInfo.
   foreach (const Environment::Variable& variable,
            executorInfo.command().environment().variables()) {
     env[variable.name()] = variable.value();
@@ -733,23 +848,78 @@ Future<bool> DockerContainerizerProcess::_launch(
 Future<bool> DockerContainerizerProcess::_launch(
     const ContainerID& containerId,
     const ExecutorInfo& executorInfo,
+    const string& directory,
     const SlaveID& slaveId,
     const PID<Slave>& slavePid,
     bool checkpoint)
 {
-  return docker.inspect(containerName(containerId))
+  Owned<Promise<containerizer::Termination> > promise(
+      new Promise<containerizer::Termination>());
+
+  promises.put(containerId, promise);
+
+  // Construct the Docker container name.
+  string name = containerName(containerId);
+
+  map<string, string> env = executorEnvironment(
+      executorInfo,
+      directory,
+      slaveId,
+      slavePid,
+      checkpoint,
+      flags.recovery_timeout);
+
+  // Include any environment variables from CommandInfo.
+  foreach (const Environment::Variable& variable,
+           executorInfo.command().environment().variables()) {
+    env[variable.name()] = variable.value();
+  }
+
+  const ContainerInfo& container = executorInfo.container();
+  const CommandInfo& command = executorInfo.command();
+
+  // Start a docker container which is an executor and docker logs.
+  // Docker logs will automatically exit once the container is
+  // stopped.
+  return docker.run(
+      container,
+      command,
+      name,
+      directory,
+      flags.docker_sandbox_directory,
+      None(),
+      env)
     .then(defer(self(),
                 &Self::__launch,
                 containerId,
                 executorInfo,
                 slaveId,
                 slavePid,
+                checkpoint))
+    .onFailed(defer(self(), &Self::destroy, containerId, false));
+}
+
+
+Future<bool> DockerContainerizerProcess::__launch(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const SlaveID& slaveId,
+    const PID<Slave>& slavePid,
+    bool checkpoint)
+{
+  return docker.inspect(containerName(containerId))
+     .then(defer(self(),
+                &Self::___launch,
+                containerId,
+                executorInfo,
+                slaveId,
+                slavePid,
                 checkpoint,
                 lambda::_1));
 }
 
 
-Future<bool> DockerContainerizerProcess::__launch(
+Future<bool> DockerContainerizerProcess::___launch(
     const ContainerID& containerId,
     const ExecutorInfo& executorInfo,
     const SlaveID& slaveId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 694c9d1..d0676c5 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -67,41 +67,6 @@ using state::RunState;
 Future<Nothing> _nothing() { return Nothing(); }
 
 
-// Helper method to build the environment map used to launch fetcher.
-map<string, string> fetcherEnvironment(
-    const CommandInfo& commandInfo,
-    const std::string& directory,
-    const Option<std::string>& user,
-    const Flags& flags)
-{
-  // Prepare the environment variables to pass to mesos-fetcher.
-  string uris = "";
-  foreach (const CommandInfo::URI& uri, commandInfo.uris()) {
-    uris += uri.value() + "+" +
-            (uri.has_executable() && uri.executable() ? "1" : "0") +
-            (uri.extract() ? "X" : "N");
-    uris += " ";
-  }
-  // Remove extra space at the end.
-  uris = strings::trim(uris);
-
-  map<string, string> environment;
-  environment["MESOS_EXECUTOR_URIS"] = uris;
-  environment["MESOS_WORK_DIRECTORY"] = directory;
-  if (user.isSome()) {
-    environment["MESOS_USER"] = user.get();
-  }
-  if (!flags.frameworks_home.empty()) {
-    environment["MESOS_FRAMEWORKS_HOME"] = flags.frameworks_home;
-  }
-  if (!flags.hadoop_home.empty()) {
-    environment["HADOOP_HOME"] = flags.hadoop_home;
-  }
-
-  return environment;
-}
-
-
 Try<MesosContainerizer*> MesosContainerizer::create(
     const Flags& flags,
     bool local)

http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 1e36c51..44ad632 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -289,6 +289,12 @@ public:
         "The absolute path to the docker executable for docker containerizer.",
         "docker");
 
+    add(&Flags::docker_sandbox_directory,
+        "docker_sandbox_directory",
+        "The absolute path for the directory in the container where the sandbox\n"
+        "is mapped to",
+        "/mnt/mesos/sandbox");
+
 #ifdef WITH_NETWORK_ISOLATOR
     add(&Flags::ephemeral_ports_per_container,
         "ephemeral_ports_per_container",
@@ -347,6 +353,7 @@ public:
   std::string containerizers;
   Option<std::string> default_container_image;
   std::string docker;
+  std::string docker_sandbox_directory;
 #ifdef WITH_NETWORK_ISOLATOR
   uint16_t ephemeral_ports_per_container;
   Option<std::string> eth0_name;

http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/src/tests/docker_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/docker_containerizer_tests.cpp b/src/tests/docker_containerizer_tests.cpp
index a559836..3099063 100644
--- a/src/tests/docker_containerizer_tests.cpp
+++ b/src/tests/docker_containerizer_tests.cpp
@@ -226,11 +226,20 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch_Executor)
   ExecutorID executorId;
   executorId.set_value("e1");
   executorInfo.mutable_executor_id()->CopyFrom(executorId);
+
   CommandInfo command;
   command.set_value("test-executor");
-  command.mutable_container()->set_image("docker:///mesosphere/test-executor");
   executorInfo.mutable_command()->CopyFrom(command);
 
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("mesosphere/test-executor");
+
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
+  executorInfo.mutable_container()->CopyFrom(containerInfo);
+
   task.mutable_executor()->CopyFrom(executorInfo);
 
   vector<TaskInfo> tasks;
@@ -324,11 +333,17 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Launch)
   task.mutable_resources()->CopyFrom(offer.resources());
 
   CommandInfo command;
-  CommandInfo::ContainerInfo* containerInfo = command.mutable_container();
-  containerInfo->set_image("docker:///busybox");
-  command.set_value("sleep 120");
+  command.set_value("sleep 1000");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("busybox");
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
 
   task.mutable_command()->CopyFrom(command);
+  task.mutable_container()->CopyFrom(containerInfo);
 
   vector<TaskInfo> tasks;
   tasks.push_back(task);
@@ -411,11 +426,17 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Kill)
   task.mutable_resources()->CopyFrom(offer.resources());
 
   CommandInfo command;
-  CommandInfo::ContainerInfo* containerInfo = command.mutable_container();
-  containerInfo->set_image("docker:///busybox");
-  command.set_value("sleep 120");
+  command.set_value("sleep 1000");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("busybox");
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
 
   task.mutable_command()->CopyFrom(command);
+  task.mutable_container()->CopyFrom(containerInfo);
 
   vector<TaskInfo> tasks;
   tasks.push_back(task);
@@ -509,13 +530,18 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Usage)
   task.mutable_resources()->CopyFrom(offer.resources());
 
   CommandInfo command;
-  CommandInfo::ContainerInfo* containerInfo = command.mutable_container();
-  containerInfo->set_image("docker:///busybox");
-
   // Run a CPU intensive command, so we can measure utime and stime later.
   command.set_value("dd if=/dev/zero of=/dev/null");
 
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("busybox");
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
+
   task.mutable_command()->CopyFrom(command);
+  task.mutable_container()->CopyFrom(containerInfo);
 
   vector<TaskInfo> tasks;
   tasks.push_back(task);
@@ -628,11 +654,17 @@ TEST_F(DockerContainerizerTest, ROOT_DOCKER_Update)
   task.mutable_resources()->CopyFrom(offer.resources());
 
   CommandInfo command;
-  CommandInfo::ContainerInfo* containerInfo = command.mutable_container();
-  containerInfo->set_image("docker:///busybox");
-  command.set_value("sleep 180");
+  command.set_value("sleep 1000");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("busybox");
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
 
   task.mutable_command()->CopyFrom(command);
+  task.mutable_container()->CopyFrom(containerInfo);
 
   vector<TaskInfo> tasks;
   tasks.push_back(task);
@@ -736,18 +768,32 @@ TEST_F(DockerContainerizerTest, DISABLED_ROOT_DOCKER_Recover)
 
   Resources resources = Resources::parse("cpus:1;mem:512").get();
 
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("busybox");
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
+
+  CommandInfo commandInfo;
+  commandInfo.set_value("sleep 1000");
+
   Future<Nothing> d1 =
     docker.run(
-        "busybox",
-        "sleep 360",
+        containerInfo,
+        commandInfo,
         slave::DOCKER_NAME_PREFIX + stringify(containerId),
+        flags.work_dir,
+        flags.docker_sandbox_directory,
         resources);
 
   Future<Nothing> d2 =
     docker.run(
-        "busybox",
-        "sleep 360",
+        containerInfo,
+        commandInfo,
         slave::DOCKER_NAME_PREFIX + stringify(reapedContainerId),
+        flags.work_dir,
+        flags.docker_sandbox_directory,
         resources);
 
   AWAIT_READY(d1);

http://git-wip-us.apache.org/repos/asf/mesos/blob/2057e3fa/src/tests/docker_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/docker_tests.cpp b/src/tests/docker_tests.cpp
index 4ef1df4..3d02702 100644
--- a/src/tests/docker_tests.cpp
+++ b/src/tests/docker_tests.cpp
@@ -28,10 +28,12 @@
 
 #include "mesos/resources.hpp"
 
+#include "tests/environment.hpp"
 #include "tests/flags.hpp"
 
 using namespace mesos;
 using namespace mesos::internal;
+using namespace mesos::internal::tests;
 
 using namespace process;
 
@@ -57,8 +59,28 @@ TEST(DockerTest, ROOT_DOCKER_interface)
     EXPECT_NE("/" + containerName, container.name);
   }
 
+  Try<string> directory = environment->mkdtemp();
+  CHECK_SOME(directory) << "Failed to create temporary directory";
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::DOCKER);
+
+  ContainerInfo::DockerInfo dockerInfo;
+  dockerInfo.set_image("busybox");
+  containerInfo.mutable_docker()->CopyFrom(dockerInfo);
+
+  CommandInfo commandInfo;
+  commandInfo.set_value("sleep 120");
+
   // Start the container.
-  status = docker.run("busybox", "sleep 120", containerName, resources);
+  status = docker.run(
+      containerInfo,
+      commandInfo,
+      containerName,
+      directory.get(),
+      "/mnt/mesos/sandbox",
+      resources);
+
   AWAIT_READY(status);
 
   // Should be able to see the container now.
@@ -132,7 +154,14 @@ TEST(DockerTest, ROOT_DOCKER_interface)
 
   // Start the container again, this time we will do a "rm -f"
   // directly, instead of killing and rm.
-  status = docker.run("busybox", "sleep 120", containerName, resources);
+  status = docker.run(
+      containerInfo,
+      commandInfo,
+      containerName,
+      directory.get(),
+      "/mnt/mesos/sandbox",
+      resources);
+
   AWAIT_READY(status);
 
   // Verify that the container is there.


[2/2] git commit: Used new CommandInfo with Docker::run.

Posted by be...@apache.org.
Used new CommandInfo with Docker::run.

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


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

Branch: refs/heads/master
Commit: 5165a4a54ae5f45d79af40336c056ab0dcbdd8ec
Parents: 2057e3f
Author: Benjamin Hindman <be...@gmail.com>
Authored: Thu Aug 14 10:14:04 2014 -0700
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Thu Aug 14 10:14:04 2014 -0700

----------------------------------------------------------------------
 src/docker/docker.cpp | 64 +++++++++++++++++++++++++++++++---------------
 1 file changed, 44 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5165a4a5/src/docker/docker.cpp
----------------------------------------------------------------------
diff --git a/src/docker/docker.cpp b/src/docker/docker.cpp
index 71dbb13..b6b8aab 100644
--- a/src/docker/docker.cpp
+++ b/src/docker/docker.cpp
@@ -51,7 +51,7 @@ using std::string;
 using std::vector;
 
 
-template<class T>
+template <typename T>
 static Future<T> failure(
     const string& cmd,
     int status,
@@ -225,7 +225,10 @@ Future<Nothing> Docker::run(
 
   const ContainerInfo::DockerInfo& dockerInfo = containerInfo.docker();
 
-  string cmd = path + " run -d";
+  vector<string> argv;
+  argv.push_back(path);
+  argv.push_back("run");
+  argv.push_back("-d");
 
   if (resources.isSome()) {
     // TODO(yifan): Support other resources (e.g. disk, ports).
@@ -233,36 +236,33 @@ Future<Nothing> Docker::run(
     if (cpus.isSome()) {
       uint64_t cpuShare =
         std::max((uint64_t) (CPU_SHARES_PER_CPU * cpus.get()), MIN_CPU_SHARES);
-      cmd += " -c " + stringify(cpuShare);
+      argv.push_back("-c");
+      argv.push_back(stringify(cpuShare));
     }
 
     Option<Bytes> mem = resources.get().mem();
     if (mem.isSome()) {
       Bytes memLimit = std::max(mem.get(), MIN_MEMORY);
-      cmd += " -m " + stringify(memLimit.bytes());
+      argv.push_back("-m");
+      argv.push_back(stringify(memLimit.bytes()));
     }
   }
 
   if (env.isSome()) {
-    // TODO(tnachen): Use subprocess with args instead once we can
-    // handle splitting command string into args.
     foreachpair (string key, string value, env.get()) {
-      key = strings::replace(key, "\"", "\\\"");
-      value = strings::replace(value, "\"", "\\\"");
-      cmd += " -e \"" + key + "=" + value + "\"";
+      argv.push_back("-e");
+      argv.push_back(key + "=" + value);
     }
   }
 
   foreach (const Environment::Variable& variable,
            commandInfo.environment().variables()) {
-    // TODO(tnachen): Use subprocess with args instead once we can
-    // handle splitting command string into args.
-    string key = strings::replace(variable.name(), "\"", "\\\"");
-    string value = strings::replace(variable.value(), "\"", "\\\"");
-    cmd += " -e \"" + key + "=" + value + "\"";
+    argv.push_back("-e");
+    argv.push_back(variable.name() + "=" + variable.value());
   }
 
-  cmd += " -e \"MESOS_SANDBOX=" + mappedDirectory + "\"";
+  argv.push_back("-e");
+  argv.push_back("MESOS_SANDBOX=" + mappedDirectory);
 
   foreach (const Volume& volume, containerInfo.volumes()) {
     string volumeConfig = volume.container_path();
@@ -279,11 +279,13 @@ Future<Nothing> Docker::run(
       return Failure("Host path is required with mode");
     }
 
-    cmd += " -v=" + volumeConfig;
+    argv.push_back("-v");
+    argv.push_back(volumeConfig);
   }
 
   // Mapping sandbox directory into the contianer mapped directory.
-  cmd += " -v=" + sandboxDirectory + ":" + mappedDirectory;
+  argv.push_back("-v");
+  argv.push_back(sandboxDirectory + ":" + mappedDirectory);
 
   const string& image = dockerInfo.image();
 
@@ -293,8 +295,28 @@ Future<Nothing> Docker::run(
   // expected to be an executor it needs to be able to communicate
   // with the slave by the slave's PID. There can be more future work
   // to allow a bridge to connect but this is not yet implemented.
-  cmd += " --net=host --name=" + name + " " + image + " " +
-         commandInfo.value();
+  argv.push_back("--net");
+  argv.push_back("host");
+
+  argv.push_back("--name");
+  argv.push_back(name);
+  argv.push_back(image);
+
+  if (commandInfo.shell()) {
+    argv.push_back("/bin/sh");
+    argv.push_back("-c");
+    argv.push_back(commandInfo.value());
+  } else {
+    if (commandInfo.has_value()) {
+      argv.push_back(commandInfo.value());
+    }
+
+    foreach (const string& argument, commandInfo.arguments()) {
+      argv.push_back(argument);
+    }
+  }
+
+  string cmd = strings::join(" ", argv);
 
   VLOG(1) << "Running " << cmd;
 
@@ -308,10 +330,12 @@ Future<Nothing> Docker::run(
   environment["HOME"] = sandboxDirectory;
 
   Try<Subprocess> s = subprocess(
-      cmd,
+      path,
+      argv,
       Subprocess::PATH("/dev/null"),
       Subprocess::PIPE(),
       Subprocess::PIPE(),
+      None(),
       environment);
 
   if (s.isError()) {