You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ji...@apache.org on 2016/12/05 05:42:40 UTC

[8/9] mesos git commit: Supported TTY in I/O switchboard.

Supported TTY in I/O switchboard.

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


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

Branch: refs/heads/master
Commit: fd142a0d7b7ec27fdcffc3d679db46edc9432de3
Parents: c7f953a
Author: Jie Yu <yu...@gmail.com>
Authored: Thu Dec 1 18:16:06 2016 -0800
Committer: Jie Yu <yu...@gmail.com>
Committed: Sun Dec 4 21:41:51 2016 -0800

----------------------------------------------------------------------
 include/mesos/slave/containerizer.proto         |   3 +
 src/slave/containerizer/mesos/containerizer.cpp |   5 +
 .../containerizer/mesos/io/switchboard.cpp      | 280 ++++++++++++++-----
 .../containerizer/mesos/io/switchboard.hpp      |  31 +-
 .../containerizer/mesos/io/switchboard_main.cpp |   1 +
 src/slave/containerizer/mesos/launch.cpp        |  10 +
 .../containerizer/io_switchboard_tests.cpp      |   2 +
 7 files changed, 251 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/fd142a0d/include/mesos/slave/containerizer.proto
----------------------------------------------------------------------
diff --git a/include/mesos/slave/containerizer.proto b/include/mesos/slave/containerizer.proto
index 33b4c23..c70d437 100644
--- a/include/mesos/slave/containerizer.proto
+++ b/include/mesos/slave/containerizer.proto
@@ -212,6 +212,9 @@ message ContainerLaunchInfo {
   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/fd142a0d/src/slave/containerizer/mesos/containerizer.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/containerizer.cpp b/src/slave/containerizer/mesos/containerizer.cpp
index a7e2665..13cf757 100644
--- a/src/slave/containerizer/mesos/containerizer.cpp
+++ b/src/slave/containerizer/mesos/containerizer.cpp
@@ -1272,6 +1272,11 @@ Future<bool> MesosContainerizerProcess::_launch(
       return Failure("Multiple isolators specify stderr");
     }
 
+    if (isolatorLaunchInfo->has_tty_slave_path() &&
+        launchInfo.has_tty_slave_path()) {
+      return Failure("Multiple isolators specify tty");
+    }
+
     launchInfo.MergeFrom(isolatorLaunchInfo.get());
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd142a0d/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 2362f1b..d5211b9 100644
--- a/src/slave/containerizer/mesos/io/switchboard.cpp
+++ b/src/slave/containerizer/mesos/io/switchboard.cpp
@@ -14,6 +14,9 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+#include <stdio.h>
+#include <stdlib.h>
+
 #include <list>
 #include <map>
 #include <string>
@@ -38,6 +41,10 @@
 #include <stout/path.hpp>
 #include <stout/recordio.hpp>
 
+#ifndef __WINDOWS__
+#include <stout/posix/os.hpp>
+#endif // __WINDOWS__
+
 #include <mesos/http.hpp>
 #include <mesos/type_utils.hpp>
 
@@ -64,6 +71,7 @@ namespace unix = process::network::unix;
 
 using std::string;
 
+using process::ErrnoFailure;
 using process::Failure;
 using process::Future;
 using process::Owned;
@@ -170,11 +178,13 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
   }
 #endif
 
