You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by gi...@apache.org on 2019/07/13 17:18:11 UTC

[mesos] branch master updated (575d47b -> e04b445)

This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git.


    from 575d47b  Updated a comment in the `filesystem/linux` isolator.
     new 6f402d5  Added the test `NamespacesIsolatorTest.ROOT_ShareIPCNamespace`.
     new f26b2aa  Added the test `NamespacesIsolatorTest.ROOT_PrivateIPCNamespace`.
     new 82305ea  Added the test `NamespacesIsolatorTest.ROOT_ShareAgentIPCNamespace`.
     new c4ce908  Added the test `ROOT_DisallowShareAgentIPCNamespace`.
     new e3b2edb  Updated the test `NamespacesIsolatorTest.ROOT_IPCNamespace`.
     new e04b445  Added the test `ROOT_IPCNamespaceWithIPCIsolatorDisabled`.

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/tests/containerizer/isolator_tests.cpp | 798 ++++++++++++++++++++++++++++-
 1 file changed, 778 insertions(+), 20 deletions(-)


[mesos] 04/06: Added the test `ROOT_DisallowShareAgentIPCNamespace`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit c4ce90884e2a93e331a6d1bbbe9ed960c5872d24
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Sat Jul 13 10:07:57 2019 -0700

    Added the test `ROOT_DisallowShareAgentIPCNamespace`.
    
    Review: https://reviews.apache.org/r/70857/
---
 src/tests/containerizer/isolator_tests.cpp | 30 ++++++++++++++++++++++++++++++
 1 file changed, 30 insertions(+)

diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 75b9520..8668088 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -865,6 +865,36 @@ TEST_F(NamespacesIsolatorTest, ROOT_ShareAgentIPCNamespace)
   ASSERT_SOME(os::rm("/dev/shm/nest1"));
   ASSERT_SOME(os::rm("/dev/shm/nest2"));
 }
+
+
+// This test verifies that top-level container with `SHARE_PARENT` IPC mode
+// will fail to launch when `--disallow_sharing_agent_ipc_namespace = true`.
+TEST_F(NamespacesIsolatorTest, ROOT_DisallowShareAgentIPCNamespace)
+{
+  Try<Owned<MesosContainerizer>> containerizer =
+    createContainerizer("filesystem/linux,namespaces/ipc", None(), true);
+
+  ASSERT_SOME(containerizer);
+
+  // Launch a top-level container with `SHARE_PARENT` IPC mode.
+  mesos::slave::ContainerConfig containerConfig = createContainerConfig(
+      None(),
+      createExecutorInfo("executor", "sleep 1000"),
+      directory);
+
+  ContainerInfo* container = containerConfig.mutable_container_info();
+  container->set_type(ContainerInfo::MESOS);
+  container->mutable_linux_info()->set_ipc_mode(LinuxInfo::SHARE_PARENT);
+
+  process::Future<Containerizer::LaunchResult> launch =
+    containerizer.get()->launch(
+        containerId,
+        containerConfig,
+        std::map<string, string>(),
+        None());
+
+  AWAIT_FAILED(launch);
+}
 #endif // __linux__
 
 } // namespace tests {


[mesos] 01/06: Added the test `NamespacesIsolatorTest.ROOT_ShareIPCNamespace`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 6f402d5f193262e854826cac4bc14f29395ca02f
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Sat Jul 13 10:07:52 2019 -0700

    Added the test `NamespacesIsolatorTest.ROOT_ShareIPCNamespace`.
    
    Review: https://reviews.apache.org/r/70845/
---
 src/tests/containerizer/isolator_tests.cpp | 267 ++++++++++++++++++++++++++++-
 1 file changed, 265 insertions(+), 2 deletions(-)

diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index bb2cda4..82081d3 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -39,6 +39,7 @@
 #include "slave/containerizer/fetcher.hpp"
 
 #include "slave/containerizer/mesos/containerizer.hpp"
+#include "slave/containerizer/mesos/paths.hpp"
 
 #include "tests/mesos.hpp"
 
@@ -51,6 +52,9 @@ using mesos::internal::slave::Containerizer;
 using mesos::internal::slave::Fetcher;
 using mesos::internal::slave::MesosContainerizer;
 
