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/10/07 16:25:57 UTC

[1/5] mesos git commit: Created an isolator for Linux capabilities.

Repository: mesos
Updated Branches:
  refs/heads/master 4773b27dc -> 4ea9651aa


Created an isolator for Linux capabilities.

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


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

Branch: refs/heads/master
Commit: 4ea9651aabd01f623f2578d2823271488d924c5b
Parents: f6a2536
Author: Benjamin Bannier <be...@mesosphere.io>
Authored: Wed Oct 5 21:44:04 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Oct 7 09:22:19 2016 -0700

----------------------------------------------------------------------
 include/mesos/slave/containerizer.proto         |   4 +
 src/CMakeLists.txt                              |   1 +
 src/Makefile.am                                 |   5 +-
 src/launcher/posix/executor.cpp                 |   4 +-
 src/slave/containerizer/mesos/containerizer.cpp |  19 +
 .../mesos/isolators/linux/capabilities.cpp      | 143 ++++++++
 .../mesos/isolators/linux/capabilities.hpp      |  50 +++
 src/slave/containerizer/mesos/launch.cpp        |  26 +-
 src/slave/containerizer/mesos/launch.hpp        |   4 +-
 .../linux_capabilities_isolator_tests.cpp       | 354 +++++++++++++++++++
 10 files changed, 589 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/include/mesos/slave/containerizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/slave/containerizer.proto b/include/mesos/slave/containerizer.proto