+  bool hasTTY = containerConfig.has_container_info() &&
+                containerConfig.container_info().has_tty_info();
+
   if (!flags.io_switchboard_enable_server) {
     // TTY support requires I/O switchboard server so that stdio can
     // be properly redirected to logger.
-    if (containerConfig.has_container_info() &&
-        containerConfig.container_info().has_tty_info()) {
+    if (hasTTY) {
       return Failure("TTY support requires I/O switchboard server");
     }
 
@@ -220,78 +230,175 @@ 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.
+  ContainerLaunchInfo launchInfo;
+
   // 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
   // be solely responsible for closing that end. The ownership of
   // the other end will be passed to the caller of this function
   // and eventually passed to the container being launched.
-  int infds[2];
-  int outfds[2];
-  int errfds[2];
+  int stdinToFd = -1;
+  int stdoutFromFd = -1;
+  int stderrFromFd = -1;
 
   // A list of file decriptors we've opened so far.
-  vector<int> fds = {};
+  hashset<int> openedFds = {};
+
+  // A list of file descriptors that will be passed to the I/O
+  // switchboard. We need to close those file descriptors once the
+  // I/O switchboard server is forked.
+  hashset<int> ioSwitchboardFds = {};
 
-  // Helper for closing the list of file
-  // descriptors we've opened so far.
-  auto close = [](const vector<int>& fds) {
+  // Helper for closing a set of file descriptors.
+  auto close = [](const hashset<int>& fds) {
     foreach (int fd, fds) {
       os::close(fd);
     }
   };
 
-  Try<Nothing> pipe = os::pipe(infds);
-  if (pipe.isError()) {
-    close(fds);
-    return Failure("Failed to create stdin pipe: " + pipe.error());
-  }
+  // Setup a pseudo terminal for the container.
+  if (hasTTY) {
+    // TODO(jieyu): Consider moving all TTY related method to stout.
+    // For instance, 'stout/posix/tty.hpp'.
 
-  fds.push_back(infds[0]);
-  fds.push_back(infds[1]);
+    // Set flag 'O_NOCTTY' so that the terminal device will not become
+    // the controlling terminal for the process.
+    int master = posix_openpt(O_RDWR | O_NOCTTY | O_CLOEXEC);
+    if (master == -1) {
+      return Failure("Failed to open a master pseudo terminal");
+    }
 
-  pipe = os::pipe(outfds);
-  if (pipe.isError()) {
-    close(fds);
-    return Failure("Failed to create stdout pipe: " + pipe.error());
-  }
+    openedFds.insert(master);
 
-  fds.push_back(outfds[0]);
-  fds.push_back(outfds[1]);
+    Try<string> slavePath = os::ptsname(master);
+    if (slavePath.isError()) {
+      close(openedFds);
+      return Failure("Failed to get the slave pseudo terminal path: " +
+                     slavePath.error());
+    }
 
-  pipe = os::pipe(errfds);
-  if (pipe.isError()) {
-    close(fds);
-    return Failure("Failed to create stderr pipe: " + pipe.error());
-  }
+    // Unlock the slave end of the pseudo terminal.
+    if (unlockpt(master) != 0) {
+      close(openedFds);
+      return ErrnoFailure("Failed to unlock the slave pseudo terminal");
+    }
+
+    // Set proper permission and ownership for the device.
+    if (grantpt(master) != 0) {
+      close(openedFds);
+      return ErrnoFailure("Failed to grant the slave pseudo terminal");
+    }
 
-  fds.push_back(errfds[0]);
-  fds.push_back(errfds[1]);
+    if (containerConfig.has_user()) {
+      Try<Nothing> chown = os::chown(
+          containerConfig.user(),
+          slavePath.get(),
+          false);
 
-  Try<Nothing> cloexec = os::cloexec(infds[0]);
-  if (cloexec.isError()) {
-    close(fds);
-    return Failure("Failed to cloexec infds.read: " + cloexec.error());
-  }
+      if (chown.isError()) {
+        close(openedFds);
+        return Failure("Failed to chown the slave pseudo terminal: " +
+                       chown.error());
+      }
+    }
+
+    // Open the slave end of the pseudo terminal. The opened file
+    // descriptor will be dup'ed to stdin/out/err of the container.
+    Try<int> slave = os::open(slavePath.get(), O_RDWR | O_NOCTTY | O_CLOEXEC);
+    if (slave.isError()) {
+      return Failure("Failed to open the slave pseudo terminal: " +
+                     slave.error());
+    }
 
-  cloexec = os::cloexec(outfds[1]);
-  if (cloexec.isError()) {
-    close(fds);
-    return Failure("Failed to cloexec outfds.write: " + cloexec.error());
+    openedFds.insert(slave.get());
+
+    LOG(INFO) << "Allocated pseudo terminal '" << slavePath.get()
+              << "' for container " << containerId;
+
+    stdinToFd = master;
+    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());
+
+    launchInfo.set_tty_slave_path(slavePath.get());
+  } else {
+    int infds[2];
+    int outfds[2];
+    int errfds[2];
+
+    Try<Nothing> pipe = os::pipe(infds);
+    if (pipe.isError()) {
+      close(openedFds);
+      return Failure("Failed to create stdin pipe: " + pipe.error());
+    }
+
+    openedFds.insert(infds[0]);
+    openedFds.insert(infds[1]);
+
+    pipe = os::pipe(outfds);
+    if (pipe.isError()) {
+      close(openedFds);
+      return Failure("Failed to create stdout pipe: " + pipe.error());
+    }
+
+    openedFds.insert(outfds[0]);
+    openedFds.insert(outfds[1]);
+
+    pipe = os::pipe(errfds);
+    if (pipe.isError()) {
+      close(openedFds);
+      return Failure("Failed to create stderr pipe: " + pipe.error());
+    }
+
+    openedFds.insert(errfds[0]);
+    openedFds.insert(errfds[1]);
+
+    stdinToFd = infds[1];
+    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]);
   }
 
-  cloexec = os::cloexec(errfds[1]);
-  if (cloexec.isError()) {
-    close(fds);
-    return Failure("Failed to cloexec errfds.write: " + cloexec.error());
+  // Make sure all file descriptors opened have CLOEXEC set.
+  foreach (int fd, openedFds) {
+    Try<Nothing> cloexec = os::cloexec(fd);
+    if (cloexec.isError()) {
+      close(openedFds);
+      return Failure("Failed to set cloexec: " + cloexec.error());
+    }
   }
 
+  ioSwitchboardFds.insert(stdinToFd);
+  ioSwitchboardFds.insert(stdoutFromFd);
+  ioSwitchboardFds.insert(stderrFromFd);
+
   // Set up our flags to send to the io switchboard server process.
   IOSwitchboardServerFlags switchboardFlags;
-  switchboardFlags.stdin_to_fd = infds[1];
-  switchboardFlags.stdout_from_fd = outfds[0];
+  switchboardFlags.tty = hasTTY;
+
+  // We use the default values for other file descriptor flags. Since
+  // I/O switchboard server's stdout and stderr will be redirected to
+  // the logger, we explicitly set the flags here.
   switchboardFlags.stdout_to_fd = STDOUT_FILENO;
-  switchboardFlags.stderr_from_fd = errfds[0];
   switchboardFlags.stderr_to_fd = STDERR_FILENO;
 
   if (containerConfig.container_class() == ContainerClass::DEBUG) {
@@ -321,17 +428,30 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
       map<string, string>(),
       None(),
       {},
-      {Subprocess::ChildHook::SETSID()});
+      {Subprocess::ChildHook::SETSID(),
+       Subprocess::ChildHook::DUP2(
+           stdinToFd,
+           IOSwitchboardServer::STDIN_TO_FD),
+       Subprocess::ChildHook::DUP2(
+           stdoutFromFd,
+           IOSwitchboardServer::STDOUT_FROM_FD),
+       Subprocess::ChildHook::DUP2(
+           stderrFromFd,
+           IOSwitchboardServer::STDERR_FROM_FD)});
 
   if (child.isError()) {
-    close(fds);
+    close(openedFds);
     return Failure("Failed to create io switchboard"
                    " server process: " + child.error());
   }
 
-  os::close(infds[1]);
-  os::close(outfds[0]);
-  os::close(errfds[0]);
+  close(ioSwitchboardFds);
+
+  // We remove the already closed file descriptors from 'openedFds' so
+  // that we don't close multiple times if failures happen below.
+  foreach (int fd, ioSwitchboardFds) {
+    openedFds.erase(fd);
+  }
 
   // Now that the child has come up, we checkpoint the socket
   // address we told it to bind to so we can access it later.
@@ -343,7 +463,7 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
       path, switchboardFlags.socket_path);
 
   if (checkpointed.isError()) {
-    close(fds);
+    close(openedFds);
     return Failure("Failed to checkpoint container's socket path to"
                    " '" + path + "': " + checkpointed.error());
   }
