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 2018/06/25 21:43:32 UTC

[1/5] mesos git commit: Allowed mounts if the container is launched in a new mount namespace.

Repository: mesos
Updated Branches:
  refs/heads/master 004fb5fa2 -> 00b688815


Allowed mounts if the container is launched in a new mount namespace.

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


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

Branch: refs/heads/master
Commit: eb61bbcccc3f076d02b1726bfdaa247a22980eea
Parents: 004fb5f
Author: Qian Zhang <zh...@gmail.com>
Authored: Mon Jun 25 14:39:31 2018 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jun 25 14:39:31 2018 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/launch.cpp | 6 ------
 1 file changed, 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/eb61bbcc/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index 65b795a..7193da0 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -708,12 +708,6 @@ int MesosContainerizerLaunch::execute()
   // If we need a new mount namespace, we have to do it before
   // we make the mounts needed to prepare the rootfs template.
   if (flags.unshare_namespace_mnt) {
-    if (!launchInfo.mounts().empty()) {
-      cerr << "Mounts are not supported if "
-           << "'unshare_namespace_mnt' is set" << endl;
-      exitWithStatus(EXIT_FAILURE);
-    }
-
     if (unshare(CLONE_NEWNS) != 0) {
       cerr << "Failed to unshare mount namespace: "
            << os::strerror(errno) << endl;


[4/5] mesos git commit: Added a test `ROOT_CGROUPS_CommandTaskSpecificCgroupsMount`.

Posted by gi...@apache.org.
Added a test `ROOT_CGROUPS_CommandTaskSpecificCgroupsMount`.

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


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

Branch: refs/heads/master
Commit: 5feaad6f35f2e278c30d30716a71650dca4735c6
Parents: 7616c4c
Author: Qian Zhang <zh...@gmail.com>
Authored: Mon Jun 25 14:39:44 2018 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jun 25 14:41:48 2018 -0700

----------------------------------------------------------------------
 .../containerizer/cgroups_isolator_tests.cpp    | 72 ++++++++++++++++++++
 1 file changed, 72 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5feaad6f/src/tests/containerizer/cgroups_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/cgroups_isolator_tests.cpp b/src/tests/containerizer/cgroups_isolator_tests.cpp
index e541150..3041640 100644
--- a/src/tests/containerizer/cgroups_isolator_tests.cpp
+++ b/src/tests/containerizer/cgroups_isolator_tests.cpp
@@ -2009,6 +2009,78 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_NestedContainerSpecificCgroupsMount)
   ASSERT_EQ(taskInfo.task_id(), finishedUpdate->status().task_id());
 }
 
