You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by kl...@apache.org on 2017/03/01 18:07:38 UTC

[3/4] mesos git commit: Fixed ContainerLogger / IOSwitchboard FD leaks.

Fixed ContainerLogger / IOSwitchboard FD leaks.

Previously, the containizer launch path would leak FDs if the
containerizer launch path failed between successfully calling
prepare() on either the ContainerLogger (in the case of the Docker
containerizer) or the IOSwitchboard (in the case of the mesos
containerizer) and forking the actual container.

These components relied on the Subprocess call inside launcher->fork()
to close these FDS on their behalf. If the containerizer launch path
failed somewhere between calling prepare() and making this fork()
call, these FDs would never be closed.

In the case of the IOSwitchboard, this would lead to deadlock in the
destroy path because the future returned by the IOSwitchboard's
cleanup function would never be satisfied. The IOSwitchboard doesn't
shutdown until the FDs it allocates to the container have been closed.

This commit fixes this problem by updating the
ContainerLogger::ContainerIO::FD abstraction to change the way it
manages FDS. Instead of tagging each FD with the Subprocess::IO::OWNED
label and forcing the launcher->fork() call to deal with closing the
FDs once it's forked a new subprocess, we now do things slightly
differently now.

We now keep the default DUP label on each FD (instead fo changing it
to OWNED) to cause launcher->fork() to dup it before mapping it onto
the stdin/stdout/stderr of the subprocess. It then only closes the
duped FD, leaving the original one open.

In doing so, it's now the containerizer's responsibility to ensure
that these FDs are closed properly (whether that's between a
successful prepare() call and launcher->fork()) or after
launcher->fork() has completed successfully). While this has the
potential to complicate things slightly on the SUCCESS path,
at least it is now the containerizers's responsibility to close these
FDS in *all* cases, rather than splitting that responsibility across
components.

In order to simplify this, we've also modified the
ContainerLogger::ContainerIO::FD abstraction to hold a Shared
pointer to its underlying file descriptor and (optionally) close it on
destruction. With this, we can ensure that all file descriptors
created through this abstraction will be automatically closed onced
their final reference goes out of scope (even if its been copied
around several times).

In essence, this releases the containerizer from the burden of manually
closing these FDS itself. So long as it holds the final reference to
these FDs (which it does), they will be automatically closed along
*any* path out of containerizer->launch(). These are exactly the
semantics we want to achieve.

In the case of the the ContainerLogger, ownership of these FDs (and
thus their final reference) is passed to the containerizer in the
ContainerIO struct returned by prepare(). In the case of the
IOSwitchboard, we had to add a new API call to transfer ownership
(since it is an isolator and prepare() can only return a protobuf),
but the end result is the same.

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


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

Branch: refs/heads/master
Commit: 48c376a5fdbee4569e4af9144139d638c7924548
Parents: 3aa7e1b
Author: Kevin Klues <kl...@gmail.com>
Authored: Wed Mar 1 09:55:21 2017 -0800
Committer: Kevin Klues <kl...@gmail.com>
Committed: Wed Mar 1 09:55:21 2017 -0800

----------------------------------------------------------------------
 include/mesos/slave/container_logger.hpp        |  65 ++++++++----
 include/mesos/slave/containerizer.proto         |  21 ----
 src/slave/containerizer/mesos/containerizer.cpp |  78 +++-----------
 src/slave/containerizer/mesos/containerizer.hpp |   1 +
 .../containerizer/mesos/io/switchboard.cpp      | 106 +++++++++----------
 .../containerizer/mesos/io/switchboard.hpp      |  17 +++
 6 files changed, 127 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/48c376a5/include/mesos/slave/container_logger.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/slave/container_logger.hpp b/include/mesos/slave/container_logger.hpp
index e0cf320..f759640 100644
--- a/include/mesos/slave/container_logger.hpp
+++ b/include/mesos/slave/container_logger.hpp
@@ -25,6 +25,7 @@
 
 #include <process/future.hpp>
 #include <process/subprocess.hpp>
