You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "lordgamez (via GitHub)" <gi...@apache.org> on 2023/05/31 15:48:53 UTC

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request, #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

lordgamez opened a new pull request, #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583

   https://issues.apache.org/jira/browse/MINIFICPP-1719
   
   --------------------
   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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1229841318


##########
README.md:
##########
@@ -151,6 +151,10 @@ and rebuild.
 * Lua and development headers -- Required if Lua support is enabled
 * libgps-dev -- Required if building libGPS support
 * Zlib headers
+* perl -- Required for OpenSSL configuration
+* NASM -- Required for OpenSSL only on Windows
+
+**NOTE** On Windows if Strawberry Perl is used the `${StrawberryPerlRoot}\c\bin` directory should not be part of the %PATH% variable as Strawberry Perl's patch.exe will be found as the patch executable in the configure phase instead if the git patch executable.

Review Comment:
   But don't we only use CMake's cpack? If so we are independent from Chocolatey, so I don't think it should be entirely necessary to note how we can conflict with it, but anyway I can add a note about it to have some additional information.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1222552988


##########
extensions/standard-processors/tests/unit/ListenTcpTests.cpp:
##########
@@ -287,7 +287,7 @@ TEST_CASE("Test ListenTCP SSL/TLS compatibility", "[ListenTCP][NetworkListenerPr
 
   SECTION("tlsv13 should be enabled") {
     client_method = asio::ssl::context::method::tlsv13_client;
-    expected_to_work = true;
+    expected_to_work = false;

Review Comment:
   thanks, makes sense



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#issuecomment-1614391963

   > There is an issue present in OpenSSL3.1.1 which makes debugging on Apple sillicon impossible [openssl/openssl#20753](https://github.com/openssl/openssl/issues/20753) Fortunetly there is already a fix on OpenSSL master branch [openssl/openssl@52a3814](https://github.com/openssl/openssl/commit/52a38144b019cfda6b0e5eaa0aca88ae11661a26) merged from [openssl/openssl#21094](https://github.com/openssl/openssl/issues/21094) , but its not yet backported to 3.1.x
   > 
   > I've backported (fixed one tiny merge conflict), which fixed the debugging issues on Apple sillicon. I think we should include this patch. Created a PR [lordgamez#228](https://github.com/lordgamez/nifi-minifi-cpp/pull/228)
   
   Thanks! Merged the fix to the branch.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1222921837


##########
extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp:
##########


Review Comment:
   Added separate jira ticket for TLS 1.3 support: https://issues.apache.org/jira/browse/MINIFICPP-2133



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1224341797


##########
libminifi/test/SimpleSSLTestServer.h:
##########
@@ -50,11 +50,12 @@ class SimpleSSLTestServer  {
   };
 
  public:
-  SimpleSSLTestServer(const SSL_METHOD* method, int port, const std::filesystem::path& key_dir)
+  SimpleSSLTestServer(uint64_t disable_version, int port, const std::filesystem::path& key_dir)

Review Comment:
   Updated in 67eb58112fe948432498c39cae18aec78251fd22



##########
docker/test/integration/steps/steps.py:
##########
@@ -358,8 +358,8 @@ def step_impl(context):
 @given("an ssl context service set up for {producer_name} and {consumer_name}")
 def step_impl(context, producer_name, consumer_name):
     root_ca_cert, root_ca_key = make_ca("root CA")
-    minifi_cert, minifi_key = make_cert("minifi-cpp-flow", root_ca_cert, root_ca_key)
-    secondary_cert, secondary_key = make_cert("secondary", root_ca_cert, root_ca_key)
+    minifi_cert, minifi_key = make_cert("minifi-cpp-flow", root_ca_cert, root_ca_key, False)
+    secondary_cert, secondary_key = make_cert("secondary", root_ca_cert, root_ca_key, True)

Review Comment:
   Changed in 67eb58112fe948432498c39cae18aec78251fd22



##########
cmake/BundledOpenSSL.cmake:
##########
@@ -0,0 +1,128 @@
+# 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.
+
+function(use_openssl SOURCE_DIR BINARY_DIR)
+    message("Using bundled OpenSSL")
+
+    if(APPLE OR WIN32 OR CMAKE_SIZEOF_VOID_P EQUAL 4)
+        set(LIBDIR "lib")
+    else()
+        set(LIBDIR "lib64")
+    endif()
+
+    # Define byproducts
+    set(BYPRODUCT_PREFIX "lib" CACHE STRING "" FORCE)
+    if (WIN32)
+        set(BYPRODUCT_SUFFIX ".lib" CACHE STRING "" FORCE)
+    elseif (APPLE AND (CMAKE_SYSTEM_PROCESSOR MATCHES "x86_64|amd64|AMD64"))
+        set(BYPRODUCT_SUFFIX ".dylib" CACHE STRING "" FORCE)
+    else()
+        set(BYPRODUCT_SUFFIX ".a" CACHE STRING "" FORCE)
+    endif()
+
+    if (APPLE AND (CMAKE_SYSTEM_PROCESSOR MATCHES "x86_64|amd64|AMD64"))
+        set(OPENSSL_SHARED_FLAG "" CACHE STRING "" FORCE)

Review Comment:
   Added a comment in 67eb58112fe948432498c39cae18aec78251fd22



##########
README.md:
##########
@@ -151,6 +151,10 @@ and rebuild.
 * Lua and development headers -- Required if Lua support is enabled
 * libgps-dev -- Required if building libGPS support
 * Zlib headers
+* perl -- Required for OpenSSL configuration
+* NASM -- Required for OpenSSL only on Windows
+
+**NOTE** On Windows if Strawberry Perl is used the `${StrawberryPerlRoot}\c\bin` directory should not be part of the %PATH% variable as Strawberry Perl's patch.exe will be found as the patch executable in the configure phase instead if the git patch executable.

Review Comment:
   Fixed in 67eb58112fe948432498c39cae18aec78251fd22
   
   I'm not sure if we should add a note about Chocolatey as it is not a requirement for our build whereas Perl is, and StrawberryPerl is the most common implementation on Windows.



##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -112,6 +112,22 @@ const core::Property SSLContextService::ClientCertKeyUsage(
         ->build());
 #endif  // WIN32
 
+namespace {
+bool is_valid_and_readable_path(const std::filesystem::path& path_to_be_tested) {
+  std::ifstream file_to_be_tested(path_to_be_tested);
+  return file_to_be_tested.good();
+}
+
+#ifdef WIN32
+std::string getCertName(const utils::tls::X509_unique_ptr& cert) {
+  const size_t BUFFER_SIZE = 256;
+  char name_buffer[BUFFER_SIZE];
+  X509_NAME_oneline(X509_get_subject_name(cert.get()), name_buffer, BUFFER_SIZE);

Review Comment:
   Updated in 67eb58112fe948432498c39cae18aec78251fd22



##########
extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp:
##########
@@ -194,21 +195,21 @@ class SimpleSSLTestClient  {
 class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
  public:
   SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
-      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+      : SimpleSSLTestClient(SSL_OP_NO_SSLv3 | SSL_OP_NO_TLSv1_2 | SSL_OP_NO_TLSv1_1 | SSL_OP_NO_TLSv1_3, host, port) {

Review Comment:
   Updated to set_max and set_min in 67eb58112fe948432498c39cae18aec78251fd22



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1247505883


##########
extensions/standard-processors/processors/HashContent.h:
##########
@@ -49,21 +48,25 @@ namespace { // NOLINT
     HashReturnType ret_val;
     ret_val.second = 0;
     std::array<std::byte, HASH_BUFFER_SIZE> buffer{};
-    MD5_CTX context;
-    MD5_Init(&context);
+    EVP_MD_CTX *context = EVP_MD_CTX_new();
+    const auto guard = gsl::finally([&context]() {
+      EVP_MD_CTX_free(context);
+    });
+    const EVP_MD *md = EVP_md5();
+    EVP_DigestInit_ex(context, md, nullptr);

Review Comment:
   I can be inlined, updated in 5ebb285e42f5ea89391d5b126bcbecb2b27c77d9



##########
extensions/standard-processors/processors/HashContent.h:
##########
@@ -49,21 +48,25 @@ namespace { // NOLINT
     HashReturnType ret_val;
     ret_val.second = 0;
     std::array<std::byte, HASH_BUFFER_SIZE> buffer{};
-    MD5_CTX context;
-    MD5_Init(&context);
+    EVP_MD_CTX *context = EVP_MD_CTX_new();
+    const auto guard = gsl::finally([&context]() {
+      EVP_MD_CTX_free(context);
+    });
+    const EVP_MD *md = EVP_md5();
+    EVP_DigestInit_ex(context, md, nullptr);
 
     size_t ret = 0;
     do {
       ret = stream->read(buffer);
       if (ret > 0) {
-        MD5_Update(&context, buffer.data(), ret);
+        EVP_DigestUpdate(context, buffer.data(), ret);
         ret_val.second += gsl::narrow<int64_t>(ret);
       }
     } while (ret > 0);
 
     if (ret_val.second > 0) {
-      std::array<std::byte, MD5_DIGEST_LENGTH> digest{};
-      MD5_Final(reinterpret_cast<unsigned char*>(digest.data()), &context);
+      std::array<std::byte, EVP_MAX_MD_SIZE> digest{};
+      EVP_DigestFinal_ex(context, reinterpret_cast<unsigned char*>(digest.data()), nullptr);

Review Comment:
   Great catch! I didn't realize this because the HashContentTest was passing. After checking the test I saw that it was passing on the wrong hash value because only a partial match was required in the assertion instead of an exact match. Fixed in 5ebb285e42f5ea89391d5b126bcbecb2b27c77d9



##########
libminifi/src/core/state/Value.cpp:
##########
@@ -34,25 +34,29 @@ const std::type_index Value::BOOL_TYPE = std::type_index(typeid(bool));
 const std::type_index Value::DOUBLE_TYPE = std::type_index(typeid(double));
 const std::type_index Value::STRING_TYPE = std::type_index(typeid(std::string));
 
-void hashNode(const SerializedResponseNode& node, SHA512_CTX& ctx) {
-  SHA512_Update(&ctx, node.name.c_str(), node.name.length());
+void hashNode(const SerializedResponseNode& node, EVP_MD_CTX* ctx) {

Review Comment:
   Updated in 5ebb285e42f5ea89391d5b126bcbecb2b27c77d9



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1254155822


##########
thirdparty/libwebsockets/openssl3.patch:
##########


Review Comment:
   I wrote it :) The patch manually sets all openssl3 features ON as the libwebsockets configuration tries to look for them checking the SSL implementation installed on the system instead of the bundled OpenSSL version.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1221678296


##########
docker/test/integration/steps/steps.py:
##########
@@ -358,8 +358,8 @@ def step_impl(context):
 @given("an ssl context service set up for {producer_name} and {consumer_name}")
 def step_impl(context, producer_name, consumer_name):
     root_ca_cert, root_ca_key = make_ca("root CA")
-    minifi_cert, minifi_key = make_cert("minifi-cpp-flow", root_ca_cert, root_ca_key)
-    secondary_cert, secondary_key = make_cert("secondary", root_ca_cert, root_ca_key)
+    minifi_cert, minifi_key = make_cert("minifi-cpp-flow", root_ca_cert, root_ca_key, False)
+    secondary_cert, secondary_key = make_cert("secondary", root_ca_cert, root_ca_key, True)

Review Comment:
   I think it would be nicer to make the new parameter a string (or a list of strings), so these would be `make_cert(..., b"clientAuth")` and `make_cert(..., b"serverAuth")` instead of `False` and `True`.



##########
extensions/standard-processors/tests/integration/TLSClientSocketSupportedProtocolsTest.cpp:
##########


Review Comment:
   Can we add a `SimpleSSLTestServerTLSv1_3` (and client)?  It's OK if you want that to be done in a separate PR.



##########
README.md:
##########
@@ -151,6 +151,10 @@ and rebuild.
 * Lua and development headers -- Required if Lua support is enabled
 * libgps-dev -- Required if building libGPS support
 * Zlib headers
+* perl -- Required for OpenSSL configuration
+* NASM -- Required for OpenSSL only on Windows
+
+**NOTE** On Windows if Strawberry Perl is used the `${StrawberryPerlRoot}\c\bin` directory should not be part of the %PATH% variable as Strawberry Perl's patch.exe will be found as the patch executable in the configure phase instead if the git patch executable.

Review Comment:
   tiny typo:
   ```suggestion
   **NOTE** On Windows if Strawberry Perl is used the `${StrawberryPerlRoot}\c\bin` directory should not be part of the %PATH% variable as Strawberry Perl's patch.exe will be found as the patch executable in the configure phase instead of the git patch executable.
   ```



##########
cmake/BundledOpenSSL.cmake:
##########
@@ -0,0 +1,128 @@
+# 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.
+
+function(use_openssl SOURCE_DIR BINARY_DIR)
+    message("Using bundled OpenSSL")
+
+    if(APPLE OR WIN32 OR CMAKE_SIZEOF_VOID_P EQUAL 4)
+        set(LIBDIR "lib")
+    else()
+        set(LIBDIR "lib64")
+    endif()
+
+    # Define byproducts
+    set(BYPRODUCT_PREFIX "lib" CACHE STRING "" FORCE)
+    if (WIN32)
+        set(BYPRODUCT_SUFFIX ".lib" CACHE STRING "" FORCE)
+    elseif (APPLE AND (CMAKE_SYSTEM_PROCESSOR MATCHES "x86_64|amd64|AMD64"))
+        set(BYPRODUCT_SUFFIX ".dylib" CACHE STRING "" FORCE)
+    else()
+        set(BYPRODUCT_SUFFIX ".a" CACHE STRING "" FORCE)
+    endif()
+
+    if (APPLE AND (CMAKE_SYSTEM_PROCESSOR MATCHES "x86_64|amd64|AMD64"))
+        set(OPENSSL_SHARED_FLAG "" CACHE STRING "" FORCE)

Review Comment:
   Can you add a comment to explain why we need to do this, please?



##########
libminifi/test/SimpleSSLTestServer.h:
##########
@@ -50,11 +50,12 @@ class SimpleSSLTestServer  {
   };
 
  public:
-  SimpleSSLTestServer(const SSL_METHOD* method, int port, const std::filesystem::path& key_dir)
+  SimpleSSLTestServer(uint64_t disable_version, int port, const std::filesystem::path& key_dir)

Review Comment:
   I think `tls_version_flags` would be a better parameter name than `disable_version`.



##########
extensions/standard-processors/tests/unit/ListenTcpTests.cpp:
##########
@@ -287,7 +287,7 @@ TEST_CASE("Test ListenTCP SSL/TLS compatibility", "[ListenTCP][NetworkListenerPr
 
   SECTION("tlsv13 should be enabled") {
     client_method = asio::ssl::context::method::tlsv13_client;
-    expected_to_work = true;
+    expected_to_work = false;

Review Comment:
   What's going on here?  I would expect TLS 1.3 to go from `expected_to_work == false` to `true`, but apparently it's changing from `true` to `false`...



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1222987263


##########
NOTICE:
##########


Review Comment:
   Could you update the copyright year near the top?



##########
bootstrap.sh:
##########
@@ -338,6 +338,9 @@ add_option PROCFS_ENABLED ${TRUE} "ENABLE_PROCFS"
 
 add_option PROMETHEUS_ENABLED ${FALSE} "ENABLE_PROMETHEUS"
 
+add_option OPENSSL_ENABLED ${TRUE} "OPENSSL_OFF"
+add_dependency OPENSSL_ENABLED "opensslbuild"

Review Comment:
   why call it "opensslbuild" instead of just "openssl", like the other dependencies?



##########
cmake/GoogleCloudCpp.cmake:
##########
@@ -40,3 +40,9 @@ FetchContent_MakeAvailable(google-cloud-cpp)
 if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang" AND CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL "14.0.0" )
     target_compile_options(google_cloud_cpp_common PUBLIC -Wno-error=deprecated-pragma)
 endif()
+
+if (WIN32)
+    target_compile_options(google_cloud_cpp_storage PUBLIC /wd4996)
+else()
+    target_compile_options(google_cloud_cpp_storage PUBLIC -Wno-error=deprecated-declarations)

Review Comment:
   Would you mind adding a note with an explanation or related error messages?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1221822270


##########
extensions/standard-processors/tests/unit/ListenTcpTests.cpp:
##########
@@ -287,7 +287,7 @@ TEST_CASE("Test ListenTCP SSL/TLS compatibility", "[ListenTCP][NetworkListenerPr
 
   SECTION("tlsv13 should be enabled") {
     client_method = asio::ssl::context::method::tlsv13_client;
-    expected_to_work = true;
+    expected_to_work = false;

Review Comment:
   Even though we had a test for TLS v1.3 previously the test case was NOOP as it was not supported by the LibreSSL library so it passed anyway. Now we set it to false as we do not support it at the moment so the test would fail because the library now actually supports TLS 1.3



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "adamdebreceni (via GitHub)" <gi...@apache.org>.
adamdebreceni commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1219180190


##########
libminifi/src/controllers/SSLContextService.cpp:
##########
@@ -112,6 +112,22 @@ const core::Property SSLContextService::ClientCertKeyUsage(
         ->build());
 #endif  // WIN32
 
+namespace {
+bool is_valid_and_readable_path(const std::filesystem::path& path_to_be_tested) {
+  std::ifstream file_to_be_tested(path_to_be_tested);
+  return file_to_be_tested.good();
+}
+
+#ifdef WIN32
+std::string getCertName(const utils::tls::X509_unique_ptr& cert) {
+  const size_t BUFFER_SIZE = 256;
+  char name_buffer[BUFFER_SIZE];
+  X509_NAME_oneline(X509_get_subject_name(cert.get()), name_buffer, BUFFER_SIZE);

Review Comment:
   according to [this](https://www.openssl.org/docs/man3.1/man3/X509_NAME_oneline.html) the usage of `X509_NAME_oneline` is "strongly discouraged in new applications" although not yet deprecated



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1254148221


##########
thirdparty/openssl/Tidy-up-aarch64-feature-detection-code-in-armcap.c.patch:
##########


Review Comment:
   Is this patch licensed in a way that allows us to use and redistribute it under Apache License? 



##########
thirdparty/libwebsockets/openssl3.patch:
##########


Review Comment:
   What's the source of this patch?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1230573291


##########
README.md:
##########
@@ -151,6 +151,10 @@ and rebuild.
 * Lua and development headers -- Required if Lua support is enabled
 * libgps-dev -- Required if building libGPS support
 * Zlib headers
+* perl -- Required for OpenSSL configuration
+* NASM -- Required for OpenSSL only on Windows
+
+**NOTE** On Windows if Strawberry Perl is used the `${StrawberryPerlRoot}\c\bin` directory should not be part of the %PATH% variable as Strawberry Perl's patch.exe will be found as the patch executable in the configure phase instead if the git patch executable.

Review Comment:
   Updated in 1a573202fd6a65332b4f7d8e8f3b6b3ecac2673a



##########
NOTICE:
##########


Review Comment:
   Updated in 1a573202fd6a65332b4f7d8e8f3b6b3ecac2673a



##########
cmake/GoogleCloudCpp.cmake:
##########
@@ -40,3 +40,9 @@ FetchContent_MakeAvailable(google-cloud-cpp)
 if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang" AND CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL "14.0.0" )
     target_compile_options(google_cloud_cpp_common PUBLIC -Wno-error=deprecated-pragma)
 endif()
+
+if (WIN32)
+    target_compile_options(google_cloud_cpp_storage PUBLIC /wd4996)
+else()
+    target_compile_options(google_cloud_cpp_storage PUBLIC -Wno-error=deprecated-declarations)

Review Comment:
   Added in 1a573202fd6a65332b4f7d8e8f3b6b3ecac2673a



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1229850078


##########
bootstrap.sh:
##########
@@ -338,6 +338,9 @@ add_option PROCFS_ENABLED ${TRUE} "ENABLE_PROCFS"
 
 add_option PROMETHEUS_ENABLED ${FALSE} "ENABLE_PROMETHEUS"
 
+add_option OPENSSL_ENABLED ${TRUE} "OPENSSL_OFF"
+add_dependency OPENSSL_ENABLED "opensslbuild"

Review Comment:
   "openssl" is already listed in some of the files like in aptitude.sh as a direct dependency, so I used the same pattern used the same pattern used for JNI where it is called "jnibuild" for the JNI's build dependencies.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#issuecomment-1584614132

   There was an issue with `HTTPClient` when specifying a TLS version in `ListenHTTPTests` with versions lower that 1.2. The version specification was ignored due to TLS versions below v1.2 are not supported by default in OpenSSL 3: https://github.com/openssl/openssl/issues/13299#issuecomment-721955354
   
   To support older TLS versions the security level has to be lowered to 0 in the SSL context. If the security level is set to the default value, the curl option changing the minimum and maximum TLS versions are ignored. As the SSL context of the `CURL` object cannot be directly modified it defaults to security level 1 and the `CURLOPT_SSLVERSION` change does not take effect. To circumvent this issue we can modify the security level and the TLS version inside the function set in `CURLOPT_SSL_CTX_FUNCTION` option where the `SSL_CTX` object is available. This change was introduced in 67eb58112fe948432498c39cae18aec78251fd22


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "adamdebreceni (via GitHub)" <gi...@apache.org>.
adamdebreceni commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1219166825


##########
extensions/standard-processors/tests/integration/TLSServerSocketSupportedProtocolsTest.cpp:
##########
@@ -194,21 +195,21 @@ class SimpleSSLTestClient  {
 class SimpleSSLTestClientTLSv1  : public SimpleSSLTestClient {
  public:
   SimpleSSLTestClientTLSv1(const std::string& host, const std::string& port)
-      : SimpleSSLTestClient(TLSv1_client_method(), host, port) {
+      : SimpleSSLTestClient(SSL_OP_NO_SSLv3 | SSL_OP_NO_TLSv1_2 | SSL_OP_NO_TLSv1_1 | SSL_OP_NO_TLSv1_3, host, port) {

Review Comment:
   [this](https://www.openssl.org/docs/man3.1/man3/SSL_CTX_set_options.html) documentation claims that these are deprecated and that `SSL_CTX_set_min_proto_version` and `SSL_CTX_set_max_proto_version` should be used instead



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1229788131


##########
README.md:
##########
@@ -151,6 +151,10 @@ and rebuild.
 * Lua and development headers -- Required if Lua support is enabled
 * libgps-dev -- Required if building libGPS support
 * Zlib headers
+* perl -- Required for OpenSSL configuration
+* NASM -- Required for OpenSSL only on Windows
+
+**NOTE** On Windows if Strawberry Perl is used the `${StrawberryPerlRoot}\c\bin` directory should not be part of the %PATH% variable as Strawberry Perl's patch.exe will be found as the patch executable in the configure phase instead if the git patch executable.

Review Comment:
   Cpack is used for MSI generation. It builds WIX files and calls WIX toolset tools to create an MSI.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1246256189


##########
extensions/standard-processors/processors/HashContent.h:
##########
@@ -49,21 +48,25 @@ namespace { // NOLINT
     HashReturnType ret_val;
     ret_val.second = 0;
     std::array<std::byte, HASH_BUFFER_SIZE> buffer{};
-    MD5_CTX context;
-    MD5_Init(&context);
+    EVP_MD_CTX *context = EVP_MD_CTX_new();
+    const auto guard = gsl::finally([&context]() {
+      EVP_MD_CTX_free(context);
+    });
+    const EVP_MD *md = EVP_md5();
+    EVP_DigestInit_ex(context, md, nullptr);

Review Comment:
   Is there any reason not to inline the digest type call?
   ```suggestion
       EVP_DigestInit_ex(context, EVP_md5(), nullptr);
   ```



##########
libminifi/src/core/state/Value.cpp:
##########
@@ -34,25 +34,29 @@ const std::type_index Value::BOOL_TYPE = std::type_index(typeid(bool));
 const std::type_index Value::DOUBLE_TYPE = std::type_index(typeid(double));
 const std::type_index Value::STRING_TYPE = std::type_index(typeid(std::string));
 
-void hashNode(const SerializedResponseNode& node, SHA512_CTX& ctx) {
-  SHA512_Update(&ctx, node.name.c_str(), node.name.length());
+void hashNode(const SerializedResponseNode& node, EVP_MD_CTX* ctx) {

Review Comment:
   If ctx is expected to be valid/not-null, then keeping it as a reference would document this on the interface.



##########
extensions/standard-processors/processors/HashContent.h:
##########
@@ -49,21 +48,25 @@ namespace { // NOLINT
     HashReturnType ret_val;
     ret_val.second = 0;
     std::array<std::byte, HASH_BUFFER_SIZE> buffer{};
-    MD5_CTX context;
-    MD5_Init(&context);
+    EVP_MD_CTX *context = EVP_MD_CTX_new();
+    const auto guard = gsl::finally([&context]() {
+      EVP_MD_CTX_free(context);
+    });
+    const EVP_MD *md = EVP_md5();
+    EVP_DigestInit_ex(context, md, nullptr);
 
     size_t ret = 0;
     do {
       ret = stream->read(buffer);
       if (ret > 0) {
-        MD5_Update(&context, buffer.data(), ret);
+        EVP_DigestUpdate(context, buffer.data(), ret);
         ret_val.second += gsl::narrow<int64_t>(ret);
       }
     } while (ret > 0);
 
     if (ret_val.second > 0) {
-      std::array<std::byte, MD5_DIGEST_LENGTH> digest{};
-      MD5_Final(reinterpret_cast<unsigned char*>(digest.data()), &context);
+      std::array<std::byte, EVP_MAX_MD_SIZE> digest{};
+      EVP_DigestFinal_ex(context, reinterpret_cast<unsigned char*>(digest.data()), nullptr);

Review Comment:
   The array is sized for SHA512 now. Did you verify that the resulting hash is trimmed to MD5 size?



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1254160568


##########
thirdparty/openssl/Tidy-up-aarch64-feature-detection-code-in-armcap.c.patch:
##########


Review Comment:
   As I understood this patch is [already merged](https://github.com/openssl/openssl/pull/20305) to OpenSSL which is under Apache License, but not yet released, but maybe @martinzink has checked this when opening the PR for it and have more information on 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits closed pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits closed pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1583: MINIFICPP-1719 Replace LibreSSL with OpenSSL 3.1

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1583:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1583#discussion_r1222986101


##########
README.md:
##########
@@ -151,6 +151,10 @@ and rebuild.
 * Lua and development headers -- Required if Lua support is enabled
 * libgps-dev -- Required if building libGPS support
 * Zlib headers
+* perl -- Required for OpenSSL configuration
+* NASM -- Required for OpenSSL only on Windows
+
+**NOTE** On Windows if Strawberry Perl is used the `${StrawberryPerlRoot}\c\bin` directory should not be part of the %PATH% variable as Strawberry Perl's patch.exe will be found as the patch executable in the configure phase instead if the git patch executable.

Review Comment:
   If we're adding warnings like this, we should add another one about Chocolatey's cpack conflicting with CMake cpack. One could also solve the problem by moving the build utils' PATH entries before the conflicting ones, at least in the terminals used to build.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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