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/02/05 20:33:27 UTC

mesos git commit: Supported working dir in docker runtime isolator.

Repository: mesos
Updated Branches:
  refs/heads/master d27776d42 -> 4a3870991


Supported working dir in docker runtime isolator.

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


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

Branch: refs/heads/master
Commit: 4a387099128767bca34b54b8951ff6decfa434f5
Parents: d27776d
Author: Gilbert Song <so...@gmail.com>
Authored: Fri Feb 5 10:39:32 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Feb 5 11:18:40 2016 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp |  27 ++-
 .../mesos/isolators/docker/runtime.cpp          | 209 +++++++++++--------
 .../mesos/isolators/docker/runtime.hpp          |   5 +-
 3 files changed, 151 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4a387099/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index dc0868e..1e362d2 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -929,6 +929,7 @@ Future<bool> MesosContainerizerProcess::__launch(
   // At most one command can be returned from docker runtime
   // isolator if a docker image is specifed.
   Option<CommandInfo> executorLaunchCommand;
+  Option<string> workingDirectory;
 
   foreach (const Option<ContainerLaunchInfo>& launchInfo, launchInfos) {
     if (launchInfo.isSome() && launchInfo->has_rootfs()) {
@@ -964,6 +965,15 @@ Future<bool> MesosContainerizerProcess::__launch(
         executorLaunchCommand = launchInfo->command();
       }
     }
+
+    if (launchInfo.isSome() && launchInfo->has_working_directory()) {
+      if (workingDirectory.isSome()) {
+        return Failure(
+            "At most one working directory can be returned from isolators");
+      } else {
+        workingDirectory = launchInfo->working_directory();
+      }
+    }
   }
 
   // TODO(jieyu): Consider moving this to 'executorEnvironment' and
@@ -1015,8 +1025,8 @@ Future<bool> MesosContainerizerProcess::__launch(
     // Use a pipe to block the child until it's been isolated.
     int pipes[2];
 
-    // We assume this should not fail under reasonable conditions so we
-    // use CHECK.
+    // We assume this should not fail under reasonable conditions so
+    // we use CHECK.
     CHECK(pipe(pipes) == 0);
 
     // Prepare the flags to pass to the launch process.
@@ -1029,6 +1039,19 @@ Future<bool> MesosContainerizerProcess::__launch(
     launchFlags.sandbox = rootfs.isSome()
       ? flags.sandbox_directory
       : directory;
+
+    // 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 (rootfs.isNone() && 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";
+    } else {
+      launchFlags.working_directory = workingDirectory;
+    }
+
     launchFlags.rootfs = rootfs;
     launchFlags.user = user;
     launchFlags.pipe_read = pipes[0];

http://git-wip-us.apache.org/repos/asf/mesos/blob/4a387099/src/slave/containerizer/mesos/isolators/docker/runtime.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/docker/runtime.cpp b/src/slave/containerizer/mesos/isolators/docker/runtime.cpp
index c8a9372..71c287c 100644
--- a/src/slave/containerizer/mesos/isolators/docker/runtime.cpp
+++ b/src/slave/containerizer/mesos/isolators/docker/runtime.cpp
@@ -19,13 +19,13 @@
 
 #include <glog/logging.h>
 
-#include <mesos/docker/v1.hpp>
-
 #include <stout/error.hpp>
 #include <stout/foreach.hpp>
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
 
+#include <mesos/docker/v1.hpp>
+
 #include "slave/flags.hpp"
 
 #include "slave/containerizer/mesos/isolators/docker/runtime.hpp"
@@ -92,28 +92,62 @@ Future<Option<ContainerLaunchInfo>> DockerRuntimeIsolatorProcess::prepare(
     return None();
   }
 
-  // Contains docker image default environment variables, merged
-  // command, and working directory.
-  ContainerLaunchInfo launchInfo;
+  Option<Environment> environment =
+    getLaunchEnvironment(containerId, containerConfig);
 
-  Option<Environment> environment = getLaunchEnvironment(
-      containerId,
-      containerConfig);
+  Option<string> workingDirectory =
+    getWorkingDirectory(containerConfig);
 
-  if (environment.isSome()) {
-    launchInfo.mutable_environment()->CopyFrom(environment.get());
-  }
-
-  Try<CommandInfo> command = getExecutorLaunchCommand(
-      containerId,
-      containerConfig);
+  Result<CommandInfo> command =
+    getExecutorLaunchCommand(containerId, containerConfig);
 
   if (command.isError()) {
     return Failure("Failed to determine the executor launch command: " +
                    command.error());
   }
 
-  launchInfo.mutable_command()->CopyFrom(command.get());
+  // Set 'launchInfo'.
+  ContainerLaunchInfo launchInfo;
+
+  if (environment.isSome()) {
+    launchInfo.mutable_environment()->CopyFrom(environment.get());
+  }
+
+  // If working directory or command exists, operation has to be
+  // distinguished for either custom executor or command task. For
+  // custom executor case, info will be included in 'launchInfo', and
+  // will be passed back to containerizer. For command task case, info
+  // will be passed to command executor as flags.
+  if (!containerConfig.has_task_info()) {
+    // Custom executor case.
+    if (workingDirectory.isSome()) {
+      launchInfo.set_working_directory(workingDirectory.get());
+    }
+
+    if (command.isSome()) {
+      launchInfo.mutable_command()->CopyFrom(command.get());
+    }
+  } else {
+    // Command task case. The 'executorCommand' below is the
+    // command with value as 'mesos-executor'.
+    CommandInfo executorCommand = containerConfig.executor_info().command();
+
+    // Pass working directory to command executor as a flag.
+    if (workingDirectory.isSome()) {
+      executorCommand.add_arguments(
+          "--working_directory=" + workingDirectory.get());
+    }
+
+    // Pass task command as a flag, which will be loaded by
+    // command executor.
+    if (command.isSome()) {
+      executorCommand.add_arguments(
+          "--task_command=" +
+          stringify(JSON::protobuf(command.get())));
+    }
+
+    launchInfo.mutable_command()->CopyFrom(executorCommand);
+  }
 
   return launchInfo;
 }
@@ -162,8 +196,9 @@ Option<Environment> DockerRuntimeIsolatorProcess::getLaunchEnvironment(
 // This method reads the CommandInfo form ExecutorInfo and optional
 // TaskInfo, and merge them with docker image default Entrypoint and
 // Cmd. It returns a merged CommandInfo which will be used to launch
-// the executor.
-Try<CommandInfo> DockerRuntimeIsolatorProcess::getExecutorLaunchCommand(
+// the executor. If no need to modify the command, this method will
+// return none.
+Result<CommandInfo> DockerRuntimeIsolatorProcess::getExecutorLaunchCommand(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig)
 {
@@ -239,87 +274,87 @@ Try<CommandInfo> DockerRuntimeIsolatorProcess::getExecutorLaunchCommand(
     if (!command.has_value()) {
       return Error("Shell specified but no command value provided");
     }
-  } else {
-    if (!command.has_value()) {
-      // TODO(gilbert): Deprecate 'override' flag option in command
-      // task. We do not exclude override case here.
-
-      // We keep the arguments of commandInfo while it does not have a
-      // value, so that arguments can be specified by user, which is
-      // running with default image Entrypoint.
-      const docker::spec::v1::ImageManifest::Config& config =
-        containerConfig.docker().manifest().config();
-
-      // Filter out executable for commandInfo value.
-      if (config.entrypoint_size() > 0) {
-        command.set_value(config.entrypoint(0));
-
-        // Put user defined argv after default entrypoint argv
-        // in sequence.
-        command.clear_arguments();
-
-        for (int i = 1; i < config.entrypoint_size(); i++) {
-          command.add_arguments(config.entrypoint(i));
-        }
-
-        // Append all possible user argv after entrypoint arguments.
-        if (!containerConfig.has_task_info()) {
-          // Custom executor case.
-          command.mutable_arguments()->MergeFrom(
-              containerConfig.executor_info().command().arguments());
-        } else {
-          // Command task case.
-          command.mutable_arguments()->MergeFrom(
-              containerConfig.task_info().command().arguments());
-        }
-
-        // Overwrite default cmd arguments if CommandInfo arguments
-        // are set by user. The logic below is the case that no
-        // argument is set by user.
-        if (command.arguments_size() == config.entrypoint_size() - 1) {
-          foreach (const string& cmd, config.cmd()) {
-            command.add_arguments(cmd);
-          }
-        }
-      } else if (config.cmd_size() > 0) {
-        command.set_value(config.cmd(0));
-
-        // Overwrite default cmd arguments if CommandInfo arguments
-        // are set by user.
-        if (command.arguments_size() == 0) {
-          for (int i = 1; i < config.cmd_size(); i++) {
-            command.add_arguments(config.cmd(i));
-          }
-        }
-      } else {
-        return Error("No executable is found for container: '" +
-                     containerId.value() + "'");
-      }
-    }
+
+    // No need to mutate command (row 7-8).
+    return None();
   }
 
-  if (containerConfig.has_task_info()) {
-    // For command executor, with command value as 'mesos-executor'.
-    CommandInfo executorCommand = containerConfig.executor_info().command();
+  if (command.has_value()) {
+    // No need to mutate command (row 3-4).
+    return None();
+  }
+
+  // TODO(gilbert): Deprecate 'override' flag option in command task.
+  // We do not exclude override case here.
+
+  // We keep the arguments of commandInfo while it does not have a
+  // value, so that arguments can be specified by user, which is
+  // running with default image Entrypoint.
+  const docker::spec::v1::ImageManifest::Config& config =
+    containerConfig.docker().manifest().config();
+
+  // Filter out executable for commandInfo value.
+  if (config.entrypoint_size() > 0) {
+    command.set_value(config.entrypoint(0));
 
-    // Only pass the mutated command to command executor as a flag if
-    // image default config is included (see table above: row 1-2).
-    if (!containerConfig.task_info().command().shell() &&
-        !containerConfig.task_info().command().has_value()) {
-      JSON::Object object = JSON::protobuf(command);
+    // Put user defined argv after default entrypoint argv
+    // in sequence.
+    command.clear_arguments();
 
-      // Pass task command as a flag, which will be loaded by
-      // command executor.
-      executorCommand.add_arguments("--task_command=" + stringify(object));
+    for (int i = 1; i < config.entrypoint_size(); i++) {
+      command.add_arguments(config.entrypoint(i));
     }
 
-    return executorCommand;
+    // Append all possible user argv after entrypoint arguments.
+    if (!containerConfig.has_task_info()) {
+      // Custom executor case.
+      command.mutable_arguments()->MergeFrom(
+          containerConfig.executor_info().command().arguments());
+    } else {
+      // Command task case.
+      command.mutable_arguments()->MergeFrom(
+          containerConfig.task_info().command().arguments());
+    }
+
+    // Overwrite default cmd arguments if CommandInfo arguments are
+    // set by user. The logic below is the case that no argument is
+    // set by user.
+    if (command.arguments_size() == config.entrypoint_size() - 1) {
+      foreach (const string& cmd, config.cmd()) {
+        command.add_arguments(cmd);
+      }
+    }
+  } else if (config.cmd_size() > 0) {
+    command.set_value(config.cmd(0));
+
+    // Overwrite default cmd arguments if CommandInfo arguments
+    // are set by user.
+    if (command.arguments_size() == 0) {
+      for (int i = 1; i < config.cmd_size(); i++) {
+        command.add_arguments(config.cmd(i));
+      }
+    }
+  } else {
+    return Error("No executable is found");
   }
 
   return command;
 }
 
 
+Option<string> DockerRuntimeIsolatorProcess::getWorkingDirectory(
+    const ContainerConfig& containerConfig)
+{
+  CHECK(containerConfig.docker().manifest().has_config());
+
+  if (!containerConfig.docker().manifest().config().has_workingdir()) {
+    return None();
+  }
+
+  return containerConfig.docker().manifest().config().workingdir();
+}
+
+
 Future<Nothing> DockerRuntimeIsolatorProcess::isolate(
     const ContainerID& containerId,
     pid_t pid)

http://git-wip-us.apache.org/repos/asf/mesos/blob/4a387099/src/slave/containerizer/mesos/isolators/docker/runtime.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/docker/runtime.hpp b/src/slave/containerizer/mesos/isolators/docker/runtime.hpp
index 6e6ec86..507505a 100644
--- a/src/slave/containerizer/mesos/isolators/docker/runtime.hpp
+++ b/src/slave/containerizer/mesos/isolators/docker/runtime.hpp
@@ -65,10 +65,13 @@ private:
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig);
 
-  Try<CommandInfo> getExecutorLaunchCommand(
+  Result<CommandInfo> getExecutorLaunchCommand(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig);
 
+  Option<std::string> getWorkingDirectory(
+      const mesos::slave::ContainerConfig& containerConfig);
+
   const Flags flags;
 };