You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/01/25 13:30:10 UTC

[GitHub] [nifi-minifi-cpp] martinzink opened a new pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

martinzink opened a new pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978


   Currently org::apache::nifi:minifi::io::TLSSocket
   (which, among other things, handles communications with C2)
   only accept TLSv1.2 protocol (hardcoded behaviour)
   
   Added integration tests to verify this behaviour
     TLSClientSocketSupportedProtocolsTest
       creates simple TLS enabled test servers with various protocols
       org::apache::nifi::minifi::io::TLSSocket only connects
       to the TLSv1.2 enabled server
   
     TLSServerSocketSupportedProtocolsTest
       creates an org::apache::nifi::minifi::io::TLSServerSocket
       various test clients try to connect to it,
       but only the TLSv1.2 enabled client succeeds
   
   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI results for build issues and submit an update to your PR as soon as possible.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#issuecomment-766835726


   And thanks for your first contribution. The general workflow is that we merge after every comment is resolved, there are 2-3 approvals on the pull request, no conflicts (rebase as needed) and no related CI issues.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564478408



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;
+  }
+
+ protected:
+  void configureSecurity() {
+    host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+    port_ = "3684";
+    if (!key_dir.empty()) {
+      configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+      configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir + "cn.crt.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir + "cn.ckey.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir + "cn.pass");
+      configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir + "nifi-cert.pem");
+      configuration_->set(minifi::Configure::nifi_default_directory, key_dir);
+    }
+  }
+
+  void runAssertions() {

Review comment:
       Agree, renamed and refactored it to be more readable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564388566



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)

Review comment:
       When storing them, pass-by-value and move is better IMO, but pass-by-ref-to-const is fine, too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564474474



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)

Review comment:
       fixed by third commit in PR




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r563733304



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;
+  }
+
+ protected:
+  void configureSecurity() {
+    host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+    port_ = "3684";
+    if (!key_dir.empty()) {
+      configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+      configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir + "cn.crt.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir + "cn.ckey.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir + "cn.pass");
+      configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir + "nifi-cert.pem");
+      configuration_->set(minifi::Configure::nifi_default_directory, key_dir);
+    }
+  }
+
+  void runAssertions() {
+    {
+      SimpleSSLTestServerTLSv1 server(port_, key_dir);
+      server.waitForConnection();
+
+      std::shared_ptr<org::apache::nifi::minifi::io::TLSContext> socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      client_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSSocket>(socket_context, host_, std::stoi(port_), 0);
+      assert(client_socket_->initialize() != 0);
+      shutdown(server.sock_, SHUT_RD);
+      close(server.sock_);
+      server.isRunning_ = false;
+      server.server_read_thread_.join();
+    }
+    {
+      SimpleSSLTestServerTLSv1_1 server(port_, key_dir);
+      server.waitForConnection();
+
+      std::shared_ptr<org::apache::nifi::minifi::io::TLSContext> socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);

Review comment:
       [Use auto to avoid redundant repetition of type names](https://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#es11-use-auto-to-avoid-redundant-repetition-of-type-names) (multiple occurences thoughout the PR, but I don't want to spam the same comment)
   
   ```suggestion
         const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
   ```

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);

Review comment:
       Is this intentionally unix-only? On Windows, the socket type is `SOCKET`, and other parts of the networking code of the test are unix-only as well.

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,217 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, std::string host, std::string port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str());
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+    close(sfd_);
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    bool successfulConnection = (status == 1);

Review comment:
       I would make `successfulConnection` `const` as well, since it's just a derived value from `status`.
   
   To be honest, I would make as many things `const` as reasonably possible, but this is not common practice in most contributions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r566678512



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_info("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_info("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::cout << "oh write!" << std::endl;
+        bytes_written->reserve(20);
+        memset(bytes_written->data(), 0x00, 20);
+        memcpy(bytes_written->data(), "hello world", 11);
+        *size = 20;
+        return *size;

Review comment:
       Thanks, looks way more readable. I included this with the other suggested changes in the latest commit.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r567732532



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,310 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::function<bool()> check_;
+    std::function<int(std::vector<uint8_t> *bytes_written, int *size)> handler_;
+    std::atomic<bool> is_running_;
+    std::shared_ptr<org::apache::nifi::minifi::io::TLSServerSocket> server_socket_;

Review comment:
       I see your point, you are right, an explicit reset call in the destructor before the destruction of the members seems the best way. fixed 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r566676706



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,213 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), had_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            had_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hadConnection() const {
+    return had_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;

Review comment:
       fixed it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm closed pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm closed pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r566675893



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_info("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_info("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());

Review comment:
       agreed, changed it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r567727415



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,310 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::function<bool()> check_;
+    std::function<int(std::vector<uint8_t> *bytes_written, int *size)> handler_;
+    std::atomic<bool> is_running_;
+    std::shared_ptr<org::apache::nifi::minifi::io::TLSServerSocket> server_socket_;

Review comment:
       well if in the future somebody switches up `is_running_` and `server_socket_`, `is_running_` will get destroyed first, and then the still running `server_socket_` could accidentally access the then destroyed `is_running_` (and setting `is_running_` in the destructor does not seem to mitigate this), but you are right that it is currently not an issue, maybe a comment then, that the order of these fields is important?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r566675746



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,213 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), had_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            had_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hadConnection() const {
+    return had_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+  SocketDescriptor socket_descriptor_;
+  bool had_connection_;
+  std::thread server_read_thread_;
+
+  void configureContext(const std::string& path) {
+    SSL_CTX_set_ecdh_auto(ctx_, 1);
+    /* Set the key and cert */
+    assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+    assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);

Review comment:
       fixed

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;

Review comment:
       fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r565260645



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,233 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str());
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port) {
+    struct hostent *host;
+    assert((host = gethostbyname(host_name)) != nullptr);

Review comment:
       Is this a leftover? The result of this name resolution doesn't seem to be used later aside from the assertion.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r567674682



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,310 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {

Review comment:
       I would argue that the templates forwarding to this template are unnecessary and we could call this one directly from `verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2 `




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564368411



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;

Review comment:
       Use snake_case for variables. Same in TLSServerSocketSupportedProtocolsTest.cpp 

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)

Review comment:
       Use const& for std::string parameters. This should apply for all functions below.

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;

Review comment:
       use "_" suffix for members to distinguish from local variables, you can avoid the `this` keyword that way. Same inTLSServerSocketSupportedProtocolsTest.cpp 

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;
+  }
+
+ protected:
+  void configureSecurity() {
+    host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+    port_ = "3684";
+    if (!key_dir.empty()) {
+      configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+      configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir + "cn.crt.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir + "cn.ckey.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir + "cn.pass");
+      configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir + "nifi-cert.pem");
+      configuration_->set(minifi::Configure::nifi_default_directory, key_dir);
+    }
+  }
+
+  void runAssertions() {
+    {
+      SimpleSSLTestServerTLSv1 server(port_, key_dir);
+      server.waitForConnection();
+
+      std::shared_ptr<org::apache::nifi::minifi::io::TLSContext> socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      client_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSSocket>(socket_context, host_, std::stoi(port_), 0);
+      assert(client_socket_->initialize() != 0);
+      shutdown(server.sock_, SHUT_RD);
+      close(server.sock_);
+      server.isRunning_ = false;
+      server.server_read_thread_.join();
+    }
+    {
+      SimpleSSLTestServerTLSv1_1 server(port_, key_dir);
+      server.waitForConnection();
+
+      std::shared_ptr<org::apache::nifi::minifi::io::TLSContext> socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      client_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSSocket>(socket_context, host_, std::stoi(port_), 0);
+      assert(client_socket_->initialize() != 0);
+      shutdown(server.sock_, SHUT_RD);
+      close(server.sock_);
+      server.isRunning_ = false;
+      server.server_read_thread_.join();
+    }
+    {
+      SimpleSSLTestServerTLSv1_2 server(port_, key_dir);
+      server.waitForConnection();
+
+      std::shared_ptr<org::apache::nifi::minifi::io::TLSContext> socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      client_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSSocket>(socket_context, host_, std::stoi(port_), 0);
+      assert(client_socket_->initialize() == 0);
+      shutdown(server.sock_, SHUT_RD);
+      close(server.sock_);
+      server.isRunning_ = false;
+      server.server_read_thread_.join();
+    }
+  }
+
+ protected:
+    std::shared_ptr<org::apache::nifi::minifi::io::TLSSocket> client_socket_;
+    std::string host_;
+    std::string port_;
+    std::string key_dir;
+    std::shared_ptr<minifi::Configure> configuration_;
+};
+
+static void sigpipe_handle(int /*x*/) {

Review comment:
       I don't think `/*x*/` is needed here as that name does not have additional info about the parameter. Either remove or rename it. Same in TLSServerSocketSupportedProtocolsTest.cpp 

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;

Review comment:
       Use a more readable name, also this declaration can be moved to the place of initialization.

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,217 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, std::string host, std::string port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str());
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+    close(sfd_);
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    bool successfulConnection = (status == 1);
+    return successfulConnection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  int sfd_;
+  std::string host_;
+  std::string port_;
+
+  int openConnection(const char *hostname, const char *port) {
+    constexpr int ERROR_STATUS = -1;
+    struct hostent *host;
+    if ((host = gethostbyname(hostname)) == nullptr) {
+        perror(hostname);
+        exit(EXIT_FAILURE);
+    }
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(hostname, port, &hints, &addrs);
+    if (status != 0) {
+        fprintf(stderr, "%s: %s\n", hostname, gai_strerror(status));
+        exit(EXIT_FAILURE);
+    }
+    int sfd, err;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+        sfd = socket(addrs->ai_family, addrs->ai_socktype, addrs->ai_protocol);
+        if (sfd == ERROR_STATUS) {
+            err = errno;
+            continue;
+        }
+        if (connect(sfd, addr->ai_addr, addr->ai_addrlen) == 0) {
+            break;
+        }
+        err = errno;
+        sfd = ERROR_STATUS;
+        close(sfd);
+    }
+    freeaddrinfo(addrs);
+    if (sfd == ERROR_STATUS) {
+        fprintf(stderr, "%s: %s\n", hostname, strerror(err));
+        exit(EXIT_FAILURE);
+    }
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(std::string host, std::string port) : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(std::string host, std::string port) : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(std::string host, std::string port) : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    TLSServerSocketSupportedProtocolsTest()
+        : isRunning_{ false }, configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      runAssertions();
+    }
+
+    void setKeyDir(const std::string key_dir) {
+      this->key_dir = key_dir;
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "3684";
+      if (!key_dir.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir);
+      }
+    }
+
+    void createServerSocket() {
+      std::shared_ptr<org::apache::nifi::minifi::io::TLSContext> socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      isRunning_ = true;
+      check = [this]() -> bool {
+        return isRunning_;
+      };
+      handler = [this](std::vector<uint8_t> *b, int *size) {

Review comment:
       Use a more descriptive name for `b`

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");

Review comment:
       We could use our core::logging::Logger instead

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;
+  }
+
+ protected:
+  void configureSecurity() {
+    host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+    port_ = "3684";
+    if (!key_dir.empty()) {
+      configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+      configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir + "cn.crt.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir + "cn.ckey.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir + "cn.pass");
+      configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir + "nifi-cert.pem");
+      configuration_->set(minifi::Configure::nifi_default_directory, key_dir);
+    }
+  }
+
+  void runAssertions() {

Review comment:
       To me this doesn't really seem like running assertions, but running separate testcases with different servers types and verifying the same behavior. It might be better to use the catch2 test framework here with different sections for different servers, or just separate test cases.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r567672227



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,310 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::function<bool()> check_;
+    std::function<int(std::vector<uint8_t> *bytes_written, int *size)> handler_;

Review comment:
       why store these functions, it seems to me that they are passed by copy to `server_socket_`

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,310 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::function<bool()> check_;
+    std::function<int(std::vector<uint8_t> *bytes_written, int *size)> handler_;

Review comment:
       why store these functions, it seems to me that they are passed by copy to `server_socket_` after creation




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564391950



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;

Review comment:
       source: According to [CONTRIB.md](https://github.com/apache/nifi-minifi-cpp/blob/main/CONTRIB.md), contributions are expected to follow the Google (C++) Style guide, and it says "[Data members of classes, both static and non-static, are named like ordinary nonmember variables, but with a trailing underscore.](https://google.github.io/styleguide/cppguide.html#Variable_Names)".




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564477962



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;

Review comment:
       Should be fixed by third commit in PR.
   Copy paste mistakes... I've originally used SecureSocketGetTCPTest.cpp and IntegrationBase.h as template. Any chance that linter could also catch these kind of style-guide violations?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564556332



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,233 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str());
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port) {
+    struct hostent *host;
+    assert((host = gethostbyname(host_name)) != nullptr);
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+        sfd = socket(addrs->ai_family, addrs->ai_socktype, addrs->ai_protocol);
+        if (sfd == INVALID_SOCKET) {
+            continue;
+        }
+        if (connect(sfd, addr->ai_addr, addr->ai_addrlen) == 0) {
+            break;
+        }
+        sfd = INVALID_SOCKET;
+#ifdef WIN32
+        closesocket(sfd);
+#else
+        close(sfd);
+#endif

Review comment:
       Minor indent issues

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,213 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), has_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            has_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hasConnection() const {
+    return has_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+  SocketDescriptor socket_descriptor_;
+  bool has_connection_;
+  std::thread server_read_thread_;
+
+  void configureContext(const std::string& path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);

Review comment:
       Minor indent issue




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
fgerlits commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r566232247



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,213 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), had_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            had_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hadConnection() const {
+    return had_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;

Review comment:
       `ssl_` will not get initialized if `accept()` in `waitForConnection()` fails, so `SSL_free()` will be called on garbage in the destructor.  It should probably be initialized to `nullptr`.

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,213 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), had_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            had_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hadConnection() const {
+    return had_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+  SocketDescriptor socket_descriptor_;
+  bool had_connection_;
+  std::thread server_read_thread_;
+
+  void configureContext(const std::string& path) {
+    SSL_CTX_set_ecdh_auto(ctx_, 1);
+    /* Set the key and cert */
+    assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+    assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  static SocketDescriptor createSocket(int port) {
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    SocketDescriptor socket_descriptor = socket(AF_INET, SOCK_STREAM, 0);
+    assert(socket_descriptor >= 0);
+    assert(bind(socket_descriptor, (struct sockaddr*)&addr, sizeof(addr)) >= 0);
+    assert(listen(socket_descriptor, 1) >= 0);
+
+    return socket_descriptor;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(const std::string& port, const std::string& path)
+      : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(const std::string& port, const std::string& path)
+      : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(const std::string& port, const std::string& path)
+      : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  explicit TLSClientSocketSupportedProtocolsTest(const std::string& key_dir)
+      : key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    verifyTLSClientSocketExclusiveCompatibilityWithTLSv1_2();
+  }
+
+
+ protected:
+  void configureSecurity() {
+    host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+    port_ = "38777";
+    if (!key_dir_.empty()) {
+      configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+      configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+      configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+      configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+    }
+  }
+
+  void verifyTLSClientSocketExclusiveCompatibilityWithTLSv1_2() {
+    verifyTLSProtocolIncompatibility<SimpleSSLTestServerTLSv1>();
+    verifyTLSProtocolIncompatibility<SimpleSSLTestServerTLSv1_1>();
+    verifyTLSProtocolCompatibility<SimpleSSLTestServerTLSv1_2>();
+  }
+
+  template <class TLSTestSever>
+  void verifyTLSProtocolIncompatibility() {
+    verifyTLSProtocolCompatibility<TLSTestSever>(false);
+  }
+
+  template <class TLSTestSever>
+  void verifyTLSProtocolCompatibility() {
+    verifyTLSProtocolCompatibility<TLSTestSever>(true);
+  }
+
+  template <class TLSTestSever>
+  void verifyTLSProtocolCompatibility(const bool should_be_compatible) {
+    TLSTestSever server(port_, key_dir_);
+    server.waitForConnection();
+
+    const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+    client_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSSocket>(socket_context, host_, std::stoi(port_), 0);
+    const bool client_initialized_successfully = (client_socket_->initialize() == 0);
+    assert(client_initialized_successfully == should_be_compatible);
+    server.shutdownServer();
+    assert(server.hadConnection() == should_be_compatible);
+  }
+
+ protected:
+    std::shared_ptr<org::apache::nifi::minifi::io::TLSSocket> client_socket_;
+    std::string host_;
+    std::string port_;
+    std::string key_dir_;
+    std::shared_ptr<minifi::Configure> configuration_;
+};
+
+static void sigpipe_handle(int) {
+}
+
+int main(int argc, char **argv) {
+  std::string key_dir;
+  if (argc > 1) {
+    key_dir = argv[1];
+  }
+#ifndef WIN32
+  signal(SIGPIPE, sigpipe_handle);

Review comment:
       why is this needed?

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,213 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), had_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            had_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hadConnection() const {
+    return had_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+  SocketDescriptor socket_descriptor_;
+  bool had_connection_;
+  std::thread server_read_thread_;
+
+  void configureContext(const std::string& path) {
+    SSL_CTX_set_ecdh_auto(ctx_, 1);
+    /* Set the key and cert */
+    assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+    assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);

Review comment:
       I think these should be `== 1` instead of `> 0`.

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_info("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_info("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::cout << "oh write!" << std::endl;
+        bytes_written->reserve(20);
+        memset(bytes_written->data(), 0x00, 20);
+        memcpy(bytes_written->data(), "hello world", 11);
+        *size = 20;
+        return *size;

Review comment:
       I don't like this, as `*bytes_written` will have `size() == 0` after the function returns, and the data is hidden inside an unused memory area.
   
   ```suggestion
           std::string contents = "hello world";
           *bytes_written = {contents.begin(), contents.end()};
           bytes_written->push_back(0);
           *size = bytes_written->size();
           return *size;
   ```

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;

Review comment:
       here too, `ssl_` may be uninitialized

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,213 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), had_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            had_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hadConnection() const {
+    return had_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;

Review comment:
       `listeners` is unused

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_info("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_info("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());

Review comment:
       minor, but should these logs on lines 165 and 172 be `log_error`?

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_info("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_info("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::cout << "oh write!" << std::endl;
+        bytes_written->reserve(20);
+        memset(bytes_written->data(), 0x00, 20);
+        memcpy(bytes_written->data(), "hello world", 11);
+        *size = 20;
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::function<bool()> check_;
+    std::function<int(std::vector<uint8_t>*b, int *size)> handler_;

Review comment:
       very minor, but I would either not include parameter names here, or rename `b` to something more descriptive, eg. `buffer_out`

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_info("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_info("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::cout << "oh write!" << std::endl;

Review comment:
       is this left over from debugging?  if so, please remove it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r567716547



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,310 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::function<bool()> check_;
+    std::function<int(std::vector<uint8_t> *bytes_written, int *size)> handler_;
+    std::atomic<bool> is_running_;
+    std::shared_ptr<org::apache::nifi::minifi::io::TLSServerSocket> server_socket_;

Review comment:
       Made it unique to be more clear, calling reset however seemed little excessive since unique_ptr destructor will be called and that achieves the same thing. I did however changed the destructor to set is_running_ to false. This will basically achieve that the external check function inside the socket will return false, after that the unique_ptr destructor will destroy the socket which will set its own inside check to false and it'll wait for the thread to finish. 

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,310 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::function<bool()> check_;
+    std::function<int(std::vector<uint8_t> *bytes_written, int *size)> handler_;

Review comment:
       fixed

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,310 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {

Review comment:
       fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r566674422



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,213 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), had_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            had_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hadConnection() const {
+    return had_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+  SocketDescriptor socket_descriptor_;
+  bool had_connection_;
+  std::thread server_read_thread_;
+
+  void configureContext(const std::string& path) {
+    SSL_CTX_set_ecdh_auto(ctx_, 1);
+    /* Set the key and cert */
+    assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+    assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  static SocketDescriptor createSocket(int port) {
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    SocketDescriptor socket_descriptor = socket(AF_INET, SOCK_STREAM, 0);
+    assert(socket_descriptor >= 0);
+    assert(bind(socket_descriptor, (struct sockaddr*)&addr, sizeof(addr)) >= 0);
+    assert(listen(socket_descriptor, 1) >= 0);
+
+    return socket_descriptor;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(const std::string& port, const std::string& path)
+      : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(const std::string& port, const std::string& path)
+      : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(const std::string& port, const std::string& path)
+      : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  explicit TLSClientSocketSupportedProtocolsTest(const std::string& key_dir)
+      : key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    verifyTLSClientSocketExclusiveCompatibilityWithTLSv1_2();
+  }
+
+
+ protected:
+  void configureSecurity() {
+    host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+    port_ = "38777";
+    if (!key_dir_.empty()) {
+      configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+      configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+      configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+      configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+    }
+  }
+
+  void verifyTLSClientSocketExclusiveCompatibilityWithTLSv1_2() {
+    verifyTLSProtocolIncompatibility<SimpleSSLTestServerTLSv1>();
+    verifyTLSProtocolIncompatibility<SimpleSSLTestServerTLSv1_1>();
+    verifyTLSProtocolCompatibility<SimpleSSLTestServerTLSv1_2>();
+  }
+
+  template <class TLSTestSever>
+  void verifyTLSProtocolIncompatibility() {
+    verifyTLSProtocolCompatibility<TLSTestSever>(false);
+  }
+
+  template <class TLSTestSever>
+  void verifyTLSProtocolCompatibility() {
+    verifyTLSProtocolCompatibility<TLSTestSever>(true);
+  }
+
+  template <class TLSTestSever>
+  void verifyTLSProtocolCompatibility(const bool should_be_compatible) {
+    TLSTestSever server(port_, key_dir_);
+    server.waitForConnection();
+
+    const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+    client_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSSocket>(socket_context, host_, std::stoi(port_), 0);
+    const bool client_initialized_successfully = (client_socket_->initialize() == 0);
+    assert(client_initialized_successfully == should_be_compatible);
+    server.shutdownServer();
+    assert(server.hadConnection() == should_be_compatible);
+  }
+
+ protected:
+    std::shared_ptr<org::apache::nifi::minifi::io::TLSSocket> client_socket_;
+    std::string host_;
+    std::string port_;
+    std::string key_dir_;
+    std::shared_ptr<minifi::Configure> configuration_;
+};
+
+static void sigpipe_handle(int) {
+}
+
+int main(int argc, char **argv) {
+  std::string key_dir;
+  if (argc > 1) {
+    key_dir = argv[1];
+  }
+#ifndef WIN32
+  signal(SIGPIPE, sigpipe_handle);

Review comment:
       This overrides how to handle the broken pipe signal (which will happen because we are testing that you cannot connect with outdated protocol). The default behavior for this signal is to end the process.
   This workaround is already used by extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp (I used that class as template for these tests)
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r566674901



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,213 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, const std::string& port, const std::string& path)
+      : port_(port), had_connection_(false) {
+    ctx_ = SSL_CTX_new(method);
+    configureContext(path);
+    socket_descriptor_ = createSocket(std::stoi(port_));
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    server_read_thread_ = std::thread([this]() -> void {
+        SocketDescriptor client = accept(socket_descriptor_, nullptr, nullptr);
+        if (client != INVALID_SOCKET) {
+            ssl_ = SSL_new(ctx_);
+            SSL_set_fd(ssl_, client);
+            had_connection_ = (SSL_accept(ssl_) == 1);
+        }
+    });
+  }
+
+  void shutdownServer() {
+#ifdef WIN32
+    shutdown(socket_descriptor_, SD_BOTH);
+    closesocket(socket_descriptor_);
+#else
+    shutdown(socket_descriptor_, SHUT_RDWR);
+    close(socket_descriptor_);
+#endif
+    server_read_thread_.join();
+  }
+
+  bool hadConnection() const {
+    return had_connection_;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;

Review comment:
       thanks, fixed it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#issuecomment-768192917


   There was a race condition which could cause one of the asserts to fail, fixed it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564487891



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;

Review comment:
       Currently cpplint is run for the PRs which should catch a lot of violations to the style-guide, but unfortunately it cannot catch all violations like variable namings.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564473775



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);

Review comment:
       No, I've forgot about windows, should be fixed now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r566677495



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_info("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_info("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::cout << "oh write!" << std::endl;
+        bytes_written->reserve(20);
+        memset(bytes_written->data(), 0x00, 20);
+        memcpy(bytes_written->data(), "hello world", 11);
+        *size = 20;
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::function<bool()> check_;
+    std::function<int(std::vector<uint8_t>*b, int *size)> handler_;

Review comment:
       changed it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r565889194



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,233 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str());
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port) {
+    struct hostent *host;
+    assert((host = gethostbyname(host_name)) != nullptr);

Review comment:
       yeah it was leftover, I removed it 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r567671195



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,310 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_ = nullptr;
+  SSL* ssl_ = nullptr;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_error("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_error("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::string contents = "hello world";
+        *bytes_written = {contents.begin(), contents.end()};
+        bytes_written->push_back(0);
+        *size = bytes_written->size();
+        return *size;
+      };
+      server_socket_->registerCallback(check_, handler_, std::chrono::milliseconds(50));
+    }
+
+    void verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2() {
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1>();
+      verifyTLSProtocolIncompatibility<SimpleSSLTestClientTLSv1_1>();
+      verifyTLSProtocolCompatibility<SimpleSSLTestClientTLSv1_2>();
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolIncompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(false);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility() {
+      verifyTLSProtocolCompatibility<TLSTestClient>(true);
+    }
+
+    template <class TLSTestClient>
+    void verifyTLSProtocolCompatibility(bool should_be_compatible) {
+      TLSTestClient client(host_, port_);
+      assert(client.canConnect() == should_be_compatible);
+    }
+
+    void shutdownServerSocket() {
+      is_running_ = false;
+    }
+
+    std::function<bool()> check_;
+    std::function<int(std::vector<uint8_t> *bytes_written, int *size)> handler_;
+    std::atomic<bool> is_running_;
+    std::shared_ptr<org::apache::nifi::minifi::io::TLSServerSocket> server_socket_;

Review comment:
       we could use a `unique_ptr` for the `server_socket_` (to make it clear that it is getting destroyed at the end of this object) and maybe even call an explicit `reset` on it in the destructor of `TLSServerSocketSupportedProtocolsTest ` (since currently the correctness hinges on the order of `server_socket_` and `is_running_`)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#issuecomment-769132298


   I'm waiting for one more approval before merging due to the extent of changes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564586153



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,233 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str());
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port) {
+    struct hostent *host;
+    assert((host = gethostbyname(host_name)) != nullptr);
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+        sfd = socket(addrs->ai_family, addrs->ai_socktype, addrs->ai_protocol);
+        if (sfd == INVALID_SOCKET) {
+            continue;
+        }
+        if (connect(sfd, addr->ai_addr, addr->ai_addrlen) == 0) {
+            break;
+        }
+        sfd = INVALID_SOCKET;
+#ifdef WIN32
+        closesocket(sfd);
+#else
+        close(sfd);
+#endif

Review comment:
       Fixed the indentations, thanks for the review




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564498125



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");

Review comment:
       moved them inside asserts instead, because they are not expected to fail but any further testing is impossible without these succeeding




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r564489250



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;

Review comment:
       I'm not sure if other tools like A-Style or clang-format might be able to catch more violations, we need to check as possible options. Some static code analysis tools like CodeChecker allow custom checkers to be created that can be used even for variable name checks.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#issuecomment-767707569


   The issue on the ubuntu-16.04-all CI run seems to be related, could you check that?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] martinzink commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
martinzink commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r566677637



##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,311 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#ifndef WIN32
+#include <arpa/inet.h>
+#endif
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#include <cerrno>
+#include <cinttypes>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#ifdef WIN32
+#include <ws2tcpip.h>
+#include <winsock2.h>
+#endif
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+#ifdef WIN32
+using SocketDescriptor = SOCKET;
+#else
+using SocketDescriptor = int;
+static constexpr SocketDescriptor INVALID_SOCKET = -1;
+#endif /* WIN32 */
+
+namespace {
+const char* str_family(int family) {
+  switch (family) {
+    case AF_INET: return "AF_INET";
+    case AF_INET6: return "AF_INET6";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_socktype(int socktype) {
+  switch (socktype) {
+    case SOCK_STREAM: return "SOCK_STREAM";
+    case SOCK_DGRAM: return "SOCK_DGRAM";
+    default: return "(n/a)";
+  }
+}
+
+const char* str_proto(int protocol) {
+  switch (protocol) {
+    case IPPROTO_TCP: return "IPPROTO_TCP";
+    case IPPROTO_UDP: return "IPPROTO_UDP";
+    case IPPROTO_IP: return "IPPROTO_IP";
+    case IPPROTO_ICMP: return "IPPROTO_ICMP";
+    default: return "(n/a)";
+  }
+}
+
+std::string str_addr(const sockaddr* const sa) {
+  char buf[128] = {0};
+  switch (sa->sa_family) {
+    case AF_INET: {
+      sockaddr_in sin{};
+      memcpy(&sin, sa, sizeof(sockaddr_in));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin.sin_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    case AF_INET6: {
+      sockaddr_in6 sin6{};
+      memcpy(&sin6, sa, sizeof(sockaddr_in6));
+#ifdef WIN32
+      const auto addr_str = InetNtop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#else
+      const auto addr_str = inet_ntop(AF_INET, &sin6.sin6_addr, buf, sizeof(buf));
+#endif
+      if (!addr_str) {
+        throw std::runtime_error{minifi::io::get_last_socket_error_message()};
+      }
+      return std::string{addr_str};
+    }
+    default: return "(n/a)";
+  }
+}
+
+void log_addrinfo(addrinfo* const ai, logging::Logger& logger) {
+  logger.log_debug(".ai_family: %d %s\n", ai->ai_family, str_family(ai->ai_family));
+  logger.log_debug(".ai_socktype: %d %s\n", ai->ai_socktype, str_socktype(ai->ai_socktype));
+  logger.log_debug(".ai_protocol: %d %s\n", ai->ai_protocol, str_proto(ai->ai_protocol));
+  logger.log_debug(".ai_addr: %s\n", str_addr(ai->ai_addr).c_str());
+  logger.log_debug(".ai_addrlen: %" PRIu32 "\n", ai->ai_addrlen);
+}
+}  // namespace
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, const std::string& host, const std::string& port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str(), *logger_);
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+#ifdef WIN32
+    closesocket(sfd_);
+#else
+    close(sfd_);
+#endif
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    const bool successful_connection = (status == 1);
+    return successful_connection;
+  }
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  SocketDescriptor sfd_;
+  std::string host_;
+  std::string port_;
+  gsl::not_null<std::shared_ptr<logging::Logger>> logger_{gsl::make_not_null(logging::LoggerFactory<SimpleSSLTestClient>::getLogger())};
+
+  static SocketDescriptor openConnection(const char *host_name, const char *port, logging::Logger& logger) {
+    struct addrinfo hints = {0}, *addrs;
+    hints.ai_family = AF_UNSPEC;
+    hints.ai_socktype = SOCK_STREAM;
+    hints.ai_protocol = IPPROTO_TCP;
+    const int status = getaddrinfo(host_name, port, &hints, &addrs);
+    assert(status == 0);
+    SocketDescriptor sfd = INVALID_SOCKET;
+    for (struct addrinfo *addr = addrs; addr != nullptr; addr = addr->ai_next) {
+      log_addrinfo(addr, logger);
+      sfd = socket(addr->ai_family, addr->ai_socktype, addr->ai_protocol);
+      if (sfd == INVALID_SOCKET) {
+        logger.log_info("socket: %s\n", minifi::io::get_last_socket_error_message());
+        continue;
+      }
+      const auto connect_result = connect(sfd, addr->ai_addr, addr->ai_addrlen);
+      if (connect_result == 0) {
+        break;
+      } else {
+        logger.log_info("connect to %s: %s\n", str_addr(addr->ai_addr), minifi::io::get_last_socket_error_message());
+      }
+      sfd = INVALID_SOCKET;
+#ifdef WIN32
+      closesocket(sfd);
+#else
+      close(sfd);
+#endif
+    }
+    freeaddrinfo(addrs);
+    assert(sfd != INVALID_SOCKET);
+    return sfd;
+  }
+};
+
+class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_1  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_1(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_1_client_method(), host, port) {
+  }
+};
+
+class SimpleSSLTestClientTLSv1_2  : public SimpleSSLTestClient {
+ public:
+  SimpleSSLTestClientTLSv1_2(const std::string& host, const std::string& port)
+      : SimpleSSLTestClient(TLSv1_2_client_method(), host, port) {
+  }
+};
+
+class TLSServerSocketSupportedProtocolsTest {
+ public:
+    explicit TLSServerSocketSupportedProtocolsTest(const std::string& key_dir)
+        : is_running_(false), key_dir_(key_dir), configuration_(std::make_shared<minifi::Configure>()) {
+    }
+
+    void run() {
+      configureSecurity();
+
+      createServerSocket();
+
+      verifyTLSServerSocketExclusiveCompatibilityWithTLSv1_2();
+
+      shutdownServerSocket();
+    }
+
+ protected:
+    void configureSecurity() {
+      host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+      port_ = "38778";
+      if (!key_dir_.empty()) {
+        configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+        configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir_ + "cn.crt.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir_ + "cn.ckey.pem");
+        configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir_ + "cn.pass");
+        configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir_ + "nifi-cert.pem");
+        configuration_->set(minifi::Configure::nifi_default_directory, key_dir_);
+      }
+    }
+
+    void createServerSocket() {
+      const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      server_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSServerSocket>(socket_context, host_, std::stoi(port_), 3);
+      assert(0 == server_socket_->initialize());
+
+      is_running_ = true;
+      check_ = [this]() -> bool {
+        return is_running_;
+      };
+      handler_ = [this](std::vector<uint8_t> *bytes_written, int *size) {
+        std::cout << "oh write!" << std::endl;

Review comment:
       Good catch, it came from extensions/standard-processors/tests/integration/SecureSocketGetTCPTest.cpp which I used as template, removed it nonetheless.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#discussion_r563733304



##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);
+        ssl_ = SSL_new(ctx_);
+        SSL_set_fd(ssl_, client);
+        successful = (SSL_accept(ssl_) == 1);
+      }
+    });
+  }
+
+  bool isRunning_;
+  std::thread server_read_thread_;
+  int sock_;
+  bool successful;
+
+ private:
+  SSL_CTX *ctx_;
+  SSL* ssl_;
+  std::string port_;
+  uint16_t listeners_;
+
+  void configure_context(std::string path) {
+      SSL_CTX_set_ecdh_auto(ctx_, 1);
+      /* Set the key and cert */
+      assert(SSL_CTX_use_certificate_file(ctx_, (path + "cn.crt.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+      assert(SSL_CTX_use_PrivateKey_file(ctx_, (path + "cn.ckey.pem").c_str(), SSL_FILETYPE_PEM) > 0);
+  }
+
+  int create_socket(int port) {
+    int s;
+    struct sockaddr_in addr;
+
+    addr.sin_family = AF_INET;
+    addr.sin_port = htons(port);
+    addr.sin_addr.s_addr = htonl(INADDR_ANY);
+
+    s = socket(AF_INET, SOCK_STREAM, 0);
+    if (s < 0) {
+      perror("Unable to create socket");
+      exit(EXIT_FAILURE);
+    }
+
+    if (bind(s, (struct sockaddr*)&addr, sizeof(addr)) < 0) {
+      perror("Unable to bind");
+      exit(EXIT_FAILURE);
+    }
+
+    if (listen(s, 1) < 0) {
+      perror("Unable to listen");
+      exit(EXIT_FAILURE);
+    }
+
+    return s;
+  }
+};
+
+class SimpleSSLTestServerTLSv1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_1  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_1(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_1_server_method(), port, path) {
+  }
+};
+
+class SimpleSSLTestServerTLSv1_2  : public SimpleSSLTestServer {
+ public:
+  SimpleSSLTestServerTLSv1_2(std::string port, std::string path) : SimpleSSLTestServer(TLSv1_2_server_method(), port, path) {
+  }
+};
+
+class TLSClientSocketSupportedProtocolsTest {
+ public:
+  TLSClientSocketSupportedProtocolsTest()
+      : configuration_(std::make_shared<minifi::Configure>()) {
+  }
+
+  void run() {
+    configureSecurity();
+
+    runAssertions();
+  }
+
+  void setKeyDir(const std::string key_dir) {
+    this->key_dir = key_dir;
+  }
+
+ protected:
+  void configureSecurity() {
+    host_ = org::apache::nifi::minifi::io::Socket::getMyHostName();
+    port_ = "3684";
+    if (!key_dir.empty()) {
+      configuration_->set(minifi::Configure::nifi_remote_input_secure, "true");
+      configuration_->set(minifi::Configure::nifi_security_client_certificate, key_dir + "cn.crt.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_private_key, key_dir + "cn.ckey.pem");
+      configuration_->set(minifi::Configure::nifi_security_client_pass_phrase, key_dir + "cn.pass");
+      configuration_->set(minifi::Configure::nifi_security_client_ca_certificate, key_dir + "nifi-cert.pem");
+      configuration_->set(minifi::Configure::nifi_default_directory, key_dir);
+    }
+  }
+
+  void runAssertions() {
+    {
+      SimpleSSLTestServerTLSv1 server(port_, key_dir);
+      server.waitForConnection();
+
+      std::shared_ptr<org::apache::nifi::minifi::io::TLSContext> socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
+      client_socket_ = std::make_shared<org::apache::nifi::minifi::io::TLSSocket>(socket_context, host_, std::stoi(port_), 0);
+      assert(client_socket_->initialize() != 0);
+      shutdown(server.sock_, SHUT_RD);
+      close(server.sock_);
+      server.isRunning_ = false;
+      server.server_read_thread_.join();
+    }
+    {
+      SimpleSSLTestServerTLSv1_1 server(port_, key_dir);
+      server.waitForConnection();
+
+      std::shared_ptr<org::apache::nifi::minifi::io::TLSContext> socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);

Review comment:
       [Use auto to avoid redundant repetition of type names](https://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#es11-use-auto-to-avoid-redundant-repetition-of-type-names) (multiple occurences thoughout the PR, but I don't want to spam the same comment)
   
   ```suggestion
         const auto socket_context = std::make_shared<org::apache::nifi::minifi::io::TLSContext>(configuration_);
   ```

##########
File path: extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,221 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestServer  {
+ public:
+  SimpleSSLTestServer(const SSL_METHOD* method, std::string port, std::string path)
+      : port_(port) {
+    ctx_ = SSL_CTX_new(method);
+    configure_context(path);
+  }
+
+  ~SimpleSSLTestServer() {
+      SSL_shutdown(ssl_);
+      SSL_free(ssl_);
+      SSL_CTX_free(ctx_);
+  }
+
+  void waitForConnection() {
+    isRunning_ = true;
+    sock_ = create_socket(std::stoi(port_));
+    server_read_thread_ = std::thread([this]() -> void {
+      while (isRunning_) {
+        struct sockaddr_in addr;
+        uint len = sizeof(addr);
+
+        int client = accept(sock_, (struct sockaddr*)&addr, &len);

Review comment:
       Is this intentionally unix-only? On Windows, the socket type is `SOCKET`, and other parts of the networking code of the test are unix-only as well.

##########
File path: extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp
##########
@@ -0,0 +1,217 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include <signal.h>
+#include <sys/stat.h>
+#include <chrono>
+#include <thread>
+#undef NDEBUG
+#include <cassert>
+#include <utility>
+#include <memory>
+#include <string>
+#include "properties/Configure.h"
+#include "io/tls/TLSSocket.h"
+#include "io/tls/TLSServerSocket.h"
+
+class SimpleSSLTestClient  {
+ public:
+  SimpleSSLTestClient(const SSL_METHOD* method, std::string host, std::string port) :
+    host_(host),
+    port_(port) {
+      ctx_ = SSL_CTX_new(method);
+      sfd_ = openConnection(host_.c_str(), port_.c_str());
+      if (ctx_ != nullptr)
+        ssl_ = SSL_new(ctx_);
+      if (ssl_ != nullptr)
+        SSL_set_fd(ssl_, sfd_);
+  }
+
+  ~SimpleSSLTestClient() {
+    SSL_free(ssl_);
+    close(sfd_);
+    SSL_CTX_free(ctx_);
+  }
+
+  bool canConnect() {
+    const int status = SSL_connect(ssl_);
+    bool successfulConnection = (status == 1);

Review comment:
       I would make `successfulConnection` `const` as well, since it's just a derived value from `status`.
   
   To be honest, I would make as many things `const` as reasonably possible, but this is not common practice in most contributions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#issuecomment-768407474


   The issue is with the `socket` call. I fixed it and added extra logging in martinzink#5


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on pull request #978: MINIFICPP-1453: Verify Supported TLS Protocols

Posted by GitBox <gi...@apache.org>.
szaszm commented on pull request #978:
URL: https://github.com/apache/nifi-minifi-cpp/pull/978#issuecomment-766835726


   And thanks for your first contribution. The general workflow is that we merge after every comment is resolved, there are 2-3 approvals on the pull request, no conflicts (rebase as needed) and no related CI issues.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org