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/27 01:30:32 UTC

mesos git commit: Symlink the namespace handle with ContainerID for the port mapping isolator.

Repository: mesos
Updated Branches:
  refs/heads/master b9a8dcac7 -> de0307cb8


Symlink the namespace handle with ContainerID for the port mapping isolator.

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


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

Branch: refs/heads/master
Commit: de0307cb8bd615dcffd09f7816d7de06deeba185
Parents: b9a8dca
Author: Jie Yu <yu...@gmail.com>
Authored: Mon Mar 23 17:34:40 2015 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Thu Mar 26 17:29:57 2015 -0700

----------------------------------------------------------------------
 .../isolators/network/port_mapping.cpp          | 348 ++++++++++++++++---
 .../isolators/network/port_mapping.hpp          |   6 +-
 src/tests/port_mapping_tests.cpp                |  96 ++++-
 3 files changed, 399 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/de0307cb/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 4bf0ade..e691d46 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -35,10 +35,12 @@
 
 #include <stout/error.hpp>
 #include <stout/foreach.hpp>
+#include <stout/fs.hpp>
 #include <stout/hashset.hpp>
 #include <stout/json.hpp>
 #include <stout/lambda.hpp>
 #include <stout/mac.hpp>
+#include <stout/multihashmap.hpp>
 #include <stout/numify.hpp>
 #include <stout/os.hpp>
 #include <stout/option.hpp>
@@ -46,8 +48,10 @@
 #include <stout/result.hpp>
 #include <stout/stringify.hpp>
 #include <stout/strings.hpp>
+#include <stout/utils.hpp>
 
 #include <stout/os/exists.hpp>
+#include <stout/os/stat.hpp>
 
 #include "common/status_utils.hpp"
 
@@ -95,6 +99,13 @@ using std::vector;
 
 using filter::ip::PortRange;
 
+
+// An old glibc might not have this symbol.
+#ifndef MNT_DETACH
+#define MNT_DETACH 2
+#endif
+
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -175,7 +186,7 @@ static string veth(pid_t pid)
 
 
 // Extracts the pid from the given veth name.
