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 2017/09/05 22:17:28 UTC

[2/2] mesos git commit: Moved sandbox path volume logic to the dedicated isolator.

Moved sandbox path volume logic to the dedicated isolator.

Before this patch, the sandbox path volume logics are in two places:
the 'filesystem/linux' isolator and the 'volume/sandbox_path'
isolator, depending on the type of the sandbox path volume (SELF or
PARENT). This patch moved all the sandbox path volume related logics
to the 'volume/sandbox_path' isolator.

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


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

Branch: refs/heads/master
Commit: 14920628c684fb3cb35adeb8714cf4654d5402a8
Parents: 2bb7a8d
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Sep 4 08:27:45 2017 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Sep 5 15:17:24 2017 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp |   6 +-
 .../mesos/isolators/filesystem/linux.cpp        | 139 ------------
 .../mesos/isolators/volume/sandbox_path.cpp     | 214 ++++++++++++++-----
 3 files changed, 160 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/14920628/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 4ff014e..4d5dc13 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -261,10 +261,12 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   // after all volume isolators, so that the nvidia gpu libraries
   // '/usr/local/nvidia' will be overwritten.
   if (isolations->contains("filesystem/linux")) {
-    // Always enable 'volume/image', 'volume/host_path'  on linux if
-    // 'filesystem/linux' is enabled for backwards compatibility.
+    // Always enable 'volume/image', 'volume/host_path',
+    // 'volume/sandbox_path' on linux if 'filesystem/linux' is enabled
+    // for backwards compatibility.
     isolations->insert("volume/image");
     isolations->insert("volume/host_path");
+    isolations->insert("volume/sandbox_path");
   }
 #endif // __linux__
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/14920628/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 bc14324..faf9490 100644
--- a/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
+++ b/src/slave/containerizer/mesos/isolators/filesystem/linux.cpp
@@ -436,145 +436,6 @@ Try<vector<CommandInfo>> LinuxFilesystemIsolatorProcess::getPreExecCommands(
     commands.push_back(command);
   }
 
-  // Get the parent sandbox user and group info for the source path.
-  struct stat s;
-  if (::stat(containerConfig.directory().c_str(), &s) < 0) {
-    return ErrnoError("Failed to stat '" + containerConfig.directory() + "'");
-  }
-
-  const uid_t uid = s.st_uid;
-  const gid_t gid = s.st_gid;
-
-  foreach (const Volume& volume, containerConfig.container_info().volumes()) {
-    // NOTE: Volumes with source will be handled by the corresponding
-    // isolators (e.g., docker/volume).
-    if (volume.has_source()) {
-      VLOG(1) << "Ignored a volume with source for container "
-              << containerId;
-      continue;
-    }
-
-    if (volume.has_image()) {
-      VLOG(1) << "Ignored an image volume for container " << containerId;
-      continue;
-    }
-
-    if (!volume.has_host_path()) {
-      return Error("A volume misses 'host_path'");
-    }
-
-    // If both 'host_path' and 'container_path' are relative paths,
-    // return an error because the user can just directly access the
-    // volume in the work directory.
-    if (!path::absolute(volume.host_path()) &&
-        !path::absolute(volume.container_path())) {
-      return Error(
-          "Both 'host_path' and 'container_path' of a volume are relative");
-    }
-
-    // Host volumes are now handled by 'volume/host_path' isolator.
-    if (path::absolute(volume.host_path())) {
-      continue;
-    }
-
-    // Determine the source of the mount.
-    // Path is interpreted as relative to the work directory.
-    string source = path::join(
-        containerConfig.directory(),
-        volume.host_path());
-
-    // TODO(jieyu): We need to check that source resolves under the
-    // work directory because a user can potentially use a container
-    // path like '../../abc'.
-
-    // NOTE: Chown should be avoided if the source directory already
-    // exists because it may be owned by some other user and should
-    // not be mutated.
-    if (!os::exists(source)) {
-      Try<Nothing> mkdir = os::mkdir(source);
-      if (mkdir.isError()) {
-        return Error(
-            "Failed to create the source of the mount at '" +
-            source + "': " + mkdir.error());
-      }
-
-      LOG(INFO) << "Changing the ownership of the sandbox volume at '"
-                << source << "' with UID " << uid << " and GID " << gid;
-
-      Try<Nothing> chown = os::chown(uid, gid, source, false);
-      if (chown.isError()) {
-        return Error(
-            "Failed to change the ownership of the sandbox volume at '" +
-            source + "' with UID " + stringify(uid) + " and GID " +
-            stringify(gid) + ": " + chown.error());
-      }
-    }
-
-    // Determine the target of the mount. The mount target
-    // is determined by 'container_path'. It can be either
-    // a directory, or the path of a file.
-    string target;
-
-    CHECK(path::absolute(volume.container_path()));
-
-    if (containerConfig.has_rootfs()) {
-      target = path::join(
-          containerConfig.rootfs(),
-          volume.container_path());
-
-      if (os::stat::isfile(source)) {
-        // The file volume case.
-        Try<Nothing> mkdir = os::mkdir(Path(target).dirname());
-        if (mkdir.isError()) {
-          return Error(
-              "Failed to create directory '" +
-              Path(target).dirname() + "' "
-              "for the target mount file: " + mkdir.error());
-        }
-
-        Try<Nothing> touch = os::touch(target);
-        if (touch.isError()) {
-          return Error(
-              "Failed to create the target mount file at '" +
-              target + "': " + touch.error());
-        }
-      } else {
-        Try<Nothing> mkdir = os::mkdir(target);
-        if (mkdir.isError()) {
-          return Error(
-              "Failed to create the target of the mount at '" +
-              target + "': " + mkdir.error());
-        }
-      }
-    } else {
-      target = volume.container_path();
-
-      // An absolute path must already exist. This is because we
-      // want to avoid creating mount points outside the work
-      // directory in the host filesystem.
-      if (!os::exists(target)) {
-        return Error("Absolute container path '" + target + "' "
-                     "does not exist");
-      }
-    }
-
-    // TODO(jieyu): We need to check that target resolves under
-    // 'rootfs' because a user can potentially use a container path
-    // like '/../../abc'.
-
-    // TODO(jieyu): Consider the mode in the volume.
-    CommandInfo command;
-    command.set_shell(false);
-    command.set_value("mount");
-    command.add_arguments("mount");
-    command.add_arguments("-n");
-    command.add_arguments("--rbind");
-    command.add_arguments(source);
-    command.add_arguments(target);
-
-    commands.push_back(command);
-  }
-
   return commands;
 }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/14920628/src/slave/containerizer/mesos/isolators/volume/sandbox_path.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/volume/sandbox_path.cpp b/src/slave/containerizer/mesos/isolators/volume/sandbox_path.cpp
