You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2015/12/30 00:13:32 UTC

[1/4] mesos git commit: Update tests that use the MesosContainerizer to use ContainerLogger.

Repository: mesos
Updated Branches:
  refs/heads/master 0d66b6915 -> d5bc8de98


Update tests that use the MesosContainerizer to use ContainerLogger.

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


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

Branch: refs/heads/master
Commit: d5bc8de98b206843495674a6210bd76f8b24c038
Parents: d041363
Author: Joseph Wu <jo...@mesosphere.io>
Authored: Tue Dec 29 15:01:09 2015 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Tue Dec 29 15:13:23 2015 -0800

----------------------------------------------------------------------
 .../containerizer/filesystem_isolator_tests.cpp | 30 ++++++++++++-------
 .../containerizer/mesos_containerizer_tests.cpp | 31 ++++++++++++++++++++
 2 files changed, 50 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d5bc8de9/src/tests/containerizer/filesystem_isolator_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/filesystem_isolator_tests.cpp b/src/tests/containerizer/filesystem_isolator_tests.cpp
index f1dd81b..5bb8503 100644
--- a/src/tests/containerizer/filesystem_isolator_tests.cpp
+++ b/src/tests/containerizer/filesystem_isolator_tests.cpp
@@ -18,6 +18,8 @@
 
 #include <mesos/mesos.hpp>
 
+#include <mesos/slave/container_logger.hpp>
+
 #include <process/owned.hpp>
 #include <process/gtest.hpp>
 
@@ -76,6 +78,7 @@ using mesos::internal::slave::ProvisionerProcess;
 using mesos::internal::slave::Slave;
 using mesos::internal::slave::Store;
 
+using mesos::slave::ContainerLogger;
 using mesos::slave::Isolator;
 
 namespace mesos {
@@ -151,32 +154,37 @@ protected:
 
     Owned<Provisioner> provisioner(new Provisioner(provisionerProcess));
 
-    Try<Isolator*> _isolator =
+    Try<Isolator*> isolator =
       LinuxFilesystemIsolatorProcess::create(flags, provisioner);
 
-    if (_isolator.isError()) {
+    if (isolator.isError()) {
       return Error(
           "Failed to create LinuxFilesystemIsolatorProcess: " +
-          _isolator.error());
+          isolator.error());
     }
 
-    Owned<Isolator> isolator(_isolator.get());
-
-    Try<Launcher*> _launcher = LinuxLauncher::create(flags);
+    Try<Launcher*> launcher = LinuxLauncher::create(flags);
 
-    if (_launcher.isError()) {
-      return Error("Failed to create LinuxLauncher: " + _launcher.error());
+    if (launcher.isError()) {
+      return Error("Failed to create LinuxLauncher: " + launcher.error());
     }
 
-    Owned<Launcher> launcher(_launcher.get());
+    // Create and initialize a new container logger.
+    Try<ContainerLogger*> logger =
+      ContainerLogger::create(flags.container_logger);
+
+    if (logger.isError()) {
+      return Error("Failed to create container logger: " + logger.error());
+    }
 
     return Owned<MesosContainerizer>(
         new MesosContainerizer(
             flags,
             true,
             &fetcher,
-            launcher,
-            {isolator}));
+            Owned<ContainerLogger>(logger.get()),
+            Owned<Launcher>(launcher.get()),
+            {Owned<Isolator>(isolator.get())}));
   }
 
   ContainerInfo createContainerInfo(

http://git-wip-us.apache.org/repos/asf/mesos/blob/d5bc8de9/src/tests/containerizer/mesos_containerizer_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/mesos_containerizer_tests.cpp b/src/tests/containerizer/mesos_containerizer_tests.cpp
index fe67935..95b493c 100644
--- a/src/tests/containerizer/mesos_containerizer_tests.cpp
+++ b/src/tests/containerizer/mesos_containerizer_tests.cpp
@@ -23,6 +23,7 @@
 
 #include <mesos/mesos.hpp>
 
+#include <mesos/slave/container_logger.hpp>
 #include <mesos/slave/isolator.hpp>
 
 #include <process/future.hpp>
@@ -62,6 +63,7 @@ using mesos::internal::slave::state::RunState;
 using mesos::internal::slave::state::SlaveState;
 
 using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerLogger;
 using mesos::slave::ContainerPrepareInfo;
 using mesos::slave::ContainerState;
 using mesos::slave::Isolator;
@@ -109,10 +111,19 @@ public:
       return Error(launcher.error());
     }
 
