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 2016/12/05 21:24:38 UTC

[1/3] mesos git commit: Implement a namespaces/ipc isolator.

Repository: mesos
Updated Branches:
  refs/heads/master 5abda76d6 -> bc4a83b49


Implement a namespaces/ipc isolator.

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


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

Branch: refs/heads/master
Commit: 25ac581966eace322bf3062a315bbc6735aaec4c
Parents: 5abda76
Author: James Peach <jp...@apache.org>
Authored: Mon Dec 5 13:18:14 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Dec 5 13:18:14 2016 -0800

----------------------------------------------------------------------
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   2 +
 src/slave/containerizer/mesos/containerizer.cpp |   2 +
 .../mesos/isolators/namespaces/ipc.cpp          |  90 +++++++++++++++
 .../mesos/isolators/namespaces/ipc.hpp          |  49 ++++++++
 src/tests/containerizer/isolator_tests.cpp      | 113 +++++++++++++++++++
 6 files changed, 257 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/25ac5819/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index ea6e399..4d8658b 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -287,6 +287,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/gpu/nvml.cpp
   slave/containerizer/mesos/isolators/gpu/volume.cpp
   slave/containerizer/mesos/isolators/linux/capabilities.cpp
+  slave/containerizer/mesos/isolators/namespaces/ipc.cpp
   slave/containerizer/mesos/isolators/namespaces/pid.cpp
   slave/containerizer/mesos/isolators/network/cni/cni.cpp
   slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/25ac5819/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index c7f78c1..5736349 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -1128,6 +1128,7 @@ MESOS_LINUX_FILES =									\
   slave/containerizer/mesos/isolators/gpu/nvml.cpp					\
   slave/containerizer/mesos/isolators/gpu/volume.cpp					\
   slave/containerizer/mesos/isolators/linux/capabilities.cpp				\
+  slave/containerizer/mesos/isolators/namespaces/ipc.cpp				\
   slave/containerizer/mesos/isolators/namespaces/pid.cpp				\
   slave/containerizer/mesos/isolators/network/cni/cni.cpp				\
   slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.cpp	\
@@ -1167,6 +1168,7 @@ MESOS_LINUX_FILES +=									\
   slave/containerizer/mesos/isolators/gpu/nvml.hpp					\
   slave/containerizer/mesos/isolators/gpu/volume.hpp					\
   slave/containerizer/mesos/isolators/linux/capabilities.hpp				\
+  slave/containerizer/mesos/isolators/namespaces/ipc.hpp				\
   slave/containerizer/mesos/isolators/namespaces/pid.hpp				\
   slave/containerizer/mesos/isolators/network/cni/cni.hpp				\
   slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.hpp	\

http://git-wip-us.apache.org/repos/asf/mesos/blob/25ac5819/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 13cf757..cb23122 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -87,6 +87,7 @@
 #include "slave/containerizer/mesos/isolators/filesystem/shared.hpp"
 #include "slave/containerizer/mesos/isolators/gpu/nvidia.hpp"
 #include "slave/containerizer/mesos/isolators/linux/capabilities.hpp"
+#include "slave/containerizer/mesos/isolators/namespaces/ipc.hpp"
 #include "slave/containerizer/mesos/isolators/namespaces/pid.hpp"
 #include "slave/containerizer/mesos/isolators/network/cni/cni.hpp"
 #include "slave/containerizer/mesos/isolators/volume/image.hpp"
@@ -322,6 +323,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
         return NvidiaGpuIsolatorProcess::create(flags, nvidia.get());
       }},
 
+    {"namespaces/ipc", &NamespacesIPCIsolatorProcess::create},
     {"namespaces/pid", &NamespacesPidIsolatorProcess::create},
     {"network/cni", &NetworkCniIsolatorProcess::create},
 #endif // __linux__