index b321b86..ee5ea3d 100644
--- a/src/slave/containerizer/mesos/isolators/volume/sandbox_path.cpp
+++ b/src/slave/containerizer/mesos/isolators/volume/sandbox_path.cpp
@@ -14,24 +14,31 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <glog/logging.h>
+
+#include <process/future.hpp>
 #include <process/id.hpp>
 
 #include <stout/foreach.hpp>
 #include <stout/fs.hpp>
+#include <stout/option.hpp>
+#include <stout/path.hpp>
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
 
+#include <stout/os/exists.hpp>
 #include <stout/os/mkdir.hpp>
+#include <stout/os/stat.hpp>
+#include <stout/os/touch.hpp>
 
-#ifdef __linux__
-#include "linux/ns.hpp"
-#endif // __linux__
+#include "common/validation.hpp"
 
 #include "slave/containerizer/mesos/isolators/volume/sandbox_path.hpp"
 
 using std::list;
 using std::string;
 
+using process::ErrnoFailure;
 using process::Failure;
 using process::Future;
 using process::Owned;
@@ -108,9 +115,7 @@ Future<Option<ContainerLaunchInfo>> VolumeSandboxPathIsolatorProcess::prepare(
   const ContainerInfo& containerInfo = containerConfig.container_info();
 
   if (containerInfo.type() != ContainerInfo::MESOS) {
-    return Failure(
-        "Can only prepare the sandbox volume isolator "
-        "for a MESOS container");
+    return Failure("Only support MESOS containers");
   }
 
   if (!bindMountSupported && containerConfig.has_rootfs()) {
@@ -122,9 +127,41 @@ Future<Option<ContainerLaunchInfo>> VolumeSandboxPathIsolatorProcess::prepare(
   ContainerLaunchInfo launchInfo;
 
   foreach (const Volume& volume, containerInfo.volumes()) {
-    if (!volume.has_source() ||
-        !volume.source().has_type() ||
-        volume.source().type() != Volume::Source::SANDBOX_PATH) {
+    // NOTE: The validation here is for backwards compatibility. For
+    // example, if an old master (no validation code) is used to
+    // launch a task with a volume.
+    Option<Error> error = common::validation::validateVolume(volume);
+    if (error.isSome()) {
+      return Failure("Invalid volume: " + error->message);
+    }
+
+    Option<Volume::Source::SandboxPath> sandboxPath;
+
+    // NOTE: This is the legacy way of specifying the Volume. The
+    // 'host_path' can be relative in legacy mode, representing
+    // SANDBOX_PATH volumes.
+    if (volume.has_host_path() &&
+        !path::absolute(volume.host_path())) {
+      sandboxPath = Volume::Source::SandboxPath();
+      sandboxPath->set_type(Volume::Source::SandboxPath::SELF);
+      sandboxPath->set_path(volume.host_path());
+    }
+
+    if (volume.has_source() &&
+        volume.source().has_type() &&
+        volume.source().type() == Volume::Source::SANDBOX_PATH) {
+      CHECK(volume.source().has_sandbox_path());
+
+      if (path::absolute(volume.source().sandbox_path().path())) {
+        return Failure(
+            "Path '" + volume.source().sandbox_path().path() + "' "
+            "in SANDBOX_PATH volume is absolute");
+      }
+
+      sandboxPath = volume.source().sandbox_path();
+    }
+
+    if (sandboxPath.isNone()) {
       continue;
     }
 
@@ -134,64 +171,83 @@ Future<Option<ContainerLaunchInfo>> VolumeSandboxPathIsolatorProcess::prepare(
           "SANDBOX_PATH volume is not supported for DEBUG containers");
     }
 
-    if (!volume.source().has_sandbox_path()) {
-      return Failure("volume.source.sandbox_path is not specified");
+    if (!bindMountSupported && path::absolute(volume.container_path())) {
+      return Failure(
+          "The 'linux' launcher and 'filesystem/linux' isolator "
+          "must be enabled to support SANDBOX_PATH volume with "
+          "absolute container path");
     }
 
-    const Volume::Source::SandboxPath& sandboxPath =
-      volume.source().sandbox_path();
+    // TODO(jieyu): We need to check that source resolves under the
+    // work directory because a user can potentially use a container
+    // path like '../../abc'.
 
-    // TODO(jieyu): Support other type of SANDBOX_PATH (e.g., SELF).
-    if (!sandboxPath.has_type() ||
-        sandboxPath.type() != Volume::Source::SandboxPath::PARENT) {
-      return Failure("Only PARENT sandbox path is supported");
+    if (!sandboxPath->has_type()) {
+      return Failure("Unknown SANDBOX_PATH volume type");
     }
 
-    if (!containerId.has_parent()) {
-      return Failure("PARENT sandbox path only works for nested container");
-    }
+    // Prepare the source.
+    string source;
+    string sourceRoot; // The parent directory of 'source'.
+
+    switch (sandboxPath->type()) {
+      case Volume::Source::SandboxPath::SELF:
+        // NOTE: For this case, the user can simply create a symlink
+        // in its sandbox. No need for a volume.
+        if (!path::absolute(volume.container_path())) {
+          return Failure(
+              "'container_path' is relative for "
+              "SANDBOX_PATH volume SELF type");
+        }
 
-    // TODO(jieyu): Validate sandboxPath.path for other invalid chars.
-    if (strings::contains(sandboxPath.path(), ".") ||
-        strings::contains(sandboxPath.path(), " ")) {
-      return Failure("Invalid char found in volume.source.sandbox_path.path");
-    }
+        sourceRoot = containerConfig.directory();
+        source = path::join(sourceRoot, sandboxPath->path());
+        break;
+      case Volume::Source::SandboxPath::PARENT:
+        if (!containerId.has_parent()) {
+          return Failure(
+              "SANDBOX_PATH volume PARENT type "
+              "only works for nested container");
+        }
 
-    if (!sandboxes.contains(containerId.parent())) {
-      return Failure("Failed to locate the sandbox for the parent container");
-    }
+        if (!sandboxes.contains(containerId.parent())) {
+          return Failure(
+              "Failed to locate the sandbox for the parent container");
+        }
 
-    // Prepare the source.
-    const string source = path::join(
-        sandboxes[containerId.parent()],
-        sandboxPath.path());
+        sourceRoot = sandboxes[containerId.parent()];
+        source = path::join(sourceRoot, sandboxPath->path());
+        break;
+      default:
+        return Failure("Unknown SANDBOX_PATH volume type");
+    }
 
-    // NOTE: Chown should be avoided if the source directory already
+    // NOTE: Chown should be avoided if the 'source' directory already
     // exists because it may be owned by some other user and should
     // not be mutated.
     if (!os::exists(source)) {
       Try<Nothing> mkdir = os::mkdir(source);
       if (mkdir.isError()) {
         return Failure(
-            "Failed to create the directory in the parent sandbox: " +
-            mkdir.error());
+            "Failed to create the directory '" + source + "' "
+            "in the sandbox: " + mkdir.error());
       }
 
-      // Get the parent sandbox user and group info for the source path.
+      // Get 'sourceRoot''s user and group info for the source path.
       struct stat s;
-      if (::stat(sandboxes[containerId.parent()].c_str(), &s) < 0) {
-        return Failure(ErrnoError(
-            "Failed to stat '" + sandboxes[containerId.parent()] + "'"));
+
+      if (::stat(sourceRoot.c_str(), &s) < 0) {
+        return ErrnoFailure("Failed to stat '" + sourceRoot + "'");
       }
 
-      LOG(INFO) << "Changing the ownership of the sandbox_path volume at '"
+      LOG(INFO) << "Changing the ownership of the SANDBOX_PATH volume at '"
                 << source << "' with UID " << s.st_uid << " and GID "
                 << s.st_gid;
 
       Try<Nothing> chown = os::chown(s.st_uid, s.st_gid, source, false);
       if (chown.isError()) {
         return Failure(
-            "Failed to change the ownership of the sandbox_path volume at '" +
+            "Failed to change the ownership of the SANDBOX_PATH volume at '" +
             source + "' with UID " + stringify(s.st_uid) + " and GID " +
             stringify(s.st_gid) + ": " + chown.error());
       }
@@ -201,33 +257,57 @@ Future<Option<ContainerLaunchInfo>> VolumeSandboxPathIsolatorProcess::prepare(
     string target;
 
     if (path::absolute(volume.container_path())) {
-      if (!bindMountSupported) {
-        return Failure(
-            "The 'linux' launcher and 'filesystem/linux' isolator must be "
-            "enabled to support absolute container path");
-      }
+      CHECK(bindMountSupported);
 
       if (containerConfig.has_rootfs()) {
         target = path::join(
             containerConfig.rootfs(),
             volume.container_path());
 
-        Try<Nothing> mkdir = os::mkdir(target);
-        if (mkdir.isError()) {
-          return Failure(
-              "Failed to create the target of the mount at '" +
-              target + "': " + mkdir.error());
+        if (os::stat::isdir(source)) {
+          Try<Nothing> mkdir = os::mkdir(target);
+          if (mkdir.isError()) {
+            return Failure(
+                "Failed to create the mount point at "
+                "'" + target + "': " + mkdir.error());
+          }
+        } else {
+          // The file (regular file or device file) bind mount case.
+          Try<Nothing> mkdir = os::mkdir(Path(target).dirname());
+          if (mkdir.isError()) {
+            return Failure(
+                "Failed to create directory "
+                "'" + Path(target).dirname() + "' "
+                "for the mount point: " + mkdir.error());
+          }
+
+          Try<Nothing> touch = os::touch(target);
+          if (touch.isError()) {
+            return Failure(
+                "Failed to touch the mount point at "
+                "'" + target + "': " + touch.error());
+          }
         }
       } else {
         target = volume.container_path();
 
+        // An absolute 'container_path' must already exist if the
+        // container rootfs is the same as the host. This is because
+        // we want to avoid creating mount points outside the work
+        // directory in the host filesystem.
         if (!os::exists(target)) {
           return Failure(
-              "Absolute container path '" + target + "' "
-              "does not exist");
+              "Mount point '" + target + "' is an absolute path. "
+              "It must exist if the container shares the host filesystem");
         }
       }
+
+      // TODO(jieyu): We need to check that target resolves under
+      // 'rootfs' because a user can potentially use a container path
+      // like '/../../abc'.
     } else {
+      CHECK_EQ(Volume::Source::SandboxPath::PARENT, sandboxPath->type());
+
       if (containerConfig.has_rootfs()) {
         target = path::join(
             containerConfig.rootfs(),
@@ -249,11 +329,29 @@ Future<Option<ContainerLaunchInfo>> VolumeSandboxPathIsolatorProcess::prepare(
             containerConfig.directory(),
             volume.container_path());
 
-        Try<Nothing> mkdir = os::mkdir(mountPoint);
-        if (mkdir.isError()) {
-          return Failure(
-              "Failed to create the target of the mount at '" +
-              mountPoint + "': " + mkdir.error());
+        if (os::stat::isdir(source)) {
+          Try<Nothing> mkdir = os::mkdir(mountPoint);
+          if (mkdir.isError()) {
+            return Failure(
+                "Failed to create the mount point at "
+                "'" + mountPoint + "': " + mkdir.error());
+          }
+        } else {
+          // The file (regular file or device file) bind mount case.
+          Try<Nothing> mkdir = os::mkdir(Path(mountPoint).dirname());
+          if (mkdir.isError()) {
+            return Failure(
+                "Failed to create the directory "
+                "'" + Path(mountPoint).dirname() + "' "
+                "for the mount point: " + mkdir.error());
+          }
+
+          Try<Nothing> touch = os::touch(mountPoint);
+          if (touch.isError()) {
+            return Failure(
+                "Failed to touch the mount point at "
+                "'" + mountPoint+ "': " + touch.error());
+          }
         }
       }
     }