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/04/05 22:08:17 UTC

[1/2] mesos git commit: Ensured the bind mount root is a shared mount in its own peer group.

Repository: mesos
Updated Branches:
  refs/heads/master 4c078d89a -> 634f11b27


Ensured the bind mount root is a shared mount in its own peer group.

This is for the port mapping isolator.

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


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

Branch: refs/heads/master
Commit: 634f11b27170454b542f00f42f0c406faf144cce
Parents: eaecce0
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Apr 4 09:58:27 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Apr 5 13:08:01 2016 -0700

----------------------------------------------------------------------
 .../mesos/isolators/network/port_mapping.cpp    | 65 +++++++++++++++-----
 1 file changed, 51 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/634f11b2/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 5f91fa8..4f3f210 100644
--- a/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
@@ -1916,12 +1916,18 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
   if (bindMountEntry.isNone()) {
     // NOTE: Instead of using fs::mount to perform the bind mount, we
     // use the shell command here because the syscall 'mount' does not
-    // update the mount table (i.e., /etc/mtab), which could cause
-    // issues for the shell command 'mount --make-rslave' inside the
-    // container. It's OK to use the blocking os::shell here because
+    // update the mount table (i.e., /etc/mtab). In other words, the
+    // mount will not be visible if the operator types command
+    // 'mount'. Since this mount will still be presented after all
+    // containers and the slave are stopped, it's better to make it
+    // visible. It's OK to use the blocking os::shell here because
     // 'create' will only be invoked during initialization.
     Try<string> mount = os::shell(
-        "mount --bind %s %s",
+        "mount --bind %s %s && "
+        "mount --make-slave %s && "
+        "mount --make-shared %s",
+        bindMountRoot->c_str(),
+        bindMountRoot->c_str(),
         bindMountRoot->c_str(),
         bindMountRoot->c_str());
 
@@ -1930,17 +1936,48 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
           "Failed to self bind mount '" + bindMountRoot.get() +
           "' and make it a shared mount: " + mount.error());
     }
-  }
-
-  // Mark the mount point bindMountRoot as recursively shared.
-  Try<string> mountShared = os::shell(
-      "mount --make-rshared %s",
-      bindMountRoot->c_str());
+  } else {
+    if (bindMountEntry->shared().isNone()) {
+      // This is the case where the work directory mount is not a
+      // shared mount yet (possibly due to slave crash while preparing
+      // the work directory mount). It's safe to re-do the following.
+      Try<string> mount = os::shell(
+          "mount --make-slave %s && "
+          "mount --make-shared %s",
+          bindMountRoot->c_str(),
+          bindMountRoot->c_str());
+
+      if (mount.isError()) {
+        return Error(
+            "Failed to self bind mount '" + bindMountRoot.get() +
+            "' and make it a shared mount: " + mount.error());
+      }
+    } else {
+      // We need to make sure that the shared mount is in its own peer
+      // group. To check that, we need to get the parent mount.
+      foreach (const fs::MountInfoTable::Entry& entry, mountTable->entries) {
+        if (entry.id == bindMountEntry->parent) {
+          // If the bind mount root and its parent mount are in the
+          // same peer group, we need to re-do the following commands
+          // so that they are in different peer groups.
+          if (entry.shared() == bindMountEntry->shared()) {
+            Try<string> mount = os::shell(
+                "mount --make-slave %s && "
+                "mount --make-shared %s",
+                bindMountRoot->c_str(),
+                bindMountRoot->c_str());
+
+            if (mount.isError()) {
+              return Error(
+                  "Failed to self bind mount '" + bindMountRoot.get() +
+                  "' and make it a shared mount: " + mount.error());
+            }
+          }
 
-  if (mountShared.isError()) {
-    return Error(
-        "Failed to mark '" + bindMountRoot.get() +
-        "' as recursively shared: " + mountShared.error());
+          break;
+        }
+      }
+    }
   }
 
   // Create the network namespace handle symlink directory if it does


[2/2] mesos git commit: Used realpath for the bind mount root in port mapping isolator.

Posted by ji...@apache.org.
Used realpath for the bind mount root in port mapping isolator.

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


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

Branch: refs/heads/master
Commit: eaecce08a7ce0d72537512cbfccb77e545bb10b8
Parents: 4c078d8
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Apr 4 09:54:58 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Tue Apr 5 13:08:01 2016 -0700

----------------------------------------------------------------------
 .../mesos/isolators/network/port_mapping.cpp    | 74 +++++++++++---------
 .../mesos/isolators/network/port_mapping.hpp    |  3 +
 src/tests/containerizer/port_mapping_tests.cpp  | 26 +++----
 3 files changed, 58 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/eaecce08/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 a103a28..5f91fa8 100644
--- a/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/port_mapping.cpp
@@ -50,6 +50,7 @@
 #include <stout/utils.hpp>
 
 #include <stout/os/exists.hpp>
+#include <stout/os/realpath.hpp>
 #include <stout/os/stat.hpp>
 
 #include "common/status_utils.hpp"
@@ -364,11 +365,9 @@ static string getSymlinkPath(const ContainerID& containerId)
 }
 
 