http://git-wip-us.apache.org/repos/asf/mesos/blob/25ac5819/src/slave/containerizer/mesos/isolators/namespaces/ipc.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/namespaces/ipc.cpp b/src/slave/containerizer/mesos/isolators/namespaces/ipc.cpp
new file mode 100644
index 0000000..8675640
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/namespaces/ipc.cpp
@@ -0,0 +1,90 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#include "linux/ns.hpp"
+
+#include "slave/containerizer/mesos/isolators/namespaces/ipc.hpp"
+
+using process::Future;
+
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+Try<Isolator*> NamespacesIPCIsolatorProcess::create(const Flags& flags)
+{
+  // Check for root permission.
+  if (geteuid() != 0) {
+    return Error("The IPC namespace isolator requires root permissions");
+  }
+
+  // Verify that IPC namespaces are available on this kernel.
+  if (ns::namespaces().count("ipc") == 0) {
+    return Error("IPC namespaces are not supported by this kernel");
+  }
+
+  // Make sure the 'linux' launcher is used because only 'linux' launcher
+  // supports cloning namespaces for the container.
+  if (flags.launcher != "linux") {
+    return Error(
+        "The 'linux' launcher must be used to enable the IPC namespace");
+  }
+
+  return new MesosIsolator(process::Owned<MesosIsolatorProcess>(
+      new NamespacesIPCIsolatorProcess()));
+}
+
+
+NamespacesIPCIsolatorProcess::NamespacesIPCIsolatorProcess()
+  : ProcessBase(process::ID::generate("ipc-namespace-isolator")) {}
+
+
+bool NamespacesIPCIsolatorProcess::supportsNesting()
+{
+  return true;
+}
+
+
+// IPC isolation on Linux just requires that a process be placed in an IPC
+// namespace. Neither /proc, nor any of the special SVIPC filesystem need
+// to be remounted for this to work. IPC namespaces are disjoint. That is,
+// once you enter an IPC namespace, IPC objects from the host namespace are
+// no longer visible (and vice versa). Since IPC namespaces do not nest,
+// we always place nested containers into the IPC namespace of the parent
+// container. That is, containers in the same group share an IPC namespace,
+// but groups are isolated from each other.
+Future<Option<ContainerLaunchInfo>> NamespacesIPCIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  ContainerLaunchInfo launchInfo;
+
+  if (containerId.has_parent()) {
+    launchInfo.add_enter_namespaces(CLONE_NEWIPC);
+  } else {
+    launchInfo.add_clone_namespaces(CLONE_NEWIPC);
+  }
+
+  return launchInfo;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/25ac5819/src/slave/containerizer/mesos/isolators/namespaces/ipc.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/namespaces/ipc.hpp b/src/slave/containerizer/mesos/isolators/namespaces/ipc.hpp
new file mode 100644
index 0000000..9850407
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/namespaces/ipc.hpp
@@ -0,0 +1,49 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+#ifndef __NAMESPACES_IPC_ISOLATOR_HPP__
+#define __NAMESPACES_IPC_ISOLATOR_HPP__
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class NamespacesIPCIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(const Flags& flags);
+
+  virtual ~NamespacesIPCIsolatorProcess() {}
+
+  virtual bool supportsNesting();
+
+  virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
+private:
+  NamespacesIPCIsolatorProcess();
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __NAMESPACES_IPC_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/25ac5819/src/tests/containerizer/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index da46278..111e1ab 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -55,6 +55,50 @@ namespace internal {
 namespace tests {
 
 #ifdef __linux__
+class NamespacesIsolatorTest : public MesosTest
+{
+public:
+  virtual void SetUp()
+  {
+    MesosTest::SetUp();
+
+    directory = os::getcwd(); // We're inside a temporary sandbox.
+    containerId.set_value(UUID::random().toString());
+  }
+
+  Try<Owned<MesosContainerizer>> createContainerizer(const string& isolation)
+  {
+    slave::Flags flags = CreateSlaveFlags();
+    flags.isolation = isolation;
+
+    Try<MesosContainerizer*> _containerizer =
+      MesosContainerizer::create(flags, false, &fetcher);
+
+    if (_containerizer.isError()) {
+      return Error(_containerizer.error());
+    }
+
+    return Owned<MesosContainerizer>(_containerizer.get());
+  }
+
+  // Read a uint64_t value from the given path.
+  Try<uint64_t> readValue(const string& path)
+  {
+    Try<string> value = os::read(path);
+
+    if (value.isError()) {
+      return Error("Failed to read '" + path + "': " + value.error());
+    }
+
+    return numify<uint64_t>(strings::trim(value.get()));
+  }
+
+  string directory;
+  Fetcher fetcher;
+  ContainerID containerId;
+};
+
+
 class NamespacesPidIsolatorTest : public MesosTest {};
 
 
@@ -120,6 +164,75 @@ TEST_F(NamespacesPidIsolatorTest, ROOT_PidNamespace)
 
   EXPECT_TRUE(strings::contains(init.get(), "mesos"));
 }
+
+
+// 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)
+{
+  Try<Owned<MesosContainerizer>> containerizer =
+    createContainerizer("namespaces/ipc");
+  ASSERT_SOME(containerizer);
+
+  // Value we will set the child namespace shmmax to.
+  uint64_t shmmaxValue = static_cast<uint64_t>(::getpid());
+
+  Try<uint64_t> hostShmmax = readValue("/proc/sys/kernel/shmmax");
+  ASSERT_SOME(hostShmmax);
+
+  // Verify that the host namespace shmmax is different.
+  ASSERT_NE(hostShmmax.get(), shmmaxValue);
+
+  const string command =
+    "stat -c %i /proc/self/ns/ipc > ns;"
+    "echo " + stringify(shmmaxValue) + " > /proc/sys/kernel/shmmax;"
+    "cp /proc/sys/kernel/shmmax shmmax";
+
+  process::Future<bool> launch = containerizer.get()->launch(
+      containerId,
+      None(),
+      createExecutorInfo("executor", command),
+      directory,
+      None(),
+      SlaveID(),
+      std::map<string, string>(),
+      false);
+
+  AWAIT_READY(launch);
+  ASSERT_TRUE(launch.get());
+
+  // Wait on the container.
+  Future<Option<ContainerTermination>> wait =
+    containerizer.get()->wait(containerId);
+
+  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.
+  Try<ino_t> testIPCNamespace = ns::getns(::getpid(), "ipc");
+  ASSERT_SOME(testIPCNamespace);
+
+  Try<string> containerIPCNamespace = os::read(path::join(directory, "ns"));
+  ASSERT_SOME(containerIPCNamespace);
+
+  EXPECT_NE(stringify(testIPCNamespace.get()),
+            strings::trim(containerIPCNamespace.get()));
+
+  // Check that we modified the IPC shmmax of the namespace, not the host.
+  Try<uint64_t> childShmmax = readValue("shmmax");
+  ASSERT_SOME(childShmmax);
+
+  // 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());
+}
 #endif // __linux__
 
 } // namespace tests {


