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 2018/03/27 06:34:29 UTC

[3/3] mesos git commit: Allowed a nested container to have a separate network namespace.

Allowed a nested container to have a separate network namespace.

Previously, nested containers always share the same network namespace as
their parent. This patch allows a nested container to have a separate
network namespace than its parent.

Continued from https://github.com/apache/mesos/pull/263

JIRA: MESOS-8534

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


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

Branch: refs/heads/master
Commit: a741b15e889de3242e3aa7878105ab9d946f6ea2
Parents: 020b8cb
Author: Sagar Patwardhan <sa...@gmail.com>
Authored: Mon Mar 26 21:13:17 2018 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Mar 26 23:29:01 2018 -0700

----------------------------------------------------------------------
 src/master/validation.cpp                       |   9 +-
 .../mesos/isolators/network/cni/cni.cpp         | 463 ++++++++++---------
 .../mesos/isolators/network/cni/cni.hpp         |  11 +-
 .../mesos/isolators/network/cni/paths.cpp       |  31 +-
 .../mesos/isolators/network/cni/paths.hpp       |  66 +--
 src/tests/containerizer/cni_isolator_tests.cpp  | 173 ++++++-
 src/tests/master_validation_tests.cpp           | 173 +++++--
 7 files changed, 614 insertions(+), 312 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a741b15e/src/master/validation.cpp