-static string getNamespaceHandlePath(pid_t pid)
+static string getNamespaceHandlePath(const string& bindMountRoot, pid_t pid)
 {
-  return path::join(
-      PORT_MAPPING_BIND_MOUNT_ROOT(),
-      stringify(pid));
+  return path::join(bindMountRoot, stringify(pid));
 }
 
 
@@ -1879,32 +1878,42 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
   // network namespace handles on the host, those handles will be
   // unmounted in the containers as well, but NOT vice versa.
 
+  // We need to get the realpath for the bind mount root since on some
+  // Linux distribution, The bind mount root (i.e., /var/run/netns)
+  // might contain symlink.
+  Result<string> bindMountRoot = os::realpath(PORT_MAPPING_BIND_MOUNT_ROOT());
+  if (!bindMountRoot.isSome()) {
+    return Error(
+        "Failed to get realpath for bind mount root '" +
+        PORT_MAPPING_BIND_MOUNT_ROOT() + "': " +
+        (bindMountRoot.isError() ? bindMountRoot.error() : "Not found"));
+  }
+
   // We first create the bind mount directory if it does not exist.
-  Try<Nothing> mkdir = os::mkdir(PORT_MAPPING_BIND_MOUNT_ROOT());
+  Try<Nothing> mkdir = os::mkdir(bindMountRoot.get());
   if (mkdir.isError()) {
     return Error(
         "Failed to create the bind mount root directory at " +
-        PORT_MAPPING_BIND_MOUNT_ROOT() + ": " + mkdir.error());
+        bindMountRoot.get() + ": " + mkdir.error());
   }
 
-  // Now, check '/proc/mounts' to see if
-  // PORT_MAPPING_BIND_MOUNT_ROOT() has already been self mounted.
-  Try<fs::MountTable> mountTable = fs::MountTable::read("/proc/mounts");
+  // Now, check '/proc/self/mounts' to see if the bind mount root has
+  // already been self mounted.
+  Try<fs::MountInfoTable> mountTable = fs::MountInfoTable::read();
   if (mountTable.isError()) {
-    return Error(
-        "Failed to the read the mount table at '/proc/mounts': " +
-        mountTable.error());
+    return Error("Failed to read the mount table: " + mountTable.error());
   }
 