+
+// This test verifies container-specific cgroup is correctly mounted for the
+// command task.
+TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_CommandTaskSpecificCgroupsMount)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/linux,docker/runtime,cgroups/mem";
+  flags.image_providers = "docker";
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
+  ASSERT_SOME(slave);
+
+  MockScheduler sched;
+  MesosSchedulerDriver driver(
+      &sched, DEFAULT_FRAMEWORK_INFO, master.get()->pid, DEFAULT_CREDENTIAL);
+
+  EXPECT_CALL(sched, registered(&driver, _, _));
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(&driver, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_EQ(1u, offers->size());
+
+  // Create a task to check if its memory (including both executor and task's
+  // memory) is correctly set in its specific cgroup, e.g.:
+  //  `/sys/fs/cgroup/memory/memory.soft_limit_in_bytes`
+  TaskInfo task = createTask(
+      offers->front().slave_id(),
+      Resources::parse("cpus:0.1;mem:32;disk:32").get(),
+      "test `cat /sys/fs/cgroup/memory/memory.soft_limit_in_bytes` = 67108864");
+
+  Image image;
+  image.set_type(Image::DOCKER);
+  image.mutable_docker()->set_name("alpine");
+
+  ContainerInfo* container = task.mutable_container();
+  container->set_type(ContainerInfo::MESOS);
+  container->mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  Future<TaskStatus> statusStarting;
+  Future<TaskStatus> statusRunning;
+  Future<TaskStatus> statusFinished;
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillOnce(FutureArg<1>(&statusStarting))
+    .WillOnce(FutureArg<1>(&statusRunning))
+    .WillOnce(FutureArg<1>(&statusFinished));
+
+  driver.launchTasks(offers->front().id(), {task});
+
+  AWAIT_READY(statusStarting);
+  EXPECT_EQ(TASK_STARTING, statusStarting->state());
+
+  AWAIT_READY(statusRunning);
+  EXPECT_EQ(TASK_RUNNING, statusRunning->state());
+
+  AWAIT_READY(statusFinished);
+  EXPECT_EQ(TASK_FINISHED, statusFinished->state());
+
+  driver.stop();
+  driver.join();
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {


[5/5] mesos git commit: Removed an unused member variable `hierarchies` from cgroups isolator.

Posted by gi...@apache.org.
Removed an unused member variable `hierarchies` from cgroups isolator.

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


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

Branch: refs/heads/master
Commit: 00b688815be6f3d429b2035e7e926a56f00fd18b
Parents: 5feaad6
Author: Qian Zhang <zh...@gmail.com>
Authored: Mon Jun 25 14:39:48 2018 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jun 25 14:41:48 2018 -0700

----------------------------------------------------------------------
 .../containerizer/mesos/isolators/cgroups/cgroups.cpp  | 13 +------------
 .../containerizer/mesos/isolators/cgroups/cgroups.hpp  |  4 ----
 2 files changed, 1 insertion(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/00b68881/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
index 316d2c0..8fba6f9 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
@@ -60,11 +60,9 @@ namespace slave {
 
 CgroupsIsolatorProcess::CgroupsIsolatorProcess(
     const Flags& _flags,
-    const hashmap<string, string>& _hierarchies,
     const multihashmap<string, Owned<Subsystem>>& _subsystems)
   : ProcessBase(process::ID::generate("cgroups-isolator")),
     flags(_flags),
-    hierarchies(_hierarchies),
     subsystems(_subsystems) {}
 
 
@@ -73,9 +71,6 @@ CgroupsIsolatorProcess::~CgroupsIsolatorProcess() {}
 
 Try<Isolator*> CgroupsIsolatorProcess::create(const Flags& flags)
 {
-  // Subsystem name -> hierarchy path.
-  hashmap<string, string> hierarchies;
-
   // Hierarchy path -> subsystem object.
   multihashmap<string, Owned<Subsystem>> subsystems;
 
@@ -153,11 +148,6 @@ Try<Isolator*> CgroupsIsolatorProcess::create(const Flags& flags)
   CHECK(!subsystemSet.empty());
 
   foreach (const string& subsystemName, subsystemSet) {
-    if (hierarchies.contains(subsystemName)) {
-      // Skip when the subsystem exists.
-      continue;
-    }
-
     // Prepare hierarchy if it does not exist.
     Try<string> hierarchy = cgroups::prepare(
         flags.cgroups_hierarchy,
@@ -181,11 +171,10 @@ Try<Isolator*> CgroupsIsolatorProcess::create(const Flags& flags)
     }
 
     subsystems.put(hierarchy.get(), subsystem.get());
-    hierarchies.put(subsystemName, hierarchy.get());
   }
 
   Owned<MesosIsolatorProcess> process(
-      new CgroupsIsolatorProcess(flags, hierarchies, subsystems));
+      new CgroupsIsolatorProcess(flags, subsystems));
 
   return new MesosIsolator(process);
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/00b68881/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
index 480aedf..cde1ace 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
@@ -101,7 +101,6 @@ private:
 
   CgroupsIsolatorProcess(
       const Flags& _flags,
-      const hashmap<std::string, std::string>& _hierarchies,
       const multihashmap<std::string, process::Owned<Subsystem>>&
         _subsystems);
 
@@ -150,9 +149,6 @@ private:
 
   const Flags flags;
 
-  // Map from subsystem name to hierarchy path.
-  hashmap<std::string, std::string> hierarchies;
-
   // We map hierarchy path and `Subsystem` in subsystems. Same hierarchy may
   // map to multiple Subsystems. For example, our cgroups hierarchies may
   // mount like below in the machine:


[2/5] mesos git commit: Added container-specific cgroup FS mounts.

Posted by gi...@apache.org.
Added container-specific cgroup FS mounts.

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


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

Branch: refs/heads/master
Commit: ae90054f6d8fcd7387bac26a4caa458e4ddb597c
Parents: eb61bbc
Author: Qian Zhang <zh...@gmail.com>
Authored: Mon Jun 25 14:39:36 2018 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jun 25 14:39:36 2018 -0700

----------------------------------------------------------------------
 src/launcher/executor.cpp                       | 41 +++++++-
 src/linux/fs.cpp                                |  7 ++
 .../mesos/isolators/cgroups/cgroups.cpp         | 99 +++++++++++++++++++-
 .../mesos/isolators/cgroups/cgroups.hpp         | 12 ++-
 4 files changed, 147 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ae90054f/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 541ca5b..9fb43c0 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -111,6 +111,8 @@ using mesos::Environment;
 using mesos::executor::Call;
 using mesos::executor::Event;
 
+using mesos::slave::ContainerLaunchInfo;
+
 using mesos::v1::executor::Mesos;
 using mesos::v1::executor::MesosBase;
 using mesos::v1::executor::V0ToV1Adapter;
@@ -131,6 +133,7 @@ public:
       const Option<Environment>& _taskEnvironment,
       const Option<CapabilityInfo>& _effectiveCapabilities,
       const Option<CapabilityInfo>& _boundingCapabilities,
+      const Option<ContainerLaunchInfo>& _taskLaunchInfo,
       const FrameworkID& _frameworkId,
       const ExecutorID& _executorId,
       const Duration& _shutdownGracePeriod)
@@ -155,6 +158,7 @@ public:
       taskEnvironment(_taskEnvironment),
       effectiveCapabilities(_effectiveCapabilities),
       boundingCapabilities(_boundingCapabilities),
+      taskLaunchInfo(_taskLaunchInfo),
       frameworkId(_frameworkId),
       executorId(_executorId),
       lastTaskStatus(None()) {}
@@ -406,12 +410,13 @@ protected:
       const Option<string>& sandboxDirectory,
       const Option<string>& workingDirectory,
       const Option<CapabilityInfo>& effectiveCapabilities,
-      const Option<CapabilityInfo>& boundingCapabilities)
+      const Option<CapabilityInfo>& boundingCapabilities,
+      const Option<ContainerLaunchInfo>& taskLaunchInfo)
   {
     // Prepare the flags to pass to the launch process.
     slave::MesosContainerizerLaunch::Flags launchFlags;
 
-    ::mesos::slave::ContainerLaunchInfo launchInfo;
+    ContainerLaunchInfo launchInfo;
     launchInfo.mutable_command()->CopyFrom(command);
 
 #ifndef __WINDOWS__
@@ -473,6 +478,15 @@ protected:
           boundingCapabilities.get());
     }
 