+using mesos::internal::slave::containerizer::paths::getSandboxPath;
+
+using mesos::slave::ContainerClass;
 using mesos::slave::ContainerTermination;
 
 namespace mesos {
@@ -71,16 +75,26 @@ public:
 
   Try<Owned<MesosContainerizer>> createContainerizer(
       const string& isolation,
-      const Option<bool>& disallowSharingAgentPidNamespace = None())
+      const Option<bool>& disallowSharingAgentPidNamespace = None(),
+      const Option<bool>& disallowSharingAgentIpcNamespace = None(),
+      const Option<Bytes>& defaultShmSize = None())
   {
     slave::Flags flags = CreateSlaveFlags();
-    flags.isolation = isolation;
+    flags.image_providers = "docker";
+    flags.isolation = isolation + ",docker/runtime";
 
     if (disallowSharingAgentPidNamespace.isSome()) {
       flags.disallow_sharing_agent_pid_namespace =
         disallowSharingAgentPidNamespace.get();
     }
 
+    if (disallowSharingAgentIpcNamespace.isSome()) {
+      flags.disallow_sharing_agent_ipc_namespace =
+        disallowSharingAgentIpcNamespace.get();
+    }
+
+    flags.default_shm_size = defaultShmSize;
+
     fetcher.reset(new Fetcher(flags));
 
     Try<MesosContainerizer*> _containerizer =
@@ -321,6 +335,255 @@ TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespace)
   EXPECT_NE(hostShmmax.get(), childShmmax.get());
   EXPECT_EQ(shmmaxValue, childShmmax.get());
 }
