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 2019/07/05 12:08:39 UTC

[mesos] 10/12: Added unit tests for hostname validation.

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

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

commit 4e21abc1910caa923c5f805bab032d25bd3cf681
Author: Benno Evers <be...@mesosphere.com>
AuthorDate: Fri May 31 16:08:42 2019 +0200

    Added unit tests for hostname validation.
    
    While going through the existing tests to look for candidates
    that would benefit from being tested for both hostname validation
    schemes, I noticed a number of existing tests where test setup
    did not quite match the comment or test name. I fixed these up
    in the same review.
    
    Review: https://reviews.apache.org/r/70797
---
 3rdparty/libprocess/include/process/ssl/gtest.hpp |  53 +++++--
 3rdparty/libprocess/src/tests/ssl_client.cpp      |  14 +-
 3rdparty/libprocess/src/tests/ssl_tests.cpp       | 172 +++++++++++++++++-----
 3 files changed, 187 insertions(+), 52 deletions(-)

diff --git a/3rdparty/libprocess/include/process/ssl/gtest.hpp b/3rdparty/libprocess/include/process/ssl/gtest.hpp
index 6cdd781..30a15f4 100644
--- a/3rdparty/libprocess/include/process/ssl/gtest.hpp
+++ b/3rdparty/libprocess/include/process/ssl/gtest.hpp
@@ -333,27 +333,31 @@ protected:
    *     SSL client subprocess with.
    * @param use_ssl_socket Whether the SSL client will try to connect
    *     using an SSL socket or a POLL socket.
+   * @param hostname The hostname to use for TLS certificate validation.
+   *     It is passed separately because some tests want to provide the
+   *     "wrong" hostname to test error conditions.
    *
    * @return Subprocess if successful otherwise an Error.
    */
   Try<process::Subprocess> launch_client(
       const std::map<std::string, std::string>& environment,
-      const process::network::inet::Socket& server,
+      const Option<std::string>& hostname,
+      const net::IP& ip,
+      uint16_t port,
       bool use_ssl_socket)
   {
-    const Try<process::network::inet::Address> address = server.address();
-    if (address.isError()) {
-      return Error(address.error());
-    }
-
     // Set up arguments to be passed to the 'client-ssl' binary.
-    const std::vector<std::string> argv = {
+    std::vector<std::string> argv = {
       "ssl-client",
       "--use_ssl=" + stringify(use_ssl_socket),
-      "--server=" + stringify(address->ip),
-      "--port=" + stringify(address->port),
+      "--server=" + stringify(ip),
+      "--port=" + stringify(port),
       "--data=" + data};
 
+    if (hostname.isSome()) {
+      argv.push_back("--server_hostname=" + hostname.get());
+    }
+
     Result<std::string> path = os::realpath(BUILD_DIR);
     if (!path.isSome()) {
       return Error("Could not establish build directory path");
@@ -375,6 +379,37 @@ protected:
         full_environment);
   }
 
+  /**
+   * Launches a test SSL client as a subprocess connecting to the
+   * server. This is a convenience overload for `launch_client()`
+   * that uses the IP address from the passed Socket as the server
+   * hostname.
+   *
+   * @param environment The SSL environment variables to launch the
+   *     SSL client subprocess with.
+   * @param use_ssl_socket Whether the SSL client will try to connect
+   *     using an SSL socket or a POLL socket.
+   *
+   * @return Subprocess if successful otherwise an Error.
+   */
+  Try<process::Subprocess> launch_client(
+      const std::map<std::string, std::string>& environment,
+      const process::network::inet::Socket& server,
+      bool use_ssl_socket)
+  {
+      const Try<process::network::inet::Address> address = server.address();
+      if (address.isError()) {
+        return Error(address.error());
+      }
+
+      return launch_client(
+          environment,
+          None(),
+          address->ip,
+          address->port,
+          use_ssl_socket);
+  }
+
   static constexpr size_t BACKLOG = 5;
 
   const std::string data;
diff --git a/3rdparty/libprocess/src/tests/ssl_client.cpp b/3rdparty/libprocess/src/tests/ssl_client.cpp
index afa0a9c..1f4470b 100644
--- a/3rdparty/libprocess/src/tests/ssl_client.cpp
+++ b/3rdparty/libprocess/src/tests/ssl_client.cpp
@@ -70,7 +70,14 @@ public:
       "The message to send as the client.",
       "Hello World");
 
-    add(&Flags::server, "server", "IP address of server", "127.0.0.1");
+    add(&Flags::server,
+      "server",
+      "IP address of server",
+      "127.0.0.1");
+
+    add(&Flags::server_hostname,
+      "server_hostname",
+      "Hostname of server");
 
     add(&Flags::port, "port", "Port of server", 5050);
   }
@@ -78,6 +85,7 @@ public:
   bool use_ssl;
   string data;
   string server;
+  Option<string> server_hostname;
   uint16_t port;
 };
 
@@ -142,7 +150,7 @@ TEST_F(SSLClientTest, client)
   Try<net::IP> ip = net::IP::parse(flags.server, AF_INET);
   EXPECT_SOME(ip);
 
-  // Connect to the server socket located at `ip:port`.
+  // Connect to the server.
   Address address(ip.get(), flags.port);
   Future<Nothing> connect = [&]() {
     switch(socket.kind()) {
@@ -151,7 +159,7 @@ TEST_F(SSLClientTest, client)
       case SocketImpl::Kind::SSL:
         return socket.connect(
             address,
-            openssl::create_tls_client_config(None()));
+            openssl::create_tls_client_config(flags.server_hostname));
     }
     UNREACHABLE();
   }();
diff --git a/3rdparty/libprocess/src/tests/ssl_tests.cpp b/3rdparty/libprocess/src/tests/ssl_tests.cpp
index e1790d5..9d5ab67 100644
--- a/3rdparty/libprocess/src/tests/ssl_tests.cpp
+++ b/3rdparty/libprocess/src/tests/ssl_tests.cpp
@@ -44,6 +44,10 @@ using std::vector;
 // We only run these tests if we have configured with '--enable-ssl'.
 #ifdef USE_SSL_SOCKET
 
+#if OPENSSL_VERSION_NUMBER >= 0x10002000L
+#define HOSTNAME_VALIDATION_OPENSSL
+#endif
+
 namespace http = process::http;
 namespace io = process::io;
 namespace network = process::network;
@@ -129,6 +133,13 @@ static const vector<string> protocols = {
 #endif
 };
 
+static const vector<string> hostname_validation_schemes = {
+  "legacy",
+#ifdef HOSTNAME_VALIDATION_OPENSSL
+  "openssl",
+#endif
+};
+
 
 // Ensure that we can't create an SSL socket when SSL is not enabled.
 TEST(SSL, Disabled)
@@ -193,26 +204,37 @@ TEST_F(SSLTest, NonSSLSocket)
 }
 
 
+class SSLTestStringParameter
+  : public SSLTest,
+    public ::testing::WithParamInterface<std::string> {};
+
+
+INSTANTIATE_TEST_CASE_P(HostnameValidationScheme,
+                        SSLTestStringParameter,
+                        ::testing::ValuesIn(hostname_validation_schemes));
+
+
 // Ensure that a certificate that was not generated using the
 // certificate authority is still allowed to communicate as long as
 // the LIBPROCESS_SSL_VERIFY_CERT and LIBPROCESS_SSL_REQUIRE_CERT
 // flags are disabled.