+    if (taskLaunchInfo.isSome()) {
+      launchInfo.mutable_mounts()->CopyFrom(taskLaunchInfo->mounts());
+      launchInfo.mutable_pre_exec_commands()->CopyFrom(
+          taskLaunchInfo->pre_exec_commands());
+
+      launchInfo.mutable_clone_namespaces()->CopyFrom(
+          taskLaunchInfo->clone_namespaces());
+    }
+
     launchFlags.launch_info = JSON::protobuf(launchInfo);
 
     // TODO(tillt): Consider using a flag allowing / disallowing the
@@ -673,7 +687,8 @@ protected:
         sandboxDirectory,
         workingDirectory,
         effectiveCapabilities,
-        boundingCapabilities);
+        boundingCapabilities,
+        taskLaunchInfo);
 
     LOG(INFO) << "Forked command at " << pid.get();
 
@@ -1203,6 +1218,7 @@ private:
   Option<Environment> taskEnvironment;
   Option<CapabilityInfo> effectiveCapabilities;
   Option<CapabilityInfo> boundingCapabilities;
+  Option<ContainerLaunchInfo> taskLaunchInfo;
   const FrameworkID frameworkId;
   const ExecutorID executorId;
   Owned<MesosBase> mesos;
@@ -1262,6 +1278,10 @@ public:
         "bounding_capabilities",
         "The bounding set of capabilities the command can use.");
 
+    add(&Flags::task_launch_info,
+        "task_launch_info",
+        "The launch info to run the task.");
+
     add(&Flags::launcher_dir,
         "launcher_dir",
         "Directory path of Mesos binaries.",
@@ -1279,6 +1299,7 @@ public:
   Option<Environment> task_environment;
   Option<mesos::CapabilityInfo> effective_capabilities;
   Option<mesos::CapabilityInfo> bounding_capabilities;
+  Option<JSON::Object> task_launch_info;
   string launcher_dir;
 };
 
@@ -1343,6 +1364,19 @@ int main(int argc, char** argv)
     shutdownGracePeriod = parse.get();
   }
 