-static Option<pid_t> getPid(string veth)
+static Option<pid_t> getPidFromVeth(const string& veth)
 {
   if (strings::startsWith(veth, PORT_MAPPING_VETH_PREFIX())) {
     Try<pid_t> pid = numify<pid_t>(
@@ -190,6 +201,89 @@ static Option<pid_t> getPid(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:
+//  <PORT_MAPPING_BIND_MOUNT_ROOT()>
+//    |--- 3945 (pid)
+//    |--- ecf293e7-e6e8-4cbc-aaee-4d6c958aa276 -> 3945
+//         (symlink: container ID -> pid)
+static Try<ContainerID> getContainerIdFromSymlink(const string& symlink)
+{
+  if (!os::stat::islink(symlink)) {
+    return Error("Not a symlink");
+  }
+
+  Try<string> _containerId = os::basename(symlink);
+  if (_containerId.isError()) {
+    return Error("Failed to get the basename: " + _containerId.error());
+  }
+
+  ContainerID containerId;
+  containerId.set_value(_containerId.get());
+
+  return containerId;
+}
+
+
+// Extracts the pid from the network namespace handle. Returns None if
+// the handle is clearly not created by us.
+static Result<pid_t> getPidFromNamespaceHandle(const string& handle)
+{
+  if (os::stat::islink(handle)) {
+    return Error("Not expecting a symlink");
+  }
+
+  Try<string> _pid = os::basename(handle);
+  if (_pid.isError()) {
+    return Error("Failed to get the basename: " + _pid.error());
+  }
+
+  Try<pid_t> pid = numify<pid_t>(_pid.get());
+  if (pid.isError()) {
+    return None();
+  }
+
+  return pid.get();
+}
+
+
+// Extracts the pid from the symlink that points to the network
+// namespace handle. Returns None if it's a dangling symlink.
+static Result<pid_t> getPidFromSymlink(const string& symlink)
+{
+  if (!os::stat::islink(symlink)) {
+    return Error("Not a symlink");
+  }
+
+  Result<string> target = os::realpath(symlink);
+  if (target.isError()) {
+    return Error("Failed to follow the symlink: " + target.error());
+  } else if (target.isNone()) {
+    // This is a dangling symlink.
+    return None();
+  }
+
+  return getPidFromNamespaceHandle(target.get());
+}
+
+
+static string getSymlinkPath(const ContainerID& containerId)
+{
+  return path::join(
+      PORT_MAPPING_BIND_MOUNT_ROOT(),
+      stringify(containerId));
+}
+
+
+static string getNamespaceHandlePath(pid_t pid)
+{
+  return path::join(
+      PORT_MAPPING_BIND_MOUNT_ROOT(),
+      stringify(pid));
+}
+
+
 // Converts from value ranges to interval set.
 static IntervalSet<uint16_t> getIntervalSet(const Value::Ranges& ranges)
 {
@@ -207,7 +301,7 @@ static IntervalSet<uint16_t> getIntervalSet(const Value::Ranges& ranges)
 // Implementation for PortMappingUpdate.
 /////////////////////////////////////////////////
 
-const std::string PortMappingUpdate::NAME = "update";
+const char* PortMappingUpdate::NAME = "update";
 
 
 PortMappingUpdate::Flags::Flags()
@@ -476,7 +570,7 @@ int PortMappingUpdate::execute()
 // Implementation for PortMappingStatistics.
 /////////////////////////////////////////////////
 
-const std::string PortMappingStatistics::NAME = "statistics";
+const char* PortMappingStatistics::NAME = "statistics";
 
 
 PortMappingStatistics::Flags::Flags()
@@ -1324,16 +1418,16 @@ Try<Isolator*> PortMappingIsolatorProcess::create(const Flags& flags)
 Future<Nothing> PortMappingIsolatorProcess::recover(
     const list<ExecutorRunState>& states)
 {
-  // Extract pids from virtual device names.
+  // Extract pids from virtual device names (veth). This tells us
+  // about all the potential live containers on this slave.
   Try<set<string>> links = net::links();
   if (links.isError()) {
     return Failure("Failed to get all the links: " + links.error());
   }
 
-  // The 'pids' here are extracted from veth devices.
   hashset<pid_t> pids;
   foreach (const string& name, links.get()) {
-    Option<pid_t> pid = getPid(name);
+    Option<pid_t> pid = getPidFromVeth(name);
     // Not all links follow the naming: mesos{pid}, so we simply
     // continue, e.g., eth0.
     if (pid.isNone()) {
@@ -1345,6 +1439,131 @@ 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).
+  Try<list<string>> entries = os::ls(PORT_MAPPING_BIND_MOUNT_ROOT());
+  if (entries.isError()) {
+    return Failure(
+        "Failed to list bind mount root '" +
+        PORT_MAPPING_BIND_MOUNT_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
+  // the same pid if the removal of a symlink fails in '_cleanup()'
+  // and the pid is reused by a new container.
+  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.
+
+      Try<ContainerID> containerId = getContainerIdFromSymlink(path);
+      if (containerId.isError()) {
+        return Failure(
+            "Failed to get container ID from network namespace handle "
+            "symlink '" + path + "': " + containerId.error());
+      }
+
+      Result<pid_t> pid = getPidFromSymlink(path);
+      if (pid.isError()) {
+        return Failure(
+            "Failed to get pid from network namespace handle symlink '" +
+            path + "': " + pid.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 << "'";
+
+        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();
+
+        linkers.put(pid.get(), containerId.get());
+      }
+    } 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.
+      }
+
+      // 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();
+        }
+      }
+    }
+  }
+
+  // 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
+  // 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
+  // is reused by a new container.
+  foreach (pid_t pid, linkers.keys()) {
+    list<ContainerID> containerIds = linkers.get(pid);
+    if (containerIds.size() > 1) {
+      foreach (const ContainerID& containerId, containerIds) {
+        const string linker = getSymlinkPath(containerId);
+
+        LOG(WARNING) << "Removing duplicated network namespace handle symlink '"
+                     << linker << "'";
+
+        Try<Nothing> rm = os::rm(linker);
+        if (rm.isError()) {
+          LOG(WARNING) << "Failed to remove duplicated network namespace "
+                       << "handle symlink '" << linker << "': " << rm.error();
+        }
+      }
+
+      linkers.remove(pid);
+    }
+  }
+
+  // Now, actually recover the isolator from slave's state.
   foreach (const ExecutorRunState& state, states) {
     const ContainerID& containerId = state.id;
     pid_t pid = state.pid;
@@ -1409,8 +1628,17 @@ Future<Nothing> PortMappingIsolatorProcess::recover(
           stringify(pid) + ": " + recover.error());
     }
 
+    // TODO(jieyu): For those orphan containers that have a known
+    // container ID, consider relying on the MesosContainerizer to
+    // clean them up so that we don't block the recovery. See details
+    // in MESOS-2367.
+    Option<ContainerID> containerId;
+    if (linkers.get(pid).size() == 1) {
+      containerId = linkers.get(pid).front();
+    }
+
     // The recovery should fail if we cannot cleanup an orphan.
-    Try<Nothing> cleanup = _cleanup(recover.get());
+    Try<Nothing> cleanup = _cleanup(recover.get(), containerId);
     if (cleanup.isError()) {
       foreachvalue (Info* info, infos) {
         delete info;
@@ -1607,10 +1835,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 = path::join(
-      PORT_MAPPING_BIND_MOUNT_ROOT(),
-      stringify(pid));
+  const string target = getNamespaceHandlePath(pid);
 
   Try<Nothing> touch = os::touch(target);
   if (touch.isError()) {
@@ -1620,12 +1845,28 @@ Future<Nothing> PortMappingIsolatorProcess::isolate(
   Try<Nothing> mount = fs::mount(source, target, None(), MS_BIND, NULL);
   if (mount.isError()) {
     return Failure(
-        "Failed to mount the network namespace handle from " +
-        source + " to " + target + ": " + mount.error());
+        "Failed to mount the network namespace handle from '" +
+        source + "' to '" + target + "': " + mount.error());
   }
 
-  LOG(INFO) << "Bind mounted " << source << " to " << target
-            << " for container " << containerId;
+  LOG(INFO) << "Bind mounted '" << source << "' to '" << target
+            << "' for container " << containerId;
+
+  // Since 0.23.0, we create a symlink to the network namespace handle
+  // using the container ID. This serves two purposes. First, it
+  // allows us to recover the container ID later when slave restarts
+  // even if slave's checkpointed meta data is deleted. Second, it
+  // makes the debugging easier. See MESOS-2528 for details.
+  const string linker = getSymlinkPath(containerId);
+  Try<Nothing> symlink = ::fs::symlink(target, linker);
+  if (symlink.isError()) {
+    return Failure(
+        "Failed to symlink the network namespace handle '" +
+        linker + "' -> '" + target + "': " + symlink.error());
+  }
+
+  LOG(INFO) << "Created network namespace handle symlink '"
+            << linker << "' -> '" << target << "'";
 
   // Create a virtual ethernet pair for this container.
   Try<bool> createVethPair = link::create(veth(pid), eth0, pid);
@@ -2267,7 +2508,7 @@ Future<Nothing> PortMappingIsolatorProcess::cleanup(
   // hashmap infos before using the helper function to clean it up.
   infos.erase(containerId);
 
-  Try<Nothing> cleanup = _cleanup(info);
+  Try<Nothing> cleanup = _cleanup(info, containerId);
   if (cleanup.isError()) {
     return Failure(cleanup.error());
   }
@@ -2276,13 +2517,14 @@ Future<Nothing> PortMappingIsolatorProcess::cleanup(
 }
 
 
-// An old glibc might not have this symbol.
-#ifndef MNT_DETACH
-#define MNT_DETACH 2
-#endif
-
-
-Try<Nothing> PortMappingIsolatorProcess::_cleanup(Info* _info)
+// TODO(jieyu): We take an optional container ID here because not all
+// the containers we want to cleanup have container IDs available. For
+// instance, we cannot get container IDs for those orphan containers
+// created by older (pre 0.23.0) versions of this isolator (with no
+// associated namespace handle symlinks).
+Try<Nothing> PortMappingIsolatorProcess::_cleanup(
+    Info* _info,
+    const Option<ContainerID>& containerId)
 {
   // Set '_info' to be auto-managed so that it will be deleted when
   // this function returns.
@@ -2422,35 +2664,51 @@ Try<Nothing> PortMappingIsolatorProcess::_cleanup(Info* _info)
     }
   }
 
-  // Release the bind mount for this container.
-  const string target = path::join(
-      PORT_MAPPING_BIND_MOUNT_ROOT(),
-      stringify(pid));
+  // We manually remove veth to avoid having to wait for the kernel to
+  // do it.
+  Try<bool> remove = link::remove(veth(pid));
+  if (remove.isError()) {
+    errors.push_back(
+        "Failed to remove the link " + veth(pid) + ": " + remove.error());
+  }
+
+  // Remove the symlink for the network namespace handle if
+  // containerId is specified.
+  if (containerId.isSome()) {
+    const string linker = getSymlinkPath(containerId.get());
 
+    // NOTE: Since we introduced the network namespace handle symlink
+    // in 0.23.0, it's likely that the symlink does not exist.
+    if (os::exists(linker)) {
+      Try<Nothing> rm = os::rm(linker);
+      if (rm.isError()) {
+        errors.push_back(
+            "Failed to remove the network namespace symlink '" +
+            linker + "' " + rm.error());
+      }
+    }
+  }
+
+  // Release the bind mount for this container.
+  const string target = getNamespaceHandlePath(pid);
   Try<Nothing> unmount = fs::unmount(target, MNT_DETACH);
   if (unmount.isError()) {
-    errors.push_back("Failed to umount: " + unmount.error());
+    errors.push_back(
+        "Failed to unmount the network namespace handle '" +
+        target + "': " + unmount.error());
   }
 
-  // MNT_DETACH does a lazy umount, which means umount will eventually
-  // succeed when the mount point becomes idle, but possiblely not
-  // soon enough every time for this remove to go through, e.g,
-  // someone entered into the container for debugging purpose. In that
-  // case remove will fail, which is okay, because we only leaked an
-  // empty file, which could also be reused later if the pid (the name
-  // of the file) is used again.
+  // MNT_DETACH does a lazy unmount, which means unmount will
+  // eventually succeed when the mount point becomes idle, but
+  // possiblely not soon enough every time for this remove to go
+  // through, e.g, someone entered into the container for debugging
+  // purpose. In that case remove will fail, which is okay, because we
+  // only leaked an empty file, which could also be reused later if
+  // the pid (the name of the file) is used again.
   Try<Nothing> rm = os::rm(target);
   if (rm.isError()) {
-    LOG(WARNING) << "Failed to remove bind mount '" << target
-                 << "' during cleanup: " << rm.error();
-  }
-
-  // We manually remove veth to avoid having to wait for the kernel to
-  // do it.
-  Try<bool> remove = link::remove(veth(pid));
-  if (remove.isError()) {
-    errors.push_back(
-        "Failed to remove the link " + veth(pid) + ": " + remove.error());
+    LOG(WARNING) << "Failed to remove the network namespace handle '"
+                 << target << "' during cleanup: " << rm.error();
   }
 
   // If any error happens along the way, return error.

http://git-wip-us.apache.org/repos/asf/mesos/blob/de0307cb/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 4dd066a..33837b4 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.hpp
@@ -253,7 +253,7 @@ private:
       ephemeralPortsAllocator(_ephemeralPortsAllocator) {}
 
   // Continuations.
-  Try<Nothing> _cleanup(Info* info);
+  Try<Nothing> _cleanup(Info* info, const Option<ContainerID>& containerId);
   Try<Info*> _recover(pid_t pid);
 
   void _update(
@@ -317,7 +317,7 @@ private:
 class PortMappingUpdate : public Subcommand
 {
 public:
-  static const std::string NAME;
+  static const char* NAME;
 
   struct Flags : public flags::FlagsBase
   {
@@ -347,7 +347,7 @@ protected:
 class PortMappingStatistics : public Subcommand
 {
 public:
-  static const std::string NAME;
+  static const char* NAME;
 
   struct Flags : public flags::FlagsBase
   {

http://git-wip-us.apache.org/repos/asf/mesos/blob/de0307cb/src/tests/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/port_mapping_tests.cpp b/src/tests/port_mapping_tests.cpp
index 8fc854e..f4124c3 100644
--- a/src/tests/port_mapping_tests.cpp
+++ b/src/tests/port_mapping_tests.cpp
@@ -34,6 +34,9 @@
 #include <stout/mac.hpp>
 #include <stout/net.hpp>
 
+#include <stout/os/stat.hpp>
+#include <stout/os/exists.hpp>
+
 #include "linux/fs.hpp"
 
 #include "linux/routing/utils.hpp"
@@ -130,10 +133,12 @@ static void cleanup(const string& eth0, const string& lo)
 
     // NOTE: Here, we ignore the unmount errors because previous tests
     // may have created the file and died before mounting.
-    mesos::internal::fs::unmount(target, MNT_DETACH);
+    if (!os::stat::islink(target)) {
+      mesos::internal::fs::unmount(target, MNT_DETACH);
+    }
 
-    // Use best effort to remove the bind mount file, but it is okay
-    // the file can't be removed at this point.
+    // Remove the network namespace handle and the corresponding
+    // symlinks. The removal here is best effort.
     os::rm(target);
   }
 }
@@ -1983,6 +1988,91 @@ TEST_F(PortMappingMesosTest, ROOT_CleanUpOrphanTest)
   Shutdown();
 }
 
+
+// This test verfies the creation and destroy of the network namespace
+// handle symlink. The symlink was introduced in 0.23.0.
+TEST_F(PortMappingMesosTest, ROOT_NetworkNamespaceHandleSymlink)
+{
+  Try<PID<Master> > master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "network/port_mapping";
+
+  Try<MesosContainerizer*> containerizer =
+    MesosContainerizer::create(flags, true, &fetcher);
+
+  ASSERT_SOME(containerizer);
+
+  Try<PID<Slave>> slave = StartSlave(containerizer.get(), flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get(), DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(_, _, _));
+
+  Future<vector<Offer> > offers;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(DeclineOffers());      // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers.get().size());
+
+  // Start a long running task using network islator.
+  TaskInfo task = createTask(offers.get()[0], "sleep 1000");
+
+  Future<TaskStatus> status1;
+  Future<TaskStatus> status2;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&status1))
+    .WillOnce(FutureArg<1>(&status2))
+    .WillRepeatedly(Return());       // Ignore subsequent updates.
+
+  driver.launchTasks(offers.get()[0].id(), {task});
+
+  AWAIT_READY(status1);
+  EXPECT_EQ(task.task_id(), status1.get().task_id());
+  EXPECT_EQ(TASK_RUNNING, status1.get().state());
+
+  Future<hashset<ContainerID>> containers = containerizer.get()->containers();
+  AWAIT_READY(containers);
+  ASSERT_EQ(1u, containers.get().size());
+
+  ContainerID containerId = *(containers.get().begin());
+
+  const string symlink = path::join(
+      slave::PORT_MAPPING_BIND_MOUNT_ROOT(),
+      stringify(containerId));
+
+  EXPECT_TRUE(os::exists(symlink));
+  EXPECT_TRUE(os::stat::islink(symlink));
+
+  Future<containerizer::Termination> termination =
+    containerizer.get()->wait(containerId);
+
+  driver.killTask(task.task_id());
+
+  AWAIT_READY(status2);
+  EXPECT_EQ(task.task_id(), status2.get().task_id());
+  EXPECT_EQ(TASK_KILLED, status2.get().state());
+
+  AWAIT_READY(termination);
+  EXPECT_FALSE(os::exists(symlink));
+
+  driver.stop();
+  driver.join();
+
+  Shutdown();
+
+  delete containerizer.get();
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {