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/03/23 19:46:03 UTC

[1/2] mesos git commit: Fixed the non-POD global variables in port mapping isolator.

Repository: mesos
Updated Branches:
  refs/heads/master d82ec9207 -> 1d5d6d690


Fixed the non-POD global variables in port mapping isolator.

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


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

Branch: refs/heads/master
Commit: 1d5d6d69081459231d1dde70fff6e0f699f0aeb6
Parents: 15927e0
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Mar 20 17:38:19 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Mar 23 11:45:42 2015 -0700

----------------------------------------------------------------------
 .../isolators/network/port_mapping.cpp          | 88 ++++++++++----------
 .../isolators/network/port_mapping.hpp          | 11 ++-
 src/tests/port_mapping_tests.cpp                | 10 +--
 3 files changed, 58 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/1d5d6d69/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 f1f4007..4bf0ade 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -104,16 +104,6 @@ using mesos::slave::Isolator;
 using mesos::slave::IsolatorProcess;
 using mesos::slave::Limitation;
 
-const std::string VETH_PREFIX = "mesos";
-
-
-// The root directory where we bind mount all the namespace handles.
-// We choose the directory '/var/run/netns' so that we can use
-// iproute2 suite (e.g., ip netns show/exec) to inspect or enter the
-// network namespace. This is very useful for debugging purposes.
-const string BIND_MOUNT_ROOT = "/var/run/netns";
-
-
 // The minimum number of ephemeral ports a container should have.
 static const uint16_t MIN_EPHEMERAL_PORTS_SIZE = 16;
 
@@ -131,8 +121,10 @@ static const uint8_t LOW = 3;
 
 
 // The well known ports. Used for sanity check.
-static const Interval<uint16_t> WELL_KNOWN_PORTS =
-  (Bound<uint16_t>::closed(0), Bound<uint16_t>::open(1024));
+static Interval<uint16_t> WELL_KNOWN_PORTS()
+{
+  return (Bound<uint16_t>::closed(0), Bound<uint16_t>::open(1024));
+}
 
 
 /////////////////////////////////////////////////
@@ -178,16 +170,16 @@ static uint32_t roundDownToPowerOfTwo(uint32_t x)
 // executor process.
 static string veth(pid_t pid)
 {
-  return VETH_PREFIX + stringify(pid);
+  return PORT_MAPPING_VETH_PREFIX() + stringify(pid);
 }
 
 
 // Extracts the pid from the given veth name.
 static Option<pid_t> getPid(string veth)
 {
-  if (strings::startsWith(veth, VETH_PREFIX)) {
+  if (strings::startsWith(veth, PORT_MAPPING_VETH_PREFIX())) {
     Try<pid_t> pid = numify<pid_t>(
-        strings::remove(veth, VETH_PREFIX, strings::PREFIX));
+        strings::remove(veth, PORT_MAPPING_VETH_PREFIX(), strings::PREFIX));
 
     if (pid.isSome()) {
       return pid.get();
@@ -879,10 +871,10 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
 
   // This is a sanity check to make sure that the ephemeral ports
   // specified do not intersect with the well known ports.
-  if (ephemeralPorts.intersects(WELL_KNOWN_PORTS)) {
+  if (ephemeralPorts.intersects(WELL_KNOWN_PORTS())) {
     return Error(
         "The specified ephemeral ports " + stringify(ephemeralPorts) +
-        " intersect with well known ports " + stringify(WELL_KNOWN_PORTS));
+        " intersect with well known ports " + stringify(WELL_KNOWN_PORTS()));
   }
 
   // Obtain the host ephemeral port range by reading the proc file
@@ -1239,23 +1231,23 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
     }
   }
 
-  // Self bind mount BIND_MOUNT_ROOT. Since we use a new mount
-  // namespace for each container, for this mount point, we set
-  // '--make-rshared' on the host and set '--make-rslave' inside each
-  // container. This is important because when we unmount the network
-  // namespace handles on the host, those handles will be unmounted in
-  // the containers as well, but NOT vice versa.
+  // Self bind mount PORT_MAPPING_BIND_MOUNT_ROOT(). Since we use a
+  // new mount namespace for each container, for this mount point, we
+  // set '--make-rshared' on the host and set '--make-rslave' inside
+  // each container. This is important because when we unmount the
+  // network namespace handles on the host, those handles will be
+  // unmounted in the containers as well, but NOT vice versa.
 
   // We first create the bind mount directory if it does not exist.
-  Try<Nothing> mkdir = os::mkdir(BIND_MOUNT_ROOT);
+  Try<Nothing> mkdir = os::mkdir(PORT_MAPPING_BIND_MOUNT_ROOT());
   if(mkdir.isError()) {
     return Error(
         "Failed to create the bind mount root directory at " +
-        BIND_MOUNT_ROOT + ": " + mkdir.error());
+        PORT_MAPPING_BIND_MOUNT_ROOT() + ": " + mkdir.error());
   }
 
-  // Now, check '/proc/mounts' to see if BIND_MOUNT_ROOT has already
-  // been self mounted.
+  // 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");
   if (mountTable.isError()) {
     return Error(
@@ -1265,12 +1257,12 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
 
   Option<fs::MountTable::Entry> bindMountRoot;
   foreach (const fs::MountTable::Entry& entry, mountTable.get().entries) {
-    if (entry.dir == BIND_MOUNT_ROOT) {
+    if (entry.dir == PORT_MAPPING_BIND_MOUNT_ROOT()) {
       bindMountRoot = entry;
     }
   }
 
-  // Self bind mount BIND_MOUNT_ROOT.
+  // Self bind mount PORT_MAPPING_BIND_MOUNT_ROOT().
   if (bindMountRoot.isNone()) {
     // NOTE: Instead of using fs::mount to perform the bind mount, we
     // use the shell command here because the syscall 'mount' does not
@@ -1281,34 +1273,36 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
     Try<int> mount = os::shell(
         NULL,
         "mount --bind %s %s",
-        BIND_MOUNT_ROOT.c_str(),
-        BIND_MOUNT_ROOT.c_str());
+        PORT_MAPPING_BIND_MOUNT_ROOT().c_str(),
+        PORT_MAPPING_BIND_MOUNT_ROOT().c_str());
 
     if (mount.isError()) {
       return Error(
-          "Failed to self bind mount '" + BIND_MOUNT_ROOT +
+          "Failed to self bind mount '" + PORT_MAPPING_BIND_MOUNT_ROOT() +
           "': " + mount.error());
     } else if (mount.get() != 0) {
       return Error(
-          "Failed to self bind mount '" + BIND_MOUNT_ROOT +
+          "Failed to self bind mount '" + PORT_MAPPING_BIND_MOUNT_ROOT() +
           "': non-zero exit code: " + stringify(mount.get()));
     }
   }
 
-  // Mark the mount point BIND_MOUNT_ROOT as recursively shared.
+  // Mark the mount point PORT_MAPPING_BIND_MOUNT_ROOT() as
+  // recursively shared.
   Try<int> mountShared = os::shell(
       NULL,
       "mount --make-rshared %s",
-      BIND_MOUNT_ROOT.c_str());
+      PORT_MAPPING_BIND_MOUNT_ROOT().c_str());
 
   if (mountShared.isError()) {
     return Error(
-        "Failed to mark '" + BIND_MOUNT_ROOT + "' as recursively shared: " +
-        mountShared.error());
+        "Failed to mark '" + PORT_MAPPING_BIND_MOUNT_ROOT() +
+        "' as recursively shared: " + mountShared.error());
   } else if (mountShared.get() != 0) {
     return Error(
-        "Failed to mark '" + BIND_MOUNT_ROOT + "' as recursively shared: " +
-        "non-zero exit code: " + stringify(mountShared.get()));
+        "Failed to mark '" + PORT_MAPPING_BIND_MOUNT_ROOT() +
+        "' as recursively shared: " + "non-zero exit code: " +
+        stringify(mountShared.get()));
   }
 
   return new Isolator(Owned<IsolatorProcess>(
@@ -1613,7 +1607,10 @@ 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 = path::join(BIND_MOUNT_ROOT, stringify(pid));
+
+  const string target = path::join(
+      PORT_MAPPING_BIND_MOUNT_ROOT(),
+      stringify(pid));
 
   Try<Nothing> touch = os::touch(target);
   if (touch.isError()) {
@@ -2426,7 +2423,9 @@ Try<Nothing> PortMappingIsolatorProcess::_cleanup(Info* _info)
   }
 
   // Release the bind mount for this container.
-  const string target = path::join(BIND_MOUNT_ROOT, stringify(pid));
+  const string target = path::join(
+      PORT_MAPPING_BIND_MOUNT_ROOT(),
+      stringify(pid));
 
   Try<Nothing> unmount = fs::unmount(target, MNT_DETACH);
   if (unmount.isError()) {
@@ -2746,9 +2745,10 @@ string PortMappingIsolatorProcess::scripts(Info* info)
   script << "#!/bin/sh\n";
   script << "set -x\n";
 
-  // Mark the mount point BIND_MOUNT_ROOT as slave mount so that
-  // changes in the container will not be propagated to the host.
-  script << "mount --make-rslave " << BIND_MOUNT_ROOT << "\n";
+  // 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";
 
   // 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/1d5d6d69/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 3aae7e2..4dd066a 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.hpp
@@ -53,10 +53,17 @@ namespace internal {
 namespace slave {
 
 // The prefix this isolator uses for the virtual ethernet devices.
-extern const std::string VETH_PREFIX;
+// NOTE: This constant is exposed for testing.
+inline std::string PORT_MAPPING_VETH_PREFIX() { return "mesos"; }
+
 
 // The root directory where we bind mount all the namespace handles.
-extern const std::string BIND_MOUNT_ROOT;
+// We choose the directory '/var/run/netns' so that we can use
+// iproute2 suite (e.g., ip netns show/exec) to inspect or enter the
+// network namespace. This is very useful for debugging purposes.
+// NOTE: This constant is exposed for testing.
+inline std::string PORT_MAPPING_BIND_MOUNT_ROOT() { return "/var/run/netns"; }
+
 
 // Responsible for allocating ephemeral ports for the port mapping
 // network isolator. This class is exposed mainly for unit testing.

http://git-wip-us.apache.org/repos/asf/mesos/blob/1d5d6d69/src/tests/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/port_mapping_tests.cpp b/src/tests/port_mapping_tests.cpp
index 82f98a4..8192dea 100644
--- a/src/tests/port_mapping_tests.cpp
+++ b/src/tests/port_mapping_tests.cpp
@@ -117,16 +117,16 @@ static void cleanup(const string& eth0, const string& lo)
   ASSERT_SOME(links);
 
   foreach (const string& name, links.get()) {
-    if (strings::startsWith(name, slave::VETH_PREFIX)) {
+    if (strings::startsWith(name, slave::PORT_MAPPING_VETH_PREFIX())) {
       ASSERT_SOME_TRUE(link::remove(name));
     }
   }
 
-  Try<list<string> > entries = os::ls(slave::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::BIND_MOUNT_ROOT, file);
+    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.
@@ -2035,11 +2035,11 @@ TEST_F(PortMappingMesosTest, ROOT_CleanUpOrphanTest)
   Try<set<string> > links = net::links();
   ASSERT_SOME(links);
   foreach (const string& name, links.get()) {
-    EXPECT_FALSE(strings::startsWith(name, slave::VETH_PREFIX));
+    EXPECT_FALSE(strings::startsWith(name, slave::PORT_MAPPING_VETH_PREFIX()));
   }
 
   // Expect no files in bind mount directory.
-  Try<list<string> > files = os::ls(slave::BIND_MOUNT_ROOT);
+  Try<list<string> > files = os::ls(slave::PORT_MAPPING_BIND_MOUNT_ROOT());
   ASSERT_SOME(files);
   EXPECT_EQ(0u, files.get().size());
 


[2/2] mesos git commit: Fixed the right angle brackets in isolator header.

Posted by ji...@apache.org.
Fixed the right angle brackets in isolator header.


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

Branch: refs/heads/master
Commit: 15927e0233760adb258370f3fc597b9aad53bef0
Parents: d82ec92
Author: Jie Yu <yu...@gmail.com>
Authored: Fri Mar 20 16:33:22 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Mar 23 11:45:42 2015 -0700

----------------------------------------------------------------------
 include/mesos/slave/isolator.hpp | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/15927e02/include/mesos/slave/isolator.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/isolator.hpp b/include/mesos/slave/isolator.hpp
index 907be5b..7cbce42 100644
--- a/include/mesos/slave/isolator.hpp
+++ b/include/mesos/slave/isolator.hpp
@@ -89,7 +89,7 @@ public:
   // returned in the optional CommandInfo and they will be run by the Launcher.
   // TODO(idownes): Any URIs or Environment in the CommandInfo will be ignored;
   // only the command value is used.
-  process::Future<Option<CommandInfo> > prepare(
+  process::Future<Option<CommandInfo>> prepare(
       const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
       const std::string& directory,
@@ -133,7 +133,7 @@ public:
   virtual process::Future<Nothing> recover(
       const std::list<ExecutorRunState>& state) = 0;
 
-  virtual process::Future<Option<CommandInfo> > prepare(
+  virtual process::Future<Option<CommandInfo>> prepare(
       const ContainerID& containerId,
       const ExecutorInfo& executorInfo,
       const std::string& directory,