+  Option<ContainerLaunchInfo> task_launch_info;
+  if (flags.task_launch_info.isSome()) {
+    Try<ContainerLaunchInfo> parse =
+      protobuf::parse<ContainerLaunchInfo>(flags.task_launch_info.get());
+
+    if (parse.isError()) {
+      EXIT(EXIT_FAILURE)
+        << "Failed to parse task launch info: " << parse.error();
+    }
+
+    task_launch_info = parse.get();
+  }
+
   process::initialize();
 
   Owned<mesos::internal::CommandExecutor> executor(
@@ -1356,6 +1390,7 @@ int main(int argc, char** argv)
           flags.task_environment,
           flags.effective_capabilities,
           flags.bounding_capabilities,
+          task_launch_info,
           frameworkId,
           executorId,
           shutdownGracePeriod));

http://git-wip-us.apache.org/repos/asf/mesos/blob/ae90054f/src/linux/fs.cpp
----------------------------------------------------------------------
diff --git a/src/linux/fs.cpp b/src/linux/fs.cpp
index 6b38b4a..9055ef4 100644
--- a/src/linux/fs.cpp
+++ b/src/linux/fs.cpp
@@ -781,6 +781,13 @@ Try<Nothing> mountSpecialFilesystems(const string& root)
     },
     {
       "tmpfs",
+      "/sys/fs/cgroup",
+      "tmpfs",
+      "mode=755",
+      MS_NOSUID | MS_NOEXEC | MS_NODEV
+    },
+    {
+      "tmpfs",
       "/dev",
       "tmpfs",
       "mode=755",

http://git-wip-us.apache.org/repos/asf/mesos/blob/ae90054f/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
index 6d14672..316d2c0 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.cpp
@@ -32,6 +32,8 @@
 #include "common/protobuf_utils.hpp"
 
 #include "linux/cgroups.hpp"
+#include "linux/fs.hpp"
+#include "linux/ns.hpp"
 
 #include "slave/containerizer/mesos/isolators/cgroups/cgroups.hpp"
 #include "slave/containerizer/mesos/isolators/cgroups/constants.hpp"
@@ -39,6 +41,7 @@
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerMountInfo;
 using mesos::slave::ContainerState;
 using mesos::slave::Isolator;
 
@@ -416,11 +419,11 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::prepare(
     const ContainerID& containerId,
     const ContainerConfig& containerConfig)
 {
-  // If we are a nested container, we do not need to prepare
-  // anything since only top-level containers should have cgroups
-  // created for them.
+  // Only prepare cgroups for top-level containers. Nested container
+  // will inherit cgroups from its root container, so here we just
+  // need to do the container-specific cgroups mounts.
   if (containerId.has_parent()) {
-    return None();
+    return __prepare(containerId, containerConfig);
   }
 
   if (infos.contains(containerId)) {
@@ -550,7 +553,93 @@ Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::_prepare(
   }
 
   return update(containerId, containerConfig.resources())
-    .then([]() { return Option<ContainerLaunchInfo>::none(); });
+    .then(defer(
+        PID<CgroupsIsolatorProcess>(this),
+        &CgroupsIsolatorProcess::__prepare,
+        containerId,
+        containerConfig));
+}
+
+
+Future<Option<ContainerLaunchInfo>> CgroupsIsolatorProcess::__prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  // We will do container-specific cgroups mounts
+  // only for the container with rootfs.
+  if (!containerConfig.has_rootfs()) {
+    return None();
+  }
+
+  const ContainerID rootContainerId = protobuf::getRootContainerId(containerId);
+
+  CHECK(infos.contains(rootContainerId));
+
+  ContainerLaunchInfo launchInfo;
+  launchInfo.add_clone_namespaces(CLONE_NEWNS);
+
+  // For the comounted subsystems (e.g., cpu & cpuacct, net_cls & net_prio),
+  // we need to create a symbolic link for each of them to the mount point.
+  // E.g.: ln -s /sys/fs/cgroup/cpu,cpuacct /sys/fs/cgroup/cpu
+  //       ln -s /sys/fs/cgroup/cpu,cpuacct /sys/fs/cgroup/cpuacct
+  foreach (const string& hierarchy, subsystems.keys()) {
+    if (subsystems.get(hierarchy).size() > 1) {
+      foreach (const Owned<Subsystem>& subsystem, subsystems.get(hierarchy)) {
+        CommandInfo* command = launchInfo.add_pre_exec_commands();
+        command->set_shell(false);
+        command->set_value("ln");
+        command->add_arguments("ln");
+        command->add_arguments("-s");
+        command->add_arguments(
+            path::join("/sys/fs/cgroup", Path(hierarchy).basename()));
+
+        command->add_arguments(path::join(
+            containerConfig.rootfs(),
+            "/sys/fs/cgroup",
+            subsystem->name()));
+      }
+    }
+  }
+
+  // For the subsystem loaded by this isolator, do the container-specific
+  // cgroups mount, e.g.:
+  //   mount --bind /sys/fs/cgroup/memory/mesos/<containerId> /sys/fs/cgroup/memory // NOLINT(whitespace/line_length)
+  foreach (const string& hierarchy, subsystems.keys()) {
+    ContainerMountInfo* mount = launchInfo.add_mounts();
+    mount->set_source(path::join(hierarchy, infos[rootContainerId]->cgroup));
+
+    mount->set_target(path::join(
+        containerConfig.rootfs(),
+        "/sys/fs/cgroup",
+        Path(hierarchy).basename()));
+
+    mount->set_flags(MS_BIND | MS_REC);
+  }
+
+  // TODO(qianzhang): This is a hack to pass the container-specific cgroups
+  // mounts and the symbolic links to the command executor to do for the
+  // command task. The reasons that we do it in this way are:
+  //   1. We need to ensure the container-specific cgroups mounts are done
+  //      only in the command task's mount namespace but not in the command
+  //      executor's mount namespace.
+  //   2. Even it's acceptable to do the container-specific cgroups mounts
+  //      in the command executor's mount namespace and the command task
+  //      inherit them from there (i.e., here we just return `launchInfo`
+  //      rather than passing it via `--task_launch_info`), the container
+  //      specific cgroups mounts will be hidden by the `sysfs` mounts done in
+  //      `mountSpecialFilesystems()` when the command executor launches the
+  //      command task.
+  if (containerConfig.has_task_info()) {
+    ContainerLaunchInfo _launchInfo;
+
+    _launchInfo.mutable_command()->add_arguments(
+        "--task_launch_info=" +
+        stringify(JSON::protobuf(launchInfo)));
+
+    return _launchInfo;
+  }
+
+  return launchInfo;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/ae90054f/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
index 81c9343..480aedf 100644
--- a/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
+++ b/src/slave/containerizer/mesos/isolators/cgroups/cgroups.hpp
@@ -114,18 +114,22 @@ private:
       const std::vector<process::Future<Nothing>>& futures);
 
   process::Future<Nothing> ___recover(
-    const ContainerID& containerId);
+      const ContainerID& containerId);
 
   process::Future<Nothing> ____recover(
-    const ContainerID& containerId,
-    const hashset<std::string>& recoveredSubsystems,
-    const std::vector<process::Future<Nothing>>& futures);
+      const ContainerID& containerId,
+      const hashset<std::string>& recoveredSubsystems,
+      const std::vector<process::Future<Nothing>>& futures);
 
   process::Future<Option<mesos::slave::ContainerLaunchInfo>> _prepare(
       const ContainerID& containerId,
       const mesos::slave::ContainerConfig& containerConfig,
       const std::vector<process::Future<Nothing>>& futures);
 