@@ -353,19 +473,6 @@ Future<Option<ContainerLaunchInfo>> IOSwitchboard::_prepare(
     child->pid(),
     process::reap(child->pid())));
 
-  // Return the set of fds that should be sent to the
-  // container and dup'd onto its stdin/stdout/stderr.
-  ContainerLaunchInfo launchInfo;
-
-  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]);
-
   return launchInfo;
 #endif // __WINDOWS__
 }
@@ -435,13 +542,19 @@ Future<Nothing> IOSwitchboard::cleanup(
 
 
 #ifndef __WINDOWS__
-constexpr char IOSwitchboardServer::NAME[];
+const char IOSwitchboardServer::NAME[]          = "mesos-io-switchboard";
+const int  IOSwitchboardServer::STDIN_TO_FD     = STDERR_FILENO + 1;
+const int  IOSwitchboardServer::STDOUT_FROM_FD  = STDERR_FILENO + 2;
+const int  IOSwitchboardServer::STDERR_FROM_FD  = STDERR_FILENO + 3;
+const int  IOSwitchboardServer::STDOUT_TO_FD    = STDERR_FILENO + 4;
+const int  IOSwitchboardServer::STDERR_TO_FD    = STDERR_FILENO + 5;
 
 
 class IOSwitchboardServerProcess : public Process<IOSwitchboardServerProcess>
 {
 public:
   IOSwitchboardServerProcess(
+      bool _tty,
       int _stdinToFd,
       int _stdoutFromFd,
       int _stdoutToFd,
@@ -507,6 +620,7 @@ private:
       const string& data,
       const agent::ProcessIO::Data::Type& type);
 
+  bool tty;
   int stdinToFd;
   int stdoutFromFd;
   int stdoutToFd;
@@ -524,6 +638,7 @@ private:
 
 
 Try<Owned<IOSwitchboardServer>> IOSwitchboardServer::create(
+    bool tty,
     int stdinToFd,
     int stdoutFromFd,
     int stdoutToFd,
@@ -556,6 +671,7 @@ Try<Owned<IOSwitchboardServer>> IOSwitchboardServer::create(
   }
 
   return new IOSwitchboardServer(
+      tty,
       stdinToFd,
       stdoutFromFd,
       stdoutToFd,
@@ -567,6 +683,7 @@ Try<Owned<IOSwitchboardServer>> IOSwitchboardServer::create(
 
 
 IOSwitchboardServer::IOSwitchboardServer(
+    bool tty,
     int stdinToFd,
     int stdoutFromFd,
     int stdoutToFd,
@@ -575,6 +692,7 @@ IOSwitchboardServer::IOSwitchboardServer(
     const unix::Socket& socket,
     bool waitForConnection)
   : process(new IOSwitchboardServerProcess(
+        tty,
         stdinToFd,
         stdoutFromFd,
         stdoutToFd,
@@ -601,6 +719,7 @@ Future<Nothing> IOSwitchboardServer::run()
 
 
 IOSwitchboardServerProcess::IOSwitchboardServerProcess(
+    bool _tty,
     int _stdinToFd,
     int _stdoutFromFd,
     int _stdoutToFd,
@@ -608,7 +727,8 @@ IOSwitchboardServerProcess::IOSwitchboardServerProcess(
     int _stderrToFd,
     const unix::Socket& _socket,
     bool _waitForConnection)
-  : stdinToFd(_stdinToFd),
+  : tty(_tty),
+    stdinToFd(_stdinToFd),
     stdoutFromFd(_stdoutFromFd),
     stdoutToFd(_stdoutToFd),
     stderrFromFd(_stderrFromFd),
@@ -638,14 +758,24 @@ Future<Nothing> IOSwitchboardServerProcess::run()
                  lambda::_1,
                  agent::ProcessIO::Data::STDOUT)});
 
-      Future<Nothing> stderrRedirect = process::io::redirect(
-          stderrFromFd,
-          stderrToFd,
-          4096,
-          {defer(self(),
-                 &Self::outputHook,
-                 lambda::_1,
-                 agent::ProcessIO::Data::STDERR)});
+      // NOTE: We don't need to redirect stderr if TTY is enabled. If
+      // TTY is enabled for the container, stdout and stderr for the
+      // container will be redirected to the slave end of the pseudo
+      // terminal device. Both stdout and stderr of the container will
+      // both coming out from the master end of the pseudo terminal.
+      Future<Nothing> stderrRedirect;
+      if (tty) {
+        stderrRedirect = Nothing();
+      } else {
+        stderrRedirect = process::io::redirect(
+            stderrFromFd,
+            stderrToFd,
+            4096,
+            {defer(self(),
+                   &Self::outputHook,
+                   lambda::_1,
+                   agent::ProcessIO::Data::STDERR)});
+      }
 
       // Set the future once our IO redirects finish. On failure,
       // fail the future.

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd142a0d/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 23c66bb..839665a 100644
--- a/src/slave/containerizer/mesos/io/switchboard.hpp
+++ b/src/slave/containerizer/mesos/io/switchboard.hpp
@@ -113,9 +113,17 @@ class IOSwitchboardServerProcess;
 class IOSwitchboardServer
 {
 public:
-  static constexpr char NAME[] = "mesos-io-switchboard";
+  static const char NAME[];
+
+  // Constant FD numbers used by I/O switchboard server.
+  static const int STDIN_TO_FD;
+  static const int STDOUT_FROM_FD;
+  static const int STDERR_FROM_FD;
+  static const int STDOUT_TO_FD;
+  static const int STDERR_TO_FD;
 
   static Try<process::Owned<IOSwitchboardServer>> create(
+      bool tty,
       int stdinToFd,
       int stdoutFromFd,
       int stdoutToFd,
@@ -130,6 +138,7 @@ public:
 
 private:
   IOSwitchboardServer(
+      bool tty,
       int stdinToFd,
       int stdoutFromFd,
       int stdoutToFd,
@@ -164,27 +173,36 @@ struct IOSwitchboardServerFlags : public virtual flags::FlagsBase
       "well as launch arbitrary subcommands inside a container and attach to\n"
       "its stdin/stdout/stderr.\n");
 
+    add(&IOSwitchboardServerFlags::tty,
+        "tty",
+        "If a pseudo terminal has been allocated for the container.");
+
     add(&IOSwitchboardServerFlags::stdin_to_fd,
         "stdin_to_fd",
-        "The file descriptor where incoming stdin data should be written.");
+        "The file descriptor where incoming stdin data should be written.",
+        IOSwitchboardServer::STDIN_TO_FD);
 
     add(&IOSwitchboardServerFlags::stdout_from_fd,
         "stdout_from_fd",
-        "The file descriptor that should be read to consume stdout data.");
+        "The file descriptor that should be read to consume stdout data.",
+        IOSwitchboardServer::STDOUT_FROM_FD);
 
     add(&IOSwitchboardServerFlags::stdout_to_fd,
         "stdout_to_fd",
         "A file descriptor where data read from\n"
-        "'stdout_from_fd' should be redirected to.");
+        "'stdout_from_fd' should be redirected to.",
+        IOSwitchboardServer::STDOUT_TO_FD);
 
     add(&IOSwitchboardServerFlags::stderr_from_fd,
         "stderr_from_fd",
-        "The file descriptor that should be read to consume stderr data.");
+        "The file descriptor that should be read to consume stderr data.",
+        IOSwitchboardServer::STDERR_FROM_FD);
 
     add(&IOSwitchboardServerFlags::stderr_to_fd,
         "stderr_to_fd",
         "A file descriptor where data read from\n"
-        "'stderr_from_fd' should be redirected to.");
+        "'stderr_from_fd' should be redirected to.",
+        IOSwitchboardServer::STDERR_TO_FD);
 
     add(&IOSwitchboardServerFlags::wait_for_connection,
         "wait_for_connection",
@@ -197,6 +215,7 @@ struct IOSwitchboardServerFlags : public virtual flags::FlagsBase
         "io switchboard should attach itself to.");
   }
 
+  bool tty;
   int stdin_to_fd;
   int stdout_from_fd;
   int stdout_to_fd;

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd142a0d/src/slave/containerizer/mesos/io/switchboard_main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/io/switchboard_main.cpp b/src/slave/containerizer/mesos/io/switchboard_main.cpp
index 5800217..aff6c21 100644
--- a/src/slave/containerizer/mesos/io/switchboard_main.cpp
+++ b/src/slave/containerizer/mesos/io/switchboard_main.cpp
@@ -38,6 +38,7 @@ int main(int argc, char** argv)
   }
 
   Try<Owned<IOSwitchboardServer>> server = IOSwitchboardServer::create(
+      flags.tty,
       flags.stdin_to_fd,
       flags.stdout_from_fd,
       flags.stdout_to_fd,

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd142a0d/src/slave/containerizer/mesos/launch.cpp
----------------------------------------------------------------------
diff --git a/src/slave/containerizer/mesos/launch.cpp b/src/slave/containerizer/mesos/launch.cpp
index d78ca4d..f90fce2 100644
--- a/src/slave/containerizer/mesos/launch.cpp
+++ b/src/slave/containerizer/mesos/launch.cpp
@@ -356,6 +356,16 @@ int MesosContainerizerLaunch::execute()
     }
   }
 
