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 2015/04/03 19:22:44 UTC

mesos git commit: Changed the bind mount symlink root in port mapping isolator.

Repository: mesos
Updated Branches:
  refs/heads/master 88db77f26 -> 0e4f6564a


Changed the bind mount symlink root in port mapping isolator.

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


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

Branch: refs/heads/master
Commit: 0e4f6564a0c4d0044b48419b721063de2b76416a
Parents: 88db77f
Author: Chi Zhang <ch...@gmail.com>
Authored: Fri Apr 3 10:06:06 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Apr 3 10:16:02 2015 -0700

----------------------------------------------------------------------
 .../isolators/network/port_mapping.cpp          | 186 +++++++++++--------
 .../isolators/network/port_mapping.hpp          |   9 +
 src/tests/port_mapping_tests.cpp                |   2 +-
 3 files changed, 119 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/0e4f6564/src/slave/containerizer/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.cpp b/src/slave/containerizer/isolators/network/port_mapping.cpp
index e691d46..ccdc44f 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -203,10 +203,12 @@ static Option<pid_t> getPidFromVeth(const string& veth)
 
 // Extracts the container ID from the symlink that points to the
 // network namespace handle. The following is the layout of the bind
-// mount root:
+// mount root and bind mount symlink root:
 //  <PORT_MAPPING_BIND_MOUNT_ROOT()>
-//    |--- 3945 (pid)
-//    |--- ecf293e7-e6e8-4cbc-aaee-4d6c958aa276 -> 3945
+//    |--- 3945 (pid)                           <-|
+//                                                |
+//  <PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT()>      |
+//    |--- ecf293e7-e6e8-4cbc-aaee-4d6c958aa276 --|
 //         (symlink: container ID -> pid)
 static Try<ContainerID> getContainerIdFromSymlink(const string& symlink)
 {
@@ -271,7 +273,7 @@ static Result<pid_t> getPidFromSymlink(const string& symlink)
 static string getSymlinkPath(const ContainerID& containerId)
 {
   return path::join(
-      PORT_MAPPING_BIND_MOUNT_ROOT(),
+      PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT(),
       stringify(containerId));
 }
 
@@ -1399,6 +1401,18 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
         stringify(mountShared.get()));
   }
 