+  process::Future<Option<mesos::slave::ContainerLaunchInfo>> __prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
   process::Future<Nothing> _isolate(
       const std::vector<process::Future<Nothing>>& futures);
 


[3/5] mesos git commit: Added a test `ROOT_CGROUPS_NestedContainerSpecificCgroupsMount`.

Posted by gi...@apache.org.
Added a test `ROOT_CGROUPS_NestedContainerSpecificCgroupsMount`.

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


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

Branch: refs/heads/master
Commit: 7616c4caec9d1bbb9f2ac645971244b55d2b9f42
Parents: ae90054
Author: Qian Zhang <zh...@gmail.com>
Authored: Mon Jun 25 14:39:40 2018 -0700
Committer: Gilbert Song <so...@gmail.com>
Committed: Mon Jun 25 14:41:42 2018 -0700

----------------------------------------------------------------------
 .../containerizer/cgroups_isolator_tests.cpp    | 106 +++++++++++++++++++
 1 file changed, 106 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7616c4ca/src/tests/containerizer/cgroups_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/cgroups_isolator_tests.cpp b/src/tests/containerizer/cgroups_isolator_tests.cpp
index d1028e3..e541150 100644
--- a/src/tests/containerizer/cgroups_isolator_tests.cpp
+++ b/src/tests/containerizer/cgroups_isolator_tests.cpp
@@ -1903,6 +1903,112 @@ TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_AutoLoadSubsystems)
   driver.join();
 }
 
