You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by bm...@apache.org on 2020/04/24 22:09:45 UTC

[mesos] branch master updated (00466fe -> 77fa450)

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

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


    from 00466fe  Enabled `LoggingTest*` and `MetricTest*` on Windows.
     new 413981a  Fixed discard race in libwinio io operations.
     new fc5014c  Added support for read readiness polling on Windows.
     new 8233369  Enabled io::poll read test on Windows / added io::poll write test.
     new 44bb777  Fixed a bug where the openssl socket can block forever.
     new 5cd3f11  Fixed a bug where OpenSSLSocketImpl accept loop can silently stop.
     new 77fa450  Added a regression test for MESOS-10114.

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 3rdparty/libprocess/include/process/io.hpp     | 15 +++--
 3rdparty/libprocess/src/io.cpp                 |  4 ++
 3rdparty/libprocess/src/io_internal.hpp        |  8 +++
 3rdparty/libprocess/src/ssl/openssl_socket.cpp | 36 +++++-------
 3rdparty/libprocess/src/ssl/openssl_socket.hpp |  3 +-
 3rdparty/libprocess/src/tests/io_tests.cpp     | 78 +++++++++++++++++++++++---
 3rdparty/libprocess/src/tests/ssl_tests.cpp    | 61 ++++++++++++++++++++
 3rdparty/libprocess/src/windows/io.cpp         | 19 ++++++-
 3rdparty/libprocess/src/windows/libwinio.cpp   |  6 +-
 9 files changed, 191 insertions(+), 39 deletions(-)


[mesos] 06/06: Added a regression test for MESOS-10114.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 77fa4508d0dd198040c5b673c5aa5e667ac7888b
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Fri Apr 10 17:17:01 2020 -0400

    Added a regression test for MESOS-10114.
    
    It turns out we already had a test for this case, but in MESOS-10114
    we need downgrade support enabled to exercise the bug.
    
    Review: https://reviews.apache.org/r/72353
---
 3rdparty/libprocess/src/tests/ssl_tests.cpp | 61 +++++++++++++++++++++++++++++
 1 file changed, 61 insertions(+)

diff --git a/3rdparty/libprocess/src/tests/ssl_tests.cpp b/3rdparty/libprocess/src/tests/ssl_tests.cpp
index 3f1d103..c90d1da 100644
--- a/3rdparty/libprocess/src/tests/ssl_tests.cpp
+++ b/3rdparty/libprocess/src/tests/ssl_tests.cpp
@@ -809,6 +809,67 @@ TEST_F(SSLTest, SilentSocket)
 }
 
 