-  Option<fs::MountTable::Entry> bindMountRoot;
-  foreach (const fs::MountTable::Entry& entry, mountTable.get().entries) {
-    if (entry.dir == PORT_MAPPING_BIND_MOUNT_ROOT()) {
-      bindMountRoot = entry;
+  Option<fs::MountInfoTable::Entry> bindMountEntry;
+  foreach (const fs::MountInfoTable::Entry& entry, mountTable->entries) {
+    if (entry.target == bindMountRoot.get()) {
+      bindMountEntry = entry;
     }
   }
 
-  // Self bind mount PORT_MAPPING_BIND_MOUNT_ROOT().
-  if (bindMountRoot.isNone()) {
+  // Do a self bind mount if needed. If the mount already exists, make
+  // sure it is a shared mount of its own peer group.
+  if (bindMountEntry.isNone()) {
     // NOTE: Instead of using fs::mount to perform the bind mount, we
     // use the shell command here because the syscall 'mount' does not
     // update the mount table (i.e., /etc/mtab), which could cause
@@ -1913,25 +1922,24 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
     // 'create' will only be invoked during initialization.
     Try<string> mount = os::shell(
         "mount --bind %s %s",
-        PORT_MAPPING_BIND_MOUNT_ROOT().c_str(),
-        PORT_MAPPING_BIND_MOUNT_ROOT().c_str());
+        bindMountRoot->c_str(),
+        bindMountRoot->c_str());
 
     if (mount.isError()) {
       return Error(
-          "Failed to self bind mount '" + PORT_MAPPING_BIND_MOUNT_ROOT() +
-          "': " + mount.error());
+          "Failed to self bind mount '" + bindMountRoot.get() +
+          "' and make it a shared mount: " + mount.error());
     }
   }
 
-  // Mark the mount point PORT_MAPPING_BIND_MOUNT_ROOT() as
-  // recursively shared.
+  // Mark the mount point bindMountRoot as recursively shared.
   Try<string> mountShared = os::shell(
       "mount --make-rshared %s",
-      PORT_MAPPING_BIND_MOUNT_ROOT().c_str());
+      bindMountRoot->c_str());
 
   if (mountShared.isError()) {
     return Error(
-        "Failed to mark '" + PORT_MAPPING_BIND_MOUNT_ROOT() +
+        "Failed to mark '" + bindMountRoot.get() +
         "' as recursively shared: " + mountShared.error());
   }
 