-TEST_F(SSLTest, NoVerifyBadCA)
+TEST_P(SSLTestStringParameter, NoVerifyBadCA)
 {
   Try<Socket> server = setup_server({
       {"LIBPROCESS_SSL_ENABLED", "true"},
       {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
       {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
-      {"LIBPROCESS_SSL_VERIFY_CERT", "false"},
-      {"LIBPROCESS_SSL_REQUIRE_CERT", "false"}});
+      {"LIBPROCESS_SSL_REQUIRE_CERT", "false"},
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", GetParam()}});
   ASSERT_SOME(server);
 
   Try<Subprocess> client = launch_client({
       {"LIBPROCESS_SSL_ENABLED", "true"},
       {"LIBPROCESS_SSL_KEY_FILE", scrap_key_path().string()},
       {"LIBPROCESS_SSL_CERT_FILE", scrap_certificate_path().string()},
-      {"LIBPROCESS_SSL_REQUIRE_CERT", "true"},
-      {"LIBPROCESS_SSL_CA_FILE", certificate_path().string()}},
+      {"LIBPROCESS_SSL_VERIFY_CERT", "false"},
+      {"LIBPROCESS_SSL_CA_FILE", certificate_path().string()},
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", GetParam()}},
       server.get(),
       true);
   ASSERT_SOME(client);
@@ -228,23 +250,29 @@ TEST_F(SSLTest, NoVerifyBadCA)
 }
 
 
-// Ensure that a certificate that was not generated using the
+// Ensure that a client certificate that was not generated using the
 // certificate authority is NOT allowed to communicate when the
 // LIBPROCESS_SSL_REQUIRE_CERT flag is enabled.
+//
+// NOTE: We cannot run this test with the 'legacy' hostname
+// validation scheme due to MESOS-9867.
+#ifdef HOSTNAME_VALIDATION_OPENSSL
 TEST_F(SSLTest, RequireBadCA)
 {
   Try<Socket> server = setup_server({
       {"LIBPROCESS_SSL_ENABLED", "true"},
       {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
       {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
-      {"LIBPROCESS_SSL_REQUIRE_CERT", "true"}});
+      {"LIBPROCESS_SSL_REQUIRE_CERT", "true"},
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", "openssl"}});
   ASSERT_SOME(server);
 
   Try<Subprocess> client = launch_client({
       {"LIBPROCESS_SSL_ENABLED", "true"},
       {"LIBPROCESS_SSL_KEY_FILE", scrap_key_path().string()},
       {"LIBPROCESS_SSL_CERT_FILE", scrap_certificate_path().string()},
-      {"LIBPROCESS_SSL_REQUIRE_CERT", "false"}},
+      {"LIBPROCESS_SSL_VERIFY_CERT", "false"},
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", "openssl"}},
       server.get(),
       true);
   ASSERT_SOME(client);
@@ -254,26 +282,37 @@ TEST_F(SSLTest, RequireBadCA)
 
   AWAIT_ASSERT_READY(await_subprocess(client.get(), None()));
 }
+#endif // HOSTNAME_VALIDATION_OPENSSL
 
 
-// Ensure that a certificate that was not generated using the
+// Ensure that a server certificate that was not generated using the
 // certificate authority is NOT allowed to communicate when the
 // LIBPROCESS_SSL_VERIFY_CERT flag is enabled.
-TEST_F(SSLTest, VerifyBadCA)
+TEST_P(SSLTestStringParameter, VerifyBadCA)
 {
   Try<Socket> server = setup_server({
       {"LIBPROCESS_SSL_ENABLED", "true"},
-      {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
-      {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
-      {"LIBPROCESS_SSL_REQUIRE_CERT", "true"}});
+      {"LIBPROCESS_SSL_KEY_FILE", scrap_key_path().string()},
+      {"LIBPROCESS_SSL_CERT_FILE", scrap_certificate_path().string()},
+      {"LIBPROCESS_SSL_REQUIRE_CERT", "false"},
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", GetParam()}});
   ASSERT_SOME(server);
 
+  Try<std::string> hostname = net::getHostname(process::address().ip);
+  ASSERT_SOME(hostname);
+
+  Try<Address> address = server->address();
+  ASSERT_SOME(address);
+
   Try<Subprocess> client = launch_client({
       {"LIBPROCESS_SSL_ENABLED", "true"},
-      {"LIBPROCESS_SSL_KEY_FILE", scrap_key_path().string()},
-      {"LIBPROCESS_SSL_CERT_FILE", scrap_certificate_path().string()},
-      {"LIBPROCESS_SSL_VERIFY_CERT", "false"}},
-      server.get(),
+      {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
+      {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
+      {"LIBPROCESS_SSL_VERIFY_CERT", "true"},
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", GetParam()}},
+      *hostname,
+      address->ip,
+      address->port,
       true);
   ASSERT_SOME(client);
 
@@ -284,68 +323,120 @@ TEST_F(SSLTest, VerifyBadCA)
 }
 
 
-// Ensure that a server that attempts to present no certificate at all
-// is NOT allowed to communicate when the LIBPROCESS_SSL_VERIFY_CERT
-// flag is enabled in the client.
-TEST_F(SSLTest, NoAnonymousCipherIfVerify)
+// Ensure that a certificate that WAS generated using the certificate
+// authority IS allowed to communicate when the
+// LIBPROCESS_SSL_VERIFY_CERT and LIBPROCESS_SSL_REQUIRE_CERT flags are
+// enabled.
+//
+// NOTE: If this test is failing for the 'legacy' scheme, subsequent
+// tests may be affected due to MESOS-9867.
+TEST_P(SSLTestStringParameter, VerifyCertificate)
 {
   Try<Socket> server = setup_server({
       {"LIBPROCESS_SSL_ENABLED", "true"},
       {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
       {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
-      // ADH stands for "Anonymous Diffie-Hellman", and is the only
-      // anonymous cipher supported by OpenSSL out of the box.
-      {"LIBPROCESS_SSL_CIPHERS", "ADH-AES256-SHA"}});
+      {"LIBPROCESS_SSL_CA_FILE", certificate_path().string()},
+      {"LIBPROCESS_SSL_REQUIRE_CERT", "true"},
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", GetParam()}});
   ASSERT_SOME(server);
 
+  Try<std::string> hostname = net::getHostname(process::address().ip);
+  ASSERT_SOME(hostname);
+
+  Try<Address> address = server->address();
+  ASSERT_SOME(address);
+
   Try<Subprocess> client = launch_client({
       {"LIBPROCESS_SSL_ENABLED", "true"},
       {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
       {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
+      {"LIBPROCESS_SSL_CA_FILE", certificate_path().string()},
       {"LIBPROCESS_SSL_VERIFY_CERT", "true"},
-      {"LIBPROCESS_SSL_CIPHERS", "ADH-AES256-SHA"}},
-      server.get(),
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", GetParam()}},
+      *hostname,
+      address->ip,
+      address->port,
       true);
   ASSERT_SOME(client);
 
   Future<Socket> socket = server->accept();
-  AWAIT_ASSERT_FAILED(socket);
+  AWAIT_ASSERT_READY(socket);
 
-  AWAIT_ASSERT_READY(await_subprocess(client.get(), None()));
+  // TODO(jmlvanre): Remove const copy.
+  AWAIT_ASSERT_EQ(data, Socket(socket.get()).recv());
+  AWAIT_ASSERT_READY(Socket(socket.get()).send(data));
+
+  AWAIT_ASSERT_READY(await_subprocess(client.get(), 0));
 }
 
 
-// Ensure that a certificate that WAS generated using the certificate
-// authority IS allowed to communicate when the
-// LIBPROCESS_SSL_VERIFY_CERT flag is enabled.
-TEST_F(SSLTest, VerifyCertificate)
+// Ensure that a server presenting a valid certificate with a not matching
+// hostname is NOT allowed to communicate when the
+// LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME flag is set to 'openssl'.
+#ifdef HOSTNAME_VALIDATION_OPENSSL
+TEST_F(SSLTest, HostnameMismatch)
 {
   Try<Socket> server = setup_server({
       {"LIBPROCESS_SSL_ENABLED", "true"},
       {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
       {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
       {"LIBPROCESS_SSL_CA_FILE", certificate_path().string()},
-      {"LIBPROCESS_SSL_VERIFY_CERT", "true"}});
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", "openssl"}});
   ASSERT_SOME(server);
 
+  Try<Address> address = server->address();
+  ASSERT_SOME(address);
+
   Try<Subprocess> client = launch_client({
       {"LIBPROCESS_SSL_ENABLED", "true"},
       {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
       {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
       {"LIBPROCESS_SSL_CA_FILE", certificate_path().string()},
-      {"LIBPROCESS_SSL_REQUIRE_CERT", "true"}},
-      server.get(),
+      {"LIBPROCESS_SSL_VERIFY_CERT", "true"},
+      {"LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", "openssl"}},
+      "invalid.example.org",
+      address->ip,
+      address->port,
       true);
   ASSERT_SOME(client);
 
   Future<Socket> socket = server->accept();
-  AWAIT_ASSERT_READY(socket);
+  AWAIT_ASSERT_FAILED(socket);
 
-  // TODO(jmlvanre): Remove const copy.
-  AWAIT_ASSERT_EQ(data, Socket(socket.get()).recv());
-  AWAIT_ASSERT_READY(Socket(socket.get()).send(data));
+  AWAIT_ASSERT_READY(await_subprocess(client.get(), None()));
+}
+#endif // HOSTNAME_VALIDATION_OPENSSL
 
-  AWAIT_ASSERT_READY(await_subprocess(client.get(), 0));
+
+// Ensure that a server that attempts to present no certificate at all
+// is NOT allowed to communicate when the LIBPROCESS_SSL_VERIFY_CERT
+// flag is enabled in the client.
+TEST_F(SSLTest, NoAnonymousCipherIfVerify)
+{
+  Try<Socket> server = setup_server({
+      {"LIBPROCESS_SSL_ENABLED", "true"},
+      {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
+      {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
+      // ADH stands for "Anonymous Diffie-Hellman", and is the only
+      // anonymous cipher supported by OpenSSL out of the box.
+      {"LIBPROCESS_SSL_CIPHERS", "ADH-AES256-SHA"}});
+  ASSERT_SOME(server);
+
+  Try<Subprocess> client = launch_client({
+      {"LIBPROCESS_SSL_ENABLED", "true"},
+      {"LIBPROCESS_SSL_KEY_FILE", key_path().string()},
+      {"LIBPROCESS_SSL_CERT_FILE", certificate_path().string()},
+      {"LIBPROCESS_SSL_VERIFY_CERT", "true"},
+      {"LIBPROCESS_SSL_CIPHERS", "ADH-AES256-SHA"}},
+      server.get(),
+      true);
+  ASSERT_SOME(client);
+
+  Future<Socket> socket = server->accept();
+  AWAIT_ASSERT_FAILED(socket);
+
+  AWAIT_ASSERT_READY(await_subprocess(client.get(), None()));
 }
 
 
@@ -777,6 +868,7 @@ TEST_P(SSLVerifyIPAddTest, BasicSameProcess)
   os::setenv("LIBPROCESS_SSL_CA_DIR", os::getcwd());
   os::setenv("LIBPROCESS_SSL_CA_FILE", certificate_path().string());
   os::setenv("LIBPROCESS_SSL_VERIFY_IPADD", GetParam());
+  os::setenv("LIBPROCESS_SSL_HOSTNAME_VALIDATION_SCHEME", "legacy");
 
   openssl::reinitialize();