+
+// This test verifies container-specific cgroup is correctly mounted inside the
+// nested container.
+TEST_F(CgroupsIsolatorTest, ROOT_CGROUPS_NestedContainerSpecificCgroupsMount)
+{
+  // Disable AuthN on the agent.
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/linux,docker/runtime,cgroups/mem";
+  flags.image_providers = "docker";
+  flags.authenticate_http_readwrite = false;
+
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  Try<Owned<cluster::Slave>> slave = StartSlave(detector.get(), flags);
+  ASSERT_SOME(slave);
+
+  auto scheduler = std::make_shared<v1::MockHTTPScheduler>();
+
+  EXPECT_CALL(*scheduler, connected(_))
+    .WillOnce(v1::scheduler::SendSubscribe(v1::DEFAULT_FRAMEWORK_INFO));
+
+  Future<Event::Subscribed> subscribed;
+  EXPECT_CALL(*scheduler, subscribed(_, _))
+    .WillOnce(FutureArg<1>(&subscribed));
+
+  Future<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);
+
+  AWAIT_READY(offers);
+  ASSERT_FALSE(offers->offers().empty());
+
+  const v1::Offer& offer = offers->offers(0);
+
+  // Create a task to check if its memory (including both executor and task's
+  // memory) is correctly set in its specific cgroup, e.g.:
+  //  `/sys/fs/cgroup/memory/memory.soft_limit_in_bytes`
+  v1::TaskInfo taskInfo = v1::createTask(
+      offer.agent_id(),
+      v1::Resources::parse("cpus:0.1;mem:32;disk:32").get(),
+      "test `cat /sys/fs/cgroup/memory/memory.soft_limit_in_bytes` = 67108864");
+
+  mesos::v1::Image image;
+  image.set_type(mesos::v1::Image::DOCKER);
+  image.mutable_docker()->set_name("alpine");
+
+  mesos::v1::ContainerInfo* container = taskInfo.mutable_container();
+  container->set_type(mesos::v1::ContainerInfo::MESOS);
+  container->mutable_mesos()->mutable_image()->CopyFrom(image);
+
+  Future<v1::scheduler::Event::Update> startingUpdate;
+  Future<v1::scheduler::Event::Update> runningUpdate;
+  Future<v1::scheduler::Event::Update> finishedUpdate;
+  EXPECT_CALL(*scheduler, update(_, _))
+    .WillOnce(DoAll(
+        FutureArg<1>(&startingUpdate),
+        v1::scheduler::SendAcknowledge(frameworkId, offer.agent_id())))
+    .WillOnce(DoAll(
+        FutureArg<1>(&runningUpdate),
+        v1::scheduler::SendAcknowledge(frameworkId, offer.agent_id())))
+    .WillOnce(DoAll(
+        FutureArg<1>(&finishedUpdate),
+        v1::scheduler::SendAcknowledge(frameworkId, offer.agent_id())));
+
+  mesos.send(
+      v1::createCallAccept(
+          frameworkId,
+          offer,
+          {v1::LAUNCH_GROUP(
+              executorInfo, v1::createTaskGroupInfo({taskInfo}))}));
+
+  AWAIT_READY(startingUpdate);
+  ASSERT_EQ(v1::TASK_STARTING, startingUpdate->status().state());
+  ASSERT_EQ(taskInfo.task_id(), startingUpdate->status().task_id());
+
+  AWAIT_READY(runningUpdate);
+  ASSERT_EQ(v1::TASK_RUNNING, runningUpdate->status().state());
+  ASSERT_EQ(taskInfo.task_id(), runningUpdate->status().task_id());
+
+  AWAIT_READY(finishedUpdate);
+  ASSERT_EQ(v1::TASK_FINISHED, finishedUpdate->status().state());
+  ASSERT_EQ(taskInfo.task_id(), finishedUpdate->status().task_id());
+}
+
 } // namespace tests {
 } // namespace internal {
 } // namespace mesos {