+#ifndef __WINDOWS__
+  if (launchInfo.has_tty_slave_path()) {
+    Try<Nothing> setctty = os::setctty(STDIN_FILENO);
+    if (setctty.isError()) {
+      cerr << "Failed to set control tty: " << setctty.error() << endl;
+      exitWithStatus(EXIT_FAILURE);
+    }
+  }
+#endif // __WINDOWS__
+
 #ifdef __linux__
   if (flags.namespace_mnt_target.isSome()) {
     string path = path::join(

http://git-wip-us.apache.org/repos/asf/mesos/blob/fd142a0d/src/tests/containerizer/io_switchboard_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/containerizer/io_switchboard_tests.cpp b/src/tests/containerizer/io_switchboard_tests.cpp
index d82f22b..d2db61e 100644
--- a/src/tests/containerizer/io_switchboard_tests.cpp
+++ b/src/tests/containerizer/io_switchboard_tests.cpp
@@ -96,6 +96,7 @@ TEST_F(IOSwitchboardTest, ServerRedirectLog)
       "mesos-io-switchboard-" + UUID::random().toString());
 
   Try<Owned<IOSwitchboardServer>> server = IOSwitchboardServer::create(
+      false,
       nullFd.get(),
       stdoutPipe[0],
       stdoutFd.get(),
@@ -203,6 +204,7 @@ TEST_F(IOSwitchboardTest, ServerAttachOutput)
       "mesos-io-switchboard-" + UUID::random().toString());
 
   Try<Owned<IOSwitchboardServer>> server = IOSwitchboardServer::create(
+      false,
       nullFd.get(),
       stdoutFd.get(),
       nullFd.get(),