----------------------------------------------------------------------
diff --git a/src/master/validation.cpp b/src/master/validation.cpp
index 9254cd2..ac2e1bb 100644
--- a/src/master/validation.cpp
+++ b/src/master/validation.cpp
@@ -1501,8 +1501,13 @@ Option<Error> validateTask(
   }
 
   if (task.has_container()) {
-    if (task.container().network_infos().size() > 0) {
-      return Error("NetworkInfos must not be set on the task");
+    if (!task.container().network_infos().empty()) {
+      if (task.has_health_check() &&
+          (task.health_check().type() == HealthCheck::HTTP ||
+           task.health_check().type() == HealthCheck::TCP)) {
+        return Error("HTTP and TCP health checks are not supported for "
+                     "nested containers not joining parent's network");
+      }
     }
 
     if (task.container().type() == ContainerInfo::DOCKER) {

http://git-wip-us.apache.org/repos/asf/mesos/blob/a741b15e/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
index 3a14887..6905cd6 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.cpp
@@ -14,6 +14,8 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include "slave/containerizer/mesos/isolators/network/cni/cni.hpp"
+
 #include <iostream>
 #include <list>
 #include <set>
@@ -42,8 +44,6 @@
 #include "linux/fs.hpp"
 #include "linux/ns.hpp"
 
-#include "slave/containerizer/mesos/isolators/network/cni/cni.hpp"
-
 namespace io = process::io;
 namespace paths = mesos::internal::slave::cni::paths;
 namespace spec = mesos::internal::slave::cni::spec;
@@ -391,20 +391,9 @@ Future<Nothing> NetworkCniIsolatorProcess::recover(
     return Nothing();
   }
 
+  hashmap<ContainerID, ContainerState> containerIdToState;
   foreach (const ContainerState& state, states) {
-    const ContainerID& containerId = state.container_id();
-
-    if (containerId.has_parent()) {
-      // We do not need to recover nested containers.
-      continue;
-    }
-
-    Try<Nothing> recover = _recover(containerId, state);
-    if (recover.isError()) {
-      return Failure(
-          "Failed to recover CNI network information for container " +
-          stringify(containerId) + ": " + recover.error());
-    }
+    containerIdToState.put(state.container_id(), state);
   }
 
   Try<list<string>> entries = os::ls(rootDir.get());
@@ -415,19 +404,28 @@ Future<Nothing> NetworkCniIsolatorProcess::recover(
   }
 
   foreach (const string& entry, entries.get()) {
-    ContainerID containerId;
-    containerId.set_value(Path(entry).basename());
+    ContainerID containerId =
+      protobuf::parseContainerId(Path(entry).basename());
 
-    if (infos.contains(containerId)) {
-      continue;
-    }
+    if (containerIdToState.contains(containerId)) {
+      // This container is attached to a non-host network.
+      Try<Nothing> recover = _recover(
+          containerId,
+          containerIdToState[containerId]);
 
-    // Recover CNI network information for orphan container.
-    Try<Nothing> recover = _recover(containerId);
-    if (recover.isError()) {
-      return Failure(
-          "Failed to recover CNI network information for orphan container " +
-          stringify(containerId) + ": " + recover.error());
+      if (recover.isError()) {
+        return Failure(
+            "Failed to recover CNI network information for the container " +
+            stringify(containerId) + ": " + recover.error());
+      }
+    } else {
+      // Recover CNI network information for orphan container.
+      Try<Nothing> recover = _recover(containerId);
+      if (recover.isError()) {
+        return Failure(
+            "Failed to recover CNI network information for orphaned the "
+            "container " + stringify(containerId) + ": " + recover.error());
+      }
     }
 
     // Known orphan containers will be cleaned up by containerizer
@@ -456,7 +454,7 @@ Try<Nothing> NetworkCniIsolatorProcess::_recover(
   // an 'Info' to 'infos' and the corresponding 'cleanup' will be skipped.
 
   const string containerDir =
-    paths::getContainerDir(rootDir.get(), containerId.value());
+    paths::getContainerDir(rootDir.get(), containerId);
 
   if (!os::exists(containerDir)) {
     // This may occur in the following cases:
@@ -472,7 +470,7 @@ Try<Nothing> NetworkCniIsolatorProcess::_recover(
   }
 
   Try<list<string>> networkNames =
-    paths::getNetworkNames(rootDir.get(), containerId.value());
+    paths::getNetworkNames(rootDir.get(), containerId);
 
   if (networkNames.isError()) {
     return Error("Failed to list CNI network names: " + networkNames.error());
@@ -482,7 +480,7 @@ Try<Nothing> NetworkCniIsolatorProcess::_recover(
   foreach (const string& networkName, networkNames.get()) {
     Try<list<string>> interfaces = paths::getInterfaces(
         rootDir.get(),
-        containerId.value(),
+        containerId,
         networkName);
 
     if (interfaces.isError()) {
@@ -522,7 +520,7 @@ Try<Nothing> NetworkCniIsolatorProcess::_recover(
 
     const string networkInfoPath = paths::getNetworkInfoPath(
         rootDir.get(),
-        containerId.value(),
+        containerId,
         containerNetwork.networkName,
         containerNetwork.ifName);
 
@@ -565,7 +563,11 @@ Try<Nothing> NetworkCniIsolatorProcess::_recover(
   // directories but before it is able to unmount the namespace handle and
   // remove the container directory. In that case, we still rely on 'cleanup'
   // to clean it up.
-  infos.put(containerId, Owned<Info>(new Info(containerNetworks)));
+  infos.put(containerId, Owned<Info>(new Info(
+      containerNetworks,
+      None(),
+      None(),
+      false)));
 
   return Nothing();
 }
@@ -582,89 +584,89 @@ Future<Option<ContainerLaunchInfo>> NetworkCniIsolatorProcess::prepare(
   hashmap<string, ContainerNetwork> containerNetworks;
   Option<string> hostname;
 
-  if (!containerId.has_parent()) {
-    const ExecutorInfo& executorInfo = containerConfig.executor_info();
-    if (!executorInfo.has_container()) {
-      return None();
-    }
+  const bool isNestedContainer = containerId.has_parent();
 
-    if (executorInfo.container().type() != ContainerInfo::MESOS) {
-      return Failure("Can only prepare CNI networks for a MESOS container");
-    }
+  const bool isDebugContainer =
+    containerConfig.container_class() == ContainerClass::DEBUG;
+
+  // Debug container always joins parent's network regardless if it
+  // sets ContainerInfo or not.
+  const bool joinsParentsNetwork =
+    isDebugContainer ||
+    !containerConfig.has_container_info() ||
+    containerConfig.container_info().network_infos().empty();
 
-    if (executorInfo.container().has_hostname()) {
-      hostname = executorInfo.container().hostname();
+  if (isNestedContainer && joinsParentsNetwork) {
+    // This is a nested container that wants to join parent's network.
+    //
+    // The `network/cni` isolator only checkpoints containers that
+    // joins the non-host networks. Therefore, after `recover` it can
+    // happen that `infos` does not contain top-level containers that
+    // have joined the host-network.  Hence, we cannot return a
+    // `Failure` here if we do not find the `rootContainerId` in
+    // `infos`. If the `rootContainerId` is not found, it's implied
+    // that the root container is a container attached to the host
+    // network, which in turn implies that `containerNetworks` should
+    // be left empty.
+    ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
+    if (infos.contains(rootContainerId)) {
+      containerNetworks = infos[rootContainerId]->containerNetworks;
     }
+  } else {
+    // This is either a top level container, or a nested container
+    // joining separate network than its parent.
+    if (containerConfig.has_container_info()) {
+      const ContainerInfo& containerInfo = containerConfig.container_info();
 
-    int ifIndex = 0;
-    foreach (const mesos::NetworkInfo& networkInfo,
-             executorInfo.container().network_infos()) {
-      if (!networkInfo.has_name()) {
-        continue;
+      if (containerInfo.type() != ContainerInfo::MESOS) {
+        return Failure("Can only prepare CNI networks for a MESOS container");
       }
 
-      const string& name = networkInfo.name();
-
-      Try<JSON::Object> networkConfigJSON = getNetworkConfigJSON(name);
-      if (networkConfigJSON.isError()) {
-        return Failure(networkConfigJSON.error());
+      if (containerInfo.has_hostname()) {
+        hostname = containerInfo.hostname();
       }
 
-      if (containerNetworks.contains(name)) {
-        return Failure(
-            "Attempted to join CNI network '" + name + "' multiple times");
-      }
+      int ifIndex = 0;
+      foreach (const mesos::NetworkInfo& networkInfo,
+               containerInfo.network_infos()) {
+        if (!networkInfo.has_name()) {
+          continue;
+        }
 
-      ContainerNetwork containerNetwork;
-      containerNetwork.networkName = name;
-      containerNetwork.ifName = "eth" + stringify(ifIndex++);
-      containerNetwork.networkInfo = networkInfo;
+        const string& name = networkInfo.name();
 
-      containerNetworks.put(name, containerNetwork);
-    }
-  } else {
-    // This is a nested container. If the `NetworkInfo` in
-    // `ContainerConfig.container` is set, it implies that the nested
-    // container needs to have a separate network namespace, else the
-    // nested container shares its network namespace with the parent.
-    if (containerConfig.has_container_info() &&
-        containerConfig.container_info().network_infos().size() > 0) {
-      return Failure(
-          "Currently, we don't support different network namespaces for "
-          "parent and nested containers.");
-    }
+        Try<JSON::Object> networkConfigJSON = getNetworkConfigJSON(name);
+        if (networkConfigJSON.isError()) {
+          return Failure(networkConfigJSON.error());
+        }
 
-    ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
+        if (containerNetworks.contains(name)) {
+          return Failure(
+              "Attempted to join CNI network '" + name + "' multiple times");
+        }
 
-    // NOTE: The `network/cni` isolator checkpoints only the following
-    // top-level containers:
-    // * Containers joining the host network with an image.
-    // * Containers joining a non-host network.
-    //
-    // Therefore, after `recover` it can happen that `infos` does not
-    // contain top-level containers that have joined the host-network.
-    // Hence, we cannot return a `Failure` here if we do not find the
-    // `rootContainerId` in `infos`. If the `rootContainerId` is not
-    // found, it's implied that the root container is a container
-    // attached to the host network, which in turn implies that
-    // `containerNetworks` should be left empty.
-    if (infos.contains(rootContainerId)) {
-      containerNetworks = infos[rootContainerId]->containerNetworks;
+        ContainerNetwork containerNetwork;
+        containerNetwork.networkName = name;
+        containerNetwork.ifName = "eth" + stringify(ifIndex++);
+        containerNetwork.networkInfo = networkInfo;
+
+        containerNetworks.put(name, containerNetwork);
+      }
     }
   }
 
   // There are two groups of cases that need to be handled when
   // attaching containers to networks.
-  // * Cases where the containers don't need a new mount namespace:
-  //    a) Containers (nested or stand alone) join the host network
-  //       without an image.
-  //    b) Nested DEBUG containers join the same mount namespace as
-  //       their parent.
-  // * Cases where the container needs a new mount namespace:
-  //    a) Containers (nested or stand alone) join the host network
-  //       with an image.
-  //    b) Containers (nested or stand alone) join a non-host network,
-  //       with or without image.
+  //   * Cases where the containers don't need a new mount namespace:
+  //      a) Containers (nested or top level) join the host network
+  //         without an image.
+  //      b) Nested DEBUG containers join the same mount namespace as
+  //         their parent.
+  //   * Cases where the container needs a new mount namespace:
+  //      a) Containers (nested or top level) join the host network
+  //         with an image.
+  //      b) Containers (nested or top level) join a non-host network,
+  //         with or without image.
   //
   // The `network/cni` isolator will add any container needing a new
   // mount namespace to the `infos` structure. Reason being that for
@@ -676,7 +678,12 @@ Future<Option<ContainerLaunchInfo>> NetworkCniIsolatorProcess::prepare(
     // to join host network, we will make sure it has access to host
     // /etc/* files.
     if (containerConfig.has_rootfs()) {
-      Owned<Info> info(new Info(containerNetworks, containerConfig.rootfs()));
+      Owned<Info> info(new Info(
+          containerNetworks,
+          containerConfig.rootfs(),
+          None(),
+          joinsParentsNetwork));
+
       infos.put(containerId, info);
     }
 
@@ -689,93 +696,94 @@ Future<Option<ContainerLaunchInfo>> NetworkCniIsolatorProcess::prepare(
     // rootfs, it will join the host network and there are no
     // namespaces it needs to enter.
     return None();
-  } else {
-    // This is the case where the container is joining a non-host
-    // network namespace. Non-DEBUG containers will need a new mount
-    // namespace to bind mount their network files (/etc/hosts,
-    // /etc/hostname, /etc/resolv.conf) which will be different than
-    // those on the host file system.
-    //
-    // Unlike other isolators, we can't simply rely on the
-    // `filesystem/linux` isolator to give this container a new
-    // mount namespace (because we allow the `filesystem/posix`
-    // isolator to be used here). We must set the clone flags
-    // ourselves explicitly.
-
-    if (containerId.has_parent() &&
-        containerConfig.has_container_class() &&
-        containerConfig.container_class() == ContainerClass::DEBUG) {
-      // Nested DEBUG containers never need a new MOUNT namespace, so
-      // we don't maintain information about them in the `infos` map.
-    } else {
-      Option<string> rootfs = None();
+  }
 
-      if (containerConfig.has_rootfs()) {
-        rootfs = containerConfig.rootfs();
-      }
+  // This is the case where the container is joining a non-host
+  // network namespace. Non-DEBUG containers will need a new mount
+  // namespace to bind mount their network files (/etc/hosts,
+  // /etc/hostname, /etc/resolv.conf) which will be different than
+  // those on the host file system.
+  //
+  // Unlike other isolators, we can't simply rely on the
+  // `filesystem/linux` isolator to give this container a new
+  // mount namespace (because we allow the `filesystem/posix`
+  // isolator to be used here). We must set the clone flags
+  // ourselves explicitly.
+  if (isDebugContainer) {
+    CHECK(isNestedContainer);
+
+    // Nested DEBUG containers never need a new MOUNT namespace, so
+    // we don't maintain information about them in the `infos` map.
+  } else {
+    Option<string> rootfs = None();
 
-      infos.put(containerId, Owned<Info>(
-          new Info(containerNetworks, rootfs, hostname)));
+    if (containerConfig.has_rootfs()) {
+      rootfs = containerConfig.rootfs();
     }
 
-    ContainerLaunchInfo launchInfo;
-
-    // Reset the `LIBPROCESS_IP` in the environment variable, so that
-    // the container binds to the IP address allocated by the CNI
-    // plugin. See MESOS-3553 to understand why we need to reset the
-    // `LIBPROCESS_IP`.
-    Environment_Variable* env =
-      launchInfo.mutable_environment()->add_variables();
-
-    env->set_name("LIBPROCESS_IP");
-    env->set_value("0.0.0.0");
-
-    if (!containerId.has_parent()) {
-      auto mesosTestNetwork = [=]() {
-        foreachkey (const string& networkName, containerNetworks) {
-          // We can specify test networks to the `network/cni` isolator
-          // with a name of the form "__MESOS_TEST__*".  For these test
-          // networks we will use a mock CNI plugin and a mock CNI
-          // network configuration file which has "__MESOS_TEST__*" as
-          // network name. The mock plugin will not create a new network
-          // namespace for the container. The container will be launched
-          // in the host's network namespace. The mock plugin will
-          // return the host's IP address for this test container.
-          //
-          // NOTE: There is an implicit assumption here that when used
-          // for testing, '__MESOS_TEST__*' are the only networks the
-          // container is going to join.
-          if (strings::contains(networkName, "__MESOS_TEST__")) {
-            return true;
-          }
+    infos.put(containerId, Owned<Info>(new Info(
+        containerNetworks,
+        rootfs,
+        hostname,
+        joinsParentsNetwork)));
+  }
+
+  ContainerLaunchInfo launchInfo;
+
+  // Reset the `LIBPROCESS_IP` in the environment variable, so that
+  // the container binds to the IP address allocated by the CNI
+  // plugin. See MESOS-3553 to understand why we need to reset the
+  // `LIBPROCESS_IP`.
+  Environment_Variable* env =
+    launchInfo.mutable_environment()->add_variables();
+
+  env->set_name("LIBPROCESS_IP");
+  env->set_value("0.0.0.0");
+
+  if (!isDebugContainer && (!isNestedContainer || !joinsParentsNetwork)) {
+    auto mesosTestNetwork = [=]() {
+      foreachkey (const string& networkName, containerNetworks) {
+        // We can specify test networks to the `network/cni` isolator
+        // with a name of the form "__MESOS_TEST__*".  For these test
+        // networks we will use a mock CNI plugin and a mock CNI
+        // network configuration file which has "__MESOS_TEST__*" as
+        // network name. The mock plugin will not create a new network
+        // namespace for the container. The container will be launched
+        // in the host's network namespace. The mock plugin will
+        // return the host's IP address for this test container.
+        //
+        // NOTE: There is an implicit assumption here that when used
+        // for testing, '__MESOS_TEST__*' are the only networks the
+        // container is going to join.
+        if (strings::contains(networkName, "__MESOS_TEST__")) {
+          return true;
         }
+      }
 
-        return false;
-      };
+      return false;
+    };
 
-      if (mesosTestNetwork()) {
-        launchInfo.add_clone_namespaces(CLONE_NEWNS);
-        launchInfo.add_clone_namespaces(CLONE_NEWUTS);
-      } else {
-        launchInfo.add_clone_namespaces(CLONE_NEWNET);
-        launchInfo.add_clone_namespaces(CLONE_NEWNS);
-        launchInfo.add_clone_namespaces(CLONE_NEWUTS);
-      }
+    if (mesosTestNetwork()) {
+      launchInfo.add_clone_namespaces(CLONE_NEWNS);
+      launchInfo.add_clone_namespaces(CLONE_NEWUTS);
     } else {
-      // This is a nested container. This shares the parent's network
-      // and UTS namespace. For non-DEBUG containers it also needs a
-      // new mount namespace.
-      launchInfo.add_enter_namespaces(CLONE_NEWNET);
-      launchInfo.add_enter_namespaces(CLONE_NEWUTS);
-
-      if (!containerConfig.has_container_class() ||
-          containerConfig.container_class() != ContainerClass::DEBUG) {
-        launchInfo.add_clone_namespaces(CLONE_NEWNS);
-      }
+      launchInfo.add_clone_namespaces(CLONE_NEWNS);
+      launchInfo.add_clone_namespaces(CLONE_NEWNET);
+      launchInfo.add_clone_namespaces(CLONE_NEWUTS);
+    }
+  } else {
+    // This is a nested container and wants to share parent's network
+    // and UTS namespace. For non-DEBUG containers, it also needs a
+    // new mount namespace.
+    launchInfo.add_enter_namespaces(CLONE_NEWNET);
+    launchInfo.add_enter_namespaces(CLONE_NEWUTS);
+
+    if (!isDebugContainer) {
+      launchInfo.add_clone_namespaces(CLONE_NEWNS);
     }
-
-    return launchInfo;
   }
+
+  return launchInfo;
 }
 
 
@@ -790,6 +798,8 @@ Future<Nothing> NetworkCniIsolatorProcess::isolate(
     return Nothing();
   }
 
+  const bool isNestedContainer = containerId.has_parent();
+
   // We first deal with containers (both top level or nested) that
   // want to join the host network. Given the above 'contains' check,
   // the container here must have rootfs defined (otherwise, we won't
@@ -829,18 +839,18 @@ Future<Nothing> NetworkCniIsolatorProcess::isolate(
   CHECK_SOME(rootDir);
   CHECK_SOME(pluginDir);
 
-  if (containerId.has_parent()) {
+  // NOTE: DEBUG container should not have Info struct. Thus if the
+  // control reaches here, the container is not a DEBUG container.
+  if (isNestedContainer && infos[containerId]->joinsParentsNetwork) {
     // We create network files for only those containers for which we
-    // create a new network namespace. Therefore, in a nested
-    // container hierarchy only the container at the root of the
-    // hierarchy (the top level container) would have network files
-    // created. Hence, find the top level container for the hierarchy
-    // to which this container belongs. We will use the network files
-    // of the top level root container to setup the network files for
-    // this nested container.
+    // create a new network namespace. Therefore, for nested
+    // containers that want to share network namespace with the parent
+    // container, find the top level container in the nested container
+    // hierarchy. We will use the network files of the top level root
+    // container to setup the network files for this nested container.
     ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
 
-    // Since the nested container joins non-host networks, its root
+    // Since this nested container wants to join non-host networks, its root
     // container has to join non-host networks because we have the
     // invariant that all containers in a hierarchy join the same
     // networks.
@@ -848,7 +858,7 @@ Future<Nothing> NetworkCniIsolatorProcess::isolate(
 
     const string rootContainerDir = paths::getContainerDir(
         rootDir.get(),
-        rootContainerId.value());
+        rootContainerId);
 
     CHECK(os::exists(rootContainerDir));
 
@@ -874,7 +884,7 @@ Future<Nothing> NetworkCniIsolatorProcess::isolate(
     // Setup the required network files and the hostname in the
     // container's filesystem and UTS namespace.
     //
-    // NOTE: Since nested containers share the UTS and network
+    // NOTE: For nested container that want to share the UTS and network
     // namespace with their root container, we do not need to setup
     // the hostname here. The hostname should have already been setup
     // when setting up the network namespace for the root container.
@@ -894,7 +904,7 @@ Future<Nothing> NetworkCniIsolatorProcess::isolate(
 
   // Create the container directory.
   const string containerDir =
-    paths::getContainerDir(rootDir.get(), containerId.value());
+    paths::getContainerDir(rootDir.get(), containerId);
 
   Try<Nothing> mkdir = os::mkdir(containerDir);
   if (mkdir.isError()) {
@@ -907,8 +917,7 @@ Future<Nothing> NetworkCniIsolatorProcess::isolate(
   // /var/run/mesos/isolators/network/cni/<containerId>/ns to hold an extra
   // reference to the network namespace which will be released in '_cleanup'.
   const string source = path::join("/proc", stringify(pid), "ns", "net");
-  const string target =
-    paths::getNamespacePath(rootDir.get(), containerId.value());
+  const string target = paths::getNamespacePath(rootDir.get(), containerId);
 
   Try<Nothing> touch = os::touch(target);
   if (touch.isError()) {
@@ -965,12 +974,17 @@ Future<Nothing> NetworkCniIsolatorProcess::_isolate(
   CHECK(infos.contains(containerId));
 
   const Owned<Info>& info = infos[containerId];
+
+  // ContainerId for nested containers is in this format:
+  // "parent.child". To ensure that it is not longer than
+  // HOST_NAME_MAX, we remove the parent's container ID from the
+  // hostname.
   string hostname = info->hostname.isSome()
     ? info->hostname.get()
-    : stringify(containerId);
+    : containerId.value();
 
   const string containerDir =
-    paths::getContainerDir(rootDir.get(), containerId.value());
+    paths::getContainerDir(rootDir.get(), containerId);
 
   CHECK(os::exists(containerDir));
 
@@ -1161,7 +1175,7 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
 
   const string ifDir = paths::getInterfaceDir(
       rootDir.get(),
-      containerId.value(),
+      containerId,
       networkName,
       containerNetwork.ifName);
 
@@ -1176,7 +1190,7 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
   // Prepare environment variables for CNI plugin.
   map<string, string> environment;
   environment["CNI_COMMAND"] = "ADD";
-  environment["CNI_CONTAINERID"] = containerId.value();
+  environment["CNI_CONTAINERID"] = stringify(containerId);
   environment["CNI_PATH"] = pluginDir.get();
   environment["CNI_IFNAME"] = containerNetwork.ifName;
   environment["CNI_NETNS"] = netNsHandle;
@@ -1248,7 +1262,7 @@ Future<Nothing> NetworkCniIsolatorProcess::attach(
   // the same JSON during cleanup.
   const string networkConfigPath = paths::getNetworkConfigPath(
       rootDir.get(),
-      containerId.value(),
+      containerId,
       networkName);
 
   Try<Nothing> write =
@@ -1368,7 +1382,7 @@ Future<Nothing> NetworkCniIsolatorProcess::_attach(
 
   const string networkInfoPath = paths::getNetworkInfoPath(
       rootDir.get(),
-      containerId.value(),
+      containerId,
       networkName,
       containerNetwork.ifName);
 
@@ -1388,26 +1402,25 @@ Future<Nothing> NetworkCniIsolatorProcess::_attach(
 Future<ContainerStatus> NetworkCniIsolatorProcess::status(
     const ContainerID& containerId)
 {
-  // NOTE: Currently, nested containers share their network namespace
-  // with the parent containers in the hierarchy to which they belong.
-  // Hence, in order to obtain the IP address of this nested container
-  // one should always look up the IP address of the root container of
-  // the hierarchy to which this container belongs.
-  //
-  // TODO(jieyu): Revisit this once we allow nested containers to use
-  // different network namespaces than their parent container.
-  if (containerId.has_parent()) {
-    return status(containerId.parent());
-  }
-
-  // TODO(jieyu): We don't create 'Info' struct for containers that want
-  // to join the host network and have no image. Currently, we rely on
-  // the slave/containerizer to set the IP addresses in ContainerStatus.
-  // Consider returning the IP address of the slave here.
+  // TODO(jieyu): We don't create 'Info' struct for containers that
+  // want to join the host network and have no image. Currently, we
+  // rely on the slave/containerizer to set the IP addresses in
+  // ContainerStatus.  Consider returning the IP address of the slave
+  // here.
   if (!infos.contains(containerId)) {
     return ContainerStatus();
   }
 
+  // In order to obtain the IP address of this nested container that
+  // wants to joins the parent's network, we should look up the IP
+  // address of the root container of the hierarchy to which this
+  // container belongs.
+  const bool isNestedContainer = containerId.has_parent();
+
+  if (isNestedContainer && infos[containerId]->joinsParentsNetwork) {
+    return status(containerId.parent());
+  }
+
   ContainerStatus status;
   foreachvalue (const ContainerNetwork& containerNetwork,
                 infos[containerId]->containerNetworks) {
@@ -1455,16 +1468,20 @@ Future<ContainerStatus> NetworkCniIsolatorProcess::status(
 Future<Nothing> NetworkCniIsolatorProcess::cleanup(
     const ContainerID& containerId)
 {
-  // NOTE: We don't keep an Info struct if the container is on the host network
-  // and has no image, or if during recovery, we found that the cleanup for
-  // this container is not required anymore (e.g., cleanup is done already, but
-  // the slave crashed and didn't realize that it's done).
+  // NOTE: We don't keep an Info struct if the container is on the
+  // host network and has no image, or if during recovery, we found
+  // that the cleanup for this container is not required anymore
+  // (e.g., cleanup is done already, but the slave crashed and didn't
+  // realize that it's done).
   if (!infos.contains(containerId)) {
     return Nothing();
   }
 
-  // For nested containers, we just need to remove it from `infos`.
-  if (containerId.has_parent()) {
+  // For nested containers that joins parent's network, we just need
+  // to remove it from `infos`.
+  const bool isNestedContainer = containerId.has_parent();
+
+  if (isNestedContainer && infos[containerId]->joinsParentsNetwork) {
     infos.erase(containerId);
     return Nothing();
   }
@@ -1513,10 +1530,10 @@ Future<Nothing> NetworkCniIsolatorProcess::_cleanup(
   }
 
   const string containerDir =
-    paths::getContainerDir(rootDir.get(), containerId.value());
+    paths::getContainerDir(rootDir.get(), containerId);
 
   const string target =
-    paths::getNamespacePath(rootDir.get(), containerId.value());
+    paths::getNamespacePath(rootDir.get(), containerId);
 
   if (os::exists(target)) {
     Try<Nothing> unmount = fs::unmount(target);
@@ -1558,11 +1575,11 @@ Future<Nothing> NetworkCniIsolatorProcess::detach(
   // Prepare environment variables for CNI plugin.
   map<string, string> environment;
   environment["CNI_COMMAND"] = "DEL";
-  environment["CNI_CONTAINERID"] = containerId.value();
+  environment["CNI_CONTAINERID"] = stringify(containerId);
   environment["CNI_PATH"] = pluginDir.get();
   environment["CNI_IFNAME"] = containerNetwork.ifName;
   environment["CNI_NETNS"] =
-      paths::getNamespacePath(rootDir.get(), containerId.value());
+      paths::getNamespacePath(rootDir.get(), containerId);
 
   // Some CNI plugins need to run "iptables" to set up IP Masquerade, so we
   // need to set the "PATH" environment variable so that the plugin can locate
@@ -1578,7 +1595,7 @@ Future<Nothing> NetworkCniIsolatorProcess::detach(
   // CNI plugin to detach the container from the CNI network.
   const string networkConfigPath = paths::getNetworkConfigPath(
       rootDir.get(),
-      containerId.value(),
+      containerId,
       networkName);
 
   Try<JSON::Object> networkConfigJSON = getNetworkConfigJSON(
@@ -1673,7 +1690,7 @@ Future<Nothing> NetworkCniIsolatorProcess::_detach(
   if (status.get() == 0) {
     const string ifDir = paths::getInterfaceDir(
         rootDir.get(),
-        containerId.value(),
+        containerId,
         networkName,
         infos[containerId]->containerNetworks[networkName].ifName);
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a741b15e/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
index 1d01915..ea5397d 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/cni.hpp
@@ -92,10 +92,12 @@ private:
   {
     Info (const hashmap<std::string, ContainerNetwork>& _containerNetworks,
           const Option<std::string>& _rootfs = None(),
-          const Option<std::string>& _hostname = None())
+          const Option<std::string>& _hostname = None(),
+          bool _joinsParentsNetwork = false)
       : containerNetworks (_containerNetworks),
         rootfs(_rootfs),
-        hostname(_hostname) {}
+        hostname(_hostname),
+        joinsParentsNetwork(_joinsParentsNetwork) {}
 
     // CNI network information keyed by network name.
     //
@@ -110,6 +112,7 @@ private:
     const Option<std::string> rootfs;
 
     const Option<std::string> hostname;
+    const bool joinsParentsNetwork;
   };
 
   // Reads each CNI config present in `configDir`, validates if the
@@ -140,7 +143,7 @@ private:
   process::Future<Nothing> _isolate(
       const ContainerID& containerId,
       pid_t pid,
-      const list<process::Future<Nothing>>& attaches);
+      const std::list<process::Future<Nothing>>& attaches);
 
   process::Future<Nothing> __isolate(
       const NetworkCniIsolatorSetup& setup);
@@ -202,7 +205,7 @@ private:
   hashmap<std::string, std::string> networkConfigs;
 
   // DNS informations of CNI networks keyed by CNI network name.
-  hashmap<string, ContainerDNSInfo::MesosInfo> cniDNSMap;
+  hashmap<std::string, ContainerDNSInfo::MesosInfo> cniDNSMap;
 
   // Default DNS information for all CNI networks.
   const Option<ContainerDNSInfo::MesosInfo> defaultCniDNS;

http://git-wip-us.apache.org/repos/asf/mesos/blob/a741b15e/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp b/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
index f9056c9..c1cb4f7 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/paths.cpp
@@ -14,11 +14,14 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-#include <stout/path.hpp>
-#include <stout/fs.hpp>
-
 #include "slave/containerizer/mesos/isolators/network/cni/paths.hpp"
 
+#include <mesos/type_utils.hpp>
+
+#include <stout/fs.hpp>
+#include <stout/path.hpp>
+#include <stout/stringify.hpp>
+
 using std::string;
 using std::list;
 
@@ -28,13 +31,17 @@ namespace slave {
 namespace cni {
 namespace paths {
 
-string getContainerDir(const string& rootDir, const string& containerId)
+string getContainerDir(
+    const string& rootDir,
+    const ContainerID& containerId)
 {
-  return path::join(rootDir, containerId);
+  return path::join(rootDir, stringify(containerId));
 }
 
 
-string getNamespacePath(const string& rootDir, const string& containerId)
+string getNamespacePath(
+    const string& rootDir,
+    const ContainerID& containerId)
 {
   return path::join(getContainerDir(rootDir, containerId), "ns");
 }
@@ -42,7 +49,7 @@ string getNamespacePath(const string& rootDir, const string& containerId)
 
 string getNetworkDir(
     const string& rootDir,
-    const string& containerId,
+    const ContainerID& containerId,
     const string& networkName)
 {
   return path::join(getContainerDir(rootDir, containerId), networkName);
@@ -51,7 +58,7 @@ string getNetworkDir(
 
 Try<list<string>> getNetworkNames(
     const string& rootDir,
-    const string& containerId)
+    const ContainerID& containerId)
 {
   const string& networkInfoDir = getContainerDir(rootDir, containerId);
 
@@ -77,7 +84,7 @@ Try<list<string>> getNetworkNames(
 
 string getNetworkConfigPath(
     const string& rootDir,
-    const string& containerId,
+    const ContainerID& containerId,
     const string& networkName)
 {
   return path::join(
@@ -88,7 +95,7 @@ string getNetworkConfigPath(
 
 string getInterfaceDir(
     const string& rootDir,
-    const string& containerId,
+    const ContainerID& containerId,
     const string& networkName,
     const string& ifName)
 {
@@ -98,7 +105,7 @@ string getInterfaceDir(
 
 Try<list<string>> getInterfaces(
     const string& rootDir,
-    const string& containerId,
+    const ContainerID& containerId,
     const string& networkName)
 {
   const string& networkDir = getNetworkDir(rootDir, containerId, networkName);
@@ -125,7 +132,7 @@ Try<list<string>> getInterfaces(
 
 string getNetworkInfoPath(
     const string& rootDir,
-    const string& containerId,
+    const ContainerID& containerId,
     const string& networkName,
     const string& ifName)
 {

http://git-wip-us.apache.org/repos/asf/mesos/blob/a741b15e/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp b/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
index 7678a7c..3b06fb1 100644
--- a/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
+++ b/src/slave/containerizer/mesos/isolators/network/cni/paths.hpp
@@ -17,8 +17,12 @@
 #ifndef __ISOLATOR_CNI_PATHS_HPP__
 #define __ISOLATOR_CNI_PATHS_HPP__
 
-using std::string;
-using std::list;
+#include <list>
+#include <string>
+
+#include <mesos/mesos.hpp>
+
+#include <stout/try.hpp>
 
 namespace mesos {
 namespace internal {
@@ -44,47 +48,51 @@ namespace paths {
 constexpr char ROOT_DIR[] = "/var/run/mesos/isolators/network/cni";
 
 
-string getContainerDir(const string& rootDir, const string& containerId);
+std::string getContainerDir(
+    const std::string& rootDir,
+    const ContainerID& containerId);
 
 
-string getNamespacePath(const string& rootDir, const string& containerId);
+std::string getNamespacePath(
+    const std::string& rootDir,
+    const ContainerID& containerId);
 
 
-string getNetworkDir(
-    const string& rootDir,
-    const string& containerId,
-    const string& networkName);
+std::string getNetworkDir(
+    const std::string& rootDir,
+    const ContainerID& containerId,
+    const std::string& networkName);
 
 
-Try<list<string>> getNetworkNames(
-    const string& rootDir,
-    const string& containerId);
+Try<std::list<std::string>> getNetworkNames(
+    const std::string& rootDir,
+    const ContainerID& containerId);
 
 
-string getNetworkConfigPath(
-    const string& rootDir,
-    const string& containerId,
-    const string& networkName);
+std::string getNetworkConfigPath(
+    const std::string& rootDir,
+    const ContainerID& containerId,
+    const std::string& networkName);
 
 
-string getInterfaceDir(
-    const string& rootDir,
-    const string& containerId,
-    const string& networkName,
-    const string& ifName);
+std::string getInterfaceDir(
+    const std::string& rootDir,
+    const ContainerID& containerId,
+    const std::string& networkName,
+    const std::string& ifName);
 
 
-Try<list<string>> getInterfaces(
-    const string& rootDir,
-    const string& containerId,
-    const string& networkName);
+Try<std::list<std::string>> getInterfaces(
+    const std::string& rootDir,
+    const ContainerID& containerId,
+    const std::string& networkName);
 
 
-string getNetworkInfoPath(
-    const string& rootDir,
-    const string& containerId,
-    const string& networkName,
-    const string& ifName);
+std::string getNetworkInfoPath(
+    const std::string& rootDir,
+    const ContainerID& containerId,
+    const std::string& networkName,
+    const std::string& ifName);
 
 } // namespace paths {
 } // namespace cni {

http://git-wip-us.apache.org/repos/asf/mesos/blob/a741b15e/src/tests/containerizer/cni_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/cni_isolator_tests.cpp b/src/tests/containerizer/cni_isolator_tests.cpp
index 0197123..b58a9ca 100644
--- a/src/tests/containerizer/cni_isolator_tests.cpp
+++ b/src/tests/containerizer/cni_isolator_tests.cpp
@@ -376,23 +376,23 @@ TEST_F(CniIsolatorTest, ROOT_VerifyCheckpointedInfo)
 
   // Check if the CNI related information is checkpointed successfully.
   const string containerDir =
-    paths::getContainerDir(paths::ROOT_DIR, containerId.value());
+    paths::getContainerDir(paths::ROOT_DIR, containerId);
 
   EXPECT_TRUE(os::exists(containerDir));
   EXPECT_TRUE(os::exists(paths::getNetworkDir(
-      paths::ROOT_DIR, containerId.value(), "__MESOS_TEST__")));
+      paths::ROOT_DIR, containerId, "__MESOS_TEST__")));
 
   EXPECT_TRUE(os::exists(paths::getNetworkConfigPath(
-      paths::ROOT_DIR, containerId.value(), "__MESOS_TEST__")));
+      paths::ROOT_DIR, containerId, "__MESOS_TEST__")));
 
   EXPECT_TRUE(os::exists(paths::getInterfaceDir(
-      paths::ROOT_DIR, containerId.value(), "__MESOS_TEST__", "eth0")));
+      paths::ROOT_DIR, containerId, "__MESOS_TEST__", "eth0")));
 
   EXPECT_TRUE(os::exists(paths::getNetworkInfoPath(
-      paths::ROOT_DIR, containerId.value(), "__MESOS_TEST__", "eth0")));
+      paths::ROOT_DIR, containerId, "__MESOS_TEST__", "eth0")));
 
   EXPECT_TRUE(os::exists(paths::getNamespacePath(
-      paths::ROOT_DIR, containerId.value())));
+      paths::ROOT_DIR, containerId)));
 
   EXPECT_TRUE(os::exists(path::join(containerDir, "hostname")));
   EXPECT_TRUE(os::exists(path::join(containerDir, "hosts")));
@@ -1581,6 +1581,167 @@ TEST_P(DefaultExecutorCniTest, ROOT_VerifyContainerIP)
 }
 
 
+class NestedContainerCniTest
+  : public CniIsolatorTest,
+    public WithParamInterface<bool>
+{
+protected:
+  slave::Flags CreateSlaveFlags()
+  {
+    slave::Flags flags = CniIsolatorTest::CreateSlaveFlags();
+
+    flags.network_cni_plugins_dir = cniPluginDir;
+    flags.network_cni_config_dir = cniConfigDir;
+    flags.isolation = "docker/runtime,filesystem/linux,network/cni";
+    flags.image_providers = "docker";
+    flags.launcher = "linux";
+
+    return flags;
+  }
+};
+
+
+INSTANTIATE_TEST_CASE_P(
+    JoinParentsNetworkParam,
+    NestedContainerCniTest,
+    ::testing::Values(
+        true,
+        false));
+
+
+TEST_P(NestedContainerCniTest, ROOT_INTERNET_CURL_VerifyContainerHostname)
+{
+  const string parentContainerHostname = "parent_container";
+  const string nestedContainerHostname = "nested_container";
+  const string hostPath = path::join(sandbox.get(), "volume");
+  const string containerPath = "/tmp";
+  const bool joinParentsNetwork = GetParam();
+
+  ASSERT_SOME(os::mkdir(hostPath));
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+  ASSERT_SOME(slave);
+
+  auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
+
+  EXPECT_CALL(*scheduler, connected(_))
+    .WillOnce(v1::scheduler::SendSubscribe(v1::DEFAULT_FRAMEWORK_INFO));
+
+  Future<v1::scheduler::Event::Subscribed> subscribed;
+  EXPECT_CALL(*scheduler, subscribed(_, _))
+    .WillOnce(FutureArg<1>(&subscribed));
+
+  Future<v1::scheduler::Event::Offers> offers;
+  EXPECT_CALL(*scheduler, offers(_, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return());
+
+  EXPECT_CALL(*scheduler, heartbeat(_))
+    .WillRepeatedly(Return()); // Ignore heartbeats.
+
+  v1::scheduler::TestMesos mesos(
+      master.get()->pid, ContentType::PROTOBUF, scheduler);
+
+  AWAIT_READY(subscribed);
+
+  v1::FrameworkID frameworkId(subscribed->framework_id());
+
+  v1::ExecutorInfo executorInfo = v1::createExecutorInfo(
+      "test_default_executor",
+      None(),
+      "cpus:0.1;mem:32;disk:32",
+      v1::ExecutorInfo::DEFAULT);
+
+  // Update `executorInfo` with the subscribed `frameworkId`.
+  executorInfo.mutable_framework_id()->CopyFrom(frameworkId);
+
+  mesos::v1::ContainerInfo *executorContainer =
+    executorInfo.mutable_container();
+  executorContainer->set_type(mesos::v1::ContainerInfo::MESOS);
+  executorContainer->add_network_infos()->set_name("__MESOS_TEST__");
+  executorContainer->set_hostname(parentContainerHostname);
+
+  AWAIT_READY(offers);
+  ASSERT_FALSE(offers->offers().empty());
+
+  const v1::Offer& offer = offers->offers(0);
+  const v1::AgentID& agentId = offer.agent_id();
+
+  v1::TaskInfo taskInfo = v1::createTask(
+      agentId,
+      v1::Resources::parse("cpus:0.1;mem:32;disk:32").get(),
+      "touch /tmp/$(hostname)");
+
+  mesos::v1::Image image;
+  image.set_type(mesos::v1::Image::DOCKER);
+  image.mutable_docker()->set_name("alpine");
+
+  mesos::v1::ContainerInfo* nestedContainer = taskInfo.mutable_container();
+  nestedContainer->set_type(mesos::v1::ContainerInfo::MESOS);
+  nestedContainer->mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  if (!joinParentsNetwork) {
+    nestedContainer->add_network_infos()->set_name("__MESOS_TEST__");
+    nestedContainer->set_hostname(nestedContainerHostname);
+  }
+
+  nestedContainer->add_volumes()->CopyFrom(
+      v1::createVolumeHostPath(
+          containerPath,
+          hostPath,
+          mesos::v1::Volume::RW));
+
+  Future<v1::scheduler::Event::Update> updateStarting;
+  Future<v1::scheduler::Event::Update> updateRunning;
+  Future<v1::scheduler::Event::Update> updateFinished;
+  EXPECT_CALL(*scheduler, update(_, _))
+    .WillOnce(DoAll(FutureArg<1>(&updateStarting),
+                    v1::scheduler::SendAcknowledge(
+                        frameworkId,
+                        offer.agent_id())))
+    .WillOnce(DoAll(FutureArg<1>(&updateRunning),
+                    v1::scheduler::SendAcknowledge(
+                        frameworkId,
+                        offer.agent_id())))
+    .WillOnce(FutureArg<1>(&updateFinished));
+
+  v1::Offer::Operation launchGroup = v1::LAUNCH_GROUP(
+      executorInfo,
+      v1::createTaskGroupInfo({taskInfo}));
+
+  mesos.send(v1::createCallAccept(frameworkId, offer, {launchGroup}));
+
+  AWAIT_READY(updateStarting);
+  ASSERT_EQ(v1::TASK_STARTING, updateStarting->status().state());
+  EXPECT_EQ(taskInfo.task_id(), updateStarting->status().task_id());
+
+  AWAIT_READY(updateRunning);
+  ASSERT_EQ(v1::TASK_RUNNING, updateRunning->status().state());
+  EXPECT_EQ(taskInfo.task_id(), updateRunning->status().task_id());
+
+  AWAIT_READY(updateFinished);
+  ASSERT_EQ(v1::TASK_FINISHED, updateFinished->status().state());
+  EXPECT_EQ(taskInfo.task_id(), updateFinished->status().task_id());
+
+  if (joinParentsNetwork) {
+    EXPECT_TRUE(os::exists(path::join(
+        sandbox.get(),
+        "volume",
+        parentContainerHostname)));
+  } else {
+    EXPECT_TRUE(os::exists(path::join(
+        sandbox.get(),
+        "volume",
+        nestedContainerHostname)));
+  }
+}
+
+
 class CniIsolatorPortMapperTest : public CniIsolatorTest
 {
 public:

http://git-wip-us.apache.org/repos/asf/mesos/blob/a741b15e/src/tests/master_validation_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/master_validation_tests.cpp b/src/tests/master_validation_tests.cpp
index efdd44d..a522961 100644
--- a/src/tests/master_validation_tests.cpp
+++ b/src/tests/master_validation_tests.cpp
@@ -3539,22 +3539,29 @@ TEST_F(TaskGroupValidationTest, TaskUsesDockerContainerInfo)
 }
 
 
-// Ensures that a task in a task group that has `NetworkInfo`
-// set is rejected during `TaskGroupInfo` validation.
-TEST_F(TaskGroupValidationTest, TaskUsesNetworkInfo)
+// Ensures that a task in a task group that has `NetworkInfo` set does
+// not have HTTP health checks during `TaskGroupInfo` validation.
+TEST_F(TaskGroupValidationTest, TaskWithNetworkInfosDoesNotHaveHTTPHealthChecks)
 {
   Try<Owned<cluster::Master>> master = StartMaster();
   ASSERT_SOME(master);
 
   Owned<MasterDetector> detector = master.get()->createDetector();
+
   Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
   ASSERT_SOME(slave);
 
   MockScheduler sched;
+
   MesosSchedulerDriver driver(
-      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+      &sched,
+      DEFAULT_FRAMEWORK_INFO,
+      master.get()->pid,
+      DEFAULT_CREDENTIAL);
 
-  EXPECT_CALL(sched, registered(&driver, _, _));
+  Future<FrameworkID> frameworkId;
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .WillOnce(FutureArg<1>(&frameworkId));
 
   Future<vector<Offer>> offers;
   EXPECT_CALL(sched, resourceOffers(&driver, _))
@@ -3563,41 +3570,43 @@ TEST_F(TaskGroupValidationTest, TaskUsesNetworkInfo)
 
   driver.start();
 
+  AWAIT_READY(frameworkId);
+
   AWAIT_READY(offers);
   ASSERT_FALSE(offers->empty());
   Offer offer = offers.get()[0];
 
-  Resources resources = Resources::parse("cpus:1;mem:512;disk:32").get();
+  Resources resources = Resources::parse("cpus:0.5;mem:300;disk:100").get();
 
   ExecutorInfo executor(DEFAULT_EXECUTOR_INFO);
   executor.set_type(ExecutorInfo::CUSTOM);
   executor.mutable_resources()->CopyFrom(resources);
+  executor.mutable_framework_id()->CopyFrom(frameworkId.get());
 
-  // Create an invalid task that has NetworkInfos set.
-  TaskInfo task1;
-  task1.set_name("1");
-  task1.mutable_task_id()->set_value("1");
-  task1.mutable_slave_id()->MergeFrom(offer.slave_id());
-  task1.mutable_resources()->MergeFrom(resources);
-  task1.mutable_container()->set_type(ContainerInfo::MESOS);
-  task1.mutable_container()->add_network_infos();
+  TaskInfo task;
+  task.set_name("1");
+  task.mutable_task_id()->set_value("1");
+  task.mutable_slave_id()->MergeFrom(offer.slave_id());
+  task.mutable_resources()->MergeFrom(resources);
+  task.mutable_container()->set_type(ContainerInfo::MESOS);
+  task.mutable_container()->add_network_infos();
 
-  // Create a valid task.
-  TaskInfo task2;
-  task2.set_name("2");
-  task2.mutable_task_id()->set_value("2");
-  task2.mutable_slave_id()->MergeFrom(offer.slave_id());
-  task2.mutable_resources()->MergeFrom(resources);
+  // Add a HTTP health check to this task.
+  HealthCheck healthCheck;
+  healthCheck.set_type(HealthCheck::HTTP);
+  healthCheck.mutable_http()->set_port(80);
+  healthCheck.set_delay_seconds(0);
+  healthCheck.set_interval_seconds(0);
+  healthCheck.set_grace_period_seconds(15);
+
+  task.mutable_health_check()->CopyFrom(healthCheck);
 
   TaskGroupInfo taskGroup;
-  taskGroup.add_tasks()->CopyFrom(task1);
-  taskGroup.add_tasks()->CopyFrom(task2);
+  taskGroup.add_tasks()->CopyFrom(task);
 
-  Future<TaskStatus> task1Status;
-  Future<TaskStatus> task2Status;
+  Future<TaskStatus> taskStatus;
   EXPECT_CALL(sched, statusUpdate(&driver, _))
-    .WillOnce(FutureArg<1>(&task1Status))
-    .WillOnce(FutureArg<1>(&task2Status));
+    .WillOnce(FutureArg<1>(&taskStatus));
 
   Offer::Operation operation;
   operation.set_type(Offer::Operation::LAUNCH_GROUP);
@@ -3610,17 +3619,109 @@ TEST_F(TaskGroupValidationTest, TaskUsesNetworkInfo)
 
   driver.acceptOffers({offer.id()}, {operation});
 
-  AWAIT_READY(task1Status);
-  EXPECT_EQ(task1.task_id(), task1Status->task_id());
-  EXPECT_EQ(TASK_ERROR, task1Status->state());
-  EXPECT_EQ(TaskStatus::REASON_TASK_GROUP_INVALID, task1Status->reason());
-  EXPECT_EQ("Task '1' is invalid: NetworkInfos must not be set on the task",
-            task1Status->message());
+  const string expected =
+    "Task '1' is invalid: HTTP and TCP health checks are not supported "
+    "for nested containers not joining parent's network";
 
-  AWAIT_READY(task2Status);
-  EXPECT_EQ(task2.task_id(), task2Status->task_id());
-  EXPECT_EQ(TASK_ERROR, task2Status->state());
-  EXPECT_EQ(TaskStatus::REASON_TASK_GROUP_INVALID, task2Status->reason());
+  AWAIT_READY(taskStatus);
+  EXPECT_EQ(task.task_id(), taskStatus->task_id());
+  EXPECT_EQ(TASK_ERROR, taskStatus->state());
+  EXPECT_EQ(TaskStatus::REASON_TASK_GROUP_INVALID, taskStatus->reason());
+  EXPECT_EQ(expected, taskStatus->message());
+
+  driver.stop();
+  driver.join();
+}
+
+
+// Ensures that a task in a task group that has `NetworkInfo` set does
+// not have TCP health checks during `TaskGroupInfo` validation.
+TEST_F(TaskGroupValidationTest, TaskWithNetworkInfosDoesNotHaveTCPHealthChecks)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get());
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched,
+      DEFAULT_FRAMEWORK_INFO,
+      master.get()->pid,
+      DEFAULT_CREDENTIAL);
+
+  Future<FrameworkID> frameworkId;
+  EXPECT_CALL(sched, registered(&driver, _, _))
+    .WillOnce(FutureArg<1>(&frameworkId));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(frameworkId);
+
+  AWAIT_READY(offers);
+  ASSERT_FALSE(offers->empty());
+  Offer offer = offers.get()[0];
+
+  Resources resources = Resources::parse("cpus:0.5;mem:300;disk:100").get();
+
+  ExecutorInfo executor(DEFAULT_EXECUTOR_INFO);
+  executor.set_type(ExecutorInfo::DEFAULT);
+  executor.mutable_resources()->CopyFrom(resources);
+  executor.mutable_framework_id()->CopyFrom(frameworkId.get());
+
+  TaskInfo task;
+  task.set_name("1");
+  task.mutable_task_id()->set_value("1");
+  task.mutable_slave_id()->MergeFrom(offer.slave_id());
+  task.mutable_resources()->MergeFrom(resources);
+  task.mutable_container()->set_type(ContainerInfo::MESOS);
+  task.mutable_container()->add_network_infos();
+
+  // Add a TCP health check to this task.
+  HealthCheck healthCheck;
+  healthCheck.set_type(HealthCheck::TCP);
+  healthCheck.mutable_tcp()->set_port(30000);
+  healthCheck.set_delay_seconds(0);
+  healthCheck.set_interval_seconds(0);
+  healthCheck.set_grace_period_seconds(15);
+
+  task.mutable_health_check()->CopyFrom(healthCheck);
+
+  TaskGroupInfo taskGroup;
+  taskGroup.add_tasks()->CopyFrom(task);
+
+  Future<TaskStatus> taskStatus;
+  EXPECT_CALL(sched, statusUpdate(&driver, _))
+    .WillOnce(FutureArg<1>(&taskStatus));
+
+  Offer::Operation operation;
+  operation.set_type(Offer::Operation::LAUNCH_GROUP);
+
+  Offer::Operation::LaunchGroup* launchGroup =
+    operation.mutable_launch_group();
+
+  launchGroup->mutable_executor()->CopyFrom(executor);
+  launchGroup->mutable_task_group()->CopyFrom(taskGroup);
+
+  driver.acceptOffers({offer.id()}, {operation});
+
+  const string expected =
+    "Task '1' is invalid: HTTP and TCP health checks are not supported "
+    "for nested containers not joining parent's network";
+
+  AWAIT_READY(taskStatus);
+  EXPECT_EQ(task.task_id(), taskStatus->task_id());
+  EXPECT_EQ(TASK_ERROR, taskStatus->state());
+  EXPECT_EQ(TaskStatus::REASON_TASK_GROUP_INVALID, taskStatus->reason());
+  EXPECT_EQ(expected, taskStatus->message());
 
   driver.stop();
   driver.join();