You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by jp...@apache.org on 2018/10/16 23:30:47 UTC

[mesos] 02/02: Switched to `net::socketpair` in `ns::clone`.

This is an automated email from the ASF dual-hosted git repository.

jpeach pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mesos.git

commit 2ef75068d2583b9995cf944dab7ff0311b484f51
Author: James Peach <jp...@apache.org>
AuthorDate: Tue Oct 16 16:30:24 2018 -0700

    Switched to `net::socketpair` in `ns::clone`.
    
    Started using to `net::socketpair` in `ns::clone` so that we don't
    have to deal with `SOCK_CLOEXEC` portability.
    
    Review: https://reviews.apache.org/r/63655/
---
 src/linux/ns.cpp | 69 ++++++++++++++++++++++++++++++++------------------------
 1 file changed, 39 insertions(+), 30 deletions(-)

diff --git a/src/linux/ns.cpp b/src/linux/ns.cpp
index ffa9b65..4d16381 100644
--- a/src/linux/ns.cpp
+++ b/src/linux/ns.cpp
@@ -22,6 +22,7 @@
 #include <sys/wait.h>
 
 #include <cstring>
+#include <type_traits>
 #include <vector>
 
 #include <process/collect.hpp>
@@ -43,6 +44,7 @@
 
 #include <stout/os/exists.hpp>
 #include <stout/os/ls.hpp>
+#include <stout/os/socket.hpp>
 
 #include "common/status_utils.hpp"
 
@@ -268,6 +270,23 @@ Result<ino_t> getns(pid_t pid, const string& ns)
 }
 
 
+// Helper for closing a container of file descriptors.
+template <
+  typename Iterable,
+  typename = typename std::enable_if<
+    std::is_same<typename Iterable::value_type, int>::value>::type>
+static void close(const Iterable& fds)
+{
+  int errsav = errno;
+
+  foreach (int fd, fds) {
+    ::close(fd); // Need to call the async-signal safe version.
+  }
+
+  errno = errsav;
+}
+
+
 Try<pid_t> clone(
     pid_t target,
     int nstypes,
@@ -324,13 +343,6 @@ Try<pid_t> clone(
   // File descriptors keyed by the (parent) namespace we are entering.
   hashmap<int, int> fds = {};
 
-  // Helper for closing a list of file descriptors.
-  auto close = [](const std::vector<int>& fds) {
-    foreach (int fd, fds) {
-      ::close(fd); // Need to call the async-signal safe version.
-    }
-  };
-
   // NOTE: we do all of this ahead of time so we can be async signal
   // safe after calling fork below.
   for (size_t i = 0; i < NAMESPACES; i++) {
@@ -353,10 +365,10 @@ Try<pid_t> clone(
   // `sockets[0]` and the child socket is `sockets[1]`. Note that both
   // sockets are both read/write but currently only the parent reads
   // and the child writes.
-  int sockets[2] = {-1, -1};
-  if (socketpair(AF_UNIX, SOCK_STREAM, 0, sockets) != 0) {
+  Try<std::array<int_fd, 2>> sockets = net::socketpair(AF_UNIX, SOCK_STREAM, 0);
+  if (sockets.isError()) {
     close(fds.values());
-    return ErrnoError("Failed to create Unix domain socket");
+    return Error("Failed to create Unix domain socket: " + sockets.error());
   }
 
   // Need to set SO_PASSCRED option in order to receive credentials
@@ -365,12 +377,10 @@ Try<pid_t> clone(
   // receiving, not also for sending.
   const int value = 1;
   const socklen_t size = sizeof(value);
-  if (setsockopt(sockets[0], SOL_SOCKET, SO_PASSCRED, &value, size) == -1) {
-    Error error = ErrnoError("Failed to set socket option SO_PASSCRED");
+  if (setsockopt(sockets->at(0), SOL_SOCKET, SO_PASSCRED, &value, size) == -1) {
     close(fds.values());
-    ::close(sockets[0]);
-    ::close(sockets[1]);
-    return error;
+    close(sockets.get());
+    return ErrnoError("Failed to set socket option SO_PASSCRED");
   }
 
   // NOTE: to determine the pid of the final process executing the
@@ -424,17 +434,16 @@ Try<pid_t> clone(
   if (child < 0) {
     stack->deallocate();
     close(fds.values());
-    ::close(sockets[0]);
-    ::close(sockets[1]);
+    close(sockets.get());
     return ErrnoError();
   } else if (child > 0) {
     // Parent.
     stack->deallocate();
 
     close(fds.values());
-    ::close(sockets[1]);
+    ::close(sockets->at(1));
 
-    ssize_t length = recvmsg(sockets[0], &message, 0);
+    ssize_t length = recvmsg(sockets->at(0), &message, 0);
 
     // TODO(benh): Note that whenever we 'kill(child, SIGKILL)' below
     // we don't guarantee cleanup! It's possible that the
@@ -447,17 +456,17 @@ Try<pid_t> clone(
       // (which might die on it's own trying to write to the
       // socket).
       Error error = ErrnoError("Failed to receive");
-      ::close(sockets[0]);
+      ::close(sockets->at(0));
       kill(child, SIGKILL);
       return error;
     } else if (length == 0) {
       // Socket closed, child must have died, but kill anyway.
-      ::close(sockets[0]);
+      ::close(sockets->at(0));
       kill(child, SIGKILL);
       return Error("Failed to receive: Socket closed");
     }
 
-    ::close(sockets[0]);
+    ::close(sockets->at(0));
 
     // Extract pid.
     if (CMSG_FIRSTHDR(&message) == nullptr ||
@@ -501,7 +510,7 @@ Try<pid_t> clone(
     return pid;
   } else {
     // Child.
-    ::close(sockets[0]);
+    ::close(sockets->at(0));
 
     // Loop through and 'setns' into all of the parent namespaces that
     // have been requested.
@@ -511,7 +520,7 @@ Try<pid_t> clone(
         ASSERT(namespaces[i].nstype & nstypes);
         if (::setns(fd.get(), namespaces[i].nstype) < 0) {
           close(fds.values());
-          ::close(sockets[1]);
+          ::close(sockets->at(1));
           _exit(EXIT_FAILURE);
         }
       }
@@ -542,17 +551,17 @@ Try<pid_t> clone(
             std::memcpy(
                 CMSG_DATA(CMSG_FIRSTHDR(&message)), &cred, sizeof(ucred));
 
-            if (sendmsg(sockets[1], &message, 0) == -1) {
+            if (sendmsg(sockets->at(1), &message, 0) == -1) {
               // Failed to send the pid back to the parent!
               _exit(EXIT_FAILURE);
             }
 
-            ::close(sockets[1]);
+            ::close(sockets->at(1));
 
             return f();
           });
 
-      ::close(sockets[1]);
+      ::close(sockets->at(1));
 
       // TODO(benh): Kill ourselves with an exit status that we can
       // decode above to determine why `clone` failed.
@@ -563,7 +572,7 @@ Try<pid_t> clone(
     os::Stack grandchildStack(os::Stack::DEFAULT_SIZE);
 
     if (!grandchildStack.allocate()) {
-      ::close(sockets[1]);
+      ::close(sockets->at(1));
       _exit(EXIT_FAILURE);
     }
 
@@ -590,11 +599,11 @@ Try<pid_t> clone(
 
     if (grandchild < 0) {
       // TODO(benh): Exit with `errno` in order to capture `fork` error?
-      ::close(sockets[1]);
+      ::close(sockets->at(1));
       _exit(EXIT_FAILURE);
     } else if (grandchild > 0) {
       // Still the (first) child.
-      ::close(sockets[1]);
+      ::close(sockets->at(1));
 
       // Need to reap the grandchild and then just exit since we're no
       // longer necessary. Technically when the grandchild exits it'll