+    // Create and initialize a new container logger.
+    Try<ContainerLogger*> logger =
+      ContainerLogger::create(flags.container_logger);
+
+    if (logger.isError()) {
+      return Error("Failed to create container logger: " + logger.error());
+    }
+
     return new MesosContainerizer(
         flags,
         false,
         fetcher,
+        Owned<ContainerLogger>(logger.get()),
         Owned<Launcher>(launcher.get()),
         isolators);
   }
@@ -443,12 +454,14 @@ public:
       const slave::Flags& flags,
       bool local,
       Fetcher* fetcher,
+      const Owned<ContainerLogger>& logger,
       const Owned<Launcher>& launcher,
       const vector<Owned<Isolator>>& isolators)
     : MesosContainerizerProcess(
           flags,
           local,
           fetcher,
+          logger,
           launcher,
           isolators)
   {
@@ -552,10 +565,16 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhileFetching)
 
   Fetcher fetcher;
 
+  Try<ContainerLogger*> logger =
+    ContainerLogger::create(flags.container_logger);
+
+  ASSERT_SOME(logger);
+
   MockMesosContainerizerProcess* process = new MockMesosContainerizerProcess(
       flags,
       true,
       &fetcher,
+      Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher.get()),
       vector<Owned<Isolator>>());
 
@@ -618,10 +637,16 @@ TEST_F(MesosContainerizerDestroyTest, DestroyWhilePreparing)
 
   Fetcher fetcher;
 
+  Try<ContainerLogger*> logger =
+    ContainerLogger::create(flags.container_logger);
+
+  ASSERT_SOME(logger);
+
   MockMesosContainerizerProcess* process = new MockMesosContainerizerProcess(
       flags,
       true,
       &fetcher,
+      Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher.get()),
       {Owned<Isolator>(isolator)});
 
@@ -694,10 +719,16 @@ TEST_F(MesosContainerizerDestroyTest, LauncherDestroyFailure)
 
   Fetcher fetcher;
 
+  Try<ContainerLogger*> logger =
+    ContainerLogger::create(flags.container_logger);
+
+  ASSERT_SOME(logger);
+
   MesosContainerizerProcess* process = new MesosContainerizerProcess(
       flags,
       true,
       &fetcher,
+      Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher),
       vector<Owned<Isolator>>());
 


[4/4] mesos git commit: Explicitly disallow ContainerLogger with the External Containerizer.

Posted by be...@apache.org.
Explicitly disallow ContainerLogger with the External Containerizer.

Using the `ContainerLogger` with the `ExternalContainerizer` is not
supported at the moment.  We explicitly disallowed the combination, so
that there are no incorrect assumptions.

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


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

Branch: refs/heads/master
Commit: ed7bfa96153958edae48bd553e0b1747328546a0
Parents: 0d66b69
Author: Joseph Wu <jo...@mesosphere.io>
Authored: Tue Dec 29 11:25:10 2015 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Tue Dec 29 15:13:23 2015 -0800

----------------------------------------------------------------------
 src/slave/containerizer/containerizer.cpp | 14 ++++++++++++++
 1 file changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/ed7bfa96/src/slave/containerizer/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/containerizer.cpp b/src/slave/containerizer/containerizer.cpp