+// A copy of the SilentSocket test to ensure that the issue
+// also is not present with downgrade support enabled. This
+// was added due to MESOS-10114.
+TEST_F(SSLTest, SilentSocketWithDowngrade)
+{
+  Try<Socket> server = setup_server({
+      {"LIBPROCESS_SSL_ENABLED", "true"},
+      {"LIBPROCESS_SSL_SUPPORT_DOWNGRADE", "true"},
+      {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
+      {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()}});
+
+  ASSERT_SOME(server);
+  ASSERT_SOME(server->address());
+
+  Try<std::string> serverHostname = server->address()->lookup_hostname();
+  ASSERT_SOME(serverHostname);
+
+  Future<Socket> socket = server->accept();
+
+  // We initiate a connection on which we will not send
+  // any data. This means the socket on the server will
+  // not complete the SSL handshake, nor be downgraded.
+  // As a result, we expect that the server will not see
+  // an accepted socket for this connection.
+  Try<Socket> connection = Socket::create(SocketImpl::Kind::POLL);
+  ASSERT_SOME(connection);
+  connection->connect(server->address().get());
+
+  // Note that settling libprocess is not sufficient
+  // for ensuring socket events are processed.
+  Clock::pause();
+  Clock::settle();
+  Clock::resume();
+
+  ASSERT_TRUE(socket.isPending());
+
+  // Now send an HTTP GET request, it should complete
+  // without getting blocked by the socket above
+  // undergoing the SSL handshake.
+  const http::URL url(
+      "https",
+      serverHostname.get(),
+      server->address()->port);
+
+  Future<http::Response> response = http::get(url);
+
+  AWAIT_READY(socket);
+
+  // Send the response from the server.
+  const string buffer = string() +
+    "HTTP/1.1 200 OK\r\n" +
+    "Content-Length: " + stringify(data.length()) + "\r\n" +
+    "\r\n" +
+    data;
+  AWAIT_READY(Socket(socket.get()).send(buffer));
+
+  AWAIT_EXPECT_RESPONSE_STATUS_EQ(http::OK().status, response);
+  EXPECT_EQ(data, response->body);
+}
+
+
 // This test was added due to an OOM issue: MESOS-7934.
 TEST_F(SSLTest, ShutdownThenSend)
 {


[mesos] 02/06: Added support for read readiness polling on Windows.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit fc5014c34f54ab6372781493d2d19aa6dc905535
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Tue Apr 21 13:37:51 2020 -0400

    Added support for read readiness polling on Windows.
    
    This uses the zero byte read trick on Windows in order to
    implement read readiness polling on Windows.
    
    The documentation for io::read has also been updated to explain
    how zero byte reads are treated consistently across POSIX and
    Windows.
    
    Write readiness polling is left unimplemented since it's less
    straightforward. A good explanation of the Windows landscape
    for polling is here:
    
    https://github.com/python-trio/trio/issues/52
    
    A library providing an epoll like interface on Windows can
    be found here:
    
    https://github.com/piscisaureus/wepoll
    
    Review: https://reviews.apache.org/r/72405/
---
 3rdparty/libprocess/include/process/io.hpp | 15 +++++++++++----
 3rdparty/libprocess/src/io.cpp             |  4 ++++
 3rdparty/libprocess/src/io_internal.hpp    |  8 ++++++++
 3rdparty/libprocess/src/windows/io.cpp     | 19 +++++++++++++++++--
 4 files changed, 40 insertions(+), 6 deletions(-)

diff --git a/3rdparty/libprocess/include/process/io.hpp b/3rdparty/libprocess/include/process/io.hpp
index 468b362..00519d4 100644
--- a/3rdparty/libprocess/include/process/io.hpp
+++ b/3rdparty/libprocess/include/process/io.hpp
@@ -26,7 +26,6 @@
 namespace process {
 namespace io {
 
-#ifndef ENABLE_LIBWINIO
 /**
  * A possible event while polling.
  *
@@ -34,6 +33,7 @@ namespace io {
  */
 const short READ = 0x01;
 
+#ifndef ENABLE_LIBWINIO
 /**
  * @copydoc process::io::READ
  */
@@ -72,17 +72,18 @@ Try<Nothing> prepare_async(int_fd fd);
 Try<bool> is_async(int_fd fd);
 
 
-#ifndef ENABLE_LIBWINIO
 /**
  * Returns the events (a subset of the events specified) that can be
  * performed on the specified file descriptor without blocking.
  *
+ * Note that on windows, only io::READ is available (under the
+ * covers this is achieved via a zero byte read).
+ *
  * @see process::io::READ
  * @see process::io::WRITE
  */
 // TODO(benh): Add a version which takes multiple file descriptors.
 Future<short> poll(int_fd fd, short events);
-#endif // ENABLE_LIBWINIO
 
 
 /**
@@ -93,7 +94,13 @@ Future<short> poll(int_fd fd, short events);
  * The future will become ready when some data is read (may be less than
  * the specified size).
  *
- * @return The number of bytes read or zero on EOF.
+ * To provide a consistent interface, a zero byte will immediately
+ * return a ready future with 0 bytes. For users looking to use
+ * the zero byte read trick on windows to achieve read readiness
+ * polling, just use io::poll with io::READ.
+ *
+ * @return The number of bytes read or zero on EOF (or if zero
+ *     bytes were requested).
  *     A failure will be returned if an error is detected.
  */
 Future<size_t> read(int_fd fd, void* data, size_t size);
diff --git a/3rdparty/libprocess/src/io.cpp b/3rdparty/libprocess/src/io.cpp
index 7288d5f..5cb275d 100644
--- a/3rdparty/libprocess/src/io.cpp
+++ b/3rdparty/libprocess/src/io.cpp
@@ -65,7 +65,11 @@ Future<size_t> read(int_fd fd, void* data, size_t size)
     return Failure("Expected an asynchronous file descriptor.");
   }
 
+#ifndef ENABLE_LIBWINIO
   return internal::read(fd, data, size);
+#else
+  return internal::read(fd, data, size, true);
+#endif // ENABLE_LIBWINIO
 }
 
 
diff --git a/3rdparty/libprocess/src/io_internal.hpp b/3rdparty/libprocess/src/io_internal.hpp
index 09bfa9a..6ad134a 100644
--- a/3rdparty/libprocess/src/io_internal.hpp
+++ b/3rdparty/libprocess/src/io_internal.hpp
@@ -21,7 +21,15 @@ namespace process {
 namespace io {
 namespace internal {
 
+#ifndef ENABLE_LIBWINIO
 Future<size_t> read(int_fd fd, void* data, size_t size);
+#else
+Future<size_t> read(
+    int_fd fd,
+    void* data,
+    size_t size,
+    bool bypassZeroByteRead);
+#endif // ENABLE_LIBWINIO
 
 Future<size_t> write(int_fd fd, const void* data, size_t size);
 
diff --git a/3rdparty/libprocess/src/windows/io.cpp b/3rdparty/libprocess/src/windows/io.cpp
index 90e780d..cca6f70 100644
--- a/3rdparty/libprocess/src/windows/io.cpp
+++ b/3rdparty/libprocess/src/windows/io.cpp
@@ -13,6 +13,7 @@
 #include <string>
 
 #include <process/future.hpp>
+#include <process/io.hpp>
 #include <process/process.hpp> // For process::initialize.
 
 #include <stout/error.hpp>
@@ -29,15 +30,29 @@
 
 namespace process {
 namespace io {
+
+
+Future<short> poll(int_fd fd, short events)
+{
+  if (events != io::READ) {
+    return Failure("Expected io::READ (" + stringify(io::READ) + ")"
+                   " but received " + stringify(events));
+  }
+
+  return io::internal::read(fd, nullptr, 0, false)
+    .then([]() { return io::READ; });
+}
+
+
 namespace internal {
 
-Future<size_t> read(int_fd fd, void* data, size_t size)
+Future<size_t> read(int_fd fd, void* data, size_t size, bool bypassZeroRead)
 {
   process::initialize();
 
   // TODO(benh): Let the system calls do what ever they're supposed to
   // rather than return 0 here?
-  if (size == 0) {
+  if (size == 0 && bypassZeroRead) {
     return 0;
   }
 


[mesos] 01/06: Fixed discard race in libwinio io operations.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 413981ad87da7c4f5f312e52a1f24f11cfe87fbd
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Tue Apr 21 12:56:38 2020 -0400

    Fixed discard race in libwinio io operations.
    
    When getting a discard request for an io operation on windows,
    a cancellation is requested and when the io operation completes
    a check is performed to see if the future had a discard requested.
    However, it's possible that the operation completed successfully
    or had a failure and we should not be dropping that information
    from the caller.
    
    To correctly check whether the cancellation succeded, we need
    to check for ERROR_OPERATION_ABORTED.
    
    Review: https://reviews.apache.org/r/72404
---
 3rdparty/libprocess/src/windows/libwinio.cpp | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/3rdparty/libprocess/src/windows/libwinio.cpp b/3rdparty/libprocess/src/windows/libwinio.cpp
index 85ee7ea..453b9df 100644
--- a/3rdparty/libprocess/src/windows/libwinio.cpp
+++ b/3rdparty/libprocess/src/windows/libwinio.cpp
@@ -141,7 +141,11 @@ static_assert(
 template <typename T>
 static void set_io_promise(Promise<T>* promise, const T& data, DWORD error)
 {
-  if (promise->future().hasDiscard()) {
+  // If our discard induced CancelIoEx call succeeded, then we
+  // will see ERROR_OPERATION_ABORTED. Otherwise, the discard
+  // lost the race against the operation completing and we
+  // should just surface the result.
+  if (promise->future().hasDiscard() && error == ERROR_OPERATION_ABORTED) {
     promise->discard();
   } else if (error == ERROR_SUCCESS) {
     promise->set(data);


[mesos] 05/06: Fixed a bug where OpenSSLSocketImpl accept loop can silently stop.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 5cd3f11733856d85605609cadf80a8a51d0bb4e0
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Fri Apr 10 17:15:27 2020 -0400

    Fixed a bug where OpenSSLSocketImpl accept loop can silently stop.
    
    The accept loop was chaining the loop body Future on the result of
    io::poll/io::read, which meant that any failed poll/read would cause
    the loop body to return a failed future and the loop to stop running.
    
    This would lead to the server socket silently no longer accepting
    incoming connections.
    
    Review: https://reviews.apache.org/r/72352
---
 3rdparty/libprocess/src/ssl/openssl_socket.cpp | 29 +++++++++++++-------------
 3rdparty/libprocess/src/ssl/openssl_socket.hpp |  3 +--
 2 files changed, 15 insertions(+), 17 deletions(-)

diff --git a/3rdparty/libprocess/src/ssl/openssl_socket.cpp b/3rdparty/libprocess/src/ssl/openssl_socket.cpp
index f03a34f..31dc4a2 100644
--- a/3rdparty/libprocess/src/ssl/openssl_socket.cpp
+++ b/3rdparty/libprocess/src/ssl/openssl_socket.cpp
@@ -559,11 +559,11 @@ Future<std::shared_ptr<SocketImpl>> OpenSSLSocketImpl::accept()
         [weak_self]() -> Future<std::shared_ptr<SocketImpl>> {
           std::shared_ptr<OpenSSLSocketImpl> self(weak_self.lock());
 
-          if (self != nullptr) {
-            return self->PollSocketImpl::accept();
+          if (self == nullptr) {
+            return Failure("Socket destructed");
           }
 
-          return Failure("Socket destructed");
+          return self->PollSocketImpl::accept();
         },
         [weak_self](const std::shared_ptr<SocketImpl>& socket)
             -> Future<ControlFlow<Nothing>> {
@@ -577,12 +577,12 @@ Future<std::shared_ptr<SocketImpl>> OpenSSLSocketImpl::accept()
           // socket to become readable. We will then MSG_PEEK it to test
           // whether we want to dispatch as SSL or non-SSL.
           if (openssl::flags().support_downgrade) {
-            return io::poll(socket->get(), process::io::READ)
-              .then([weak_self, socket]() -> Future<ControlFlow<Nothing>> {
+            io::poll(socket->get(), process::io::READ)
+              .onReady([weak_self, socket]() {
                 std::shared_ptr<OpenSSLSocketImpl> self(weak_self.lock());
 
                 if (self == nullptr) {
-                  return Break();
+                  return;
                 }
 
                 char data[6];
@@ -624,15 +624,16 @@ Future<std::shared_ptr<SocketImpl>> OpenSSLSocketImpl::accept()
                 }
 
                 if (ssl) {
-                  return self->handle_accept_callback(socket);
+                  self->handle_accept_callback(socket);
                 } else {
                   self->accept_queue.put(socket);
-                  return Continue();
                 }
               });
+          } else {
+            self->handle_accept_callback(socket);
           }
 
-          return self->handle_accept_callback(socket);
+          return Continue();
         });
 
     accept_loop_started.done();
@@ -708,7 +709,7 @@ Try<Nothing, SocketError> OpenSSLSocketImpl::shutdown(int how)
 }
 
 
-Future<ControlFlow<Nothing>> OpenSSLSocketImpl::handle_accept_callback(
+void OpenSSLSocketImpl::handle_accept_callback(
     const std::shared_ptr<SocketImpl>& socket)
 {
   // Wrap this new socket up into our SSL wrapper class by releasing
@@ -720,7 +721,7 @@ Future<ControlFlow<Nothing>> OpenSSLSocketImpl::handle_accept_callback(
   SSL* accept_ssl = SSL_new(openssl::context());
   if (accept_ssl == nullptr) {
     accept_queue.put(Failure("Accept failed, SSL_new"));
-    return Continue();
+    return;
   }
 
   Try<Address> peer_address = network::peer(ssl_socket->get());
@@ -728,7 +729,7 @@ Future<ControlFlow<Nothing>> OpenSSLSocketImpl::handle_accept_callback(
     SSL_free(accept_ssl);
     accept_queue.put(
         Failure("Failed to determine peer IP: " + peer_address.error()));
-    return Continue();
+    return;
   }
 
   // NOTE: Right now, `openssl::configure_socket` does not do anything
@@ -742,7 +743,7 @@ Future<ControlFlow<Nothing>> OpenSSLSocketImpl::handle_accept_callback(
     accept_queue.put(
         Failure("Failed to openssl::configure_socket for " +
                 stringify(*peer_address) + ": " + configured.error()));
-    return Continue();
+    return;
   }
 
   // Set the SSL context in server mode.
@@ -807,8 +808,6 @@ Future<ControlFlow<Nothing>> OpenSSLSocketImpl::handle_accept_callback(
 
       self->accept_queue.put(ssl_socket);
     });
-
-  return Continue();
 }
 
 
diff --git a/3rdparty/libprocess/src/ssl/openssl_socket.hpp b/3rdparty/libprocess/src/ssl/openssl_socket.hpp
index 0528c03..2fe0f05 100644
--- a/3rdparty/libprocess/src/ssl/openssl_socket.hpp
+++ b/3rdparty/libprocess/src/ssl/openssl_socket.hpp
@@ -58,8 +58,7 @@ protected:
   // Verifies incoming sockets and initiates the SSL handshake.
   // Upon completion or failure of the SSL handshake, the peer socket
   // (or Failure object) will be enqueued on the server socket's accept queue.
-  Future<ControlFlow<Nothing>> handle_accept_callback(
-      const std::shared_ptr<SocketImpl>& socket);
+  void handle_accept_callback(const std::shared_ptr<SocketImpl>& socket);
 
   // Takes ownership of the given SSL object and performs an SSL handshake
   // with the context of the SSL object. Either `SSL_set_connect_state`


[mesos] 03/06: Enabled io::poll read test on Windows / added io::poll write test.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8233369079a261584a9da3b93aa5c5a370c38147
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Tue Apr 21 13:38:25 2020 -0400

    Enabled io::poll read test on Windows / added io::poll write test.
    
    Now that io::poll is implemented for windows we need to test it.
    
    This also adds a test of write polling, which was absent before.
    
    Review: https://reviews.apache.org/r/72407
---
 3rdparty/libprocess/src/tests/io_tests.cpp | 78 +++++++++++++++++++++++++++---
 1 file changed, 70 insertions(+), 8 deletions(-)

diff --git a/3rdparty/libprocess/src/tests/io_tests.cpp b/3rdparty/libprocess/src/tests/io_tests.cpp
index 7e1d514..c2bb36b 100644
--- a/3rdparty/libprocess/src/tests/io_tests.cpp
+++ b/3rdparty/libprocess/src/tests/io_tests.cpp
@@ -31,6 +31,7 @@
 
 namespace io = process::io;
 
+using process::Clock;
 using process::Future;
 
 using std::array;
@@ -38,26 +39,87 @@ using std::string;
 
 class IOTest: public TemporaryDirectoryTest {};
 
-// Polling is the POSIX way to do async IO and Windows does not provide a
-// generic and scalable way to poll on arbitary `HANDLEs`, so this test is
-// removed on Windows.
+
+TEST_F(IOTest, PollRead)
+{
+  Try<std::array<int_fd, 2>> pipes = os::pipe();
+  ASSERT_SOME(pipes);
+  ASSERT_SOME(io::prepare_async((*pipes)[0]));
+  ASSERT_SOME(io::prepare_async((*pipes)[1]));
+
+  // Test discard when polling.
+  Future<short> future = io::poll((*pipes)[0], io::READ);
+  EXPECT_TRUE(future.isPending());
+  future.discard();
+  AWAIT_DISCARDED(future);
+
+  // Test successful polling.
+  future = io::poll((*pipes)[0], io::READ);
+
+  // This is not sufficient for ensuring the event loop does
+  // not detect readiness, since it's happening asynchronously
+  // in the kernel.
+  Clock::pause();
+  Clock::settle();
+  Clock::resume();
+
+  EXPECT_TRUE(future.isPending());
+
+  ASSERT_EQ(2, write((*pipes)[1], "hi", 2));
+  AWAIT_EXPECT_EQ(io::READ, future);
+
+  ASSERT_SOME(os::close((*pipes)[0]));
+  ASSERT_SOME(os::close((*pipes)[1]));
+}
+
+
+// We do not support write readiness polling on Windows
+// at the current time.
 #ifndef __WINDOWS__
-TEST_F(IOTest, Poll)
+TEST_F(IOTest, PollWrite)
 {
   int pipes[2];
   ASSERT_NE(-1, pipe(pipes));
+  ASSERT_SOME(io::prepare_async(pipes[0]));
+  ASSERT_SOME(io::prepare_async(pipes[1]));
+
+  // Fill up the pipe to test write readiness polling.
+  while (true) {
+    int result = ::write(pipes[1], "hello world!", sizeof("hello world!"));
+
+    if (result < 0) {
+      if (errno == EAGAIN) {
+        break;
+      }
+      FAIL() << "Unexpected error: " << strerror(errno);
+    }
+  }
 
   // Test discard when polling.
-  Future<short> future = io::poll(pipes[0], io::READ);
+  Future<short> future = io::poll(pipes[1], io::WRITE);
   EXPECT_TRUE(future.isPending());
   future.discard();
   AWAIT_DISCARDED(future);
 
   // Test successful polling.
-  future = io::poll(pipes[0], io::READ);
+  future = io::poll(pipes[1], io::WRITE);
+
+  // This is not sufficient for ensuring the event loop does
+  // not detect readiness, since it's happening asynchronously
+  // in the kernel.
+  Clock::pause();
+  Clock::settle();
+  Clock::resume();
+
   EXPECT_TRUE(future.isPending());
-  ASSERT_EQ(3, write(pipes[1], "hi", 3));
-  AWAIT_EXPECT_EQ(io::READ, future);
+
+  // It appears that Linux does not notify of write readiness
+  // until the reader reads at least a page of data.
+  size_t size = os::pagesize();
+  std::unique_ptr<char[]> buffer(new char[size]);
+  ASSERT_EQ(size, ::read(pipes[0], buffer.get(), size));
+
+  AWAIT_EXPECT_EQ(io::WRITE, future);
 
   ASSERT_SOME(os::close(pipes[0]));
   ASSERT_SOME(os::close(pipes[1]));


[mesos] 04/06: Fixed a bug where the openssl socket can block forever.

Posted by bm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 44bb7777450e3b585cb1e821a2fd6915528b1cac
Author: Benjamin Mahler <bm...@apache.org>
AuthorDate: Mon Apr 20 14:28:33 2020 -0400

    Fixed a bug where the openssl socket can block forever.
    
    Using the zero byte read trick against io::read is problematic
    since io::read aims to provide consistent behavior across POSIX
    and Windows by returning immediately.
    
    Now that io::poll for reads is implemented, we can fix this by
    removing the Windows specialization and just using io::poll.
    
    Review: https://reviews.apache.org/r/72406
---
 3rdparty/libprocess/src/ssl/openssl_socket.cpp | 7 -------
 1 file changed, 7 deletions(-)

diff --git a/3rdparty/libprocess/src/ssl/openssl_socket.cpp b/3rdparty/libprocess/src/ssl/openssl_socket.cpp
index 3f4dab6..f03a34f 100644
--- a/3rdparty/libprocess/src/ssl/openssl_socket.cpp
+++ b/3rdparty/libprocess/src/ssl/openssl_socket.cpp
@@ -577,14 +577,7 @@ Future<std::shared_ptr<SocketImpl>> OpenSSLSocketImpl::accept()
           // socket to become readable. We will then MSG_PEEK it to test
           // whether we want to dispatch as SSL or non-SSL.
           if (openssl::flags().support_downgrade) {
-#ifdef __WINDOWS__
-            // Since there is no `io::poll` on Windows, we instead make
-            // a 0-byte read, which will only return once there is something
-            // to read.
-            return io::read(socket->get(), nullptr, 0)
-#else
             return io::poll(socket->get(), process::io::READ)
-#endif // __WINDOWS__
               .then([weak_self, socket]() -> Future<ControlFlow<Nothing>> {
                 std::shared_ptr<OpenSSLSocketImpl> self(weak_self.lock());