@@ -1950,6 +1958,7 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
   return new MesosIsolator(Owned<MesosIsolatorProcess>(
       new PortMappingIsolatorProcess(
           flags,
+          bindMountRoot.get(),
           eth0.get(),
           lo.get(),
           hostMAC.get(),
@@ -1992,16 +2001,15 @@ Future<Nothing> PortMappingIsolatorProcess::recover(
 
   // Scan the bind mount root to cleanup all stale network namespace
   // handles that do not have an active veth associated with.
-  Try<list<string>> entries = os::ls(PORT_MAPPING_BIND_MOUNT_ROOT());
+  Try<list<string>> entries = os::ls(bindMountRoot);
   if (entries.isError()) {
     return Failure(
-        "Failed to list bind mount root '" +
-        PORT_MAPPING_BIND_MOUNT_ROOT() +
+        "Failed to list bind mount root '" + bindMountRoot +
         "': " + entries.error());
   }
 
   foreach (const string& entry, entries.get()) {
-    const string path = path::join(PORT_MAPPING_BIND_MOUNT_ROOT(), entry);
+    const string path = path::join(bindMountRoot, entry);
 
     // NOTE: We expect all regular files whose names are numbers under
     // the bind mount root are network namespace handles.
@@ -2512,7 +2520,7 @@ Future<Nothing> PortMappingIsolatorProcess::isolate(
   // the process 'pid' is gone, which allows us to explicitly control
   // the network namespace life cycle.
   const string source = path::join("/proc", stringify(pid), "ns", "net");
-  const string target = getNamespaceHandlePath(pid);
+  const string target = getNamespaceHandlePath(bindMountRoot, pid);
 
   Try<Nothing> touch = os::touch(target);
   if (touch.isError()) {
@@ -3512,7 +3520,7 @@ Try<Nothing> PortMappingIsolatorProcess::_cleanup(
   }
 
   // Release the bind mount for this container.
-  const string target = getNamespaceHandlePath(pid);
+  const string target = getNamespaceHandlePath(bindMountRoot, pid);
   Try<Nothing> unmount = fs::unmount(target, MNT_DETACH);
   if (unmount.isError()) {
     errors.push_back(
@@ -3876,7 +3884,7 @@ string PortMappingIsolatorProcess::scripts(Info* info)
   // Mark the mount point PORT_MAPPING_BIND_MOUNT_ROOT() as slave
   // mount so that changes in the container will not be propagated to
   // the host.
-  script << "mount --make-rslave " << PORT_MAPPING_BIND_MOUNT_ROOT() << "\n";
+  script << "mount --make-rslave " << bindMountRoot << "\n";
 
   // Disable IPv6 as IPv6 packets won't be forwarded anyway.
   script << "echo 1 > /proc/sys/net/ipv6/conf/all/disable_ipv6\n";

http://git-wip-us.apache.org/repos/asf/mesos/blob/eaecce08/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp b/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp
index 0fe2f48..21a86e3 100644
--- a/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/port_mapping.hpp
@@ -247,6 +247,7 @@ private:
 
   PortMappingIsolatorProcess(
       const Flags& _flags,
+      const std::string& _bindMountRoot,
       const std::string& _eth0,
       const std::string& _lo,
       const net::MAC& _hostMAC,
@@ -260,6 +261,7 @@ private:
       const process::Owned<EphemeralPortsAllocator>& _ephemeralPortsAllocator,
       const std::set<uint16_t>& _flowIDs)
     : flags(_flags),
+      bindMountRoot(_bindMountRoot),
       eth0(_eth0),
       lo(_lo),
       hostMAC(_hostMAC),
@@ -306,6 +308,7 @@ private:
   uint16_t getNextFlowId();
 
   const Flags flags;
+  const std::string bindMountRoot;
 
   const std::string eth0;
   const std::string lo;

http://git-wip-us.apache.org/repos/asf/mesos/blob/eaecce08/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 e062daa..ce985f4 100644
--- a/src/tests/containerizer/port_mapping_tests.cpp
+++ b/src/tests/containerizer/port_mapping_tests.cpp
@@ -161,21 +161,23 @@ static void cleanup(const string& eth0, const string& lo)
     }
   }
 
-  Try<list<string> > entries = os::ls(slave::PORT_MAPPING_BIND_MOUNT_ROOT());
-  ASSERT_SOME(entries);
+  if (os::exists(slave::PORT_MAPPING_BIND_MOUNT_ROOT())) {
+    Try<list<string> > entries = os::ls(slave::PORT_MAPPING_BIND_MOUNT_ROOT());
+    ASSERT_SOME(entries);
 
-  foreach (const string& file, entries.get()) {
-    string target = path::join(slave::PORT_MAPPING_BIND_MOUNT_ROOT(), file);
+    foreach (const string& file, entries.get()) {
+      string target = path::join(slave::PORT_MAPPING_BIND_MOUNT_ROOT(), file);
 
-    // NOTE: Here, we ignore the unmount errors because previous tests
-    // may have created the file and died before mounting.
-    if (!os::stat::islink(target)) {
-      mesos::internal::fs::unmount(target, MNT_DETACH);
-    }
+      // NOTE: Here, we ignore the unmount errors because previous tests
+      // may have created the file and died before mounting.
+      if (!os::stat::islink(target)) {
+        mesos::internal::fs::unmount(target, MNT_DETACH);
+      }
 
-    // Remove the network namespace handle and the corresponding
-    // symlinks. The removal here is best effort.
-    os::rm(target);
+      // Remove the network namespace handle and the corresponding
+      // symlinks. The removal here is best effort.
+      os::rm(target);
+    }
   }
 }