+#include <process/shared.hpp>
 
 #include <stout/try.hpp>
 #include <stout/nothing.hpp>
@@ -80,25 +81,19 @@ public:
 
       static IO PATH(const std::string& path)
       {
-        return IO(Type::PATH, None(), path);
+        return IO(Type::PATH, path);
       }
 
-      static IO FD(int_fd fd)
+      static IO FD(int_fd fd, bool closeOnDestruction = true)
       {
-        return IO(Type::FD, fd, None());
+        return IO(Type::FD, fd, closeOnDestruction);
       }
 
       operator process::Subprocess::IO () const
       {
         switch (type_) {
           case Type::FD:
-            // NOTE: The FD is not duplicated and will be closed (as
-            // seen by the agent process) when the container is
-            // spawned.  This shifts the burden of FD-lifecycle
-            // management into the Containerizer.
-            return process::Subprocess::FD(
-                fd_.get(),
-                process::Subprocess::IO::OWNED);
+            return process::Subprocess::FD(*fd_->get());
           case Type::PATH:
             return process::Subprocess::PATH(path_.get());
           default:
@@ -106,20 +101,46 @@ public:
         }
       }
 
-      Type type() const { return type_; }
-      Option<int_fd> fd() const { return fd_; }
-      Option<std::string> path() const { return path_; }
-
     private:
-      IO(Type _type,
-         const Option<int_fd>& _fd,
-         const Option<std::string>& _path)
+      // A simple abstraction to wrap an FD and (optionally) close it
+      // on destruction. We know that we never copy instances of this
+      // class once they are instantiated, so it's OK to call
+      // `close()` in the destructor since only one reference will
+      // ever exist to it.
+      class FDWrapper
+      {
+      public:
+        FDWrapper(int_fd _fd, bool _closeOnDestruction)
+          : fd(_fd), closeOnDestruction(_closeOnDestruction) {}
+
+        ~FDWrapper() {
+          CHECK(fd >= 0);
+          if (closeOnDestruction) {
+            close(fd);
+          }
+        }
+
+        operator int_fd() const { return fd; }
+
+      private:
+        FDWrapper(const FDWrapper& fd) = delete;
+
+        int_fd fd;
+        bool closeOnDestruction;
+      };
+
+      IO(Type _type, int_fd _fd, bool closeOnDestruction)
+        : type_(_type),
+          fd_(new FDWrapper(_fd, closeOnDestruction)),
+          path_(None()) {}
+
+      IO(Type _type, const std::string& _path)
         : type_(_type),
-          fd_(_fd),
+          fd_(None()),
           path_(_path) {}
 
       Type type_;
-      Option<int_fd> fd_;
+      Option<process::Shared<FDWrapper>> fd_;
       Option<std::string> path_;
     };
 
@@ -127,18 +148,18 @@ public:
      * How to redirect the stdin of the executable.
      * See `process::Subprocess::IO`.
      */
-    IO in = IO::FD(STDIN_FILENO);
+    IO in = IO::FD(STDIN_FILENO, false);
 
     /**
      * How to redirect the stdout of the executable.
      * See `process::Subprocess::IO`.
      */
-    IO out = IO::FD(STDOUT_FILENO);
+    IO out = IO::FD(STDOUT_FILENO, false);
 
     /**
      * Similar to `out`, except this describes how to redirect stderr.
      */
