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 2015/10/26 19:40:31 UTC

[07/12] mesos git commit: Relocated MesosContainerizer specific files to the correct location.

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a722d74/src/slave/containerizer/mesos/isolators/filesystem/posix.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/filesystem/posix.cpp b/src/slave/containerizer/mesos/isolators/filesystem/posix.cpp
new file mode 100644
index 0000000..cc54e90
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/filesystem/posix.cpp
@@ -0,0 +1,294 @@
+/**
+ * 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 <list>
+#include <string>
+
+#include <stout/fs.hpp>
+#include <stout/os.hpp>
+#include <stout/path.hpp>
+
+#include "slave/paths.hpp"
+
+#include "slave/containerizer/mesos/isolators/filesystem/posix.hpp"
+
+using namespace process;
+
+using std::list;
+using std::string;
+
+using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerPrepareInfo;
+using mesos::slave::ContainerState;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+PosixFilesystemIsolatorProcess::PosixFilesystemIsolatorProcess(
+    const Flags& _flags)
+  : flags(_flags) {}
+
+
+PosixFilesystemIsolatorProcess::~PosixFilesystemIsolatorProcess() {}
+
+
+Try<Isolator*> PosixFilesystemIsolatorProcess::create(const Flags& flags)
+{
+  process::Owned<MesosIsolatorProcess> process(
+      new PosixFilesystemIsolatorProcess(flags));
+
+  return new MesosIsolator(process);
+}
+
+
+Future<Nothing> PosixFilesystemIsolatorProcess::recover(
+    const list<ContainerState>& states,
+    const hashset<ContainerID>& orphans)
+{
+  foreach (const ContainerState& state, states) {
+    infos.put(state.container_id(), Owned<Info>(new Info(state.directory())));
+  }
+
+  return Nothing();
+}
+
+
+Future<Option<ContainerPrepareInfo>> PosixFilesystemIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const Option<string>& user)
+{
+  if (infos.contains(containerId)) {
+    return Failure("Container has already been prepared");
+  }
+
+  // Return failure if the container change the filesystem root
+  // because the symlinks will become invalid in the new root.
+  if (executorInfo.has_container()) {
+    CHECK_EQ(executorInfo.container().type(), ContainerInfo::MESOS);
+
+    if (executorInfo.container().mesos().has_image()) {
+      return Failure("Container root filesystems not supported");
+    }
+
+    // TODO(jieyu): Also return a failure if there exists images in
+    // the specified volumes.
+  }
+
+  infos.put(containerId, Owned<Info>(new Info(directory)));
+
+  return update(containerId, executorInfo.resources())
+      .then([]() -> Future<Option<ContainerPrepareInfo>> { return None(); });
+}
+
+
+Future<Nothing> PosixFilesystemIsolatorProcess::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  // No-op.
+  return Nothing();
+}
+
+
+Future<ContainerLimitation> PosixFilesystemIsolatorProcess::watch(
+    const ContainerID& containerId)
+{
+  // No-op.
+  return Future<ContainerLimitation>();
+}
+
+
+Future<Nothing> PosixFilesystemIsolatorProcess::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  if (!infos.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
+  const Owned<Info>& info = infos[containerId];
+
+  // TODO(jieyu): Currently, we only allow non-nested relative
+  // container paths for volumes. This is enforced by the master. For
+  // those volumes, we create symlinks in the executor directory.
+  Resources current = info->resources;
+
+  // We first remove unneeded persistent volumes.
+  foreach (const Resource& resource, current.persistentVolumes()) {
+    // This is enforced by the master.
+    CHECK(resource.disk().has_volume());
+
+    // Ignore absolute and nested paths.
+    const string& containerPath = resource.disk().volume().container_path();
+    if (strings::contains(containerPath, "/")) {
+      LOG(WARNING) << "Skipping updating symlink for persistent volume "
+                   << resource << " of container " << containerId
+                   << " because the container path '" << containerPath
+                   << "' contains slash";
+      continue;
+    }
+
+    if (resources.contains(resource)) {
+      continue;
+    }
+
+    string link = path::join(info->directory, containerPath);
+
+    LOG(INFO) << "Removing symlink '" << link << "' for persistent volume "
+              << resource << " of container " << containerId;
+
+    Try<Nothing> rm = os::rm(link);
+    if (rm.isError()) {
+      return Failure(
+          "Failed to remove the symlink for the unneeded "
+          "persistent volume at '" + link + "'");
+    }
+  }
+
+  // We then link additional persistent volumes.
+  foreach (const Resource& resource, resources.persistentVolumes()) {
+    // This is enforced by the master.
+    CHECK(resource.disk().has_volume());
+
+    // Ignore absolute and nested paths.
+    const string& containerPath = resource.disk().volume().container_path();
+    if (strings::contains(containerPath, "/")) {
+      LOG(WARNING) << "Skipping updating symlink for persistent volume "
+                   << resource << " of container " << containerId
+                   << " because the container path '" << containerPath
+                   << "' contains slash";
+      continue;
+    }
+
+    if (current.contains(resource)) {
+      continue;
+    }
+
+    string original = paths::getPersistentVolumePath(
+        flags.work_dir,
+        resource.role(),
+        resource.disk().persistence().id());
+
+    // Set the ownership of the persistent volume to match that of the
+    // sandbox directory.
+    //
+    // NOTE: Currently, persistent volumes in Mesos are exclusive,
+    // meaning that if a persistent volume is used by one task or
+    // executor, it cannot be concurrently used by other task or
+    // executor. But if we allow multiple executors to use same
+    // persistent volume at the same time in the future, the ownership
+    // of the persistent volume may conflict here.
+    //
+    // TODO(haosdent): Consider letting the frameworks specify the
+    // user/group of the persistent volumes.
+    struct stat s;
+    if (::stat(info->directory.c_str(), &s) < 0) {
+      return Failure(
+          "Failed to get ownership for '" + info->directory +
+          "': " + strerror(errno));
+    }
+
+    LOG(INFO) << "Changing the ownership of the persistent volume at '"
+              << original << "' with uid " << s.st_uid
+              << " and gid " << s.st_gid;
+
+    Try<Nothing> chown = os::chown(s.st_uid, s.st_gid, original, true);
+    if (chown.isError()) {
+      return Failure(
+          "Failed to change the ownership of the persistent volume at '" +
+          original + "' with uid " + stringify(s.st_uid) +
+          " and gid " + stringify(s.st_gid) + ": " + chown.error());
+    }
+
+    string link = path::join(info->directory, containerPath);
+
+    if (os::exists(link)) {
+      // NOTE: This is possible because 'info->resources' will be
+      // reset when slave restarts and recovers. When the slave calls
+      // 'containerizer->update' after the executor re-registers,
+      // we'll try to relink all the already symlinked volumes.
+      Result<string> realpath = os::realpath(link);
+      if (!realpath.isSome()) {
+        return Failure(
+            "Failed to get the realpath of symlink '" + link + "': " +
+            (realpath.isError() ? realpath.error() : "No such directory"));
+      }
+
+      // A sanity check to make sure the target of the symlink does
+      // not change. In fact, this is not supposed to happen.
+      // NOTE: Here, we compare the realpaths because 'original' might
+      // contain symbolic links.
+      Result<string> _original = os::realpath(original);
+      if (!_original.isSome()) {
+        return Failure(
+            "Failed to get the realpath of volume '" + original + "': " +
+            (_original.isError() ? _original.error() : "No such directory"));
+      }
+
+      if (realpath.get() != _original.get()) {
+        return Failure(
+            "The existing symlink '" + link + "' points to '" +
+            _original.get() + "' and the new target is '" +
+            realpath.get() + "'");
+      }
+    } else {
+      LOG(INFO) << "Adding symlink from '" << original << "' to '"
+                << link << "' for persistent volume " << resource
+                << " of container " << containerId;
+
+      Try<Nothing> symlink = ::fs::symlink(original, link);
+      if (symlink.isError()) {
+        return Failure(
+            "Failed to symlink persistent volume from '" +
+            original + "' to '" + link + "'");
+      }
+    }
+  }
+
+  // Store the updated resources.
+  info->resources = resources;
+
+  return Nothing();
+}
+
+
+Future<ResourceStatistics> PosixFilesystemIsolatorProcess::usage(
+    const ContainerID& containerId)
+{
+  // No-op, no usage gathered.
+  return ResourceStatistics();
+}
+
+
+Future<Nothing> PosixFilesystemIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  // Symlinks for persistent resources will be removed when the work
+  // directory is GC'ed, therefore no need to do explicit cleanup.
+  infos.erase(containerId);
+
+  return Nothing();
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a722d74/src/slave/containerizer/mesos/isolators/filesystem/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/filesystem/posix.hpp b/src/slave/containerizer/mesos/isolators/filesystem/posix.hpp
new file mode 100644
index 0000000..61b81dd
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/filesystem/posix.hpp
@@ -0,0 +1,89 @@
+/**
+ * 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 __POSIX_FILESYSTEM_ISOLATOR_HPP__
+#define __POSIX_FILESYSTEM_ISOLATOR_HPP__
+
+#include <mesos/resources.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+class PosixFilesystemIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(const Flags& flags);
+
+  virtual ~PosixFilesystemIsolatorProcess();
+
+  virtual process::Future<Nothing> recover(
+      const std::list<mesos::slave::ContainerState>& states,
+      const hashset<ContainerID>& orphans);
+
+  virtual process::Future<Option<mesos::slave::ContainerPrepareInfo>> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const Option<std::string>& user);
+
+  virtual process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  virtual process::Future<mesos::slave::ContainerLimitation> watch(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> cleanup(
+      const ContainerID& containerId);
+
+private:
+  PosixFilesystemIsolatorProcess(const Flags& flags);
+
+  const Flags flags;
+
+  struct Info
+  {
+    explicit Info(const std::string& _directory)
+      : directory(_directory) {}
+
+    const std::string directory;
+
+    // Track resources so we can unlink unneeded persistent volumes.
+    Resources resources;
+  };
+
+  hashmap<ContainerID, process::Owned<Info>> infos;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __POSIX_FILESYSTEM_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a722d74/src/slave/containerizer/mesos/isolators/filesystem/shared.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/filesystem/shared.cpp b/src/slave/containerizer/mesos/isolators/filesystem/shared.cpp
new file mode 100644
index 0000000..4702211
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/filesystem/shared.cpp
@@ -0,0 +1,268 @@
+/**
+ * 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 <set>
+
+#include "linux/ns.hpp"
+
+#include "slave/containerizer/mesos/isolators/filesystem/shared.hpp"
+
+using namespace process;
+
+using std::list;
+using std::set;
+using std::string;
+
+using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerPrepareInfo;
+using mesos::slave::ContainerState;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+SharedFilesystemIsolatorProcess::SharedFilesystemIsolatorProcess(
+    const Flags& _flags)
+  : flags(_flags) {}
+
+
+SharedFilesystemIsolatorProcess::~SharedFilesystemIsolatorProcess() {}
+
+
+Try<Isolator*> SharedFilesystemIsolatorProcess::create(const Flags& flags)
+{
+  Result<string> user = os::user();
+  if (!user.isSome()) {
+    return Error("Failed to determine user: " +
+                 (user.isError() ? user.error() : "username not found"));
+  }
+
+  if (user.get() != "root") {
+    return Error("SharedFilesystemIsolator requires root privileges");
+  }
+
+  process::Owned<MesosIsolatorProcess> process(
+      new SharedFilesystemIsolatorProcess(flags));
+
+  return new MesosIsolator(process);
+}
+
+
+Future<Nothing> SharedFilesystemIsolatorProcess::recover(
+    const list<ContainerState>& states,
+    const hashset<ContainerID>& orphans)
+{
+  // There is nothing to recover because we do not keep any state and
+  // do not monitor filesystem usage or perform any action on cleanup.
+  return Nothing();
+}
+
+
+Future<Option<ContainerPrepareInfo>> SharedFilesystemIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const Option<string>& user)
+{
+  if (executorInfo.has_container() &&
+      executorInfo.container().type() != ContainerInfo::MESOS) {
+    return Failure("Can only prepare filesystem for a MESOS container");
+  }
+
+  LOG(INFO) << "Preparing shared filesystem for container: "
+            << stringify(containerId);
+
+  if (!executorInfo.has_container()) {
+    // We don't consider this an error, there's just nothing to do so
+    // we return None.
+
+    return None();
+  }
+
+  // We don't support mounting to a container path which is a parent
+  // to another container path as this can mask entries. We'll keep
+  // track of all container paths so we can check this.
+  set<string> containerPaths;
+  containerPaths.insert(directory);
+
+  ContainerPrepareInfo prepareInfo;
+  prepareInfo.set_namespaces(CLONE_NEWNS);
+
+  foreach (const Volume& volume, executorInfo.container().volumes()) {
+    // Because the filesystem is shared we require the container path
+    // already exist, otherwise containers can create arbitrary paths
+    // outside their sandbox.
+    if (!os::exists(volume.container_path())) {
+      return Failure("Volume with container path '" +
+                     volume.container_path() +
+                     "' must exist on host for shared filesystem isolator");
+    }
+
+    // Host path must be provided.
+    if (!volume.has_host_path()) {
+      return Failure("Volume with container path '" +
+                     volume.container_path() +
+                     "' must specify host path for shared filesystem isolator");
+    }
+
+    // Check we won't mask another volume.
+    // NOTE: Assuming here that the container path is absolute, see
+    // Volume protobuf.
+    // TODO(idownes): This test is unnecessarily strict and could be
+    // relaxed if mounts could be re-ordered.
+    foreach (const string& containerPath, containerPaths) {
+      if (strings::startsWith(volume.container_path(), containerPath)) {
+        return Failure("Cannot mount volume to '" +
+                        volume.container_path() +
+                        "' because it is under volume '" +
+                        containerPath +
+                        "'");
+      }
+
+      if (strings::startsWith(containerPath, volume.container_path())) {
+        return Failure("Cannot mount volume to '" +
+                        containerPath +
+                        "' because it is under volume '" +
+                        volume.container_path() +
+                        "'");
+      }
+    }
+    containerPaths.insert(volume.container_path());
+
+    // A relative host path will be created in the container's work
+    // directory, otherwise check it already exists.
+    string hostPath;
+    if (!strings::startsWith(volume.host_path(), "/")) {
+      hostPath = path::join(directory, volume.host_path());
+
+      // Do not support any relative components in the resulting path.
+      // There should not be any links in the work directory to
+      // resolve.
+      if (strings::contains(hostPath, "/./") ||
+          strings::contains(hostPath, "/../")) {
+        return Failure("Relative host path '" +
+                       hostPath +
+                       "' cannot contain relative components");
+      }
+
+      Try<Nothing> mkdir = os::mkdir(hostPath, true);
+      if (mkdir.isError()) {
+        return Failure("Failed to create host_path '" +
+                        hostPath +
+                        "' for mount to '" +
+                        volume.container_path() +
+                        "': " +
+                        mkdir.error());
+      }
+
+      // Set the ownership and permissions to match the container path
+      // as these are inherited from host path on bind mount.
+      struct stat stat;
+      if (::stat(volume.container_path().c_str(), &stat) < 0) {
+        return Failure("Failed to get permissions on '" +
+                        volume.container_path() + "'" +
+                        ": " + strerror(errno));
+      }
+
+      Try<Nothing> chmod = os::chmod(hostPath, stat.st_mode);
+      if (chmod.isError()) {
+        return Failure("Failed to chmod hostPath '" +
+                       hostPath +
+                       "': " +
+                       chmod.error());
+      }
+
+      Try<Nothing> chown = os::chown(stat.st_uid, stat.st_gid, hostPath, false);
+      if (chown.isError()) {
+        return Failure("Failed to chown hostPath '" +
+                       hostPath +
+                       "': " +
+                       chown.error());
+      }
+    } else {
+      hostPath = volume.host_path();
+
+      if (!os::exists(hostPath)) {
+        return Failure("Volume with container path '" +
+                       volume.container_path() +
+                       "' must have host path '" +
+                       hostPath +
+                       "' present on host for shared filesystem isolator");
+      }
+    }
+
+    prepareInfo.add_commands()->set_value(
+        "mount -n --bind " + hostPath + " " + volume.container_path());
+  }
+
+  return prepareInfo;
+}
+
+
+Future<Nothing> SharedFilesystemIsolatorProcess::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  // No-op, isolation happens when unsharing the mount namespace.
+
+  return Nothing();
+}
+
+
+Future<ContainerLimitation> SharedFilesystemIsolatorProcess::watch(
+    const ContainerID& containerId)
+{
+  // No-op, for now.
+
+  return Future<ContainerLimitation>();
+}
+
+
+Future<Nothing> SharedFilesystemIsolatorProcess::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  // No-op, nothing enforced.
+
+  return Nothing();
+}
+
+
+Future<ResourceStatistics> SharedFilesystemIsolatorProcess::usage(
+    const ContainerID& containerId)
+{
+  // No-op, no usage gathered.
+
+  return ResourceStatistics();
+}
+
+
+Future<Nothing> SharedFilesystemIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  // Cleanup of mounts is done automatically done by the kernel when
+  // the mount namespace is destroyed after the last process
+  // terminates.
+
+  return Nothing();
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a722d74/src/slave/containerizer/mesos/isolators/filesystem/shared.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/filesystem/shared.hpp b/src/slave/containerizer/mesos/isolators/filesystem/shared.hpp
new file mode 100644
index 0000000..3a2f7db
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/filesystem/shared.hpp
@@ -0,0 +1,79 @@
+/**
+ * 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 __SHARED_FILESYSTEM_ISOLATOR_HPP__
+#define __SHARED_FILESYSTEM_ISOLATOR_HPP__
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// This isolator is to be used when all containers share the host's
+// filesystem.  It supports creating mounting "volumes" from the host
+// into each container's mount namespace. In particular, this can be
+// used to give each container a "private" system directory, such as
+// /tmp and /var/tmp.
+class SharedFilesystemIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(const Flags& flags);
+
+  virtual ~SharedFilesystemIsolatorProcess();
+
+  virtual process::Future<Nothing> recover(
+      const std::list<mesos::slave::ContainerState>& states,
+      const hashset<ContainerID>& orphans);
+
+  virtual process::Future<Option<mesos::slave::ContainerPrepareInfo>> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const Option<std::string>& user);
+
+  virtual process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  virtual process::Future<mesos::slave::ContainerLimitation> watch(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> cleanup(
+      const ContainerID& containerId);
+
+private:
+  SharedFilesystemIsolatorProcess(const Flags& flags);
+
+  const Flags flags;
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __SHARED_FILESYSTEM_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a722d74/src/slave/containerizer/mesos/isolators/namespaces/pid.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/namespaces/pid.cpp b/src/slave/containerizer/mesos/isolators/namespaces/pid.cpp
new file mode 100644
index 0000000..d41efe6
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/namespaces/pid.cpp
@@ -0,0 +1,266 @@
+/**
+ * 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 <sys/mount.h>
+
+#include <list>
+#include <set>
+#include <string>
+
+#include <stout/os.hpp>
+
+#include <stout/os/exists.hpp>
+#include <stout/os/ls.hpp>
+#include <stout/os/stat.hpp>
+
+#include "linux/fs.hpp"
+#include "linux/ns.hpp"
+
+#include "slave/containerizer/mesos/isolators/namespaces/pid.hpp"
+
+using namespace process;
+
+using std::list;
+using std::set;
+using std::string;
+
+using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerPrepareInfo;
+using mesos::slave::ContainerState;
+using mesos::slave::Isolator;
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// The root directory where we bind mount all the namespace handles.
+static const char PID_NS_BIND_MOUNT_ROOT[] = "/var/run/mesos/pidns";
+
+
+// The empty directory that we'll use to mask the namespace handles
+// inside each container. This mount ensures they cannot determine the
+// namespace of another container.
+static const char PID_NS_BIND_MOUNT_MASK_DIR[] = "/var/empty/mesos";
+
+
+// Helper to construct the path to a pid's namespace file.
+inline string nsProcFile(pid_t pid)
+{
+  return path::join("/proc", stringify(pid), "ns", "pid");
+}
+
+
+// Helper to construct the path to the additional reference created
+// for a container's pid namespace.
+inline string nsExtraReference(const ContainerID& containerId)
+{
+  return path::join(PID_NS_BIND_MOUNT_ROOT, stringify(containerId));
+}
+
+
+Try<Isolator*> NamespacesPidIsolatorProcess::create(const Flags& flags)
+{
+  // Check for root permission.
+  if (geteuid() != 0) {
+    return Error("The pid namespace isolator requires root permissions");
+  }
+
+  // Verify that pid namespaces are available on this kernel.
+  if (ns::namespaces().count("pid") == 0) {
+    return Error("Pid namespaces are not supported by this kernel");
+  }
+
+  // Create the directory where bind mounts of the pid namespace will
+  // be placed.
+  Try<Nothing> mkdir = os::mkdir(PID_NS_BIND_MOUNT_ROOT);
+  if (mkdir.isError()) {
+    return Error(
+        "Failed to create the bind mount root directory at " +
+        string(PID_NS_BIND_MOUNT_ROOT) + ": " + mkdir.error());
+  }
+
+  // Create the empty directory that will be used to mask the bind
+  // mounts inside each container.
+  mkdir = os::mkdir(PID_NS_BIND_MOUNT_MASK_DIR);
+  if (mkdir.isError()) {
+    return Error(
+        "Failed to create the bind mount mask direcrory at " +
+        string(PID_NS_BIND_MOUNT_MASK_DIR) + ": " + mkdir.error());
+  }
+
+  return new MesosIsolator(Owned<MesosIsolatorProcess>(
+      new NamespacesPidIsolatorProcess()));
+}
+
+
+Result<ino_t> NamespacesPidIsolatorProcess::getNamespace(
+    const ContainerID& containerId)
+{
+  const string target = nsExtraReference(containerId);
+
+  if (os::exists(target)) {
+    return os::stat::inode(target);
+  }
+
+  return None();
+}
+
+
+Future<Nothing> NamespacesPidIsolatorProcess::recover(
+    const list<ContainerState>& states,
+    const hashset<ContainerID>& orphans)
+{
+  hashset<ContainerID> recovered;
+  foreach (const ContainerState& state, states) {
+    recovered.insert(state.container_id());
+  }
+
+  // Clean up any unknown orphaned bind mounts and empty files. Known
+  // orphan bind mounts and empty files will be destroyed by the
+  // containerizer using the normal cleanup path. See MESOS-2367 for
+  // details.
+  Try<list<string>> entries = os::ls(PID_NS_BIND_MOUNT_ROOT);
+  if (entries.isError()) {
+    return Failure("Failed to list existing containers in '" +
+                   string(PID_NS_BIND_MOUNT_ROOT) + "': " + entries.error());
+  }
+
+  foreach (const string& entry, entries.get()) {
+    ContainerID containerId;
+    containerId.set_value(entry);
+
+    if (!recovered.contains(containerId) && !orphans.contains(containerId)) {
+      cleanup(containerId);
+    }
+  }
+
+  return Nothing();
+}
+
+
+Future<Option<ContainerPrepareInfo>> NamespacesPidIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const Option<string>& user)
+{
+  ContainerPrepareInfo prepareInfo;
+  prepareInfo.set_namespaces(CLONE_NEWPID | CLONE_NEWNS);
+
+  // Mask the bind mount root directory in each container so
+  // containers cannot see the namespace bind mount of other
+  // containers.
+  prepareInfo.add_commands()->set_value(
+      "mount -n --bind " + string(PID_NS_BIND_MOUNT_MASK_DIR) +
+      " " + string(PID_NS_BIND_MOUNT_ROOT));
+
+  // Mount /proc for the container's pid namespace to show the
+  // container's pids (and other /proc files), not the parent's. We
+  // first recursively make the mount private because /proc is usually
+  // marked explicitly as shared (see /proc/self/mountinfo) and
+  // changes would propagate to the parent's /proc mount otherwise. We
+  // then mount /proc with the standard options. This technique was
+  // taken from unshare.c in utils-linux for --mount-proc. We use the
+  // -n flag so the mount is not added to the mtab where it will not
+  // be correctly removed with the namespace terminates.
+  prepareInfo.add_commands()->set_value(
+      "mount none /proc --make-private -o rec");
+  prepareInfo.add_commands()->set_value(
+      "mount -n -t proc proc /proc -o nosuid,noexec,nodev");
+
+  return prepareInfo;
+}
+
+
+Future<Nothing> NamespacesPidIsolatorProcess::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  const string source = nsProcFile(pid);
+  const string target = nsExtraReference(containerId);
+
+  // Create a bind mount of the pid namespace so we can control the
+  // lifetime of the pid namespace. This lets us identify the
+  // container's pid namespace, even if the leading pid has exited.
+  Try<Nothing> touch = os::touch(target);
+  if (touch.isError()) {
+    return Failure("Failed to create bind mount target: " + touch.error());
+  }
+
+  Try<Nothing> mount = fs::mount(source, target, None(), MS_BIND, NULL);
+  if (mount.isError()) {
+    return Failure(
+        "Failed to mount pid namespace handle from " +
+        source + " to " + target + ": " + mount.error());
+  }
+
+  return Nothing();
+}
+
+
+Future<ContainerLimitation> NamespacesPidIsolatorProcess::watch(
+    const ContainerID& containerId)
+{
+  return Future<ContainerLimitation>();
+}
+
+
+Future<Nothing> NamespacesPidIsolatorProcess::update(
+    const ContainerID& containerId,
+    const Resources& resources)
+{
+  return Nothing();
+}
+
+
+Future<ResourceStatistics> NamespacesPidIsolatorProcess::usage(
+    const ContainerID& containerId)
+{
+  return ResourceStatistics();
+}
+
+
+// An old glibc might not have this symbol.
+#ifndef MNT_DETACH
+#define MNT_DETACH 2
+#endif
+
+
+Future<Nothing> NamespacesPidIsolatorProcess::cleanup(
+    const ContainerID& containerId)
+{
+  const string target = nsExtraReference(containerId);
+
+  if (os::exists(target)) {
+    // We don't expect anyone to have a reference to target but do a
+    // lazy umount in case. We do not want to force the umount; it
+    // will not cause an issue if this umount is delayed.
+    Try<Nothing> unmount = fs::unmount(target, MNT_DETACH);
+
+    // This will fail if the unmount hasn't completed yet but this
+    // only leaks a uniquely named empty file that will cleaned up as
+    // an orphan on recovery.
+    os::rm(target);
+  }
+
+  return Nothing();
+}
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a722d74/src/slave/containerizer/mesos/isolators/namespaces/pid.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/namespaces/pid.hpp b/src/slave/containerizer/mesos/isolators/namespaces/pid.hpp
new file mode 100644
index 0000000..87270d0
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/namespaces/pid.hpp
@@ -0,0 +1,91 @@
+/**
+ * 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_PID_ISOLATOR_HPP__
+#define __NAMESPACES_PID_ISOLATOR_HPP__
+
+#include <sys/types.h>
+
+#include <string>
+
+#include <stout/result.hpp>
+
+#include "slave/flags.hpp"
+
+#include "slave/containerizer/isolator.hpp"
+
+namespace mesos {
+namespace internal {
+namespace slave {
+
+// This isolator itself does not specify the necessary clone() flags
+// (see the LinuxLauncher for that) but it is used to keep track of a
+// container's pid namespace through a bind mount and exposed by
+// getNamespace().
+class NamespacesPidIsolatorProcess : public MesosIsolatorProcess
+{
+public:
+  static Try<mesos::slave::Isolator*> create(const Flags& flags);
+
+  // Return the pid namespace of the container. Returns None if the
+  // container was not created in a separate pid namespace, i.e.,
+  // processes are in the same namespace as the slave. This is used by
+  // the LinuxLauncher to determine if it can kill the leading process
+  // in the container and let the kernel kill the remaining processes.
+  // A container may not have a pid namespace if it was created
+  // without the namespaces/pid isolator and the slave was
+  // subsequently restarted with namespaces/pid enabled.
+  static Result<ino_t> getNamespace(const ContainerID& container);
+
+  NamespacesPidIsolatorProcess() {}
+
+  virtual ~NamespacesPidIsolatorProcess() {}
+
+  virtual process::Future<Nothing> recover(
+      const std::list<mesos::slave::ContainerState>& states,
+      const hashset<ContainerID>& orphans);
+
+  virtual process::Future<Option<mesos::slave::ContainerPrepareInfo>> prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory,
+      const Option<std::string>& user);
+
+  virtual process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  virtual process::Future<mesos::slave::ContainerLimitation> watch(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> update(
+      const ContainerID& containerId,
+      const Resources& resources);
+
+  virtual process::Future<ResourceStatistics> usage(
+      const ContainerID& containerId);
+
+  virtual process::Future<Nothing> cleanup(
+      const ContainerID& containerId);
+};
+
+} // namespace slave {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __NAMESPACES_PID_ISOLATOR_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/9a722d74/src/slave/containerizer/mesos/isolators/network/helper.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/isolators/network/helper.cpp b/src/slave/containerizer/mesos/isolators/network/helper.cpp
new file mode 100644
index 0000000..d851195
--- /dev/null
+++ b/src/slave/containerizer/mesos/isolators/network/helper.cpp
@@ -0,0 +1,35 @@
+/**
+ * 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 <stout/none.hpp>
+#include <stout/subcommand.hpp>
+
+#include "slave/containerizer/mesos/isolators/network/port_mapping.hpp"
+
+using namespace mesos::internal::slave;
+
+
+int main(int argc, char** argv)
+{
+  return Subcommand::dispatch(
+      None(),
+      argc,
+      argv,
+      new PortMappingUpdate(),
+      new PortMappingStatistics());
+}