+
+
+// This test verifies that a top-level container with private IPC mode will
+// have its own IPC namespace and /dev/shm, and it can share IPC namespace
+// and /dev/shm with its child container, grandchild container and debug
+// container.
+TEST_F(NamespacesIsolatorTest, ROOT_ShareIPCNamespace)
+{
+  Try<Owned<MesosContainerizer>> containerizer =
+    createContainerizer("filesystem/linux,namespaces/ipc");
+
+  ASSERT_SOME(containerizer);
+
+  // Launch a top-level container with `PRIVATE` IPC mode and 128MB /dev/shm,
+  // check its /dev/shm size is correctly set and its IPC namespace is
+  // different than agent's IPC namespace, write its IPC namespace inode to
+  // a file under /dev/shm.
+  const string command =
+    "df -m /dev/shm | grep -w 128 && "
+    "test `stat -Lc %i /proc/self/ns/ipc` != `stat -Lc %i /proc/1/ns/ipc` && "
+    "stat -Lc %i /proc/self/ns/ipc > /dev/shm/root && "
+    "touch marker && "
+    "sleep 1000";
+
+  mesos::slave::ContainerConfig containerConfig = createContainerConfig(
+      None(),
+      createExecutorInfo("executor", command),
+      directory);
+
+  ContainerInfo* container = containerConfig.mutable_container_info();
+  container->set_type(ContainerInfo::MESOS);
+  container->mutable_linux_info()->set_ipc_mode(LinuxInfo::PRIVATE);
+  container->mutable_linux_info()->set_shm_size(128);
+
+  process::Future<Containerizer::LaunchResult> launch =
+    containerizer.get()->launch(
+        containerId,
+        containerConfig,
+        std::map<string, string>(),
+        None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Wait until the marker file is created.
+  Duration waited = Duration::zero();
+
+  do {
+    if (os::exists(path::join(directory, "marker"))) {
+      break;
+    }
+
+    os::sleep(Seconds(1));
+    waited += Seconds(1);
+  } while (waited < process::TEST_AWAIT_TIMEOUT);
+
+  EXPECT_LT(waited, process::TEST_AWAIT_TIMEOUT);
+
+  // The file created by the top-level container should only exist in
+  // its own /dev/shm rather than in agent's /dev/shm.
+  ASSERT_FALSE(os::exists("/dev/shm/root"));
+
+  // Now launch two child containers with `SHARE_PARENT` ipc mode and
+  // 256MB /dev/shm.
+  ContainerID childContainerId1, childContainerId2;
+
+  childContainerId1.mutable_parent()->CopyFrom(containerId);
+  childContainerId1.set_value(id::UUID::random().toString());
+
+  childContainerId2.mutable_parent()->CopyFrom(containerId);
+  childContainerId2.set_value(id::UUID::random().toString());
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.mutable_linux_info()->set_ipc_mode(LinuxInfo::SHARE_PARENT);
+  containerInfo.mutable_linux_info()->set_shm_size(256);
+
+  // Launch the first child container, check its /dev/shm size is 128MB
+  // rather than 256MB, it can see the file created by its parent container
+  // in /dev/shm and it is in the same IPC namespace with its parent container,
+  // and then write its IPC namespace inode to a file under /dev/shm.
+  launch = containerizer.get()->launch(
+      childContainerId1,
+      createContainerConfig(
+          createCommandInfo(
+              "df -m /dev/shm | grep -w 128 && "
+              "test `stat -Lc %i /proc/self/ns/ipc` = `cat /dev/shm/root` && "
+              "stat -Lc %i /proc/self/ns/ipc > /dev/shm/child1 && "
+              "touch marker && "
+              "sleep 1000"),
+          containerInfo),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Wait until the marker file is created.
+  waited = Duration::zero();
+  const string childSandboxPath1 = getSandboxPath(directory, childContainerId1);
+
+  do {
+    if (os::exists(path::join(childSandboxPath1, "marker"))) {
+      break;
+    }
+
+    os::sleep(Seconds(1));
+    waited += Seconds(1);
+  } while (waited < process::TEST_AWAIT_TIMEOUT);
+
+  EXPECT_LT(waited, process::TEST_AWAIT_TIMEOUT);
+
+  // Launch the second child container with its own rootfs, check its /dev/shm
+  // size is 128MB rather than 256MB, it can see the files created by its parent
+  // container and the first child container in /dev/shm and it is in the same
+  // IPC namespace with its parent container and the first child container. and
+  // then write its IPC namespace inode to a file under /dev/shm.
+  mesos::Image image;
+  image.set_type(mesos::Image::DOCKER);
+  image.mutable_docker()->set_name("alpine");
+
+  containerInfo.mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  launch = containerizer.get()->launch(
+      childContainerId2,
+      createContainerConfig(
+          createCommandInfo(
+              "df -m /dev/shm | grep -w 128 && "
+              "test `stat -Lc %i /proc/self/ns/ipc` = `cat /dev/shm/root` && "
+              "test `stat -Lc %i /proc/self/ns/ipc` = `cat /dev/shm/child1` && "
+              "stat -Lc %i /proc/self/ns/ipc > /dev/shm/child2 && "
+              "touch marker && "
+              "sleep 1000"),
+          containerInfo),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Wait until the marker file is created.
+  waited = Duration::zero();
+  const string childSandboxPath2 = getSandboxPath(directory, childContainerId2);
+
+  do {
+    if (os::exists(path::join(childSandboxPath2, "marker"))) {
+      break;
+    }
+
+    os::sleep(Seconds(1));
+    waited += Seconds(1);
+  } while (waited < process::TEST_AWAIT_TIMEOUT);
+
+  EXPECT_LT(waited, process::TEST_AWAIT_TIMEOUT);
+
+  // Launch a grandchild container with `SHARE_PARENT` ipc mode and
+  // 256MB /dev/shm under the first child container, check its /dev/shm
+  // size is 128MB rather than 256MB, it can see the files created by
+  // its parent and grandparent containers and it is in the same IPC
+  // namespace with its parent and grandparent containers.
+  ContainerID grandchildContainerId;
+  grandchildContainerId.mutable_parent()->CopyFrom(childContainerId1);
+  grandchildContainerId.set_value(id::UUID::random().toString());
+
+  launch = containerizer.get()->launch(
+      grandchildContainerId,
+      createContainerConfig(
+          createCommandInfo(
+              "df -m /dev/shm | grep -w 128 && "
+              "test `stat -Lc %i /proc/self/ns/ipc` = `cat /dev/shm/child1` && "
+              "test `stat -Lc %i /proc/self/ns/ipc` = `cat /dev/shm/root`"),
+          containerInfo),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  Future<Option<ContainerTermination>> wait =
+    containerizer.get()->wait(grandchildContainerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  // Launch a debug container with `PRIVATE` ipc mode and 256MB /dev/shm
+  // under the first child container, check its /dev/shm size is 128MB
+  // rather than 256MB and it is in the same IPC namespace with its parent
+  // container even its ipc mode is `PRIVATE`.
+  ContainerID debugContainerId1;
+  debugContainerId1.mutable_parent()->CopyFrom(childContainerId1);
+  debugContainerId1.set_value(id::UUID::random().toString());
+
+  containerInfo.clear_mesos();
+  containerInfo.mutable_linux_info()->set_ipc_mode(LinuxInfo::PRIVATE);
+
+  launch = containerizer.get()->launch(
+      debugContainerId1,
+      createContainerConfig(
+          createCommandInfo(
+              "df -m /dev/shm | grep -w 128 && "
+              "test `stat -Lc %i /proc/self/ns/ipc` = `cat /dev/shm/child1`"),
+          containerInfo,
+          ContainerClass::DEBUG),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  wait = containerizer.get()->wait(debugContainerId1);
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  // Launch a debug container with `PRIVATE` ipc mode and 256MB /dev/shm
+  // under the second child container, check its /dev/shm size is 128MB
+  // rather than 256MB and it is in the same IPC namespace with its parent
+  // container even its ipc mode is `PRIVATE`.
+  ContainerID debugContainerId2;
+  debugContainerId2.mutable_parent()->CopyFrom(childContainerId2);
+  debugContainerId2.set_value(id::UUID::random().toString());
+
+  containerInfo.mutable_linux_info()->set_ipc_mode(LinuxInfo::PRIVATE);
+
+  launch = containerizer.get()->launch(
+      debugContainerId2,
+      createContainerConfig(
+          createCommandInfo(
+              "df -m /dev/shm | grep -w 128 && "
+              "test `stat -Lc %i /proc/self/ns/ipc` = `cat /dev/shm/child2`"),
+          containerInfo,
+          ContainerClass::DEBUG),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  wait = containerizer.get()->wait(debugContainerId2);
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  Future<Option<ContainerTermination>> termination =
+    containerizer.get()->destroy(containerId);
+
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
+}
 #endif // __linux__
 
 } // namespace tests {


[mesos] 02/06: Added the test `NamespacesIsolatorTest.ROOT_PrivateIPCNamespace`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit f26b2aa93553a1a7e845b194cb9ff49fb72bae22
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Sat Jul 13 10:07:54 2019 -0700

    Added the test `NamespacesIsolatorTest.ROOT_PrivateIPCNamespace`.
    
    Review: https://reviews.apache.org/r/70849/
---
 src/tests/containerizer/isolator_tests.cpp | 159 +++++++++++++++++++++++++++++
 1 file changed, 159 insertions(+)

diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 82081d3..fad6eb4 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -584,6 +584,165 @@ TEST_F(NamespacesIsolatorTest, ROOT_ShareIPCNamespace)
   ASSERT_TRUE(termination.get()->has_status());
   EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
+
+
+// This test verifies that nested container with private IPC mode will
+// have its own IPC namespace and /dev/shm.
+TEST_F(NamespacesIsolatorTest, ROOT_PrivateIPCNamespace)
+{
+  // Create containerizer with `--default_shm_size=64MB`.
+  Try<Owned<MesosContainerizer>> containerizer = createContainerizer(
+      "filesystem/linux,namespaces/ipc",
+      None(),
+      None(),
+      Megabytes(64));
+
+  ASSERT_SOME(containerizer);
+
+  // Launch a top-level container with `PRIVATE` IPC mode, check its /dev/shm
+  // size is correctly set to the default value, touch a file in its /dev/shm,
+  // and write its IPC namespace inode to a file in its sandbox.
+  const string command =
+    "df -m /dev/shm | grep -w 64 && "
+    "touch /dev/shm/root &&"
+    "stat -Lc %i /proc/self/ns/ipc > ns && "
+    "sleep 1000";
+
+  mesos::slave::ContainerConfig containerConfig = createContainerConfig(
+      None(),
+      createExecutorInfo("executor", command),
+      directory);
+
+  ContainerInfo* container = containerConfig.mutable_container_info();
+  container->set_type(ContainerInfo::MESOS);
+  container->mutable_linux_info()->set_ipc_mode(LinuxInfo::PRIVATE);
+
+  process::Future<Containerizer::LaunchResult> launch =
+    containerizer.get()->launch(
+        containerId,
+        containerConfig,
+        std::map<string, string>(),
+        None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Wait until the `ns` file is created in the sandbox.
+  Duration waited = Duration::zero();
+
+  do {
+    if (os::exists(path::join(directory, "ns"))) {
+      break;
+    }
+
+    os::sleep(Seconds(1));
+    waited += Seconds(1);
+  } while (waited < process::TEST_AWAIT_TIMEOUT);
+
+  EXPECT_LT(waited, process::TEST_AWAIT_TIMEOUT);
+
+  // Launch a nested container with `PRIVATE` IPC mode, check its /dev/shm
+  // size is correctly set to the default value and the file created by the
+  // top-level container does not exist in its /dev/shm, touch a file in its
+  // /dev/shm and write its IPC namespace inode to a file in its sandbox.
+  ContainerID nestedContainerId1;
+  nestedContainerId1.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId1.set_value(id::UUID::random().toString());
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.mutable_linux_info()->set_ipc_mode(LinuxInfo::PRIVATE);
+
+  launch = containerizer.get()->launch(
+      nestedContainerId1,
+      createContainerConfig(
+          createCommandInfo(
+              "df -m /dev/shm | grep -w 64 &&"
+              "test ! -e /dev/shm/root &&"
+              "touch /dev/shm/nested1 &&"
+              "stat -Lc %i /proc/self/ns/ipc > ns && "
+              "sleep 1000"),
+          containerInfo),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Wait until the `ns` file is created in the sandbox.
+  waited = Duration::zero();
+  const string nestedSandboxPath1 =
+    getSandboxPath(directory, nestedContainerId1);
+
+  do {
+    if (os::exists(path::join(nestedSandboxPath1, "ns"))) {
+      break;
+    }
+
+    os::sleep(Seconds(1));
+    waited += Seconds(1);
+  } while (waited < process::TEST_AWAIT_TIMEOUT);
+
+  EXPECT_LT(waited, process::TEST_AWAIT_TIMEOUT);
+
+  // Launch another nested container with private IPC mode and 128MB
+  // /dev/shm, check its /dev/shm size is correctly set to 128MB and
+  // the files created by the top-level container and the first nested
+  // container do not exist in its /dev/shm, write its IPC namespace
+  // inode to a file in its sandbox.
+  ContainerID nestedContainerId2;
+  nestedContainerId2.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId2.set_value(id::UUID::random().toString());
+
+  containerInfo.mutable_linux_info()->set_shm_size(128);
+
+  launch = containerizer.get()->launch(
+      nestedContainerId2,
+      createContainerConfig(
+          createCommandInfo(
+              "df -m /dev/shm | grep -w 128 &&"
+              "test ! -e /dev/shm/root &&"
+              "test ! -e /dev/shm/nested1 &&"
+              "stat -Lc %i /proc/self/ns/ipc > ns"),
+          containerInfo),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer.get()->wait(
+      nestedContainerId2);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  // Check top-level container and the two nested containers
+  // have their own IPC namespaces.
+  Try<uint64_t> rootIpcNamespace = readValue(path::join(directory, "ns"));
+  ASSERT_SOME(rootIpcNamespace);
+
+  Try<uint64_t> nestedIpcNamespace1 =
+    readValue(path::join(nestedSandboxPath1, "ns"));
+
+  ASSERT_SOME(nestedIpcNamespace1);
+
+  Try<uint64_t> nestedIpcNamespace2 =
+    readValue(path::join(getSandboxPath(directory, nestedContainerId2), "ns"));
+
+  ASSERT_SOME(nestedIpcNamespace2);
+
+  EXPECT_NE(rootIpcNamespace.get(), nestedIpcNamespace1.get());
+  EXPECT_NE(rootIpcNamespace.get(), nestedIpcNamespace2.get());
+  EXPECT_NE(nestedIpcNamespace1.get(), nestedIpcNamespace2.get());
+
+  Future<Option<ContainerTermination>> termination =
+    containerizer.get()->destroy(containerId);
+
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
+}
 #endif // __linux__
 
 } // namespace tests {


[mesos] 03/06: Added the test `NamespacesIsolatorTest.ROOT_ShareAgentIPCNamespace`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 82305ea19b4fd1b04bdeb0d6f8bb1792077a6206
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Sat Jul 13 10:07:55 2019 -0700

    Added the test `NamespacesIsolatorTest.ROOT_ShareAgentIPCNamespace`.
    
    Review: https://reviews.apache.org/r/70852/
---
 src/tests/containerizer/isolator_tests.cpp | 122 +++++++++++++++++++++++++++++
 1 file changed, 122 insertions(+)

diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index fad6eb4..75b9520 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -743,6 +743,128 @@ TEST_F(NamespacesIsolatorTest, ROOT_PrivateIPCNamespace)
   ASSERT_TRUE(termination.get()->has_status());
   EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
 }
+
+
+// This test verifies that top-level container and nested
+// containers can share agent's IPC namespace and /dev/shm.
+TEST_F(NamespacesIsolatorTest, ROOT_ShareAgentIPCNamespace)
+{
+  Try<Owned<MesosContainerizer>> containerizer =
+    createContainerizer("filesystem/linux,namespaces/ipc");
+
+  ASSERT_SOME(containerizer);
+
+  // Launch a top-level container with `SHARE_PARENT` IPC mode,
+  // write its IPC namespace inode to a file under /dev/shm.
+  const string command =
+    "stat -Lc %i /proc/self/ns/ipc > /dev/shm/root && "
+    "sleep 1000";
+
+  mesos::slave::ContainerConfig containerConfig = createContainerConfig(
+      None(),
+      createExecutorInfo("executor", command),
+      directory);
+
+  ContainerInfo* container = containerConfig.mutable_container_info();
+  container->set_type(ContainerInfo::MESOS);
+  container->mutable_linux_info()->set_ipc_mode(LinuxInfo::SHARE_PARENT);
+
+  process::Future<Containerizer::LaunchResult> launch =
+    containerizer.get()->launch(
+        containerId,
+        containerConfig,
+        std::map<string, string>(),
+        None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Launch a nested container with `SHARE_PARENT` IPC mode,
+  // write its IPC namespace inode to a file under /dev/shm.
+  ContainerID nestedContainerId1;
+  nestedContainerId1.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId1.set_value(id::UUID::random().toString());
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.mutable_linux_info()->set_ipc_mode(LinuxInfo::SHARE_PARENT);
+
+  launch = containerizer.get()->launch(
+      nestedContainerId1,
+      createContainerConfig(
+          createCommandInfo("stat -Lc %i /proc/self/ns/ipc > /dev/shm/nest1"),
+          containerInfo),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  Future<Option<ContainerTermination>> wait = containerizer.get()->wait(
+      nestedContainerId1);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  // Launch another nested container with `SHARE_PARENT` IPC mode and its
+  // own rootfs, write its IPC namespace inode to a file under /dev/shm.
+  ContainerID nestedContainerId2;
+  nestedContainerId2.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId2.set_value(id::UUID::random().toString());
+
+  mesos::Image image;
+  image.set_type(mesos::Image::DOCKER);
+  image.mutable_docker()->set_name("alpine");
+
+  containerInfo.mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  launch = containerizer.get()->launch(
+      nestedContainerId2,
+      createContainerConfig(
+          createCommandInfo("stat -Lc %i /proc/self/ns/ipc > /dev/shm/nest2"),
+          containerInfo),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  wait = containerizer.get()->wait(nestedContainerId2);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  ASSERT_TRUE(wait.get()->has_status());
+  EXPECT_WEXITSTATUS_EQ(0, wait.get()->status());
+
+  // Check top-level container and the two nested containers
+  // share agent's IPC namespace and /dev/shm.
+  Try<uint64_t> rootIpcNamespace = readValue("/dev/shm/root");
+  ASSERT_SOME(rootIpcNamespace);
+
+  Try<uint64_t> nestedIpcNamespace1 = readValue("/dev/shm/nest1");
+  ASSERT_SOME(nestedIpcNamespace1);
+
+  Try<uint64_t> nestedIpcNamespace2 = readValue("/dev/shm/nest2");
+  ASSERT_SOME(nestedIpcNamespace2);
+
+  Result<ino_t> agentIpcNamespace = ns::getns(::getpid(), "ipc");
+  ASSERT_SOME(agentIpcNamespace);
+
+  EXPECT_EQ(rootIpcNamespace.get(), agentIpcNamespace.get());
+  EXPECT_EQ(nestedIpcNamespace1.get(), agentIpcNamespace.get());
+  EXPECT_EQ(nestedIpcNamespace2.get(), agentIpcNamespace.get());
+
+  Future<Option<ContainerTermination>> termination =
+    containerizer.get()->destroy(containerId);
+
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
+
+  ASSERT_SOME(os::rm("/dev/shm/root"));
+  ASSERT_SOME(os::rm("/dev/shm/nest1"));
+  ASSERT_SOME(os::rm("/dev/shm/nest2"));
+}
 #endif // __linux__
 
 } // namespace tests {


[mesos] 05/06: Updated the test `NamespacesIsolatorTest.ROOT_IPCNamespace`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit e3b2edb1f3226e9f84e22383153314dc4a212edc
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Sat Jul 13 10:07:58 2019 -0700

    Updated the test `NamespacesIsolatorTest.ROOT_IPCNamespace`.
    
    This test is updated to verify the backward compatibility is kept
    after we implement the configurable IPC namespaces and /dev/shm.
    
    Review: https://reviews.apache.org/r/70859/
---
 src/tests/containerizer/isolator_tests.cpp | 105 ++++++++++++++++++++++++-----
 1 file changed, 87 insertions(+), 18 deletions(-)

diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 8668088..47debaa 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -268,17 +268,20 @@ TEST_F(NamespacesIsolatorTest, ROOT_SharePidNamespaceWhenDisallow)
 }
 
 
-// The IPC namespace has its own copy of the svipc(7) tunables. We verify
-// that we are correctly entering the IPC namespace by verifying that we
-// can set shmmax some different value than that of the host namespace.
-TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespace)
+// This test verifies that when `namespaces/ipc` isolator is enabled and
+// container's IPC mode is not set, for backward compatibility we will
+// keep the previous behavior: Top level container will have its own IPC
+// namespace and nested container will share the IPC namespace with its
+// parent container. If the container does not have its own rootfs, it
+// will share agent's /dev/shm, otherwise it will have its own /dev/shm.
+TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespaceWithIPCModeUnset)
 {
   Try<Owned<MesosContainerizer>> containerizer =
     createContainerizer("filesystem/linux,namespaces/ipc");
 
   ASSERT_SOME(containerizer);
 
-  // Value we will set the child namespace shmmax to.
+  // Value we will set the top-level container's IPC namespace shmmax to.
   uint64_t shmmaxValue = static_cast<uint64_t>(::getpid());
 
   Try<uint64_t> hostShmmax = readValue("/proc/sys/kernel/shmmax");
@@ -287,10 +290,13 @@ TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespace)
   // Verify that the host namespace shmmax is different.
   ASSERT_NE(hostShmmax.get(), shmmaxValue);
 
-  const string command =
-    "stat -Lc %i /proc/self/ns/ipc > ns;"
-    "echo " + stringify(shmmaxValue) + " > /proc/sys/kernel/shmmax;"
-    "cp /proc/sys/kernel/shmmax shmmax";
+  // Launch a top-level container with IPC mode
+  // unset and it does not have its own rootfs.
+  string command =
+    "stat -Lc %i /proc/self/ns/ipc > ns && "
+    "echo " + stringify(shmmaxValue) + " > /proc/sys/kernel/shmmax && "
+    "cp /proc/sys/kernel/shmmax /dev/shm/shmmax && "
+    "sleep 1000";
 
   process::Future<Containerizer::LaunchResult> launch =
     containerizer.get()->launch(
@@ -304,36 +310,99 @@ TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespace)
 
   AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
 
-  // Wait on the container.
+  // Since the top-level container does not have its own
+  // rootfs, it will share host's /dev/shm, so let's wait
+  // until /dev/shm/shmmax is created in the host.
+  Duration waited = Duration::zero();
+
+  do {
+    if (os::exists("/dev/shm/shmmax")) {
+      break;
+    }
+
+    os::sleep(Seconds(1));
+    waited += Seconds(1);
+  } while (waited < process::TEST_AWAIT_TIMEOUT);
+
+  EXPECT_LT(waited, process::TEST_AWAIT_TIMEOUT);
+
+  // Launch a nested container with IPC mode unset and it has its own rootfs.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(id::UUID::random().toString());
+
+  mesos::Image image;
+  image.set_type(mesos::Image::DOCKER);
+  image.mutable_docker()->set_name("alpine");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  command =
+    "stat -Lc %i /proc/self/ns/ipc > ns && "
+    "test `cat /proc/sys/kernel/shmmax` = " + stringify(shmmaxValue) + " && "
+    "touch /dev/shm/file";
+
+  launch = containerizer.get()->launch(
+      nestedContainerId,
+      createContainerConfig(createCommandInfo(command), containerInfo),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Wait on the nested container.
   Future<Option<ContainerTermination>> wait =
-    containerizer.get()->wait(containerId);
+    containerizer.get()->wait(nestedContainerId);
 
   AWAIT_READY(wait);
   ASSERT_SOME(wait.get());
-
-  // Check the executor exited correctly.
   EXPECT_TRUE(wait->get().has_status());
   EXPECT_EQ(0, wait->get().status());
 
-  // Check that the command was run in a different IPC namespace.
+  // Check that top-level container and the nested container are in the
+  // same IPC namespace but not in the same IPC namespace with host.
   Result<ino_t> testIPCNamespace = ns::getns(::getpid(), "ipc");
   ASSERT_SOME(testIPCNamespace);
 
   Try<string> containerIPCNamespace = os::read(path::join(directory, "ns"));
   ASSERT_SOME(containerIPCNamespace);
 
+  Try<string> nestedcontainerIPCNamespace =
+    os::read(path::join(getSandboxPath(directory, nestedContainerId), "ns"));
+
+  ASSERT_SOME(nestedcontainerIPCNamespace);
+
   EXPECT_NE(stringify(testIPCNamespace.get()),
             strings::trim(containerIPCNamespace.get()));
 
+  EXPECT_EQ(strings::trim(containerIPCNamespace.get()),
+            strings::trim(nestedcontainerIPCNamespace.get()));
+
+  // The nested container will have its own /dev/shm since it has its own
+  // rootfs, so the file it created should not exist in the host.
+  ASSERT_FALSE(os::exists("/dev/shm/file"));
+
   // Check that we modified the IPC shmmax of the namespace, not the host.
-  Try<uint64_t> childShmmax = readValue("shmmax");
-  ASSERT_SOME(childShmmax);
+  Try<uint64_t> containerShmmax = readValue("/dev/shm/shmmax");
+  ASSERT_SOME(containerShmmax);
 
   // Verify that we didn't modify shmmax in the host namespace.
   ASSERT_EQ(hostShmmax.get(), readValue("/proc/sys/kernel/shmmax").get());
 
-  EXPECT_NE(hostShmmax.get(), childShmmax.get());
-  EXPECT_EQ(shmmaxValue, childShmmax.get());
+  EXPECT_NE(hostShmmax.get(), containerShmmax.get());
+  EXPECT_EQ(shmmaxValue, containerShmmax.get());
+
+  Future<Option<ContainerTermination>> termination =
+    containerizer.get()->destroy(containerId);
+
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
+
+  ASSERT_SOME(os::rm("/dev/shm/shmmax"));
 }
 
 


[mesos] 06/06: Added the test `ROOT_IPCNamespaceWithIPCIsolatorDisabled`.

Posted by gi...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

gilbert pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit e04b445f414e256866ba2e1f441bd7e86acc858e
Author: Qian Zhang <zh...@gmail.com>
AuthorDate: Sat Jul 13 10:07:59 2019 -0700

    Added the test `ROOT_IPCNamespaceWithIPCIsolatorDisabled`.
    
    Review: https://reviews.apache.org/r/70860/
---
 src/tests/containerizer/isolator_tests.cpp | 115 +++++++++++++++++++++++++++++
 1 file changed, 115 insertions(+)

diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 47debaa..a769bc5 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -406,6 +406,121 @@ TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespaceWithIPCModeUnset)
 }
 
 
+// This test verifies that when `namespaces/ipc` isolator is not enabled,
+// for backward compatibility we will keep the previous behavior: Any
+// containers will share IPC namespace with agent, and if the container
+// does not have its own rootfs, it will also share agent's /dev/shm,
+// otherwise it will have its own /dev/shm.
+TEST_F(NamespacesIsolatorTest, ROOT_IPCNamespaceWithIPCIsolatorDisabled)
+{
+  Try<Owned<MesosContainerizer>> containerizer =
+    createContainerizer("filesystem/linux");
+
+  ASSERT_SOME(containerizer);
+
+  // Launch a top-level container which does not have its own rootfs.
+  string command =
+    "stat -Lc %i /proc/self/ns/ipc > ns && "
+    "touch /dev/shm/root && "
+    "sleep 1000";
+
+  process::Future<Containerizer::LaunchResult> launch =
+    containerizer.get()->launch(
+        containerId,
+        createContainerConfig(
+            None(),
+            createExecutorInfo("executor", command),
+            directory),
+        std::map<string, string>(),
+        None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Since the top-level container does not have its own
+  // rootfs, it will share host's /dev/shm, so let's wait
+  // until /dev/shm/root is created in the host.
+  Duration waited = Duration::zero();
+
+  do {
+    if (os::exists("/dev/shm/root")) {
+      break;
+    }
+
+    os::sleep(Seconds(1));
+    waited += Seconds(1);
+  } while (waited < process::TEST_AWAIT_TIMEOUT);
+
+  EXPECT_LT(waited, process::TEST_AWAIT_TIMEOUT);
+
+  // Launch a nested container which has its own rootfs.
+  ContainerID nestedContainerId;
+  nestedContainerId.mutable_parent()->CopyFrom(containerId);
+  nestedContainerId.set_value(id::UUID::random().toString());
+
+  mesos::Image image;
+  image.set_type(mesos::Image::DOCKER);
+  image.mutable_docker()->set_name("alpine");
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  command =
+    "stat -Lc %i /proc/self/ns/ipc > ns && "
+    "touch /dev/shm/nested";
+
+  launch = containerizer.get()->launch(
+      nestedContainerId,
+      createContainerConfig(createCommandInfo(command), containerInfo),
+      std::map<string, string>(),
+      None());
+
+  AWAIT_ASSERT_EQ(Containerizer::LaunchResult::SUCCESS, launch);
+
+  // Wait on the nested container.
+  Future<Option<ContainerTermination>> wait =
+    containerizer.get()->wait(nestedContainerId);
+
+  AWAIT_READY(wait);
+  ASSERT_SOME(wait.get());
+  EXPECT_TRUE(wait->get().has_status());
+  EXPECT_EQ(0, wait->get().status());
+
+  // Check that top-level container and the nested container are
+  // in the same IPC namespace with host.
+  Result<ino_t> testIPCNamespace = ns::getns(::getpid(), "ipc");
+  ASSERT_SOME(testIPCNamespace);
+
+  Try<string> containerIPCNamespace = os::read(path::join(directory, "ns"));
+  ASSERT_SOME(containerIPCNamespace);
+
+  Try<string> nestedcontainerIPCNamespace =
+    os::read(path::join(getSandboxPath(directory, nestedContainerId), "ns"));
+
+  ASSERT_SOME(nestedcontainerIPCNamespace);
+
+  EXPECT_EQ(stringify(testIPCNamespace.get()),
+            strings::trim(containerIPCNamespace.get()));
+
+  EXPECT_EQ(strings::trim(containerIPCNamespace.get()),
+            strings::trim(nestedcontainerIPCNamespace.get()));
+
+  // The nested container will have its own /dev/shm since it has its own
+  // rootfs, so the file it created should not exist in the host.
+  ASSERT_FALSE(os::exists("/dev/shm/nested"));
+
+  Future<Option<ContainerTermination>> termination =
+    containerizer.get()->destroy(containerId);
+
+  AWAIT_READY(termination);
+  ASSERT_SOME(termination.get());
+  ASSERT_TRUE(termination.get()->has_status());
+  EXPECT_WTERMSIG_EQ(SIGKILL, termination.get()->status());
+
+  ASSERT_SOME(os::rm("/dev/shm/root"));
+}
+
+
 // This test verifies that a top-level container with private IPC mode will
 // have its own IPC namespace and /dev/shm, and it can share IPC namespace
 // and /dev/shm with its child container, grandchild container and debug