[2/3] mesos git commit: Use a common fixture for the PID namespace test.

Posted by ji...@apache.org.
Use a common fixture for the PID namespace test.

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


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

Branch: refs/heads/master
Commit: a03ee9b3563d2191bf1d5575b7a2ddb1d3566898
Parents: 25ac581
Author: James Peach <jp...@apache.org>
Authored: Mon Dec 5 13:18:18 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Dec 5 13:18:18 2016 -0800

----------------------------------------------------------------------
 src/tests/containerizer/isolator_tests.cpp | 29 +++++++------------------
 1 file changed, 8 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a03ee9b3/src/tests/containerizer/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/isolator_tests.cpp b/src/tests/containerizer/isolator_tests.cpp
index 111e1ab..355e15f 100644
--- a/src/tests/containerizer/isolator_tests.cpp
+++ b/src/tests/containerizer/isolator_tests.cpp
@@ -99,32 +99,17 @@ public:
 };
 
 
-class NamespacesPidIsolatorTest : public MesosTest {};
-
-
-TEST_F(NamespacesPidIsolatorTest, ROOT_PidNamespace)
+TEST_F(NamespacesIsolatorTest, ROOT_PidNamespace)
 {
-  slave::Flags flags = CreateSlaveFlags();
-  flags.isolation = "filesystem/linux,namespaces/pid";
-
-  string directory = os::getcwd(); // We're inside a temporary sandbox.
-
-  Fetcher fetcher;
-
-  Try<MesosContainerizer*> _containerizer =
-    MesosContainerizer::create(flags, false, &fetcher);
-
-  ASSERT_SOME(_containerizer);
-  Owned<MesosContainerizer> containerizer(_containerizer.get());
-
-  ContainerID containerId;
-  containerId.set_value(UUID::random().toString());
+  Try<Owned<MesosContainerizer>> containerizer =
+    createContainerizer("filesystem/linux,namespaces/pid");
+  ASSERT_SOME(containerizer);
 
   // Write the command's pid namespace inode and init name to files.
   const string command =
     "stat -c %i /proc/self/ns/pid > ns && (cat /proc/1/comm > init)";
 
-  Future<bool> launch = containerizer->launch(
+  process::Future<bool> launch = containerizer.get()->launch(
       containerId,
       None(),
       createExecutorInfo("executor", command),
@@ -138,7 +123,9 @@ TEST_F(NamespacesPidIsolatorTest, ROOT_PidNamespace)
   ASSERT_TRUE(launch.get());
 
   // Wait on the container.
-  Future<Option<ContainerTermination>> wait = containerizer->wait(containerId);
+  Future<Option<ContainerTermination>> wait =
+    containerizer.get()->wait(containerId);
+
   AWAIT_READY(wait);
   ASSERT_SOME(wait.get());
 


[3/3] mesos git commit: Add namespaces/ipc documentation.

Posted by ji...@apache.org.
Add namespaces/ipc documentation.

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


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

Branch: refs/heads/master
Commit: bc4a83b49ea06a9ba2992c910f3b4cd310f5a871
Parents: a03ee9b
Author: James Peach <jp...@apache.org>
Authored: Mon Dec 5 13:18:21 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Mon Dec 5 13:18:21 2016 -0800

----------------------------------------------------------------------
 docs/mesos-containerizer.md | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/bc4a83b4/docs/mesos-containerizer.md
----------------------------------------------------------------------
diff --git a/docs/mesos-containerizer.md b/docs/mesos-containerizer.md
index 2bff35f..6d13b2d 100644
--- a/docs/mesos-containerizer.md
+++ b/docs/mesos-containerizer.md
@@ -282,6 +282,15 @@ the container&mdash; in the agent's [/state](endpoints/slave/state.md) endpoint.
 This is described in a [separate document](docker-volume.md).
 
 
+### The `namespaces/ipc` Isolator
+
+The IPC Namespace isolator can be used on Linux to place tasks
+in a distinct IPC namespace. The benefit of this is that any
+[IPC objects](http://man7.org/linux/man-pages/man7/svipc.7.html) created
+in the container will be automatically removed when the container is
+destroyed.
+
+
 ### The `network/cni` Isolator
 
 This is described in a [separate document](cni.md).