You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by id...@apache.org on 2014/10/27 18:36:38 UTC

[1/4] git commit: Serialize isolator prepare and cleanup (reversed).

Repository: mesos
Updated Branches:
  refs/heads/master b4938753a -> f511395e8


Serialize isolator prepare and cleanup (reversed).

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


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

Branch: refs/heads/master
Commit: 47fa5a11ba09614c4b780b16da64bf1c276f50ef
Parents: 8e6e36a
Author: Ian Downes <id...@twitter.com>
Authored: Thu Sep 11 23:17:11 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Mon Oct 27 10:36:36 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 202 +++++++++++++------
 src/slave/containerizer/mesos/containerizer.hpp |   4 +-
 2 files changed, 147 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/47fa5a11/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 9f745d8..ce92878 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -451,22 +451,51 @@ Future<bool> MesosContainerizerProcess::launch(
       checkpoint);
 }
 
-Future<list<Option<CommandInfo> > > MesosContainerizerProcess::prepare(
+
+static list<Option<CommandInfo>> accumulate(
+    list<Option<CommandInfo>> l,
+    const Option<CommandInfo>& e)
+{
+  l.push_back(e);
+  return l;
+}
+
+
+static Future<list<Option<CommandInfo>>> _prepare(
+    const Owned<Isolator>& isolator,
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory,
+    const list<Option<CommandInfo>> commands)
+{
+  // Propagate any failure.
+  return isolator->prepare(containerId, executorInfo, directory)
+    .then(lambda::bind(&accumulate, commands, lambda::_1));
+}
+
+
+Future<list<Option<CommandInfo>>> MesosContainerizerProcess::prepare(
     const ContainerID& containerId,
     const ExecutorInfo& executorInfo,
     const string& directory,
     const Option<string>& user)
 {
-  // Start preparing all isolators (in parallel) and gather any additional
-  // preparation comands that must be run in the forked child before exec'ing
-  // the executor.
-  list<Future<Option<CommandInfo> > > futures;
+  // We prepare the isolators sequentially according to their ordering
+  // to permit basic dependency specification, e.g., preparing a
+  // filesystem isolator before other isolators.
+  Future<list<Option<CommandInfo>>> f = list<Option<CommandInfo>>();
+
   foreach (const Owned<Isolator>& isolator, isolators) {
-    futures.push_back(isolator->prepare(containerId, executorInfo));
+    // Chain together preparing each isolator.
+    f = f.then(lambda::bind(&_prepare,
+                            isolator,
+                            containerId,
+                            executorInfo,
+                            directory,
+                            lambda::_1));
   }
 
-  // Wait for all isolators to complete preparations.
-  return collect(futures);
+  return f;
 }
 
 
@@ -738,6 +767,9 @@ Future<bool> MesosContainerizerProcess::isolate(
   }
 
   // Isolate the executor with each isolator.
+  // NOTE: This is done is parallel and is not sequenced like prepare
+  // or destroy because we assume there are no dependencies in
+  // isolation.
   list<Future<Nothing> > futures;
   foreach (const Owned<Isolator>& isolator, isolators) {
     futures.push_back(isolator->isolate(containerId, _pid));
@@ -753,14 +785,14 @@ Future<bool> MesosContainerizerProcess::exec(
     const ContainerID& containerId,
     int pipeWrite)
 {
-  // The container may be destroyed before we exec the executor so return
-  // failure here.
+  // The container may be destroyed before we exec the executor so
+  // return failure here.
   if (!promises.contains(containerId)) {
     return Failure("Container destroyed during launch");
   }
 
-  // Now that we've contained the child we can signal it to continue by
-  // writing to the pipe.
+  // Now that we've contained the child we can signal it to continue
+  // by writing to the pipe.
   char dummy;
   ssize_t length;
   while ((length = write(pipeWrite, &dummy, sizeof(dummy))) == -1 &&
@@ -790,9 +822,9 @@ Future<Nothing> MesosContainerizerProcess::update(
     const ContainerID& containerId,
     const Resources& _resources)
 {
-  // The resources hashmap won't initially contain the container's resources
-  // after recovery so we must check the promises hashmap (which is set during
-  // recovery).
+  // The resources hashmap won't initially contain the container's
+  // resources after recovery so we must check the promises hashmap
+  // (which is set during recovery).
   if (!promises.contains(containerId)) {
     // It is not considered a failure if the container is not known
     // because the slave will attempt to update the container's
@@ -817,9 +849,9 @@ Future<Nothing> MesosContainerizerProcess::update(
 }
 
 
-// Resources are used to set the limit fields in the statistics but are
-// optional because they aren't known after recovery until/unless update() is
-// called.
+// Resources are used to set the limit fields in the statistics but
+// are optional because they aren't known after recovery until/unless
+// update() is called.
 Future<ResourceStatistics> _usage(
     const ContainerID& containerId,
     const Option<Resources>& resources,
@@ -871,8 +903,8 @@ Future<ResourceStatistics> MesosContainerizerProcess::usage(
   }
 
   // Use await() here so we can return partial usage statistics.
-  // TODO(idownes): After recovery resources won't be known until after an
-  // update() because they aren't part of the SlaveState.
+  // TODO(idownes): After recovery resources won't be known until
+  // after an update() because they aren't part of the SlaveState.
   return await(futures)
     .then(lambda::bind(
           _usage, containerId, resources.get(containerId), lambda::_1));
@@ -899,8 +931,8 @@ void MesosContainerizerProcess::destroy(const ContainerID& containerId)
     launcher->destroy(containerId)
       .onAny(defer(self(), &Self::_destroy, containerId, lambda::_1));
   } else {
-    // The executor never forked so no processes to kill, go straight to
-    // __destroy() with status = None().
+    // The executor never forked so no processes to kill, go straight
+    // to __destroy() with status = None().
     __destroy(containerId, None());
   }
 }
@@ -910,12 +942,12 @@ void MesosContainerizerProcess::_destroy(
     const ContainerID& containerId,
     const Future<Nothing>& future)
 {
-  // Something has gone wrong and the launcher wasn't able to kill all the
-  // processes in the container. We cannot clean up the isolators because they
-  // may require that all processes have exited so just return the failure to
-  // the slave.
-  // TODO(idownes): This is a pretty bad state to be in but we should consider
-  // cleaning up here.
+  // Something has gone wrong and the launcher wasn't able to kill all
+  // the processes in the container. We cannot clean up the isolators
+  // because they may require that all processes have exited so just
+  // return the failure to the slave.
+  // TODO(idownes): This is a pretty bad state to be in but we should
+  // consider cleaning up here.
   if (!future.isReady()) {
     promises[containerId]->fail(
         "Failed to destroy container: " +
@@ -925,53 +957,109 @@ void MesosContainerizerProcess::_destroy(
     return;
   }
 
-  // We've successfully killed all processes in the container so get the exit
-  // status of the executor when it's ready (it may already be) and continue
-  // the destroy.
+  // We've successfully killed all processes in the container so get
+  // the exit status of the executor when it's ready (it may already
+  // be) and continue the destroy.
   statuses.get(containerId).get()
     .onAny(defer(self(), &Self::__destroy, containerId, lambda::_1));
 }
 
 
+static list<Future<Nothing>> _cleanup(const list<Future<Nothing>>& cleanups)
+{
+  return cleanups;
+}
+
+
+static Future<list<Future<Nothing>>> cleanup(
+    const Owned<Isolator>& isolator,
+    const ContainerID& containerId,
+    list<Future<Nothing>> cleanups)
+{
+  // Accumulate but do not propagate any failure.
+  Future<Nothing> cleanup = isolator->cleanup(containerId);
+  cleanups.push_back(cleanup);
+
+  // Wait for the cleanup to complete/fail before returning the list.
+  // We use await here to asynchronously wait for the isolator to
+  // complete then return cleanups.
+  list<Future<Nothing>> cleanup_;
+  cleanup_.push_back(cleanup);
+
+  return await(cleanup_)
+    .then(lambda::bind(&_cleanup, cleanups));
+}
+
+
+// TODO(idownes): Use a reversed view of the container rather than
+// reversing a copy.
+template <typename T>
+static T reversed(const T& t)
+{
+  T r = t;
+  std::reverse(r.begin(), r.end());
+  return r;
+}
+
+
 void MesosContainerizerProcess::__destroy(
     const ContainerID& containerId,
     const Future<Option<int > >& status)
 {
-  // Now that all processes have exited we can now clean up all isolators.
-  list<Future<Nothing> > futures;
-  foreach (const Owned<Isolator>& isolator, isolators) {
-    futures.push_back(isolator->cleanup(containerId));
+  // We clean up each isolator in the reverse order they were
+  // prepared (see comment in prepare()).
+  Future<list<Future<Nothing>>> f = list<Future<Nothing>>();
+
+  foreach (const Owned<Isolator>& isolator, reversed(isolators)) {
+    // We'll try to clean up all isolators, waiting for each to
+    // complete and continuing if one fails.
+    f = f.then(lambda::bind(&cleanup,
+                            isolator,
+                            containerId,
+                            lambda::_1));
   }
 
-  // Wait for all isolators to complete cleanup before continuing.
-  collect(futures)
-    .onAny(defer(self(), &Self::___destroy, containerId, status, lambda::_1));
+  // Continue destroy when we're done trying to clean up.
+  f.onAny(defer(self(),
+                &Self::___destroy,
+                containerId,
+                status,
+                lambda::_1));
+
+  return;
 }
 
 
 void MesosContainerizerProcess::___destroy(
     const ContainerID& containerId,
-    const Future<Option<int > >& status,
-    const Future<list<Nothing> >& futures)
+    const Future<Option<int>>& status,
+    const Future<list<Future<Nothing>>>& cleanups)
 {
-  // Something has gone wrong with one of the Isolators and cleanup failed.
-  // We'll fail the container termination and remove the 'destroying' flag but
-  // leave all other state. The containerizer is now in a bad state because
-  // at least one isolator has failed to clean up.
-  if (!futures.isReady()) {
-    promises[containerId]->fail(
-        "Failed to clean up isolators when destroying container: " +
-        (futures.isFailed() ? futures.failure() : "discarded future"));
-
-    destroying.erase(containerId);
-
-    return;
+  // This should not occur because we only use the Future<list> to
+  // facilitate chaining.
+  CHECK_READY(cleanups);
+
+  // Check cleanup succeeded for all isolators. If not, we'll fail the
+  // container termination and remove the 'destroying' flag but leave
+  // all other state. The container is now in an inconsistent state.
+  foreach (const Future<Nothing>& cleanup, cleanups.get()) {
+    if (!cleanup.isReady()) {
+      promises[containerId]->fail(
+        "Failed to clean up an isolator when destroying container '" +
+        stringify(containerId) + "' :" +
+        (cleanup.isFailed() ? cleanup.failure() : "discarded future"));
+
+      destroying.erase(containerId);
+
+      return;
+    }
   }
 
   // A container is 'killed' if any isolator limited it.
-  // Note: We may not see a limitation in time for it to be registered. This
-  // could occur if the limitation (e.g., an OOM) killed the executor and we
-  // triggered destroy() off the executor exit.
+  // Note: We may not see a limitation in time for it to be
+  // registered. This could occur if the limitation (e.g., an OOM)
+  // killed the executor and we triggered destroy() off the executor
+  // exit.
   bool killed = false;
   string message;
   if (limitations.contains(containerId)) {
@@ -1028,8 +1116,8 @@ void MesosContainerizerProcess::limited(
               << " and will be terminated";
     limitations.put(containerId, future.get());
   } else {
-    // TODO(idownes): A discarded future will not be an error when isolators
-    // discard their promises after cleanup.
+    // TODO(idownes): A discarded future will not be an error when
+    // isolators discard their promises after cleanup.
     LOG(ERROR) << "Error in a resource limitation for container "
                << containerId << ": " << (future.isFailed() ? future.failure()
                                                             : "discarded");

http://git-wip-us.apache.org/repos/asf/mesos/blob/47fa5a11/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index bf246ca..ab3bb6f 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -193,8 +193,8 @@ private:
   // cleanup.
   void ___destroy(
       const ContainerID& containerId,
-      const process::Future<Option<int > >& status,
-      const process::Future<std::list<Nothing> >& futures);
+      const process::Future<Option<int> >& status,
+      const process::Future<std::list<process::Future<Nothing>>>& cleanups);
 
   // Call back for when an isolator limits a container and impacts the
   // processes. This will trigger container destruction.


[4/4] git commit: Introduce a shared filesytem isolator.

Posted by id...@apache.org.
Introduce a shared filesytem isolator.

Isolator supports creating private copies of system directories, e.g.,
/tmp, for each container while sharing the host's filesystem.

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


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

Branch: refs/heads/master
Commit: f511395e84b009881e10b46d2da20b673bf452cb
Parents: c18a50a
Author: Ian Downes <id...@twitter.com>
Authored: Wed Oct 1 11:05:19 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Mon Oct 27 10:36:37 2014 -0700

----------------------------------------------------------------------
 include/mesos/mesos.proto                       |   6 +-
 src/Makefile.am                                 |   4 +
 src/common/parse.hpp                            |  14 +
 src/common/type_utils.hpp                       |   8 +
 .../isolators/filesystem/shared.cpp             | 263 +++++++++++++++++++
 .../isolators/filesystem/shared.hpp             |  75 ++++++
 src/slave/containerizer/linux_launcher.cpp      |  10 +-
 src/slave/containerizer/mesos/containerizer.cpp |  24 +-
 src/slave/flags.hpp                             |  22 ++
 src/slave/slave.cpp                             |  16 +-
 src/tests/isolator_tests.cpp                    | 176 +++++++++++++
 src/tests/mesos.hpp                             |   8 +
 12 files changed, 614 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/include/mesos/mesos.proto
----------------------------------------------------------------------
diff --git a/include/mesos/mesos.proto b/include/mesos/mesos.proto
index 6b93e90..168a7a8 100644
--- a/include/mesos/mesos.proto
+++ b/include/mesos/mesos.proto
@@ -837,7 +837,8 @@ message Volume {
   // Absolute path pointing to a directory or file in the container.
   required string container_path = 1;
 
-  // Absolute path pointing to a directory or file on the host.
+  // Absolute path pointing to a directory or file on the host or a path
+  // relative to the container work directory.
   optional string host_path = 2;
 
   enum Mode {
@@ -851,12 +852,13 @@ message Volume {
 
 /**
  * Describes a container configuration and allows extensible
- * configurations for different container implementation.
+ * configurations for different container implementations.
  */
 message ContainerInfo {
   // All container implementation types.
   enum Type {
     DOCKER = 1;
+    MESOS = 2;
   }
 
   message DockerInfo {

http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 6820d8a..f177d87 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -343,6 +343,7 @@ if OS_LINUX
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/cgroups/cpushare.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/cgroups/mem.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/cgroups/perf_event.cpp
+  libmesos_no_3rdparty_la_SOURCES += slave/containerizer/isolators/filesystem/shared.cpp
   libmesos_no_3rdparty_la_SOURCES += slave/containerizer/linux_launcher.cpp
 else
   EXTRA_DIST += linux/cgroups.cpp
@@ -437,12 +438,15 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	slave/containerizer/isolator.hpp				\
 	slave/containerizer/launcher.hpp				\
 	slave/containerizer/linux_launcher.hpp				\
+	slave/containerizer/mesos/containerizer.hpp			\
+	slave/containerizer/mesos/launch.hpp				\
 	slave/containerizer/isolators/posix.hpp				\
 	slave/containerizer/isolators/cgroups/constants.hpp		\
 	slave/containerizer/isolators/cgroups/cpushare.hpp		\
 	slave/containerizer/isolators/cgroups/mem.hpp			\
 	slave/containerizer/isolators/cgroups/perf_event.hpp		\
 	slave/containerizer/mesos/containerizer.hpp			\
+	slave/containerizer/isolators/filesystem/shared.hpp		\
 	slave/containerizer/mesos/launch.hpp				\
 	tests/cluster.hpp						\
 	tests/containerizer.hpp						\

http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/src/common/parse.hpp
----------------------------------------------------------------------
diff --git a/src/common/parse.hpp b/src/common/parse.hpp
index c9ca30f..ae581e5 100644
--- a/src/common/parse.hpp
+++ b/src/common/parse.hpp
@@ -68,6 +68,20 @@ inline Try<mesos::internal::Modules> parse(const std::string& value)
   return protobuf::parse<mesos::internal::Modules>(json.get());
 }
 
+
+template<>
+inline Try<mesos::ContainerInfo> parse(const std::string& value)
+{
+  // Convert from string or file to JSON.
+  Try<JSON::Object> json = parse<JSON::Object>(value);
+  if (json.isError()) {
+    return Error(json.error());
+  }
+
+  // Convert from JSON to Protobuf.
+  return protobuf::parse<mesos::ContainerInfo>(json.get());
+}
+
 } // namespace flags {
 
 #endif // __COMMON_PARSE_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/src/common/type_utils.hpp
----------------------------------------------------------------------
diff --git a/src/common/type_utils.hpp b/src/common/type_utils.hpp
index f16beb8..2d22db8 100644
--- a/src/common/type_utils.hpp
+++ b/src/common/type_utils.hpp
@@ -55,6 +55,14 @@ inline bool operator == (const ExecutorID& left, const ExecutorID& right)
 }
 
 
+inline std::ostream& operator << (
+    std::ostream& stream,
+    const ContainerInfo& containerInfo)
+{
+  return stream << containerInfo.DebugString();
+}
+
+
 inline bool operator == (const FrameworkID& left, const FrameworkID& right)
 {
   return left.value() == right.value();

http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/src/slave/containerizer/isolators/filesystem/shared.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/shared.cpp b/src/slave/containerizer/isolators/filesystem/shared.cpp
new file mode 100644
index 0000000..49510b2
--- /dev/null
+++ b/src/slave/containerizer/isolators/filesystem/shared.cpp
@@ -0,0 +1,263 @@
+/**
+ * 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 "slave/containerizer/isolators/filesystem/shared.hpp"
+
+using namespace process;
+
+using std::list;
+using std::set;
+using std::string;
+
+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<IsolatorProcess> process(
+      new SharedFilesystemIsolatorProcess(flags));
+
+  return new Isolator(process);
+}
+
+
+Future<Nothing> SharedFilesystemIsolatorProcess::recover(
+    const list<state::RunState>& states)
+{
+  // 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<CommandInfo> > SharedFilesystemIsolatorProcess::prepare(
+    const ContainerID& containerId,
+    const ExecutorInfo& executorInfo,
+    const string& directory)
+{
+  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);
+
+  list<string> commands;
+
+  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");
+      }
+    }
+
+    commands.push_back("mount -n --bind " +
+                       hostPath +
+                       " " +
+                       volume.container_path());
+  }
+
+  CommandInfo command;
+  command.set_value(strings::join(" && ", commands));
+
+  return command;
+}
+
+
+Future<Nothing> SharedFilesystemIsolatorProcess::isolate(
+    const ContainerID& containerId,
+    pid_t pid)
+{
+  // No-op, isolation happens when unsharing the mount namespace.
+
+  return Nothing();
+}
+
+
+Future<Limitation> SharedFilesystemIsolatorProcess::watch(
+    const ContainerID& containerId)
+{
+  // No-op, for now.
+
+  return Future<Limitation>();
+}
+
+
+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/f511395e/src/slave/containerizer/isolators/filesystem/shared.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/filesystem/shared.hpp b/src/slave/containerizer/isolators/filesystem/shared.hpp
new file mode 100644
index 0000000..75172d5
--- /dev/null
+++ b/src/slave/containerizer/isolators/filesystem/shared.hpp
@@ -0,0 +1,75 @@
+/**
+ * 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/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 IsolatorProcess
+{
+public:
+  static Try<Isolator*> create(const Flags& flags);
+
+  virtual ~SharedFilesystemIsolatorProcess();
+
+  virtual process::Future<Nothing> recover(
+      const std::list<state::RunState>& states);
+
+  virtual process::Future<Option<CommandInfo> > prepare(
+      const ContainerID& containerId,
+      const ExecutorInfo& executorInfo,
+      const std::string& directory);
+
+  virtual process::Future<Nothing> isolate(
+      const ContainerID& containerId,
+      pid_t pid);
+
+  virtual process::Future<Limitation> 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/f511395e/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index 07ee643..7a5cdbb 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -91,8 +91,6 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
   LOG(INFO) << "Using " << hierarchy.get()
             << " as the freezer hierarchy for the Linux launcher";
 
-  // TODO(idownes): Inspect the isolation flag to determine namespaces
-  // to use.
   int namespaces = 0;
 
 #ifdef WITH_NETWORK_ISOLATOR
@@ -103,6 +101,10 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
   }
 #endif
 
+  if (strings::contains(flags.isolation, "filesystem/shared")) {
+    namespaces |= CLONE_NEWNS;
+  }
+
   return new LinuxLauncher(flags, namespaces, hierarchy.get());
 }
 
@@ -347,6 +349,10 @@ Try<pid_t> LinuxLauncher::fork(
 
 Future<Nothing> LinuxLauncher::destroy(const ContainerID& containerId)
 {
+  if (!pids.contains(containerId)) {
+    return Failure("Unknown container");
+  }
+
   pids.erase(containerId);
 
   return cgroups::destroy(

http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index ce92878..3fa249f 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -42,6 +42,7 @@
 #include "slave/containerizer/isolators/cgroups/cpushare.hpp"
 #include "slave/containerizer/isolators/cgroups/mem.hpp"
 #include "slave/containerizer/isolators/cgroups/perf_event.hpp"
+#include "slave/containerizer/isolators/filesystem/shared.hpp"
 #endif // __linux__
 #ifdef WITH_NETWORK_ISOLATOR
 #include "slave/containerizer/isolators/network/port_mapping.hpp"
@@ -102,6 +103,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   creators["cgroups/cpu"] = &CgroupsCpushareIsolatorProcess::create;
   creators["cgroups/mem"] = &CgroupsMemIsolatorProcess::create;
   creators["cgroups/perf_event"] = &CgroupsPerfEventIsolatorProcess::create;
+  creators["filesystem/shared"] = &SharedFilesystemIsolatorProcess::create;
 #endif // __linux__
 #ifdef WITH_NETWORK_ISOLATOR
   creators["network/port_mapping"] = &PortMappingIsolatorProcess::create;
@@ -124,7 +126,13 @@ Try<MesosContainerizer*> MesosContainerizer::create(
         return Error(
             "Could not create isolator " + type + ": " + isolator.error());
       } else {
-        isolators.push_back(Owned<Isolator>(isolator.get()));
+        if (type == "filesystem/shared") {
+          // Filesystem isolator must be the first isolator used for prepare()
+          // so any volume mounts are performed before anything else runs.
+          isolators.insert(isolators.begin(), Owned<Isolator>(isolator.get()));
+        } else {
+          isolators.push_back(Owned<Isolator>(isolator.get()));
+        }
       }
     } else {
       return Error("Unknown or unsupported isolator: " + type);
@@ -135,7 +143,8 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   // Determine which launcher to use based on the isolation flag.
   Try<Launcher*> launcher =
     (strings::contains(isolation, "cgroups") ||
-     strings::contains(isolation, "network/port_mapping"))
+     strings::contains(isolation, "network/port_mapping") ||
+     strings::contains(isolation, "filesystem/shared"))
     ? LinuxLauncher::create(flags)
     : PosixLauncher::create(flags);
 #else
@@ -384,9 +393,10 @@ Future<bool> MesosContainerizerProcess::launch(
     return Failure("Container already started");
   }
 
-  if (executorInfo.has_container()) {
-    // We return false as this containerizer does not support
-    // handling ContainerInfo.
+  // We support MESOS containers or ExecutorInfos with no
+  // ContainerInfo given.
+  if (executorInfo.has_container() &&
+      executorInfo.container().type() != ContainerInfo::MESOS) {
     return false;
   }
 
@@ -437,7 +447,7 @@ Future<bool> MesosContainerizerProcess::launch(
 {
   if (taskInfo.has_container()) {
     // We return false as this containerizer does not support
-    // handling ContainerInfo.
+    // handling TaskInfo::ContainerInfo.
     return false;
   }
 
@@ -950,7 +960,7 @@ void MesosContainerizerProcess::_destroy(
   // consider cleaning up here.
   if (!future.isReady()) {
     promises[containerId]->fail(
-        "Failed to destroy container: " +
+        "Failed to destroy container " + stringify(containerId) + ": " +
         (future.isFailed() ? future.failure() : "discarded future"));
 
     destroying.erase(containerId);

http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index bdaa5d5..621d0de 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -303,6 +303,27 @@ public:
         "sandbox is mapped to.\n",
         "/mnt/mesos/sandbox");
 
+    add(&Flags::default_container_info,
+        "default_container_info",
+        "JSON formatted ContainerInfo that will be included into\n"
+        "any ExecutorInfo that does not specify a ContainerInfo.\n"
+        "\n"
+        "See the ContainerInfo protobuf in mesos.proto for\n"
+        "the expected format.\n"
+        "\n"
+        "Example:\n"
+        "{\n"
+        "\"type\": \"MESOS\",\n"
+        "\"volumes\": [\n"
+        "  {\n"
+        "    \"host_path\": \"./.private/tmp\",\n"
+        "    \"container_path\": \"/tmp\",\n"
+        "    \"mode\": \"RW\"\n"
+        "  }\n"
+        " ]\n"
+        "}"
+        );
+
 #ifdef WITH_NETWORK_ISOLATOR
     add(&Flags::ephemeral_ports_per_container,
         "ephemeral_ports_per_container",
@@ -399,6 +420,7 @@ public:
   Option<std::string> default_container_image;
   std::string docker;
   std::string docker_sandbox_directory;
+  Option<ContainerInfo> default_container_info;
 #ifdef WITH_NETWORK_ISOLATOR
   uint16_t ephemeral_ports_per_container;
   Option<std::string> eth0_name;

http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 5e7c107..96fb5f7 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -2701,10 +2701,24 @@ ExecutorInfo Slave::getExecutorInfo(
           "cpus:" + stringify(DEFAULT_EXECUTOR_CPUS) + ";" +
           "mem:" + stringify(DEFAULT_EXECUTOR_MEM.megabytes())).get());
 
+    // Add in any default ContainerInfo.
+    if (!executor.has_container() && flags.default_container_info.isSome()) {
+      executor.mutable_container()->CopyFrom(
+          flags.default_container_info.get());
+    }
+
     return executor;
   }
 
-  return task.executor();
+  ExecutorInfo executor = task.executor();
+
+  // Add in any default ContainerInfo.
+  if (!executor.has_container() && flags.default_container_info.isSome()) {
+    executor.mutable_container()->CopyFrom(
+        flags.default_container_info.get());
+  }
+
+  return executor;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/src/tests/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator_tests.cpp b/src/tests/isolator_tests.cpp
index db7a58a..a0653e2 100644
--- a/src/tests/isolator_tests.cpp
+++ b/src/tests/isolator_tests.cpp
@@ -48,6 +48,7 @@
 #include "slave/containerizer/isolators/cgroups/cpushare.hpp"
 #include "slave/containerizer/isolators/cgroups/mem.hpp"
 #include "slave/containerizer/isolators/cgroups/perf_event.hpp"
+#include "slave/containerizer/isolators/filesystem/shared.hpp"
 #endif // __linux__
 #include "slave/containerizer/isolators/posix.hpp"
 
@@ -75,6 +76,7 @@ using mesos::internal::master::Master;
 using mesos::internal::slave::CgroupsCpushareIsolatorProcess;
 using mesos::internal::slave::CgroupsMemIsolatorProcess;
 using mesos::internal::slave::CgroupsPerfEventIsolatorProcess;
+using mesos::internal::slave::SharedFilesystemIsolatorProcess;
 #endif // __linux__
 using mesos::internal::slave::Isolator;
 using mesos::internal::slave::IsolatorProcess;
@@ -752,4 +754,178 @@ TEST_F(PerfEventIsolatorTest, ROOT_CGROUPS_Sample)
   delete isolator.get();
 }
 
+class SharedFilesystemIsolatorTest : public MesosTest {};
+
+
+// Test that a container can create a private view of a system
+// directory (/var/tmp). Check that a file written by a process inside
+// the container doesn't appear on the host filesystem but does appear
+// under the container's work directory.
+TEST_F(SharedFilesystemIsolatorTest, ROOT_RelativeVolume)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/shared";
+
+  Try<Isolator*> isolator = SharedFilesystemIsolatorProcess::create(flags);
+  CHECK_SOME(isolator);
+
+  Try<Launcher*> launcher = LinuxLauncher::create(flags);
+  CHECK_SOME(launcher);
+
+  // Use /var/tmp so we don't mask the work directory (under /tmp).
+  const string containerPath = "/var/tmp";
+  ASSERT_TRUE(os::isdir(containerPath));
+
+  // Use a host path relative to the container work directory.
+  const string hostPath = strings::remove(containerPath, "/", strings::PREFIX);
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.add_volumes()->CopyFrom(
+      CREATE_VOLUME(containerPath, hostPath, Volume::RW));
+
+  ExecutorInfo executorInfo;
+  executorInfo.mutable_container()->CopyFrom(containerInfo);
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  Future<Option<CommandInfo> > prepare =
+    isolator.get()->prepare(containerId, executorInfo, flags.work_dir);
+  AWAIT_READY(prepare);
+  ASSERT_SOME(prepare.get());
+
+  // The test will touch a file in container path.
+  const string file = path::join(containerPath, UUID::random().toString());
+  ASSERT_FALSE(os::exists(file));
+
+  // Manually run the isolator's preparation command first, then touch
+  // the file.
+  vector<string> args;
+  args.push_back("/bin/sh");
+  args.push_back("-x");
+  args.push_back("-c");
+  args.push_back(prepare.get().get().value() + " && touch " + file);
+
+  Try<pid_t> pid = launcher.get()->fork(
+      containerId,
+      "/bin/sh",
+      args,
+      Subprocess::FD(STDIN_FILENO),
+      Subprocess::FD(STDOUT_FILENO),
+      Subprocess::FD(STDERR_FILENO),
+      None(),
+      None(),
+      None());
+  ASSERT_SOME(pid);
+
+  // Set up the reaper to wait on the forked child.
+  Future<Option<int> > status = process::reap(pid.get());
+
+  AWAIT_READY(status);
+  EXPECT_SOME_EQ(0, status.get());
+
+  // Check the correct hierarchy was created under the container work
+  // directory.
+  string dir = "/";
+  foreach (const string& subdir, strings::tokenize(containerPath, "/")) {
+    dir = path::join(dir, subdir);
+
+    struct stat hostStat;
+    EXPECT_EQ(0, ::stat(dir.c_str(), &hostStat));
+
+    struct stat containerStat;
+    EXPECT_EQ(0,
+              ::stat(path::join(flags.work_dir, dir).c_str(), &containerStat));
+
+    EXPECT_EQ(hostStat.st_mode, containerStat.st_mode);
+    EXPECT_EQ(hostStat.st_uid, containerStat.st_uid);
+    EXPECT_EQ(hostStat.st_gid, containerStat.st_gid);
+  }
+
+  // Check it did *not* create a file in the host namespace.
+  EXPECT_FALSE(os::exists(file));
+
+  // Check it did create the file under the container's work directory
+  // on the host.
+  EXPECT_TRUE(os::exists(path::join(flags.work_dir, file)));
+
+  delete launcher.get();
+  delete isolator.get();
+}
+
+
+TEST_F(SharedFilesystemIsolatorTest, ROOT_AbsoluteVolume)
+{
+  slave::Flags flags = CreateSlaveFlags();
+  flags.isolation = "filesystem/shared";
+
+  Try<Isolator*> isolator = SharedFilesystemIsolatorProcess::create(flags);
+  CHECK_SOME(isolator);
+
+  Try<Launcher*> launcher = LinuxLauncher::create(flags);
+  CHECK_SOME(launcher);
+
+  // We'll mount the absolute test work directory as /var/tmp in the
+  // container.
+  const string hostPath = flags.work_dir;
+  const string containerPath = "/var/tmp";
+
+  ContainerInfo containerInfo;
+  containerInfo.set_type(ContainerInfo::MESOS);
+  containerInfo.add_volumes()->CopyFrom(
+      CREATE_VOLUME(containerPath, hostPath, Volume::RW));
+
+  ExecutorInfo executorInfo;
+  executorInfo.mutable_container()->CopyFrom(containerInfo);
+
+  ContainerID containerId;
+  containerId.set_value(UUID::random().toString());
+
+  Future<Option<CommandInfo> > prepare =
+    isolator.get()->prepare(containerId, executorInfo, flags.work_dir);
+  AWAIT_READY(prepare);
+  ASSERT_SOME(prepare.get());
+
+  // Test the volume mounting by touching a file in the container's
+  // /tmp, which should then be in flags.work_dir.
+  const string filename = UUID::random().toString();
+  ASSERT_FALSE(os::exists(path::join(containerPath, filename)));
+
+  vector<string> args;
+  args.push_back("/bin/sh");
+  args.push_back("-x");
+  args.push_back("-c");
+  args.push_back(prepare.get().get().value() +
+                 " && touch " +
+                 path::join(containerPath, filename));
+
+  Try<pid_t> pid = launcher.get()->fork(
+      containerId,
+      "/bin/sh",
+      args,
+      Subprocess::FD(STDIN_FILENO),
+      Subprocess::FD(STDOUT_FILENO),
+      Subprocess::FD(STDERR_FILENO),
+      None(),
+      None(),
+      None());
+  ASSERT_SOME(pid);
+
+  // Set up the reaper to wait on the forked child.
+  Future<Option<int> > status = process::reap(pid.get());
+
+  AWAIT_READY(status);
+  EXPECT_SOME_EQ(0, status.get());
+
+  // Check the file was created in flags.work_dir.
+  EXPECT_TRUE(os::exists(path::join(hostPath, filename)));
+
+  // Check it didn't get created in the host's view of containerPath.
+  EXPECT_FALSE(os::exists(path::join(containerPath, filename)));
+
+  delete launcher.get();
+  delete isolator.get();
+}
+
 #endif // __linux__

http://git-wip-us.apache.org/repos/asf/mesos/blob/f511395e/src/tests/mesos.hpp
----------------------------------------------------------------------
diff --git a/src/tests/mesos.hpp b/src/tests/mesos.hpp
index e36e138..c1d64a7 100644
--- a/src/tests/mesos.hpp
+++ b/src/tests/mesos.hpp
@@ -311,6 +311,14 @@ protected:
      commandInfo; })
 
 
+#define CREATE_VOLUME(containerPath, hostPath, mode)                  \
+      ({ Volume volume;                                               \
+         volume.set_container_path(containerPath);                    \
+         volume.set_host_path(hostPath);                              \
+         volume.set_mode(mode);                                       \
+         volume; })
+
+
 // TODO(bmahler): Refactor this to make the distinction between
 // command tasks and executor tasks clearer.
 inline TaskInfo createTask(


[2/4] git commit: Pass executor directory to Isolator::prepare().

Posted by id...@apache.org.
Pass executor directory to Isolator::prepare().

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


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

Branch: refs/heads/master
Commit: 8e6e36a9ab3376cb2adb9b1e4f9d00e10fec1f8c
Parents: b493875
Author: Ian Downes <id...@twitter.com>
Authored: Fri Jun 27 12:06:12 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Mon Oct 27 10:36:36 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/isolator.cpp            |  6 +-
 src/slave/containerizer/isolator.hpp            |  6 +-
 .../isolators/cgroups/cpushare.cpp              |  3 +-
 .../isolators/cgroups/cpushare.hpp              |  3 +-
 .../containerizer/isolators/cgroups/mem.cpp     |  3 +-
 .../containerizer/isolators/cgroups/mem.hpp     |  3 +-
 .../isolators/cgroups/perf_event.cpp            |  3 +-
 .../isolators/cgroups/perf_event.hpp            |  3 +-
 .../isolators/network/port_mapping.cpp          |  3 +-
 .../isolators/network/port_mapping.hpp          |  3 +-
 src/slave/containerizer/isolators/posix.hpp     |  3 +-
 src/tests/isolator.hpp                          |  3 +-
 src/tests/isolator_tests.cpp                    | 50 +++++++---
 src/tests/port_mapping_tests.cpp                | 99 +++++++++++++++++---
 14 files changed, 149 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolator.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolator.cpp b/src/slave/containerizer/isolator.cpp
index 5e61bf2..69849d2 100644
--- a/src/slave/containerizer/isolator.cpp
+++ b/src/slave/containerizer/isolator.cpp
@@ -52,12 +52,14 @@ Future<Nothing> Isolator::recover(const list<state::RunState>& state)
 
 Future<Option<CommandInfo> > Isolator::prepare(
     const ContainerID& containerId,
-    const ExecutorInfo& executorInfo)
+    const ExecutorInfo& executorInfo,
+    const string& directory)
 {
   return dispatch(process.get(),
                   &IsolatorProcess::prepare,
                   containerId,
-                  executorInfo);
+                  executorInfo,
+                  directory);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolator.hpp b/src/slave/containerizer/isolator.hpp
index e52e8b1..4c9d1d8 100644
--- a/src/slave/containerizer/isolator.hpp
+++ b/src/slave/containerizer/isolator.hpp
@@ -77,7 +77,8 @@ public:
   // only the command value is used.
   process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,
-      const ExecutorInfo& executorInfo);
+      const ExecutorInfo& executorInfo,
+      const std::string& directory);
 
   // Isolate the executor.
   process::Future<Nothing> isolate(
@@ -119,7 +120,8 @@ public:
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,
-      const ExecutorInfo& executorInfo) = 0;
+      const ExecutorInfo& executorInfo,
+      const std::string& directory) = 0;
 
   virtual process::Future<Nothing> isolate(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolators/cgroups/cpushare.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.cpp b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
index 7164ecc..f9531e4 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.cpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.cpp
@@ -249,7 +249,8 @@ Future<Nothing> CgroupsCpushareIsolatorProcess::recover(
 
 Future<Option<CommandInfo> > CgroupsCpushareIsolatorProcess::prepare(
     const ContainerID& containerId,
-    const ExecutorInfo& executorInfo)
+    const ExecutorInfo& executorInfo,
+    const string& directory)
 {
   if (infos.contains(containerId)) {
     return Failure("Container has already been prepared");

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolators/cgroups/cpushare.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/cpushare.hpp b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
index 2187c29..5d43169 100644
--- a/src/slave/containerizer/isolators/cgroups/cpushare.hpp
+++ b/src/slave/containerizer/isolators/cgroups/cpushare.hpp
@@ -47,7 +47,8 @@ public:
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,
-      const ExecutorInfo& executorInfo);
+      const ExecutorInfo& executorInfo,
+      const std::string& directory);
 
   virtual process::Future<Nothing> isolate(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolators/cgroups/mem.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.cpp b/src/slave/containerizer/isolators/cgroups/mem.cpp
index b3d4a5d..96bc506 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.cpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.cpp
@@ -204,7 +204,8 @@ Future<Nothing> CgroupsMemIsolatorProcess::recover(
 
 Future<Option<CommandInfo> > CgroupsMemIsolatorProcess::prepare(
     const ContainerID& containerId,
-    const ExecutorInfo& executorInfo)
+    const ExecutorInfo& executorInfo,
+    const string& directory)
 {
   if (infos.contains(containerId)) {
     return Failure("Container has already been prepared");

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolators/cgroups/mem.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/mem.hpp b/src/slave/containerizer/isolators/cgroups/mem.hpp
index b1b4f5a..25e4afc 100644
--- a/src/slave/containerizer/isolators/cgroups/mem.hpp
+++ b/src/slave/containerizer/isolators/cgroups/mem.hpp
@@ -41,7 +41,8 @@ public:
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,
-      const ExecutorInfo& executorInfo);
+      const ExecutorInfo& executorInfo,
+      const std::string& directory);
 
   virtual process::Future<Nothing> isolate(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolators/cgroups/perf_event.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.cpp b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
index 4ced508..7ed418a 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.cpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.cpp
@@ -213,7 +213,8 @@ Future<Nothing> CgroupsPerfEventIsolatorProcess::recover(
 
 Future<Option<CommandInfo> > CgroupsPerfEventIsolatorProcess::prepare(
     const ContainerID& containerId,
-    const ExecutorInfo& executorInfo)
+    const ExecutorInfo& executorInfo,
+    const string& directory)
 {
   if (infos.contains(containerId)) {
     return Failure("Container has already been prepared");

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolators/cgroups/perf_event.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/cgroups/perf_event.hpp b/src/slave/containerizer/isolators/cgroups/perf_event.hpp
index f7283d8..7cb2ba2 100644
--- a/src/slave/containerizer/isolators/cgroups/perf_event.hpp
+++ b/src/slave/containerizer/isolators/cgroups/perf_event.hpp
@@ -45,7 +45,8 @@ public:
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,
-      const ExecutorInfo& executorInfo);
+      const ExecutorInfo& executorInfo,
+      const std::string& directory);
 
   virtual process::Future<Nothing> isolate(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.cpp b/src/slave/containerizer/isolators/network/port_mapping.cpp
index 9cb7487..1d0609a 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -1437,7 +1437,8 @@ PortMappingIsolatorProcess::_recover(pid_t pid)
 
 Future<Option<CommandInfo> > PortMappingIsolatorProcess::prepare(
     const ContainerID& containerId,
-    const ExecutorInfo& executorInfo)
+    const ExecutorInfo& executorInfo,
+    const string& directory)
 {
   if (unmanaged.contains(containerId)) {
     return Failure("Asked to prepare an unmanaged container");

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolators/network/port_mapping.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.hpp b/src/slave/containerizer/isolators/network/port_mapping.hpp
index b3fd331..f9215b2 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.hpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.hpp
@@ -134,7 +134,8 @@ public:
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,
-      const ExecutorInfo& executorInfo);
+      const ExecutorInfo& executorInfo,
+      const std::string& directory);
 
   virtual process::Future<Nothing> isolate(
       const ContainerID& containerId,

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/slave/containerizer/isolators/posix.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/posix.hpp b/src/slave/containerizer/isolators/posix.hpp
index 6038e92..7e02f92 100644
--- a/src/slave/containerizer/isolators/posix.hpp
+++ b/src/slave/containerizer/isolators/posix.hpp
@@ -69,7 +69,8 @@ public:
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,
-      const ExecutorInfo& executorInfo)
+      const ExecutorInfo& executorInfo,
+      const std::string& directory)
   {
     if (promises.contains(containerId)) {
       return process::Failure("Container " + stringify(containerId) +

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/tests/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator.hpp b/src/tests/isolator.hpp
index 89df4c4..d8f3f09 100644
--- a/src/tests/isolator.hpp
+++ b/src/tests/isolator.hpp
@@ -45,7 +45,8 @@ public:
 
   virtual process::Future<Option<CommandInfo> > prepare(
       const ContainerID& containerId,
-      const ExecutorInfo& executorInfo)
+      const ExecutorInfo& executorInfo,
+      const std::string& directory)
   {
     return commandInfo;
   }

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/tests/isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/isolator_tests.cpp b/src/tests/isolator_tests.cpp
index 52b38a3..db7a58a 100644
--- a/src/tests/isolator_tests.cpp
+++ b/src/tests/isolator_tests.cpp
@@ -148,10 +148,13 @@ TYPED_TEST(CpuIsolatorTest, UserCpuUsage)
   ContainerID containerId;
   containerId.set_value("user_cpu_usage");
 
-  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo));
-
-  Try<string> dir = os::mkdtemp();
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
   ASSERT_SOME(dir);
+
+  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo, dir.get()));
+
   const string& file = path::join(dir.get(), "mesos_isolator_test_ready");
 
   // Max out a single core in userspace. This will run for at most one second.
@@ -230,8 +233,6 @@ TYPED_TEST(CpuIsolatorTest, UserCpuUsage)
 
   delete isolator.get();
   delete launcher.get();
-
-  CHECK_SOME(os::rmdir(dir.get()));
 }
 
 
@@ -252,10 +253,13 @@ TYPED_TEST(CpuIsolatorTest, SystemCpuUsage)
   ContainerID containerId;
   containerId.set_value("system_cpu_usage");
 
-  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo));
-
-  Try<string> dir = os::mkdtemp();
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
   ASSERT_SOME(dir);
+
+  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo, dir.get()));
+
   const string& file = path::join(dir.get(), "mesos_isolator_test_ready");
 
   // Generating random numbers is done by the kernel and will max out a single
@@ -335,8 +339,6 @@ TYPED_TEST(CpuIsolatorTest, SystemCpuUsage)
 
   delete isolator.get();
   delete launcher.get();
-
-  CHECK_SOME(os::rmdir(dir.get()));
 }
 
 
@@ -364,7 +366,12 @@ TEST_F(LimitedCpuIsolatorTest, ROOT_CGROUPS_Cfs)
   ContainerID containerId;
   containerId.set_value("mesos_test_cfs_cpu_limit");
 
-  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo));
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
+  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo, dir.get()));
 
   // Generate random numbers to max out a single core. We'll run this for 0.5
   // seconds of wall time so it should consume approximately 250 ms of total
@@ -465,7 +472,12 @@ TEST_F(LimitedCpuIsolatorTest, ROOT_CGROUPS_Cfs_Big_Quota)
   ContainerID containerId;
   containerId.set_value("mesos_test_cfs_big_cpu_limit");
 
-  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo));
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
+  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo, dir.get()));
 
   int pipes[2];
   ASSERT_NE(-1, ::pipe(pipes));
@@ -594,7 +606,12 @@ TYPED_TEST(MemIsolatorTest, MemUsage)
   ContainerID containerId;
   containerId.set_value("memory_usage");
 
-  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo));
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
+  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo, dir.get()));
 
   int pipes[2];
   ASSERT_NE(-1, ::pipe(pipes));
@@ -681,7 +698,12 @@ TEST_F(PerfEventIsolatorTest, ROOT_CGROUPS_Sample)
   ContainerID containerId;
   containerId.set_value("test");
 
-  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo));
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
+  AWAIT_READY(isolator.get()->prepare(containerId, executorInfo, dir.get()));
 
   // This first sample is likely to be empty because perf hasn't
   // completed yet but we should still have the required fields.

http://git-wip-us.apache.org/repos/asf/mesos/blob/8e6e36a9/src/tests/port_mapping_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/port_mapping_tests.cpp b/src/tests/port_mapping_tests.cpp
index 973bdef..1a5e52c 100644
--- a/src/tests/port_mapping_tests.cpp
+++ b/src/tests/port_mapping_tests.cpp
@@ -357,8 +357,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerTCPTest)
   ContainerID containerId1;
   containerId1.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir1 = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir1);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId1, executorInfo);
+    isolator.get()->prepare(containerId1, executorInfo, dir1.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -411,8 +416,14 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerTCPTest)
   executorInfo.mutable_resources()->CopyFrom(
       Resources::parse(container2Ports).get());
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir2 = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir2);
+
   Future<Option<CommandInfo> > preparation2 =
-    isolator.get()->prepare(containerId2, executorInfo);
+    isolator.get()->prepare(containerId2, executorInfo, dir2.get());
+
   AWAIT_READY(preparation2);
   ASSERT_SOME(preparation2.get());
 
@@ -493,8 +504,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerUDPTest)
   ContainerID containerId1;
   containerId1.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir1 = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir1);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId1, executorInfo);
+    isolator.get()->prepare(containerId1, executorInfo, dir1.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -548,8 +564,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerToContainerUDPTest)
   executorInfo.mutable_resources()->CopyFrom(
       Resources::parse(container2Ports).get());
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir2 = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir2);
+
   Future<Option<CommandInfo> > preparation2 =
-    isolator.get()->prepare(containerId2, executorInfo);
+    isolator.get()->prepare(containerId2, executorInfo, dir2.get());
   AWAIT_READY(preparation2);
   ASSERT_SOME(preparation2.get());
 
@@ -631,8 +652,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerUDPTest)
   ContainerID containerId;
   containerId.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId, executorInfo);
+    isolator.get()->prepare(containerId, executorInfo, dir.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -740,8 +766,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_HostToContainerTCPTest)
   ContainerID containerId;
   containerId.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId, executorInfo);
+    isolator.get()->prepare(containerId, executorInfo, dir.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -856,8 +887,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerICMPExternalTest)
   ContainerID containerId;
   containerId.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId, executorInfo);
+    isolator.get()->prepare(containerId, executorInfo, dir.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -929,8 +965,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerICMPInternalTest)
   ContainerID containerId;
   containerId.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId, executorInfo);
+    isolator.get()->prepare(containerId, executorInfo, dir.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -1006,8 +1047,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_ContainerARPExternalTest)
   ContainerID containerId;
   containerId.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId, executorInfo);
+    isolator.get()->prepare(containerId, executorInfo, dir.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -1088,8 +1134,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_DNSTest)
   ContainerID containerId;
   containerId.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId, executorInfo);
+    isolator.get()->prepare(containerId, executorInfo, dir.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -1166,8 +1217,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_TooManyContainersTest)
   ContainerID containerId1;
   containerId1.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir1 = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir1);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId1, executorInfo);
+    isolator.get()->prepare(containerId1, executorInfo, dir1.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -1205,8 +1261,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_TooManyContainersTest)
   executorInfo.mutable_resources()->CopyFrom(
       Resources::parse(container2Ports).get());
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir2 = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir2);
+
   Future<Option<CommandInfo> > preparation2 =
-    isolator.get()->prepare(containerId2, executorInfo);
+    isolator.get()->prepare(containerId2, executorInfo, dir2.get());
   AWAIT_FAILED(preparation2);
 
   // Ensure all processes are killed.
@@ -1258,8 +1319,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_SmallEgressLimitTest)
   ContainerID containerId;
   containerId.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId, executorInfo);
+    isolator.get()->prepare(containerId, executorInfo, dir.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 
@@ -1374,8 +1440,13 @@ TEST_F(PortMappingIsolatorTest, ROOT_ExportRTTTest)
   ContainerID containerId;
   containerId.set_value("container1");
 
+  // Use a relative temporary directory so it gets cleaned up
+  // automatically with the test.
+  Try<string> dir1 = os::mkdtemp(path::join(os::getcwd(), "XXXXXX"));
+  ASSERT_SOME(dir1);
+
   Future<Option<CommandInfo> > preparation1 =
-    isolator.get()->prepare(containerId, executorInfo);
+    isolator.get()->prepare(containerId, executorInfo, dir1.get());
   AWAIT_READY(preparation1);
   ASSERT_SOME(preparation1.get());
 


[3/4] git commit: Remove /proc and /sys remounts from port_mapping isolator.

Posted by id...@apache.org.
Remove /proc and /sys remounts from port_mapping isolator.

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


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

Branch: refs/heads/master
Commit: c18a50a0f199d1b8cdf121f39cd83163f0f8b152
Parents: 47fa5a1
Author: Ian Downes <id...@twitter.com>
Authored: Wed Oct 1 10:38:23 2014 -0700
Committer: Ian Downes <id...@twitter.com>
Committed: Mon Oct 27 10:36:37 2014 -0700

----------------------------------------------------------------------
 src/slave/containerizer/isolators/network/port_mapping.cpp | 5 -----
 src/slave/containerizer/linux_launcher.cpp                 | 5 ++---
 2 files changed, 2 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/c18a50a0/src/slave/containerizer/isolators/network/port_mapping.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/isolators/network/port_mapping.cpp b/src/slave/containerizer/isolators/network/port_mapping.cpp
index 1d0609a..1234d8e 100644
--- a/src/slave/containerizer/isolators/network/port_mapping.cpp
+++ b/src/slave/containerizer/isolators/network/port_mapping.cpp
@@ -2622,11 +2622,6 @@ string PortMappingIsolatorProcess::scripts(Info* info)
   script << "#!/bin/sh\n";
   script << "set -x\n";
 
-  // Remount /proc and /sys to show a separate networking stack.
-  // These should be done by a FilesystemIsolator in the future.
-  script << "mount -n -o remount -t sysfs none /sys\n";
-  script << "mount -n -o remount -t proc none /proc\n";
-
   // Mark the mount point BIND_MOUNT_ROOT as slave mount so that
   // changes in the container will not be propagated to the host.
   script << "mount --make-rslave " << BIND_MOUNT_ROOT << "\n";

http://git-wip-us.apache.org/repos/asf/mesos/blob/c18a50a0/src/slave/containerizer/linux_launcher.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/linux_launcher.cpp b/src/slave/containerizer/linux_launcher.cpp
index f7bc894..07ee643 100644
--- a/src/slave/containerizer/linux_launcher.cpp
+++ b/src/slave/containerizer/linux_launcher.cpp
@@ -96,11 +96,10 @@ Try<Launcher*> LinuxLauncher::create(const Flags& flags)
   int namespaces = 0;
 
 #ifdef WITH_NETWORK_ISOLATOR
-  // The network port mapping isolator requires network (CLONE_NEWNET)
-  // and mount (CLONE_NEWNS) namespaces.
+  // The network port mapping isolator requires network namespaces
+  // (CLONE_NEWNET).
   if (strings::contains(flags.isolation, "network/port_mapping")) {
     namespaces |= CLONE_NEWNET;
-    namespaces |= CLONE_NEWNS;
   }
 #endif