index dcdf98f..5e7e55e 100644
--- a/src/slave/containerizer/containerizer.cpp
+++ b/src/slave/containerizer/containerizer.cpp
@@ -168,6 +168,13 @@ Try<Containerizer*> Containerizer::create(
                  << "please update your flags to"
                  << " '--containerizers=external'.";
 
+    if (flags.container_logger.isSome()) {
+      return Error(
+          "The external containerizer does not support custom container "
+          "logger modules.  The '--isolation=external' flag cannot be "
+          " set along with '--container_logger=...'");
+    }
+
     Try<ExternalContainerizer*> containerizer =
       ExternalContainerizer::create(flags);
     if (containerizer.isError()) {
@@ -204,6 +211,13 @@ Try<Containerizer*> Containerizer::create(
         containerizers.push_back(containerizer.get());
       }
     } else if (type == "external") {
+      if (flags.container_logger.isSome()) {
+        return Error(
+            "The external containerizer does not support custom container "
+            "logger modules.  The '--containerizers=external' flag cannot be "
+            "set along with '--container_logger=...'");
+      }
+
       Try<ExternalContainerizer*> containerizer =
         ExternalContainerizer::create(flags);
       if (containerizer.isError()) {


[2/4] mesos git commit: Add support for the ContainerLogger to the Mesos Containerizer.

Posted by be...@apache.org.
Add support for the ContainerLogger to the Mesos Containerizer.

Changes the `MesosContainerizer` to create and initialize the
`ContainerLogger`.

The `MesosContainerizer` modifies the arguments to `launcher->fork()`
(in `::_launch`) by calling the `ContainerLogger` beforehand.

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


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

Branch: refs/heads/master
Commit: 17ef06d883a1771ce66fcc5301b765e0799ebeb8
Parents: ed7bfa9
Author: Joseph Wu <jo...@mesosphere.io>
Authored: Tue Dec 29 11:25:50 2015 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Tue Dec 29 15:13:23 2015 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 170 +++++++++++--------
 src/slave/containerizer/mesos/containerizer.hpp |   5 +
 2 files changed, 100 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/17ef06d8/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index 8242190..db8c746 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -16,6 +16,7 @@
 
 #include <mesos/module/isolator.hpp>
 
+#include <mesos/slave/container_logger.hpp>
 #include <mesos/slave/isolator.hpp>
 
 #include <process/collect.hpp>
@@ -90,6 +91,7 @@ namespace slave {
 using mesos::modules::ModuleManager;
 
 using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerLogger;
 using mesos::slave::ContainerPrepareInfo;
 using mesos::slave::ContainerState;
 using mesos::slave::Isolator;
@@ -106,6 +108,14 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     bool local,
     Fetcher* fetcher)
 {
+  // Create and initialize the container logger module.
+  Try<ContainerLogger*> logger =
+    ContainerLogger::create(flags.container_logger);
+
+  if (logger.isError()) {
+    return Error("Failed to create container logger: " + logger.error());
+  }
+
   string isolation;
 
   if (flags.isolation == "process") {
@@ -245,6 +255,7 @@ Try<MesosContainerizer*> MesosContainerizer::create(
       flags_,
       local,
       fetcher,
+      Owned<ContainerLogger>(logger.get()),
       Owned<Launcher>(launcher.get()),
       isolators);
 }
@@ -254,12 +265,14 @@ MesosContainerizer::MesosContainerizer(
     const Flags& flags,
     bool local,
     Fetcher* fetcher,
+    const Owned<ContainerLogger>& logger,
     const Owned<Launcher>& launcher,
     const vector<Owned<Isolator>>& isolators)
   : process(new MesosContainerizerProcess(
       flags,
       local,
       fetcher,
+      logger,
       launcher,
       isolators))
 {
@@ -790,89 +803,96 @@ Future<bool> MesosContainerizerProcess::_launch(
   JSON::Object commands;
   commands.values["commands"] = commandArray;
 
-  // Use a pipe to block the child until it's been isolated.
-  int pipes[2];
-
-  // We assume this should not fail under reasonable conditions so we
-  // use CHECK.
-  CHECK(pipe(pipes) == 0);
-
-  // Prepare the flags to pass to the launch process.
-  MesosContainerizerLaunch::Flags launchFlags;
-
-  launchFlags.command = JSON::protobuf(executorInfo.command());
-
-  launchFlags.directory = rootfs.isSome() ? flags.sandbox_directory : directory;
-  launchFlags.rootfs = rootfs;
-  launchFlags.user = user;
-  launchFlags.pipe_read = pipes[0];
-  launchFlags.pipe_write = pipes[1];
-  launchFlags.commands = commands;
-
-  // Fork the child using launcher.
-  vector<string> argv(2);
-  argv[0] = MESOS_CONTAINERIZER;
-  argv[1] = MesosContainerizerLaunch::NAME;
-
-  Try<pid_t> forked = launcher->fork(
-      containerId,
-      path::join(flags.launcher_dir, MESOS_CONTAINERIZER),
-      argv,
-      Subprocess::FD(STDIN_FILENO),
-      (local ? Subprocess::FD(STDOUT_FILENO)
-             : Subprocess::PATH(path::join(directory, "stdout"))),
-      (local ? Subprocess::FD(STDERR_FILENO)
-             : Subprocess::PATH(path::join(directory, "stderr"))),
-      launchFlags,
-      environment,
-      None(),
-      namespaces); // 'namespaces' will be ignored by PosixLauncher.
-
-  if (forked.isError()) {
-    return Failure("Failed to fork executor: " + forked.error());
-  }
-  pid_t pid = forked.get();
+  return logger->prepare(executorInfo, directory)
+    .then(defer(
+        self(),
+        [=](const ContainerLogger::SubprocessInfo& subprocessInfo)
+          -> Future<bool> {
+    // Use a pipe to block the child until it's been isolated.
+    int pipes[2];
+
+    // We assume this should not fail under reasonable conditions so we
+    // use CHECK.
+    CHECK(pipe(pipes) == 0);
+
+    // Prepare the flags to pass to the launch process.
+    MesosContainerizerLaunch::Flags launchFlags;
+
+    launchFlags.command = JSON::protobuf(executorInfo.command());
+
+    launchFlags.directory =
+      rootfs.isSome() ? flags.sandbox_directory : directory;
+    launchFlags.rootfs = rootfs;
+    launchFlags.user = user;
+    launchFlags.pipe_read = pipes[0];
+    launchFlags.pipe_write = pipes[1];
+    launchFlags.commands = commands;
+
+    // Fork the child using launcher.
+    vector<string> argv(2);
+    argv[0] = MESOS_CONTAINERIZER;
+    argv[1] = MesosContainerizerLaunch::NAME;
+
+    Try<pid_t> forked = launcher->fork(
+        containerId,
+        path::join(flags.launcher_dir, MESOS_CONTAINERIZER),
+        argv,
+        Subprocess::FD(STDIN_FILENO),
+        (local ? Subprocess::FD(STDOUT_FILENO)
+               : (Subprocess::IO) subprocessInfo.out),
+        (local ? Subprocess::FD(STDERR_FILENO)
+               : (Subprocess::IO) subprocessInfo.err),
+        launchFlags,
+        environment,
+        None(),
+        namespaces); // 'namespaces' will be ignored by PosixLauncher.
+
+    if (forked.isError()) {
+      return Failure("Failed to fork executor: " + forked.error());
+    }
+    pid_t pid = forked.get();
 
-  // Checkpoint the executor's pid if requested.
-  if (checkpoint) {
-    const string& path = slave::paths::getForkedPidPath(
-        slave::paths::getMetaRootDir(flags.work_dir),
-        slaveId,
-        executorInfo.framework_id(),
-        executorInfo.executor_id(),
-        containerId);
+    // Checkpoint the executor's pid if requested.
+    if (checkpoint) {
+      const string& path = slave::paths::getForkedPidPath(
+          slave::paths::getMetaRootDir(flags.work_dir),
+          slaveId,
+          executorInfo.framework_id(),
+          executorInfo.executor_id(),
+          containerId);
 
-    LOG(INFO) << "Checkpointing executor's forked pid " << pid
-              << " to '" << path <<  "'";
+      LOG(INFO) << "Checkpointing executor's forked pid " << pid
+                << " to '" << path <<  "'";
 
-    Try<Nothing> checkpointed =
-      slave::state::checkpoint(path, stringify(pid));
+      Try<Nothing> checkpointed =
+        slave::state::checkpoint(path, stringify(pid));
 
-    if (checkpointed.isError()) {
-      LOG(ERROR) << "Failed to checkpoint executor's forked pid to '"
-                 << path << "': " << checkpointed.error();
+      if (checkpointed.isError()) {
+        LOG(ERROR) << "Failed to checkpoint executor's forked pid to '"
+                   << path << "': " << checkpointed.error();
 
-      return Failure("Could not checkpoint executor's pid");
+        return Failure("Could not checkpoint executor's pid");
+      }
     }
-  }
 
-  // Monitor the executor's pid. We keep the future because we'll
-  // refer to it again during container destroy.
-  Future<Option<int>> status = process::reap(pid);
-  status.onAny(defer(self(), &Self::reaped, containerId));
-  containers_[containerId]->status = status;
+    // Monitor the executor's pid. We keep the future because we'll
+    // refer to it again during container destroy.
+    Future<Option<int>> status = process::reap(pid);
+    status.onAny(defer(self(), &Self::reaped, containerId));
+    containers_[containerId]->status = status;
 
-  return isolate(containerId, pid)
-    .then(defer(self(),
-                &Self::fetch,
-                containerId,
-                executorInfo.command(),
-                directory,
-                user,
-                slaveId))
-    .then(defer(self(), &Self::exec, containerId, pipes[1]))
-    .onAny(lambda::bind(&os::close, pipes[0]))
-    .onAny(lambda::bind(&os::close, pipes[1]));
+    return isolate(containerId, pid)
+      .then(defer(self(),
+                  &Self::fetch,
+                  containerId,
+                  executorInfo.command(),
+                  directory,
+                  user,
+                  slaveId))
+      .then(defer(self(), &Self::exec, containerId, pipes[1]))
+      .onAny(lambda::bind(&os::close, pipes[0]))
+      .onAny(lambda::bind(&os::close, pipes[1]));
+  }));
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ef06d8/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index ab87cbc..89d1862 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -20,6 +20,7 @@
 #include <list>
 #include <vector>
 
+#include <mesos/slave/container_logger.hpp>
 #include <mesos/slave/isolator.hpp>
 
 #include <process/metrics/counter.hpp>
@@ -54,6 +55,7 @@ public:
       const Flags& flags,
       bool local,
       Fetcher* fetcher,
+      const process::Owned<mesos::slave::ContainerLogger>& logger,
       const process::Owned<Launcher>& launcher,
       const std::vector<process::Owned<mesos::slave::Isolator>>& isolators);
 
@@ -111,11 +113,13 @@ public:
       const Flags& _flags,
       bool _local,
       Fetcher* _fetcher,
+      const process::Owned<mesos::slave::ContainerLogger>& _logger,
       const process::Owned<Launcher>& _launcher,
       const std::vector<process::Owned<mesos::slave::Isolator>>& _isolators)
     : flags(_flags),
       local(_local),
       fetcher(_fetcher),
+      logger(_logger),
       launcher(_launcher),
       isolators(_isolators) {}
 
@@ -233,6 +237,7 @@ private:
   const Flags flags;
   const bool local;
   Fetcher* fetcher;
+  process::Owned<mesos::slave::ContainerLogger> logger;
   const process::Owned<Launcher> launcher;
   const std::vector<process::Owned<mesos::slave::Isolator>> isolators;
 


[3/4] mesos git commit: Cleanups in containerizer.cpp.

Posted by be...@apache.org.
Cleanups in containerizer.cpp.


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

Branch: refs/heads/master
Commit: d041363f87b7c4d28df06cb96fc4fbe1189e16e4
Parents: 17ef06d
Author: Benjamin Hindman <be...@gmail.com>
Authored: Tue Dec 29 15:01:03 2015 -0800
Committer: Benjamin Hindman <be...@gmail.com>
Committed: Tue Dec 29 15:13:23 2015 -0800

----------------------------------------------------------------------
 src/slave/containerizer/mesos/containerizer.cpp | 142 +++++++++----------
 1 file changed, 67 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/d041363f/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index db8c746..f3c370a 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -108,30 +108,22 @@ Try<MesosContainerizer*> MesosContainerizer::create(
     bool local,
     Fetcher* fetcher)
 {
-  // Create and initialize the container logger module.
-  Try<ContainerLogger*> logger =
-    ContainerLogger::create(flags.container_logger);
-
-  if (logger.isError()) {
-    return Error("Failed to create container logger: " + logger.error());
-  }
-
-  string isolation;
+  // Modify `flags` based on the deprecated `isolation` flag (and then
+  // use `flags_` in the rest of this function).
+  Flags flags_ = flags;
 
   if (flags.isolation == "process") {
     LOG(WARNING) << "The 'process' isolation flag is deprecated, "
                  << "please update your flags to"
                  << " '--isolation=posix/cpu,posix/mem'.";
 
-    isolation = "posix/cpu,posix/mem";
+    flags_.isolation = "posix/cpu,posix/mem";
   } else if (flags.isolation == "cgroups") {
     LOG(WARNING) << "The 'cgroups' isolation flag is deprecated, "
                  << "please update your flags to"
                  << " '--isolation=cgroups/cpu,cgroups/mem'.";
 
-    isolation = "cgroups/cpu,cgroups/mem";
-  } else {
-    isolation = flags.isolation;
+    flags_.isolation = "cgroups/cpu,cgroups/mem";
   }
 
   // One and only one filesystem isolator is required. The filesystem
@@ -140,25 +132,61 @@ Try<MesosContainerizer*> MesosContainerizer::create(
   // the user does not specify one, 'filesystem/posix' will be used.
   //
   // TODO(jieyu): Check that only one filesystem isolator is used.
-  if (!strings::contains(isolation, "filesystem/")) {
-    isolation += ",filesystem/posix";
+  if (!strings::contains(flags_.isolation, "filesystem/")) {
+    flags_.isolation += ",filesystem/posix";
   }
 
-  // Modify the flags to include any changes to isolation.
-  Flags flags_ = flags;
-  flags_.isolation = isolation;
+  LOG(INFO) << "Using isolation: " << flags_.isolation;
+
+  // Create the container logger for the MesosContainerizer.
+  Try<ContainerLogger*> logger =
+    ContainerLogger::create(flags_.container_logger);
 
-  LOG(INFO) << "Using isolation: " << isolation;
+  if (logger.isError()) {
+    return Error("Failed to create container logger: " + logger.error());
+  }
+
+  // Create the launcher for the MesosContainerizer.
+  Try<Launcher*> launcher = [&flags_]() -> Try<Launcher*> {
+#ifdef __linux__
+    if (flags_.launcher.isSome()) {
+      // If the user has specified the launcher, use it.
+      if (flags_.launcher.get() == "linux") {
+        return LinuxLauncher::create(flags_);
+      } else if (flags_.launcher.get() == "posix") {
+        return PosixLauncher::create(flags_);
+      } else {
+        return Error(
+            "Unknown or unsupported launcher: " + flags_.launcher.get());
+      }
+    }
+
+    // Use Linux launcher if it is available, POSIX otherwise.
+    return LinuxLauncher::available()
+      ? LinuxLauncher::create(flags_)
+      : PosixLauncher::create(flags_);
+#else
+    if (flags_.launcher.isSome() && flags_.launcher.get() != "posix") {
+      return Error("Unsupported launcher: " + flags_.launcher.get());
+    }
+
+    return PosixLauncher::create(flags_);
+#endif // __linux__
+  }();
+
+  if (launcher.isError()) {
+    return Error("Failed to create launcher: " + launcher.error());
+  }
 
 #ifdef __linux__
   // The provisioner will be used by the 'filesystem/linux' isolator.
-  Try<Owned<Provisioner>> provisioner = Provisioner::create(flags, fetcher);
+  Try<Owned<Provisioner>> provisioner = Provisioner::create(flags_, fetcher);
   if (provisioner.isError()) {
     return Error("Failed to create provisioner: " + provisioner.error());
   }
 #endif
 
-  // Create a MesosContainerizerProcess using isolators and a launcher.
+  // Create the isolators for the MesosContainerizer.
   const hashmap<string, lambda::function<Try<Isolator*>(const Flags&)>>
     creators = {
     // Filesystem isolators.
@@ -189,68 +217,31 @@ Try<MesosContainerizer*> MesosContainerizer::create(
 
   vector<Owned<Isolator>> isolators;
 
-  foreach (const string& type, strings::tokenize(isolation, ",")) {
-    Owned<Isolator> isolator;
-
-    if (creators.contains(type)) {
-      Try<Isolator*> _isolator = creators.at(type)(flags_);
-      if (_isolator.isError()) {
-        return Error(
-            "Could not create isolator " + type + ": " + _isolator.error());
-      }
-
-      isolator.reset(_isolator.get());
-    } else if (ModuleManager::contains<Isolator>(type)) {
-      Try<Isolator*> _isolator = ModuleManager::create<Isolator>(type);
-      if (_isolator.isError()) {
-        return Error(
-            "Could not create isolator " + type + ": " + _isolator.error());
+  foreach (const string& type, strings::tokenize(flags_.isolation, ",")) {
+    Try<Isolator*> isolator = [&creators, &type, &flags_]() -> Try<Isolator*> {
+      if (creators.contains(type)) {
+        return creators.at(type)(flags_);
+      } else if (ModuleManager::contains<Isolator>(type)) {
+        return ModuleManager::create<Isolator>(type);
       }
+      return Error("Unknown or unsupported isolator");
+    }();
 
-      isolator.reset(_isolator.get());
-    } else {
-      return Error("Unknown or unsupported isolator: " + type);
+    if (isolator.isError()) {
+      return Error(
+          "Could not create isolator '" + type + "': " + isolator.error());
     }
 
     // NOTE: The filesystem isolator must be the first isolator used
     // so that the runtime isolators can have a consistent view on the
     // prepared filesystem (e.g., any volume mounts are performed).
     if (strings::contains(type, "filesystem/")) {
-      isolators.insert(isolators.begin(), isolator);
+      isolators.insert(isolators.begin(), Owned<Isolator>(isolator.get()));
     } else {
-      isolators.push_back(isolator);
+      isolators.push_back(Owned<Isolator>(isolator.get()));
     }
   }
 
-#ifdef __linux__
-  Try<Launcher*> launcher = (Launcher*) NULL;
-  if (flags_.launcher.isSome()) {
-    // If the user has specified the launcher, use it.
-    if (flags_.launcher.get() == "linux") {
-      launcher = LinuxLauncher::create(flags_);
-    } else if (flags_.launcher.get() == "posix") {
-      launcher = PosixLauncher::create(flags_);
-    } else {
-      return Error("Unknown or unsupported launcher: " + flags_.launcher.get());
-    }
-  } else {
-    // Use Linux launcher if it is available, POSIX otherwise.
-    launcher = LinuxLauncher::available()
-      ? LinuxLauncher::create(flags_)
-      : PosixLauncher::create(flags_);
-  }
-#else
-  if (flags_.launcher.isSome() && flags_.launcher.get() != "posix") {
-    return Error("Unsupported launcher: " + flags_.launcher.get());
-  }
-
-  Try<Launcher*> launcher = PosixLauncher::create(flags_);
-#endif // __linux__
-
-  if (launcher.isError()) {
-    return Error("Failed to create launcher: " + launcher.error());
-  }
-
   return new MesosContainerizer(
       flags_,
       local,
@@ -820,8 +811,9 @@ Future<bool> MesosContainerizerProcess::_launch(
 
     launchFlags.command = JSON::protobuf(executorInfo.command());
 
-    launchFlags.directory =
-      rootfs.isSome() ? flags.sandbox_directory : directory;
+    launchFlags.directory = rootfs.isSome()
+      ? flags.sandbox_directory
+      : directory;
     launchFlags.rootfs = rootfs;
     launchFlags.user = user;
     launchFlags.pipe_read = pipes[0];
@@ -839,9 +831,9 @@ Future<bool> MesosContainerizerProcess::_launch(
         argv,
         Subprocess::FD(STDIN_FILENO),
         (local ? Subprocess::FD(STDOUT_FILENO)
-               : (Subprocess::IO) subprocessInfo.out),
+               : Subprocess::IO(subprocessInfo.out)),
         (local ? Subprocess::FD(STDERR_FILENO)
-               : (Subprocess::IO) subprocessInfo.err),
+               : Subprocess::IO(subprocessInfo.err)),
         launchFlags,
         environment,
         None(),