+  // Create the network namespace handle symlink directory if it does
+  // not exist. It is used to host from network namespace handle
+  // symlinks whose basename is a container ID. This allows us to
+  // recover container IDs for orphan containers (i.e., not known by
+  // the slave). This is introduced in 0.23.0.
+  mkdir = os::mkdir(PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT());
+  if(mkdir.isError()) {
+    return Error(
+        "Failed to create the bind mount root directory at " +
+        PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT() + ": " + mkdir.error());
+  }
+
   return new Isolator(Owned<IsolatorProcess>(
       new PortMappingIsolatorProcess(
           flags,
@@ -1439,12 +1453,8 @@ Future<Nothing> PortMappingIsolatorProcess::recover(
     pids.insert(pid.get());
   }
 
-  // Scan the bind mount root. This serves two purposes. First, we
-  // want to cleanup all stale network namespace handles that do not
-  // have active veth associated with. Second, since 0.23.0, we added
-  // a symlink to the network namespace handle using the container ID.
-  // This allows us to recover container IDs for orphan containers
-  // (i.e., not known by the slave).
+  // 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());
   if (entries.isError()) {
     return Failure(
@@ -1453,6 +1463,53 @@ Future<Nothing> PortMappingIsolatorProcess::recover(
         "': " + entries.error());
   }
 
+  foreach (const string& entry, entries.get()) {
+    const string path = path::join(PORT_MAPPING_BIND_MOUNT_ROOT(), entry);
+
+    // NOTE: We expect all regular files whose names are numbers under
+    // the bind mount root are network namespace handles.
+    Result<pid_t> pid = getPidFromNamespaceHandle(path);
+    if (pid.isError()) {
+      return Failure(
+          "Failed to get pid from network namespace handle '" +
+          path + "': " + pid.error());
+    } else if (pid.isNone()) {
+      // We ignore files that are clearly not network namespace
+      // handles created by us. It's likely that those are created by
+      // users or other tools.
+    }
+
+    // We cleanup the network namespace handle if the associated
+    // containers have clearly exited (i.e., the veth has gone). The
+    // cleanup here is best effort.
+    if (!pids.contains(pid.get())) {
+      LOG(INFO) << "Removing stale network namespace handle '" << path << "'";
+
+      Try<Nothing> unmount = fs::unmount(path, MNT_DETACH);
+      if (unmount.isError()) {
+        LOG(WARNING) << "Failed to unmount stale network namespace handle '"
+                     << path << "': " << unmount.error();
+      }
+
+      Try<Nothing> rm = os::rm(path);
+      if (rm.isError()) {
+        LOG(WARNING) << "Failed to remove stale network namespace handle '"
+                     << path << "': " << rm.error();
+      }
+    }
+  }
+
+  // Scan the bind mount symlink root for container IDs. This allows us
+  // to recover container IDs for orphan containers (i.e., not known
+  // by the slave). This is introduced in 0.23.0.
+  entries = os::ls(PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT());
+  if (entries.isError()) {
+    return Failure(
+        "Failed to list bind mount symlink root '" +
+        PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT() +
+        "': " + entries.error());
+  }
+
   // This map stores the mapping between pids and container IDs
   // recovered from the bind mount root that have valid veth links. We
   // use a multihashmap here because multiple container IDs can map to
@@ -1461,84 +1518,59 @@ Future<Nothing> PortMappingIsolatorProcess::recover(
   multihashmap<pid_t, ContainerID> linkers;
 
   foreach (const string& entry, entries.get()) {
-    const string path = path::join(PORT_MAPPING_BIND_MOUNT_ROOT(), entry);
-
-    if (os::stat::islink(path)) {
-      // NOTE: We expect all symlinks under the bind mount root to be
-      // container ID symlinks. The 'ip netns' tool will not create
-      // symlinks under the bind mount root.
+    const string path =
+      path::join(PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT(), entry);
+
+    // We only create symlinks in this directory and assume
+    // non-symlink files are created by other users or tools,
+    // therefore will be ignored.
+    if (!os::stat::islink(path)) {
+      LOG(WARNING) << "Ignored non-symlink file '" << path
+                   << "' under bind mount symlink root '"
+                   << PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT() << "'";
+      continue;
+    }
 
-      Try<ContainerID> containerId = getContainerIdFromSymlink(path);
-      if (containerId.isError()) {
-        return Failure(
-            "Failed to get container ID from network namespace handle "
-            "symlink '" + path + "': " + containerId.error());
-      }
+    // NOTE: We expect all symlinks under the bind mount symlink root
+    // to be container ID symlinks.
 
-      Result<pid_t> pid = getPidFromSymlink(path);
-      if (pid.isError()) {
-        return Failure(
-            "Failed to get pid from network namespace handle symlink '" +
-            path + "': " + pid.error());
-      }
+    Try<ContainerID> containerId = getContainerIdFromSymlink(path);
+    if (containerId.isError()) {
+      return Failure(
+          "Failed to get container ID from network namespace handle symlink '" +
+          path + "': " + containerId.error());
+    }
 
-      // We remove the symlink if it's dangling or the associated
-      // containers have clearly exited (i.e., the veth has gone). The
-      // cleanup here is best effort.
-      if (pid.isNone() || !pids.contains(pid.get())) {
-        LOG(INFO) << "Removing stale network namespace handle symlink '"
-                  << path << "'";
+    Result<pid_t> pid = getPidFromSymlink(path);
+    if (pid.isError()) {
+      return Failure(
+          "Failed to get pid from network namespace handle symlink '" + path +
+          "': " + pid.error());
+    }
 
-        Try<Nothing> rm = os::rm(path);
-        if (rm.isError()) {
-          LOG(WARNING) << "Failed to remove stale network namespace handle "
-                       << " symlink '" << path << "': " << rm.error();
-        }
-      } else {
-        LOG(INFO) << "Discovered network namespace handle symlink "
-                  << containerId.get() << " -> " << pid.get();
+    // We remove the symlink if it's dangling or the associated
+    // containers have clearly exited (i.e., the veth has gone). The
+    // cleanup here is best effort.
+    if (pid.isNone() || !pids.contains(pid.get())) {
+      LOG(INFO) << "Removing stale network namespace handle symlink '"
+                << path << "'";
 
-        linkers.put(pid.get(), containerId.get());
+      Try<Nothing> rm = os::rm(path);
+      if (rm.isError()) {
+        LOG(WARNING) << "Failed to remove stale network namespace handle "
+                     << " symlink '" << path << "': " << rm.error();
       }
     } else {
-      // NOTE: We expect all regular files whose names are numbers
-      // under the bind mount root are network namespace handles.
-
-      Result<pid_t> pid = getPidFromNamespaceHandle(path);
-      if (pid.isError()) {
-        return Failure(
-            "Failed to get pid from network namespace handle '" +
-            path + "': " + pid.error());
-      } else if (pid.isNone()) {
-        // We ignore files that are clearly not network namespace
-        // handles created by us. It's likely that those are created
-        // by users or other tools.
-      }
+      LOG(INFO) << "Discovered network namespace handle symlink "
+                << containerId.get() << " -> " << pid.get();
 
-      // We cleanup the network namespace handle if the associated
-      // containers have clearly exited (i.e., the veth has gone). The
-      // cleanup here is best effort.
-      if (!pids.contains(pid.get())) {
-        LOG(INFO) << "Removing stale network namespace handle '" << path << "'";
-
-        Try<Nothing> unmount = fs::unmount(path, MNT_DETACH);
-        if (unmount.isError()) {
-          LOG(WARNING) << "Failed to unmount stale network namespace handle '"
-                       << path << "': " << unmount.error();
-        }
-
-        Try<Nothing> rm = os::rm(path);
-        if (rm.isError()) {
-          LOG(WARNING) << "Failed to remove stale network namespace handle '"
-                       << path << "': " << rm.error();
-        }
-      }
+      linkers.put(pid.get(), containerId.get());
     }
   }
 
   // If multiple container IDs point to the same pid, we remove both
   // symlinks for safety (as if we cannot derive the container ID for
-  // orphans, which is OK because it'll be treated the same as thosek
+  // orphans, which is OK because it'll be treated the same as those
   // containers that are created by older (pre 0.23.0) versions). Note
   // that it's possible that multiple container IDs map to the same
   // pid if the removal of a symlink fails in '_cleanup()' and the pid
@@ -2672,8 +2704,8 @@ Try<Nothing> PortMappingIsolatorProcess::_cleanup(
         "Failed to remove the link " + veth(pid) + ": " + remove.error());
   }
 
-  // Remove the symlink for the network namespace handle if
-  // containerId is specified.
+  // Remove the symlink for the network namespace handle if a
+  // container ID is specified.
   if (containerId.isSome()) {
     const string linker = getSymlinkPath(containerId.get());
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e4f6564/src/slave/containerizer/isolators/network/port_mapping.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.hpp b/src/slave/containerizer/isolators/network/port_mapping.hpp
index 33837b4..466cd82 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.hpp
@@ -65,6 +65,15 @@ inline std::string PORT_MAPPING_VETH_PREFIX() { return "mesos"; }
 inline std::string PORT_MAPPING_BIND_MOUNT_ROOT() { return "/var/run/netns"; }
 
 
+// The root directory where we keep all the namespace handle
+// symlinks. This is introduced in 0.23.0.
+// NOTE: This constant is exposed for testing.
+inline std::string PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT()
+{
+  return "/var/run/mesos/netns";
+}
+
+
 // Responsible for allocating ephemeral ports for the port mapping
 // network isolator. This class is exposed mainly for unit testing.
 class EphemeralPortsAllocator

http://git-wip-us.apache.org/repos/asf/mesos/blob/0e4f6564/src/tests/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/port_mapping_tests.cpp b/src/tests/port_mapping_tests.cpp
index f4124c3..fbb9414 100644
--- a/src/tests/port_mapping_tests.cpp
+++ b/src/tests/port_mapping_tests.cpp
@@ -2047,7 +2047,7 @@ TEST_F(PortMappingMesosTest, ROOT_NetworkNamespaceHandleSymlink)
   ContainerID containerId = *(containers.get().begin());
 
   const string symlink = path::join(
-      slave::PORT_MAPPING_BIND_MOUNT_ROOT(),
+      slave::PORT_MAPPING_BIND_MOUNT_SYMLINK_ROOT(),
       stringify(containerId));
 
   EXPECT_TRUE(os::exists(symlink));