index 86d1859..94e8cb4 100644
--- a/include/mesos/slave/containerizer.proto
+++ b/include/mesos/slave/containerizer.proto
@@ -163,6 +163,10 @@ message ContainerLaunchInfo {
   // The working directory for the container.
   // NOTE: This is different than Mesos sandbox.
   optional string working_directory = 6;
+
+  // (Linux only) The capabilities requested for the container.
+  // The capabilities are set while launching the container.
+  optional CapabilityInfo capabilities = 7;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt
index ba49d81..e60d34c 100644
--- a/src/CMakeLists.txt
+++ b/src/CMakeLists.txt
@@ -175,6 +175,7 @@ set(LINUX_SRC
   slave/containerizer/mesos/isolators/gpu/isolator.cpp
   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/pid.cpp
   slave/containerizer/mesos/isolators/network/cni/cni.cpp
   slave/containerizer/mesos/isolators/volume/image.cpp

http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 184afb5..8b507cd 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -828,9 +828,9 @@ libmesos_no_3rdparty_la_SOURCES +=					\
   slave/containerizer/mesos/launcher.cpp				\
   slave/containerizer/mesos/mount.cpp					\
   slave/containerizer/mesos/paths.cpp					\
-  slave/containerizer/mesos/isolators/filesystem/posix.cpp		\
   slave/containerizer/mesos/isolators/docker/volume/driver.cpp		\
   slave/containerizer/mesos/isolators/docker/volume/paths.cpp		\
+  slave/containerizer/mesos/isolators/filesystem/posix.cpp		\
   slave/containerizer/mesos/isolators/network/cni/paths.cpp		\
   slave/containerizer/mesos/isolators/network/cni/spec.cpp		\
   slave/containerizer/mesos/isolators/posix/disk.cpp			\
@@ -1048,6 +1048,7 @@ MESOS_LINUX_FILES =									\
   slave/containerizer/mesos/isolators/gpu/isolator.cpp					\
   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/pid.cpp				\
   slave/containerizer/mesos/isolators/network/cni/cni.cpp				\
   slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.cpp	\
@@ -1086,6 +1087,7 @@ MESOS_LINUX_FILES +=									\
   slave/containerizer/mesos/isolators/gpu/nvidia.hpp					\
   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/pid.hpp				\
   slave/containerizer/mesos/isolators/network/cni/cni.hpp				\
   slave/containerizer/mesos/isolators/network/cni/plugins/port_mapper/port_mapper.hpp	\
@@ -2201,6 +2203,7 @@ mesos_tests_DEPENDENCIES =					\
 if OS_LINUX
 mesos_tests_SOURCES +=						\
   tests/ldcache_tests.cpp					\
+  tests/containerizer/linux_capabilities_isolator_tests.cpp	\
   tests/containerizer/capabilities_tests.cpp			\
   tests/containerizer/capabilities_test_helper.cpp		\
   tests/containerizer/cgroups_isolator_tests.cpp		\

http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/src/launcher/posix/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.cpp b/src/launcher/posix/executor.cpp
index 8a21191..fdee17c 100644
--- a/src/launcher/posix/executor.cpp
+++ b/src/launcher/posix/executor.cpp
@@ -99,9 +99,7 @@ pid_t launchTaskPosix(
   launchFlags.user = user;
 
 #ifdef __linux__
-  launchFlags.capabilities = capabilities.isSome()
-    ? JSON::protobuf(capabilities.get())
-    : Option<JSON::Object>::none();
+  launchFlags.capabilities = capabilities;
 #endif // __linux__
 
   string commandString = strings::format(

http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 405e051..32058c3 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -85,6 +85,7 @@
 #include "slave/containerizer/mesos/isolators/filesystem/linux.hpp"
 #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/pid.hpp"
 #include "slave/containerizer/mesos/isolators/network/cni/cni.hpp"
 #include "slave/containerizer/mesos/isolators/volume/image.hpp"
@@ -304,6 +305,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     {"appc/runtime", &AppcRuntimeIsolatorProcess::create},
     {"docker/runtime", &DockerRuntimeIsolatorProcess::create},
     {"docker/volume", &DockerVolumeIsolatorProcess::create},
+    {"linux/capabilities", &LinuxCapabilitiesIsolatorProcess::create},
 
     {"volume/image",
       [&provisioner] (const Flags& flags) -> Try<Isolator*> {
@@ -1226,6 +1228,7 @@ Future<bool> MesosContainerizerProcess::_launch(
   Option<CommandInfo> launchCommand;
   Option<string> workingDirectory;
   JSON::Array preExecCommands;
+  Option<CapabilityInfo> capabilities;
 
   // TODO(jieyu): We should use Option here. If no namespace is
   // required, we should pass None() to 'launcher->fork'.
@@ -1288,6 +1291,15 @@ Future<bool> MesosContainerizerProcess::_launch(
     if (launchInfo->has_namespaces()) {
       namespaces |= launchInfo->namespaces();
     }
+
+    if (launchInfo->has_capabilities()) {
+      if (capabilities.isSome()) {
+        return Failure(
+            "At most one capabilities set can be returned from isolators");
+      } else {
+        capabilities = launchInfo->capabilities();
+      }
+    }
   }
 
   // Determine the launch command for the container.
@@ -1378,6 +1390,13 @@ Future<bool> MesosContainerizerProcess::_launch(
         : flags.sandbox_directory;
     }
 
+#ifdef __linux__
+    // TODO(bbannier): For the case where the user requested
+    // capabilities, but no capabilities isolation was configured for
+    // the agent, the master should reject the task.
+    launchFlags.capabilities = capabilities;
+#endif // __linux__
+
 #ifdef __WINDOWS__
     if (rootfs.isSome()) {
       return Failure(

http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/src/slave/containerizer/mesos/isolators/linux/capabilities.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/linux/capabilities.cpp b/src/slave/containerizer/mesos/isolators/linux/capabilities.cpp
new file mode 100644
index 0000000..60d22aa
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/linux/capabilities.cpp
@@ -0,0 +1,143 @@
+// 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 <unistd.h>
+
+#include <sys/types.h>
+
+#include <algorithm>
+#include <set>
+
+#include <stout/set.hpp>
+
+#include "linux/capabilities.hpp"
+
+#include "slave/containerizer/mesos/isolators/linux/capabilities.hpp"
+
+using std::set;
+
+using process::Failure;
+using process::Future;
+using process::Owned;
+
+using mesos::internal::capabilities::Capabilities;
+using mesos::internal::capabilities::Capability;
+using mesos::internal::capabilities::convert;
+
+using mesos::slave::ContainerConfig;
+using mesos::slave::ContainerLaunchInfo;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+Try<Isolator*> LinuxCapabilitiesIsolatorProcess::create(const Flags& flags)
+{
+  if (geteuid() != 0) {
+    return Error("Linux capabilities isolator requires root permissions");
+  }
+
+  Try<Capabilities> create = Capabilities::create();
+  if (create.isError()) {
+    return Error("Failed to initialize capabilities: " + create.error());
+  }
+
+  return new MesosIsolator(
+      Owned<MesosIsolatorProcess>(new LinuxCapabilitiesIsolatorProcess(flags)));
+}
+
+
+Future<Option<ContainerLaunchInfo>> LinuxCapabilitiesIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ContainerConfig& containerConfig)
+{
+  // Determine the capabilities of the container that we want to
+  // launch. None() here means that we don't want to set capabilities
+  // for the container.
+  Option<CapabilityInfo> capabilities = None();
+
+  if (containerConfig.has_container_info() &&
+      containerConfig.container_info().has_linux_info() &&
+      containerConfig.container_info().linux_info().has_capability_info()) {
+    capabilities =
+      containerConfig.container_info().linux_info().capability_info();
+  }
+
+  // If both the framework sets the capabilities for the container and
+  // the operator sets the allowed capabilities on the agent, we need
+  // to verify that the request from the framework is allowed.
+  if (capabilities.isSome() && flags.allowed_capabilities.isSome()) {
+    const set<Capability> requested = convert(capabilities.get());
+    const set<Capability> allowed = convert(flags.allowed_capabilities.get());
+
+    if ((requested & allowed).size() != requested.size()) {
+      return Failure(
+          "Capabilities requested '" + stringify(requested) + "', "
+          "but only '" + stringify(allowed) + "' are allowed");
+    }
+  }
+
+  // If the framework does not set the capabilities and the operator
+  // sets the allowed capabilities, use that as the capabilities for
+  // the container.
+  if (capabilities.isNone() && flags.allowed_capabilities.isSome()) {
+    capabilities = flags.allowed_capabilities.get();
+  }
+
+  // If no capabilities need to be set for the container, we do not
+  // need to modify the container launch info.
+  if (capabilities.isNone()) {
+    return None();
+  }
+
+  ContainerLaunchInfo launchInfo;
+
+  if (containerConfig.has_task_info()) {
+    // Command task case.
+    // 1) If the command task specifies a root filesystem, we need the
+    //    command executor to be run with full privileges to perform
+    //    operations like 'pivot_root'. Therefore, we set the command
+    //    executor flags and the command executor will set the
+    //    capabilities when it launches the task.
+    // 2) If the command task does not specify a root filesystem, we
+    //    will set the capabilities before we execute the command
+    //    executor. Command executor itself does not require any
+    //    capabilities to execute the user task in that case.
+    //
+    // TODO(jieyu): The caveat in case 2) is that the command executor
+    // should be executable after the process drops capabilities.  For
+    // containers that want to run under root, if the requested
+    // capability for the container does not have CAP_DAC_READ_SEARCH,
+    // the exec will fail with EACCESS, which is not the typical
+    // behavior a user would expect from a root user.
+    if (containerConfig.has_rootfs()) {
+      launchInfo.mutable_command()->add_arguments(
+          "--capabilities=" + stringify(JSON::protobuf(capabilities.get())));
+    } else {
+      launchInfo.mutable_capabilities()->CopyFrom(capabilities.get());
+    }
+  } else {
+    // Custom executor or nested container.
+    launchInfo.mutable_capabilities()->CopyFrom(capabilities.get());
+  }
+
+  return launchInfo;
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/src/slave/containerizer/mesos/isolators/linux/capabilities.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/linux/capabilities.hpp b/src/slave/containerizer/mesos/isolators/linux/capabilities.hpp
new file mode 100644
index 0000000..c3afe20
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/linux/capabilities.hpp
@@ -0,0 +1,50 @@
+// 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 __LINUX_CAPABILITIES_ISOLATOR_HPP__
+#define __LINUX_CAPABILITIES_ISOLATOR_HPP__
+
+#include <stout/try.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/mesos/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class LinuxCapabilitiesIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(const Flags& flags);
+
+  virtual process::Future<Option<mesos::slave::ContainerLaunchInfo>> prepare(
+      const ContainerID& containerId,
+      const mesos::slave::ContainerConfig& containerConfig);
+
+private:
+  LinuxCapabilitiesIsolatorProcess(const Flags& _flags)
+    : flags(_flags) {}
+
+  const Flags flags;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif  // __LINUX_CAPABILITIES_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index 7dd10d5..c6b669a 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -31,14 +31,17 @@
 #include <stout/path.hpp>
 #include <stout/unreachable.hpp>
 
+#include <mesos/mesos.hpp>
+#include <mesos/type_utils.hpp>
+
+#include "common/parse.hpp"
+
 #ifdef __linux__
 #include "linux/capabilities.hpp"
 #include "linux/fs.hpp"
 #include "linux/ns.hpp"
 #endif
 
-#include "mesos/mesos.hpp"
-
 #include "slave/containerizer/mesos/launch.hpp"
 #include "slave/containerizer/mesos/paths.hpp"
 
@@ -110,14 +113,14 @@ MesosContainerizerLaunch::Flags::Flags()
       "executing the command.");
 
 #ifdef __linux__
+  add(&capabilities,
+      "capabilities",
+      "Capabilities the command can use.");
+
   add(&unshare_namespace_mnt,
       "unshare_namespace_mnt",
       "Whether to launch the command in a new mount namespace.",
       false);
-
-  add(&capabilities,
-      "capabilities",
-      "Capabilities of the command can use.");
 #endif // __linux__
 }
 
@@ -557,15 +560,6 @@ int MesosContainerizerLaunch::execute()
 
 #ifdef __linux__
   if (flags.capabilities.isSome()) {
-    Try<CapabilityInfo> requestedCapabilities =
-      ::protobuf::parse<CapabilityInfo>(flags.capabilities.get());
-
-    if (requestedCapabilities.isError()) {
-      cerr << "Failed to parse capabilities: "
-           << requestedCapabilities.error() << endl;
-      exitWithStatus(EXIT_FAILURE);
-    }
-
     Try<ProcessCapabilities> capabilities = capabilitiesManager->get();
     if (capabilities.isError()) {
       cerr << "Failed to get capabilities for the current process: "
@@ -586,7 +580,7 @@ int MesosContainerizerLaunch::execute()
     }
 
     // Set up requested capabilities.
-    set<Capability> target = capabilities::convert(requestedCapabilities.get());
+    set<Capability> target = capabilities::convert(flags.capabilities.get());
 
     capabilities->set(capabilities::EFFECTIVE, target);
     capabilities->set(capabilities::PERMITTED, target);

http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/src/slave/containerizer/mesos/launch.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.hpp b/src/slave/containerizer/mesos/launch.hpp
index a9b6ee2..208ad2b 100644
--- a/src/slave/containerizer/mesos/launch.hpp
+++ b/src/slave/containerizer/mesos/launch.hpp
@@ -21,6 +21,8 @@
 #include <stout/option.hpp>
 #include <stout/subcommand.hpp>
 
+#include <mesos/mesos.hpp>
+
 namespace mesos {
 namespace internal {
 namespace slave {
@@ -45,8 +47,8 @@ public:
     Option<int> pipe_write;
     Option<JSON::Array> pre_exec_commands;
 #ifdef __linux__
+    Option<CapabilityInfo> capabilities;
     bool unshare_namespace_mnt;
-    Option<JSON::Object> capabilities;
 #endif // __linux__
   };
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/4ea9651a/src/tests/containerizer/linux_capabilities_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/linux_capabilities_isolator_tests.cpp b/src/tests/containerizer/linux_capabilities_isolator_tests.cpp
new file mode 100644
index 0000000..f040c20
--- /dev/null
+++ b/src/tests/containerizer/linux_capabilities_isolator_tests.cpp
@@ -0,0 +1,354 @@
+// 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 <ostream>
+#include <set>
+#include <string>
+#include <vector>
+
+#include <gmock/gmock.h>
+
+#include <process/future.hpp>
+#include <process/gmock.hpp>
+#include <process/gtest.hpp>
+#include <process/owned.hpp>
+#include <process/queue.hpp>
+
+#include <stout/foreach.hpp>
+#include <stout/gtest.hpp>
+#include <stout/none.hpp>
+#include <stout/nothing.hpp>
+#include <stout/option.hpp>
+#include <stout/protobuf.hpp>
+#include <stout/stringify.hpp>
+#include <stout/try.hpp>
+
+#include <mesos/mesos.hpp>
+#include <mesos/scheduler.hpp>
+
+#include <mesos/master/detector.hpp>
+
+#include "linux/capabilities.hpp"
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/fetcher.hpp"
+
+#include "tests/cluster.hpp"
+#include "tests/mesos.hpp"
+
+#include "tests/containerizer/docker_archive.hpp"
+
+using mesos::internal::capabilities::Capability;
+using mesos::internal::capabilities::CHOWN;
+using mesos::internal::capabilities::DAC_READ_SEARCH;
+using mesos::internal::capabilities::NET_ADMIN;
+using mesos::internal::capabilities::NET_RAW;
+
+using mesos::internal::slave::Fetcher;
+
+using mesos::master::detector::MasterDetector;
+
+using process::Future;
+using process::Owned;
+using process::Queue;
+
+using std::initializer_list;
+using std::ostream;
+using std::set;
+using std::string;
+using std::vector;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+// Param for the tests:
+//   'requested'  Framework specified capabilities for the container.
+//   'allowed'    Allowed capabilities configured by the operator.
+//   'success'    True if the task should finish normally.
+struct TestParam
+{
+  TestParam(
+      const Option<set<Capability>>& _requested,
+      const Option<set<Capability>>& _allowed,
+      bool _image,
+      bool _success)
+    : requested(_requested.isSome()
+        ? convert(_requested.get())
+        : Option<CapabilityInfo>::none()),
+      allowed(_allowed.isSome()
+        ? convert(_allowed.get())
+        : Option<CapabilityInfo>::none()),
+      image(_image),
+      success(_success) {}
+
+  const Option<CapabilityInfo> requested;
+  const Option<CapabilityInfo> allowed;
+  const bool image;
+  const bool success;
+};
+
+
+ostream& operator<<(ostream& stream, const TestParam& param)
+{
+  if (param.requested.isSome()) {
+    stream << "requested='" << JSON::protobuf(param.requested.get()) << "', ";
+  } else {
+    stream << "requested='none', ";
+  }
+
+  if (param.allowed.isSome()) {
+    stream << "allowed='" << JSON::protobuf(param.allowed.get()) << "', ";
+  } else {
+    stream << "allowed='none', ";
+  }
+
+  stream << "image='" << (param.image ? "true" : "false") << "', ";
+  stream << "success='" << (param.success ? "true" : "false") << "'";
+
+  return stream;
+}
+
+
+class LinuxCapabilitiesIsolatorTest
+  : public MesosTest,
+    public ::testing::WithParamInterface<TestParam>
+{
+public:
+  LinuxCapabilitiesIsolatorTest()
+    : param(GetParam()) {}
+
+protected:
+  TestParam param;
+};
+
+
+ACTION_TEMPLATE(PushTaskStatus,
+                HAS_1_TEMPLATE_PARAMS(int, k),
+                AND_1_VALUE_PARAMS(statuses))
+{
+  statuses->put(std::tr1::get<k>(args));
+}
+
+
+// Parameterized test confirming the behavior of the capabilities
+// isolator. We here use the fact has `ping` has `NET_RAW` and
+// `NET_ADMIN` in its file capabilities. This test should be
+// instantiated with above `TestParam` struct.
+TEST_P(LinuxCapabilitiesIsolatorTest, ROOT_Ping)
+{
+  Try<Owned<cluster::Master>> master = StartMaster();
+  ASSERT_SOME(master);
+
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "linux/capabilities";
+  flags.allowed_capabilities = param.allowed;
+
+  if (param.image) {
+    const string registry = path::join(sandbox.get(), "registry");
+    AWAIT_READY(DockerArchive::create(registry, "test_image"));
+
+    flags.docker_registry = registry;
+    flags.docker_store_dir = path::join(os::getcwd(), "store");
+    flags.image_providers = "docker";
+    flags.isolation += ",docker/runtime,filesystem/linux";
+  }
+
+  Owned<MasterDetector> detector = master.get()->createDetector();
+
+  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(_, _, _))
+    .Times(1);
+
+  Future<vector<Offer>> offers;
+  EXPECT_CALL(sched, resourceOffers(_, _))
+    .WillOnce(FutureArg<1>(&offers))
+    .WillRepeatedly(Return()); // Ignore subsequent offers.
+
+  driver.start();
+
+  AWAIT_READY(offers);
+  EXPECT_NE(0u, offers->size());
+
+  // We use 'ping' as the command since it has file capabilities
+  // (`NET_RAW` and `NET_ADMIN` in permitted set). This allows us to
+  // test if capabilities are properly set.
+  CommandInfo command;
+  command.set_shell(false);
+  command.set_value("/bin/ping");
+  command.add_arguments("ping");
+  command.add_arguments("-c");
+  command.add_arguments("1");
+  command.add_arguments("127.0.0.1");
+
+  TaskInfo task = createTask(
+      offers.get()[0].slave_id(),
+      offers.get()[0].resources(),
+      command);
+
+  if (param.requested.isSome()) {
+    ContainerInfo* container = task.mutable_container();
+    container->set_type(ContainerInfo::MESOS);
+
+    LinuxInfo* linux = container->mutable_linux_info();
+    CapabilityInfo* capabilities = linux->mutable_capability_info();
+    capabilities->CopyFrom(param.requested.get());
+  }
+
+  if (param.image) {
+    ContainerInfo* container = task.mutable_container();
+    container->set_type(ContainerInfo::MESOS);
+
+    Image* image = container->mutable_mesos()->mutable_image();
+    image->set_type(Image::DOCKER);
+    image->mutable_docker()->set_name("test_image");
+  }
+
+  Queue<TaskStatus> statuses;
+  EXPECT_CALL(sched, statusUpdate(_, _))
+    .WillRepeatedly(PushTaskStatus<1>(&statuses));
+
+  driver.launchTasks(offers.get()[0].id(), {task});
+
+  // Wait for the terminal status update.
+  for (;;) {
+    Future<TaskStatus> status = statuses.get();
+    AWAIT_READY(status);
+
+    TaskState state = status->state();
+    if (protobuf::isTerminalState(state)) {
+      EXPECT_EQ((param.success ? TASK_FINISHED : TASK_FAILED), state);
+      break;
+    }
+  }
+
+  driver.stop();
+  driver.join();
+}
+
+
+// TODO(jieyu): We used DAC_READ_SEARCH capability below so that test
+// results won't be affected even if the executable (e.g., command
+// executor) to launch is not accessible (e.g., under someone's home
+// directory). Without that, even ROOT user will receive EACCESS if
+// DAC_READ_SEARCH is dropped.
+INSTANTIATE_TEST_CASE_P(
+    TestParam,
+    LinuxCapabilitiesIsolatorTest,
+    ::testing::Values(
+        // Dropped all capabilities, thus ping will fail.
+        TestParam(set<Capability>(), None(), false, false),
+        TestParam(set<Capability>(), None(), true, false),
+        TestParam(
+            set<Capability>(),
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            false,
+            false),
+        TestParam(
+            set<Capability>(),
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            true,
+            false),
+        TestParam(
+            set<Capability>(),
+            set<Capability>({CHOWN, DAC_READ_SEARCH}),
+            false,
+            false),
+        TestParam(
+            set<Capability>(),
+            set<Capability>({CHOWN, DAC_READ_SEARCH}),
+            true,
+            false),
+
+        // Allowed capabilities do not contain that ping needs, thus
+        // ping will fail.
+        TestParam(
+            None(),
+            set<Capability>({CHOWN, DAC_READ_SEARCH}),
+            false,
+            false),
+        TestParam(
+            None(),
+            set<Capability>({CHOWN, DAC_READ_SEARCH}),
+            true,
+            false),
+
+        // Requested capabilities are not allowed, task will fail.
+        TestParam(
+            set<Capability>({NET_RAW, NET_ADMIN}),
+            set<Capability>({CHOWN}),
+            false,
+            false),
+        TestParam(
+            set<Capability>({NET_RAW, NET_ADMIN}),
+            set<Capability>({CHOWN}),
+            true,
+            false),
+
+        // Dropped all capabilities but those that ping needs, thus
+        // ping will finish normally.
+        TestParam(
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            None(),
+            false,
+            true),
+        TestParam(
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            None(),
+            true,
+            true),
+        TestParam(
+            None(),
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            false,
+            true),
+        TestParam(
+            None(),
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            true,
+            true),
+        TestParam(
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            false,
+            true),
+        TestParam(
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            set<Capability>({NET_RAW, NET_ADMIN, DAC_READ_SEARCH}),
+            true,
+            true)));
+
+
+// TODO(bbannier): Add test cases for running the container as non-root.
+
+
+// TODO(bbannier): Reject these tasks that specify capabilities if
+// capabilities isolator is not enabled.
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {


[3/5] mesos git commit: Supported merging the launch command from isolators.

Posted by ji...@apache.org.
Supported merging the launch command from isolators.

Previously, we only allow one isolator to specify the launch command
for the container. This is not ideal because multiple isolators might
want to add some flags to the command executor. For instance, the
'docker/runtime' isolator wants to specify '--task_command' and
'--working_directory', and 'linux/capabilities' isolator wants to
specify '--capabilities'.

This patch changes the semantics so that launch command from isolators
are merged. However, it is isolator's responsibility to make sure the
merged command is a valid command.

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


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

Branch: refs/heads/master
Commit: a7d567c849ab9044642481271becc2f877d97617
Parents: 4773b27
Author: Jie Yu <yu...@gmail.com>
Authored: Wed Oct 5 20:11:27 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Oct 7 09:22:19 2016 -0700

----------------------------------------------------------------------
 include/mesos/type_utils.hpp                    |  3 +++
 src/common/type_utils.cpp                       | 18 ++++++++++++------
 src/slave/containerizer/mesos/containerizer.cpp | 19 ++++++++++++++++++-
 3 files changed, 33 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/a7d567c8/include/mesos/type_utils.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/type_utils.hpp b/include/mesos/type_utils.hpp
index 4f4f46e..1d09b4f 100644
--- a/include/mesos/type_utils.hpp
+++ b/include/mesos/type_utils.hpp
@@ -248,6 +248,9 @@ std::ostream& operator<<(
     const CapabilityInfo& capabilityInfo);
 
 
+std::ostream& operator<<(std::ostream& stream, const CommandInfo& commandInfo);
+
+
 std::ostream& operator<<(std::ostream& stream, const ContainerID& containerId);
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7d567c8/src/common/type_utils.cpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.cpp b/src/common/type_utils.cpp
index 8fde673..cad243f 100644
--- a/src/common/type_utils.cpp
+++ b/src/common/type_utils.cpp
@@ -442,6 +442,18 @@ bool operator!=(const TaskStatus& left, const TaskStatus& right)
 }
 
 
+ostream& operator<<(std::ostream& stream, const CapabilityInfo& capabilityInfo)
+{
+  return stream << JSON::protobuf(capabilityInfo);
+}
+
+
+ostream& operator<<(ostream& stream, const CommandInfo& commandInfo)
+{
+  return stream << JSON::protobuf(commandInfo);
+}
+
+
 ostream& operator<<(ostream& stream, const ContainerID& containerId)
 {
   return containerId.has_parent()
@@ -462,12 +474,6 @@ ostream& operator<<(ostream& stream, const ExecutorID& executorId)
 }
 
 
-ostream& operator<<(std::ostream& stream, const CapabilityInfo& capabilityInfo)
-{
-  return stream << stringify(JSON::protobuf(capabilityInfo));
-}
-
-
 ostream& operator<<(ostream& stream, const ExecutorInfo& executor)
 {
   return stream << executor.DebugString();

http://git-wip-us.apache.org/repos/asf/mesos/blob/a7d567c8/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index e6bd9f7..11f7adb 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1284,8 +1284,15 @@ Future<bool> MesosContainerizerProcess::_launch(
     }
 
     if (launchInfo->has_command()) {
+      // NOTE: 'command' from 'launchInfo' will be merged. It is
+      // isolators' responsibility to make sure that the merged
+      // command is a valid command.
       if (launchCommand.isSome()) {
-        return Failure("At most one command can be returned from isolators");
+        VLOG(1) << "Merging launch commands '" << launchCommand.get()
+                << "' and '" << launchInfo->command()
+                << "' from two different isolators";
+
+        launchCommand->MergeFrom(launchInfo->command());
       } else {
         launchCommand = launchInfo->command();
       }
@@ -1309,12 +1316,15 @@ Future<bool> MesosContainerizerProcess::_launch(
     }
   }
 
+  // Determine the launch command for the container.
   if (launchCommand.isNone()) {
     launchCommand = container->config.command_info();
   }
 
   // For the command executor case, we should add the rootfs flag to
   // the launch command of the command executor.
+  // TODO(jieyu): Remove this once we no longer support the old style
+  // command task (i.e., that uses mesos-execute).
   if (container->config.has_task_info() &&
       container->config.has_rootfs()) {
     CHECK_SOME(launchCommand);
@@ -1322,6 +1332,13 @@ Future<bool> MesosContainerizerProcess::_launch(
         "--rootfs=" + container->config.rootfs());
   }
 
+  // TODO(jieyu): 'uris', 'environment' and 'user' in 'launchCommand'
+  // will be ignored. In fact, the above fields should be moved to
+  // TaskInfo or ExecutorInfo, instead of putting them in CommandInfo.
+  launchCommand->clear_uris();
+  launchCommand->clear_environment();
+  launchCommand->clear_user();
+
   // Include any enviroment variables from CommandInfo.
   foreach (const Environment::Variable& variable,
            container->config.command_info().environment().variables()) {


[2/5] mesos git commit: Added `ping` to test linux rootfs.

Posted by ji...@apache.org.
Added `ping` to test linux rootfs.

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


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

Branch: refs/heads/master
Commit: e7d1f53621a09da47ee7dc5d6fcd6326cb72792d
Parents: 5e3648c
Author: Benjamin Bannier <be...@mesosphere.io>
Authored: Wed Oct 5 21:28:12 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Oct 7 09:22:19 2016 -0700

----------------------------------------------------------------------
 src/tests/containerizer/rootfs.hpp | 12 ++++++++++--
 1 file changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/e7d1f536/src/tests/containerizer/rootfs.hpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/rootfs.hpp b/src/tests/containerizer/rootfs.hpp
index f8fcfd4..6bc3835 100644
--- a/src/tests/containerizer/rootfs.hpp
+++ b/src/tests/containerizer/rootfs.hpp
@@ -67,12 +67,18 @@ public:
 
     // TODO(jieyu): Make sure 'path' is not under 'root'.
 
+    // Copy the files. We perserve all attributes so that e.g., `ping`
+    // keeps its file-based capabilities.
     if (os::stat::isdir(path)) {
-      if (os::system("cp -r '" + path + "' '" + target + "'") != 0) {
+      if (os::system(strings::format(
+              "cp -r --preserve=all '%s' '%s'",
+              path, target).get()) != 0) {
         return ErrnoError("Failed to copy '" + path + "' to rootfs");
       }
     } else {
-      if (os::system("cp '" + path + "' '" + target + "'") != 0) {
+      if (os::system(strings::format(
+              "cp --preserve=all '%s' '%s'",
+              path, target).get()) != 0) {
         return ErrnoError("Failed to copy '" + path + "' to rootfs");
       }
     }
@@ -104,6 +110,7 @@ public:
     std::vector<std::string> files = {
       "/bin/echo",
       "/bin/ls",
+      "/bin/ping",
       "/bin/sh",
       "/bin/sleep",
       "/usr/bin/sh",
@@ -111,6 +118,7 @@ public:
       "/lib64/ld-linux-x86-64.so.2",
       "/lib64/libc.so.6",
       "/lib64/libdl.so.2",
+      "/lib64/libidn.so.11",
       "/lib64/libtinfo.so.5",
       "/lib64/libselinux.so.1",
       "/lib64/libpcre.so.1",


[5/5] mesos git commit: Introduced Linux capabilities support for Mesos executor.

Posted by ji...@apache.org.
Introduced Linux capabilities support for Mesos executor.

This change introduces Linux capability-based security the Mesos
exector. A new flag `capabilities` is introduced to optionally specify
the capabilities tasks launched by the Mesos executor are allowed to
use.

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


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

Branch: refs/heads/master
Commit: 5e3648c871f8008d8e11390b2ccba86c59d82f70
Parents: a7d567c
Author: Benjamin Bannier <be...@mesosphere.io>
Authored: Wed Oct 5 20:55:42 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Oct 7 09:22:19 2016 -0700

----------------------------------------------------------------------
 src/launcher/executor.cpp       | 13 ++++++++++++-
 src/launcher/posix/executor.cpp |  9 ++++++++-
 src/launcher/posix/executor.hpp |  3 ++-
 3 files changed, 22 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/5e3648c8/src/launcher/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/executor.cpp b/src/launcher/executor.cpp
index 8a1051b..3e95d60 100644
--- a/src/launcher/executor.cpp
+++ b/src/launcher/executor.cpp
@@ -65,6 +65,7 @@
 #include <stout/os/killtree.hpp>
 
 #include "common/http.hpp"
+#include "common/parse.hpp"
 #include "common/protobuf_utils.hpp"
 #include "common/status_utils.hpp"
 
@@ -127,6 +128,7 @@ public:
       const Option<string>& _workingDirectory,
       const Option<string>& _user,
       const Option<string>& _taskCommand,
+      const Option<CapabilityInfo>& _capabilities,
       const FrameworkID& _frameworkId,
       const ExecutorID& _executorId,
       const Duration& _shutdownGracePeriod)
@@ -146,6 +148,7 @@ public:
       workingDirectory(_workingDirectory),
       user(_user),
       taskCommand(_taskCommand),
+      capabilities(_capabilities),
       frameworkId(_frameworkId),
       executorId(_executorId),
       task(None())
@@ -410,7 +413,8 @@ protected:
         user,
         rootfs,
         sandboxDirectory,
-        workingDirectory);
+        workingDirectory,
+        capabilities);
 #else
     // A Windows process is started using the `CREATE_SUSPENDED` flag
     // and is part of a job object. While the process handle is kept
@@ -799,6 +803,7 @@ private:
   Option<string> workingDirectory;
   Option<string> user;
   Option<string> taskCommand;
+  Option<CapabilityInfo> capabilities;
   const FrameworkID frameworkId;
   const ExecutorID executorId;
   Owned<MesosBase> mesos;
@@ -840,6 +845,10 @@ public:
         "If specified, this is the overrided command for launching the\n"
         "task (instead of the command from TaskInfo).");
 
+    add(&capabilities,
+        "capabilities",
+        "Capabilities the command can use.");
+
     add(&launcher_dir,
         "launcher_dir",
         "Directory path of Mesos binaries.",
@@ -854,6 +863,7 @@ public:
   Option<string> working_directory;
   Option<string> user;
   Option<string> task_command;
+  Option<mesos::CapabilityInfo> capabilities;
   string launcher_dir;
 };
 
@@ -927,6 +937,7 @@ int main(int argc, char** argv)
           flags.working_directory,
           flags.user,
           flags.task_command,
+          flags.capabilities,
           frameworkId,
           executorId,
           shutdownGracePeriod));

http://git-wip-us.apache.org/repos/asf/mesos/blob/5e3648c8/src/launcher/posix/executor.cpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.cpp b/src/launcher/posix/executor.cpp
index 0f3fed3..8a21191 100644
--- a/src/launcher/posix/executor.cpp
+++ b/src/launcher/posix/executor.cpp
@@ -57,7 +57,8 @@ pid_t launchTaskPosix(
     const Option<string>& user,
     const Option<string>& rootfs,
     const Option<string>& sandboxDirectory,
-    const Option<string>& workingDirectory)
+    const Option<string>& workingDirectory,
+    const Option<CapabilityInfo>& capabilities)
 {
   // Prepare the flags to pass to the launch process.
   MesosContainerizerLaunch::Flags launchFlags;
@@ -97,6 +98,12 @@ pid_t launchTaskPosix(
   launchFlags.rootfs = rootfs;
   launchFlags.user = user;
 
+#ifdef __linux__
+  launchFlags.capabilities = capabilities.isSome()
+    ? JSON::protobuf(capabilities.get())
+    : Option<JSON::Object>::none();
+#endif // __linux__
+
   string commandString = strings::format(
       "%s %s %s",
       path::join(launcherDir, MESOS_CONTAINERIZER),

http://git-wip-us.apache.org/repos/asf/mesos/blob/5e3648c8/src/launcher/posix/executor.hpp
----------------------------------------------------------------------
diff --git a/src/launcher/posix/executor.hpp b/src/launcher/posix/executor.hpp
index 9e46726..d057ff6 100644
--- a/src/launcher/posix/executor.hpp
+++ b/src/launcher/posix/executor.hpp
@@ -32,7 +32,8 @@ pid_t launchTaskPosix(
     const Option<std::string>& user,
     const Option<std::string>& rootfs,
     const Option<std::string>& sandboxDirectory,
-    const Option<std::string>& workingDirectory);
+    const Option<std::string>& workingDirectory,
+    const Option<CapabilityInfo>& capabilities);
 
 } // namespace internal {
 } // namespace mesos {


[4/5] mesos git commit: Reorganized includes in containerizer.

Posted by ji...@apache.org.
Reorganized includes in containerizer.

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


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

Branch: refs/heads/master
Commit: f6a25360053fc38e843129cc7e1f9fe4cf757ecd
Parents: e7d1f53
Author: Benjamin Bannier <be...@mesosphere.io>
Authored: Wed Oct 5 21:35:40 2016 -0700
Committer: Jie Yu <yu...@gmail.com>
Committed: Fri Oct 7 09:22:19 2016 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 62 ++++++--------------
 1 file changed, 18 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f6a25360/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 11f7adb..405e051 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -56,74 +56,48 @@
 #include "slave/containerizer/containerizer.hpp"
 #include "slave/containerizer/fetcher.hpp"
 
+#include "slave/containerizer/mesos/constants.hpp"
+#include "slave/containerizer/mesos/launch.hpp"
 #include "slave/containerizer/mesos/launcher.hpp"
-#ifdef __linux__
-#include "slave/containerizer/mesos/linux_launcher.hpp"
-#endif // __linux__
+#include "slave/containerizer/mesos/containerizer.hpp"
+#include "slave/containerizer/mesos/paths.hpp"
+#include "slave/containerizer/mesos/utils.hpp"
 
+#include "slave/containerizer/mesos/isolators/filesystem/posix.hpp"
 #include "slave/containerizer/mesos/isolators/posix.hpp"
+#include "slave/containerizer/mesos/isolators/posix/disk.hpp"
+#include "slave/containerizer/mesos/isolators/volume/sandbox_path.hpp"
+
+#include "slave/containerizer/mesos/provisioner/provisioner.hpp"
+
 #ifdef __WINDOWS__
 #include "slave/containerizer/mesos/isolators/windows.hpp"
+#include "slave/containerizer/mesos/isolators/filesystem/windows.hpp"
 #endif // __WINDOWS__
 
-#include "slave/containerizer/mesos/isolators/posix/disk.hpp"
-
-#if ENABLE_XFS_DISK_ISOLATOR
-#include "slave/containerizer/mesos/isolators/xfs/disk.hpp"
-#endif
-
 #ifdef __linux__
-#include "slave/containerizer/mesos/isolators/appc/runtime.hpp"
-#endif // __linux__
+#include "slave/containerizer/mesos/linux_launcher.hpp"
 
-#ifdef __linux__
+#include "slave/containerizer/mesos/isolators/appc/runtime.hpp"
 #include "slave/containerizer/mesos/isolators/cgroups/cgroups.hpp"
-#endif // __linux__
-
-#ifdef __linux__
 #include "slave/containerizer/mesos/isolators/docker/runtime.hpp"
-#endif // __linux__
-
-#ifdef __linux__
 #include "slave/containerizer/mesos/isolators/docker/volume/isolator.hpp"
-#endif // __linux__
-
-#ifdef __linux__
 #include "slave/containerizer/mesos/isolators/filesystem/linux.hpp"
-#endif // __linux__
-#include "slave/containerizer/mesos/isolators/filesystem/posix.hpp"
-#ifdef __WINDOWS__
-#include "slave/containerizer/mesos/isolators/filesystem/windows.hpp"
-#endif // __WINDOWS__
-#ifdef __linux__
 #include "slave/containerizer/mesos/isolators/filesystem/shared.hpp"
-#endif // __linux__
-
 #include "slave/containerizer/mesos/isolators/gpu/nvidia.hpp"
-
-#ifdef __linux__
 #include "slave/containerizer/mesos/isolators/namespaces/pid.hpp"
 #include "slave/containerizer/mesos/isolators/network/cni/cni.hpp"
-#endif
+#include "slave/containerizer/mesos/isolators/volume/image.hpp"
+#endif // __linux__
 
 #ifdef WITH_NETWORK_ISOLATOR
 #include "slave/containerizer/mesos/isolators/network/port_mapping.hpp"
 #endif
 
-#ifdef __linux__
-#include "slave/containerizer/mesos/isolators/volume/image.hpp"
+#if ENABLE_XFS_DISK_ISOLATOR
+#include "slave/containerizer/mesos/isolators/xfs/disk.hpp"
 #endif
 
-#include "slave/containerizer/mesos/isolators/volume/sandbox_path.hpp"
-
-#include "slave/containerizer/mesos/constants.hpp"
-#include "slave/containerizer/mesos/containerizer.hpp"
-#include "slave/containerizer/mesos/launch.hpp"
-#include "slave/containerizer/mesos/paths.hpp"
-#include "slave/containerizer/mesos/utils.hpp"
-
-#include "slave/containerizer/mesos/provisioner/provisioner.hpp"
-
 using process::collect;
 using process::dispatch;
 using process::defer;