-    IO err = IO::FD(STDERR_FILENO);
+    IO err = IO::FD(STDERR_FILENO, false);
   };
 
   /**

http://git-wip-us.apache.org/repos/asf/mesos/blob/48c376a5/include/mesos/slave/containerizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/slave/containerizer.proto b/include/mesos/slave/containerizer.proto
index 76fde8a..c30b1fc 100644
--- a/include/mesos/slave/containerizer.proto
+++ b/include/mesos/slave/containerizer.proto
@@ -150,21 +150,6 @@ message ContainerConfig {
 }
 
 
-// This message is used to describe how to handle standard
-// input/output/error for the container.
-message ContainerIO {
-  enum Type {
-    UNKNOWN = 0;
-    FD = 1;       // File descriptor. Ownership will be transferred.
-    PATH = 2;     // File path.
-  }
-
-  optional Type type = 1;
-  optional int32 fd = 2;
-  optional string path = 3;
-}
-
-
 /**
  * Protobuf returned by Isolator::prepare(). The command is executed
  * by the Launcher in the containerized context.
@@ -209,12 +194,6 @@ message ContainerLaunchInfo {
   // The rlimits are set while launching the container.
   optional RLimitInfo rlimits = 8;
 
-  // I/O information about the container to be launched. If not set,
-  // the container will inherit the agent standard input/output/error.
-  optional ContainerIO in = 11;
-  optional ContainerIO out = 12;
-  optional ContainerIO err = 13;
-
   // (POSIX only) The slave path of the pseudo terminal.
   optional string tty_slave_path = 14;
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/48c376a5/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index d2b4f75..e99db1b 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -129,9 +129,9 @@ using mesos::modules::ModuleManager;
 
 using mesos::slave::ContainerClass;
 using mesos::slave::ContainerConfig;
-using mesos::slave::ContainerIO;
 using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerLimitation;
+using mesos::slave::ContainerLogger;
 using mesos::slave::ContainerState;
 using mesos::slave::ContainerTermination;
 using mesos::slave::Isolator;
@@ -1095,9 +1095,13 @@ Future<bool> MesosContainerizerProcess::launch(
   if (!containerConfig.has_container_info() ||
       !containerConfig.container_info().mesos().has_image()) {
     return prepare(containerId, None())
+      .then(defer(self(), [this, containerId] () {
+        return ioSwitchboard->extractContainerIO(containerId);
+      }))
       .then(defer(self(),
                   &Self::_launch,
                   containerId,
+                  lambda::_1,
                   environment,
                   slaveId,
                   checkpoint));
@@ -1111,9 +1115,13 @@ Future<bool> MesosContainerizerProcess::launch(
     .then(defer(self(),
                 [=](const ProvisionInfo& provisionInfo) -> Future<bool> {
       return prepare(containerId, provisionInfo)
+        .then(defer(self(), [this, containerId] () {
+          return ioSwitchboard->extractContainerIO(containerId);
+        }))
         .then(defer(self(),
                     &Self::_launch,
                     containerId,
+                    lambda::_1,
                     environment,
                     slaveId,
                     checkpoint));
@@ -1242,6 +1250,7 @@ Future<Nothing> MesosContainerizerProcess::fetch(
 
 Future<bool> MesosContainerizerProcess::_launch(
     const ContainerID& containerId,
+    const Option<ContainerLogger::ContainerIO>& containerIO,
     const map<string, string>& environment,
     const SlaveID& slaveId,
     bool checkpoint)
@@ -1256,6 +1265,7 @@ Future<bool> MesosContainerizerProcess::_launch(
     return Failure("Container is being destroyed during preparing");
   }
 
+  CHECK(containerIO.isSome());
   CHECK_EQ(container->state, PREPARING);
   CHECK_READY(container->launchInfos);
 
@@ -1302,21 +1312,6 @@ Future<bool> MesosContainerizerProcess::_launch(
       return Failure("Multiple isolators specify rlimits");
     }
 
-    if (isolatorLaunchInfo->has_in() &&
-        launchInfo.has_in()) {
-      return Failure("Multiple isolators specify stdin");
-    }
-
-    if (isolatorLaunchInfo->has_out() &&
-        launchInfo.has_out()) {
-      return Failure("Multiple isolators specify stdout");
-    }
-
-    if (isolatorLaunchInfo->has_err() &&
-        launchInfo.has_err()) {
-      return Failure("Multiple isolators specify stderr");
-    }
-
     if (isolatorLaunchInfo->has_tty_slave_path() &&
         launchInfo.has_tty_slave_path()) {
       return Failure("Multiple isolators specify tty");
@@ -1325,51 +1320,6 @@ Future<bool> MesosContainerizerProcess::_launch(
     launchInfo.MergeFrom(isolatorLaunchInfo.get());
   }
 
-  // Determine the I/O for the container.
-  // TODO(jieyu): Close 'fd' on error before 'launcher->fork'.
-  Option<Subprocess::IO> in;
-  Option<Subprocess::IO> out;
-  Option<Subprocess::IO> err;
-
-  if (launchInfo.has_in()) {
-    switch (launchInfo.in().type()) {
-      case ContainerIO::FD:
-        in = Subprocess::FD(launchInfo.in().fd(), Subprocess::IO::OWNED);
-        break;
-      case ContainerIO::PATH:
-        in = Subprocess::PATH(launchInfo.in().path());
-        break;
-      default:
-        break;
-    }
-  }
-
-  if (launchInfo.has_out()) {
-    switch (launchInfo.out().type()) {
-      case ContainerIO::FD:
-        out = Subprocess::FD(launchInfo.out().fd(), Subprocess::IO::OWNED);
-        break;
-      case ContainerIO::PATH:
-        out = Subprocess::PATH(launchInfo.out().path());
-        break;
-      default:
-        break;
-    }
-  }
-
-  if (launchInfo.has_err()) {
-    switch (launchInfo.err().type()) {
-      case ContainerIO::FD:
-        err = Subprocess::FD(launchInfo.err().fd(), Subprocess::IO::OWNED);
-        break;
-      case ContainerIO::PATH:
-        err = Subprocess::PATH(launchInfo.err().path());
-        break;
-      default:
-        break;
-    }
-  }
-
   // Remove duplicated entries in enter and clone namespaces.
   set<int> enterNamespaces(
       launchInfo.enter_namespaces().begin(),
@@ -1608,9 +1558,9 @@ Future<bool> MesosContainerizerProcess::_launch(
       containerId,
       argv[0],
       argv,
-      in.isSome() ? in.get() : Subprocess::FD(STDIN_FILENO),
-      out.isSome() ? out.get() : Subprocess::FD(STDOUT_FILENO),
-      err.isSome() ? err.get() : Subprocess::FD(STDERR_FILENO),
+      containerIO->in,
+      containerIO->out,
+      containerIO->err,
       nullptr,
       launchEnvironment,
       // 'enterNamespaces' will be ignored by PosixLauncher.

http://git-wip-us.apache.org/repos/asf/mesos/blob/48c376a5/src/slave/containerizer/mesos/containerizer.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.hpp b/src/slave/containerizer/mesos/containerizer.hpp
index 10a9b57..8010a08 100644
--- a/src/slave/containerizer/mesos/containerizer.hpp
+++ b/src/slave/containerizer/mesos/containerizer.hpp
@@ -235,6 +235,7 @@ private:
 
   process::Future<bool> _launch(
       const ContainerID& containerId,
+      const Option<mesos::slave::ContainerLogger::ContainerIO>& containerIO,
       const std::map<std::string, std::string>& environment,
       const SlaveID& slaveId,
       bool checkpoint);

http://git-wip-us.apache.org/repos/asf/mesos/blob/48c376a5/src/slave/containerizer/mesos/io/switchboard.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/io/switchboard.cpp b/src/slave/containerizer/mesos/io/switchboard.cpp
index c9e6f63..895f619 100644
--- a/src/slave/containerizer/mesos/io/switchboard.cpp
+++ b/src/slave/containerizer/mesos/io/switchboard.cpp
@@ -101,7 +101,6 @@ using process::network::internal::SocketImpl;
 
 using mesos::slave::ContainerConfig;
 using mesos::slave::ContainerClass;
-using mesos::slave::ContainerIO;
 using mesos::slave::ContainerLaunchInfo;
 using mesos::slave::ContainerLimitation;
 using mesos::slave::ContainerLogger;
@@ -264,6 +263,7 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::prepare(
 {
   // In local mode, the container will inherit agent's stdio.
   if (local) {
+    containerIOs[containerId] = ContainerLogger::ContainerIO();
     return None();
   }
 
@@ -305,42 +305,10 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
                 containerConfig.container_info().has_tty_info();
 
   if (!IOSwitchboard::requiresServer(containerConfig)) {
-    ContainerLaunchInfo launchInfo;
+    CHECK(!containerIOs.contains(containerId));
+    containerIOs[containerId] = loggerIO;
 
-    ContainerIO* out = launchInfo.mutable_out();
-    ContainerIO* err = launchInfo.mutable_err();
-
-    switch (loggerIO.out.type()) {
-#ifndef __WINDOWS__
-      case ContainerLogger::ContainerIO::IO::Type::FD:
-        out->set_type(ContainerIO::FD);
-        out->set_fd(loggerIO.out.fd().get());
-        break;
-#endif
-      case ContainerLogger::ContainerIO::IO::Type::PATH:
-        out->set_type(ContainerIO::PATH);
-        out->set_path(loggerIO.out.path().get());
-        break;
-      default:
-        UNREACHABLE();
-    }
-
-    switch (loggerIO.err.type()) {
-#ifndef __WINDOWS__
-      case ContainerLogger::ContainerIO::IO::Type::FD:
-        err->set_type(ContainerIO::FD);
-        err->set_fd(loggerIO.err.fd().get());
-        break;
-#endif
-      case ContainerLogger::ContainerIO::IO::Type::PATH:
-        err->set_type(ContainerIO::PATH);
-        err->set_path(loggerIO.err.path().get());
-        break;
-      default:
-        UNREACHABLE();
-    }
-
-    return launchInfo;
+    return ContainerLaunchInfo();
   }
 
 #ifndef __WINDOWS__
@@ -351,10 +319,16 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
                    " '" + stringify(containerId) + "'");
   }
 
-  // Return the set of fds that should be sent to the
-  // container and dup'd onto its stdin/stdout/stderr.
+  // We need this so we can return the
+  // `tty_slave_path` if there is one.
   ContainerLaunchInfo launchInfo;
 
+  // We assign this variable to an entry in the `containerIOs` hashmap
+  // at the bottom of this function. We declare it here so we can
+  // populate it throughout this function and only store it back to
+  // the hashmap once we know this function has succeeded.
+  ContainerLogger::ContainerIO containerIO;
+
   // Manually construct pipes instead of using `Subprocess::PIPE`
   // so that the ownership of the FDs is properly represented. The
   // `Subprocess` spawned below owns one end of each pipe and will
@@ -443,14 +417,9 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
     stdoutFromFd = master;
     stderrFromFd = master;
 
-    launchInfo.mutable_in()->set_type(ContainerIO::FD);
-    launchInfo.mutable_in()->set_fd(slave.get());
-
-    launchInfo.mutable_out()->set_type(ContainerIO::FD);
-    launchInfo.mutable_out()->set_fd(slave.get());
-
-    launchInfo.mutable_err()->set_type(ContainerIO::FD);
-    launchInfo.mutable_err()->set_fd(slave.get());
+    containerIO.in = ContainerLogger::ContainerIO::IO::FD(slave.get());
+    containerIO.out = containerIO.in;
+    containerIO.err = containerIO.in;
 
     launchInfo.set_tty_slave_path(slavePath.get());
   } else {
@@ -491,14 +460,9 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
     stdoutFromFd = outfds[0];
     stderrFromFd = errfds[0];
 
-    launchInfo.mutable_in()->set_type(ContainerIO::FD);
-    launchInfo.mutable_in()->set_fd(infds[0]);
-
-    launchInfo.mutable_out()->set_type(ContainerIO::FD);
-    launchInfo.mutable_out()->set_fd(outfds[1]);
-
-    launchInfo.mutable_err()->set_type(ContainerIO::FD);
-    launchInfo.mutable_err()->set_fd(errfds[1]);
+    containerIO.in = ContainerLogger::ContainerIO::IO::FD(infds[0]);
+    containerIO.out = ContainerLogger::ContainerIO::IO::FD(outfds[1]);
+    containerIO.err = ContainerLogger::ContainerIO::IO::FD(errfds[1]);
   }
 
   // Make sure all file descriptors opened have CLOEXEC set.
@@ -653,6 +617,9 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
         containerId,
         lambda::_1))));
 
+  // Populate the `containerIOs` hashmap.
+  containerIOs[containerId] = containerIO;
+
   return launchInfo;
 #endif // __WINDOWS__
 }
@@ -722,6 +689,29 @@ Future<http::Connection> IOSwitchboard::_connect(
 }
 
 
+Future<Option<ContainerLogger::ContainerIO>> IOSwitchboard::extractContainerIO(
+    const ContainerID& containerId)
+{
+  return dispatch(self(), [this, containerId]() {
+    return _extractContainerIO(containerId);
+  });
+}
+
+
+Future<Option<ContainerLogger::ContainerIO>> IOSwitchboard::_extractContainerIO(
+    const ContainerID& containerId)
+{
+  if (!containerIOs.contains(containerId)) {
+    return None();
+  }
+
+  ContainerLogger::ContainerIO containerIO = containerIOs[containerId];
+  containerIOs.erase(containerId);
+
+  return containerIO;
+}
+
+
 Future<ContainerLimitation> IOSwitchboard::watch(
     const ContainerID& containerId)
 {
@@ -806,6 +796,14 @@ Future<Nothing> IOSwitchboard::cleanup(
   // DISCARDED cases as well.
   return await(list<Future<Option<int>>>{status}).then(
       defer(self(), [this, containerId]() -> Future<Nothing> {
+        // We need to call `_extractContainerIO` here in case the
+        // `IOSwitchboard` still holds a reference to the container's
+        // `ContainerIO` struct. We don't care about its value at this
+        // point. We just need to extract it out of the hashmap (if
+        // it's in there) so it can drop out of scope and all open
+        // file descriptors will be closed.
+        _extractContainerIO(containerId);
+
         // We only remove the 'containerId from our info struct once
         // we are sure that the I/O switchboard has shutdown. If we
         // removed it any earlier, attempts to connect to the I/O

http://git-wip-us.apache.org/repos/asf/mesos/blob/48c376a5/src/slave/containerizer/mesos/io/switchboard.hpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/io/switchboard.hpp b/src/slave/containerizer/mesos/io/switchboard.hpp
index 7189602..83ed029 100644
--- a/src/slave/containerizer/mesos/io/switchboard.hpp
+++ b/src/slave/containerizer/mesos/io/switchboard.hpp
@@ -79,6 +79,11 @@ public:
   process::Future<process::http::Connection> connect(
       const ContainerID& containerId) const;
 
+  // Transfer ownership of a `ContainerIO` struct for a given
+  // container out of the `IOSwitchboard` and into the caller.
+  process::Future<Option<mesos::slave::ContainerLogger::ContainerIO>>
+      extractContainerIO(const ContainerID& containerID);
+
   // Helper function that returns `true` if `IOSwitchboardServer`
   // needs to be enabled for the given `ContainerConfig`. It must
   // be enabled for `DEBUG` containers and ones that need `TTYInfo`.
@@ -110,6 +115,9 @@ private:
   process::Future<process::http::Connection> _connect(
       const ContainerID& containerId) const;
 
+  process::Future<Option<mesos::slave::ContainerLogger::ContainerIO>>
+       _extractContainerIO(const ContainerID& containerID);
+
 #ifndef __WINDOWS__
   void reaped(
       const ContainerID& containerId,
@@ -120,6 +128,15 @@ private:
   bool local;
   process::Owned<mesos::slave::ContainerLogger> logger;
   hashmap<ContainerID, process::Owned<Info>> infos;
+
+  // We use a separate hashmap to hold the `ContainerIO` for each
+  // container because we need to maintain this information even in
+  // the case were we only instantiate the logger and never spawn an
+  // `IOSwitchbaordProcess`. Also, the lifetime of the `ContainerIO`
+  // is shorter lived than the `Info` struct, as it should be removed
+  // from this hash as soon as ownership is transferred out of the
+  // `IOSwitchboard` via a call to `extractContainerIO()`.
+  hashmap<ContainerID, mesos::slave::ContainerLogger::ContainerIO> containerIOs;
 };