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 2022/07/15 09:58:30 UTC

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request, #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

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

   https://issues.apache.org/jira/browse/MINIFICPP-1866
   
   ------------------------------------
   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] martinzink commented on a diff in pull request #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r925340171


##########
extensions/standard-processors/processors/NetworkListenerProcessor.cpp:
##########
@@ -37,7 +38,8 @@ void NetworkListenerProcessor::onTrigger(const std::shared_ptr<core::ProcessCont
 }
 
 void NetworkListenerProcessor::startServer(

Review Comment:
   Based on our conversation I think we should split this to startTCP/UDPServer and remove the arguments and create virtual functions to get max_queue_size, port, etc.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r947892457


##########
extensions/civetweb/tests/ListenHTTPTests.cpp:
##########
@@ -60,15 +60,15 @@ class ListenHTTPTestsFixture {
     LogTestController::getInstance().setTrace<minifi::processors::ListenHTTP>();
     LogTestController::getInstance().setTrace<minifi::processors::ListenHTTP::Handler>();
     LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
-    LogTestController::getInstance().setDebug<utils::HTTPClient>();

Review Comment:
   The minifi prefix is not necessary, since we're in the minifi namespace.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r949165864


##########
libminifi/include/utils/net/SessionHandlingServer.h:
##########
@@ -0,0 +1,67 @@
+/**
+ * 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.
+ */
+#pragma once
+
+#include <utility>
+#include <memory>
+
+#include "Server.h"
+#include "asio/ssl.hpp"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+template<typename SessionType>
+class SessionHandlingServer : public Server {
+ public:
+  SessionHandlingServer(std::optional<size_t> max_queue_size, uint16_t port, std::shared_ptr<core::logging::Logger> logger)
+      : Server(max_queue_size, std::move(logger)),
+        acceptor_(io_context_, asio::ip::tcp::endpoint(asio::ip::tcp::v4(), port)) {
+  }
+
+  void run() override {
+    startAccept();
+    Server::run();
+  }
+
+ protected:
+  void startAccept() {
+    auto new_session = createSession();
+    acceptor_.async_accept(new_session->getSocket(),
+                           [this, new_session](const auto& error_code) -> void {
+                             handleAccept(new_session, error_code);
+                           });

Review Comment:
   ok, let's leave it then



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r926601399


##########
PROCESSORS.md:
##########
@@ -1292,11 +1293,13 @@ Listens for incoming TCP connections and reads data from each connection using a
 
 In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
 
-| Name                          | Default Value | Allowable Values | Description                                                                                                                                                                                   |
-|-------------------------------|---------------|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| **Listening Port**            |               |                  | The port to listen on for communication.                                                                                                                                                      |
-| **Max Batch Size**            | 500           |                  | The maximum number of messages to process at a time.                                                                                                                                          |
-| **Max Size of Message Queue** | 10000         |                  | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| Name                          | Default Value | Allowable Values           | Description                                                                                                                                                                                      |
+|-------------------------------|---------------|----------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| **Listening Port**            |               |                            | The port to listen on for communication.                                                                                                                                                         |
+| **Max Batch Size**            | 500           |                            | The maximum number of messages to process at a time.                                                                                                                                             |
+| **Max Size of Message Queue** | 10000         |                            | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| SSL Context Service           |               |                            | The Controller Service to use in order to obtain an SSL Context. If this property is set, messages will be received over a secure connection.                                                    |
+| Client Auth                   | NONE          | NONE<br/>WANT<br/>REQUIRED | The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.                                                                                    |

Review Comment:
   The client could use a different certificate than the one used for the server side authentication. It's more rare to have a client side certificate configured as well. Currently NiFi has these same options for Client Auth property.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948126915


##########
libminifi/test/Utils.h:
##########
@@ -135,3 +142,37 @@ struct FlowFileQueueTestAccessor {
   FIELD_ACCESSOR(load_task_);
   FIELD_ACCESSOR(queue_);
 };
+
+bool sendMessagesViaSSL(const std::vector<std::string_view>& contents, uint64_t port, const std::string& ca_cert_path, const std::optional<minifi::utils::net::SslData>& ssl_data = std::nullopt) {
+  asio::ssl::context ctx(asio::ssl::context::sslv23);
+  ctx.load_verify_file(ca_cert_path);
+  if (ssl_data) {
+    ctx.set_verify_mode(asio::ssl::verify_peer);
+    ctx.use_certificate_file(ssl_data->cert_loc, asio::ssl::context::pem);
+    ctx.use_private_key_file(ssl_data->key_loc, asio::ssl::context::pem);

Review Comment:
   Added in f3fb107052af05c62635ba3d4a9c1906c714d6fe



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r926789874


##########
extensions/standard-processors/processors/ListenSyslog.h:
##########
@@ -45,13 +45,15 @@ class ListenSyslog : public NetworkListenerProcessor {
   EXTENSIONAPI static const core::Property MaxBatchSize;
   EXTENSIONAPI static const core::Property ParseMessages;
   EXTENSIONAPI static const core::Property MaxQueueSize;
+  EXTENSIONAPI static const core::Property SSLContextService;

Review Comment:
   Updated in d0e2ac6c3abeff8d46f8e15fd80816dd86fa9b13



##########
extensions/standard-processors/processors/NetworkListenerProcessor.cpp:
##########
@@ -37,7 +38,8 @@ void NetworkListenerProcessor::onTrigger(const std::shared_ptr<core::ProcessCont
 }
 
 void NetworkListenerProcessor::startServer(

Review Comment:
   Updated in d0e2ac6c3abeff8d46f8e15fd80816dd86fa9b13



##########
PROCESSORS.md:
##########
@@ -1292,11 +1293,13 @@ Listens for incoming TCP connections and reads data from each connection using a
 
 In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
 
-| Name                          | Default Value | Allowable Values | Description                                                                                                                                                                                   |
-|-------------------------------|---------------|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| **Listening Port**            |               |                  | The port to listen on for communication.                                                                                                                                                      |
-| **Max Batch Size**            | 500           |                  | The maximum number of messages to process at a time.                                                                                                                                          |
-| **Max Size of Message Queue** | 10000         |                  | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| Name                          | Default Value | Allowable Values           | Description                                                                                                                                                                                      |
+|-------------------------------|---------------|----------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| **Listening Port**            |               |                            | The port to listen on for communication.                                                                                                                                                         |
+| **Max Batch Size**            | 500           |                            | The maximum number of messages to process at a time.                                                                                                                                             |
+| **Max Size of Message Queue** | 10000         |                            | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| SSL Context Service           |               |                            | The Controller Service to use in order to obtain an SSL Context. If this property is set, messages will be received over a secure connection.                                                    |
+| Client Auth                   | NONE          | NONE<br/>WANT<br/>REQUIRED | The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.                                                                                    |
 

Review Comment:
   Updated in d0e2ac6c3abeff8d46f8e15fd80816dd86fa9b13



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r947892457


##########
extensions/civetweb/tests/ListenHTTPTests.cpp:
##########
@@ -60,15 +60,15 @@ class ListenHTTPTestsFixture {
     LogTestController::getInstance().setTrace<minifi::processors::ListenHTTP>();
     LogTestController::getInstance().setTrace<minifi::processors::ListenHTTP::Handler>();
     LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
-    LogTestController::getInstance().setDebug<utils::HTTPClient>();

Review Comment:
   The minifi prefix is not necessary, since we're in the minifi namespace. Please check the other changes, too. I see a bunch of similar changes, but didn't check separately for each file.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948553599


##########
libminifi/include/utils/net/SessionHandlingServer.h:
##########
@@ -0,0 +1,67 @@
+/**
+ * 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.
+ */
+#pragma once
+
+#include <utility>
+#include <memory>
+
+#include "Server.h"
+#include "asio/ssl.hpp"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+template<typename SessionType>
+class SessionHandlingServer : public Server {
+ public:
+  SessionHandlingServer(std::optional<size_t> max_queue_size, uint16_t port, std::shared_ptr<core::logging::Logger> logger)
+      : Server(max_queue_size, std::move(logger)),
+        acceptor_(io_context_, asio::ip::tcp::endpoint(asio::ip::tcp::v4(), port)) {
+  }
+
+  void run() override {
+    startAccept();
+    Server::run();
+  }
+
+ protected:
+  void startAccept() {
+    auto new_session = createSession();
+    acceptor_.async_accept(new_session->getSocket(),
+                           [this, new_session](const auto& error_code) -> void {
+                             handleAccept(new_session, error_code);
+                           });

Review Comment:
   Did you query the socket in a separate statement? Because the evaluation order of function arguments are unspecified, so if you don't extract the socket into a separate variable, you can easily move from new_session (as part of the lambda expression) before calling getSocket 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] adamdebreceni commented on a diff in pull request #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r930973465


##########
libminifi/test/Utils.h:
##########
@@ -115,6 +119,43 @@ void sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t
     tcp_message += '\n';
     socket.send(asio::buffer(tcp_message, tcp_message.size()), 0, err);

Review Comment:
   similarly to `write_some`, `send` might also send less bytes than supplied



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948021879


##########
extensions/civetweb/tests/ListenHTTPTests.cpp:
##########
@@ -60,15 +60,15 @@ class ListenHTTPTestsFixture {
     LogTestController::getInstance().setTrace<minifi::processors::ListenHTTP>();
     LogTestController::getInstance().setTrace<minifi::processors::ListenHTTP::Handler>();
     LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
-    LogTestController::getInstance().setDebug<utils::HTTPClient>();

Review Comment:
   You're right, I was wrong about the name lookup rules, sorry.



##########
libminifi/include/utils/net/SessionHandlingServer.h:
##########
@@ -0,0 +1,67 @@
+/**
+ * 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.
+ */
+#pragma once
+
+#include <utility>
+#include <memory>
+
+#include "Server.h"
+#include "asio/ssl.hpp"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+template<typename SessionType>
+class SessionHandlingServer : public Server {
+ public:
+  SessionHandlingServer(std::optional<size_t> max_queue_size, uint16_t port, std::shared_ptr<core::logging::Logger> logger)
+      : Server(max_queue_size, std::move(logger)),
+        acceptor_(io_context_, asio::ip::tcp::endpoint(asio::ip::tcp::v4(), port)) {
+  }
+
+  void run() override {
+    startAccept();
+    Server::run();
+  }
+
+ protected:
+  void startAccept() {
+    auto new_session = createSession();
+    acceptor_.async_accept(new_session->getSocket(),
+                           [this, new_session](const auto& error_code) -> void {
+                             handleAccept(new_session, error_code);
+                           });

Review Comment:
   Shouldn't the session be moved into the lambda? After querying the socket, it's no longer used outside. It could be moved further into handleAccept later, on the accepting thread. This would allow it to not be a shared_ptr.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948147376


##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -439,12 +439,13 @@ TEST_CASE("Test ListenSyslog via TCP with SSL connection", "[ListenSyslog]") {
 
   SingleProcessorTestController controller{listen_syslog};
   auto ssl_context_service = controller.plan->addController("SSLContextService", "SSLContextService");
+  const auto executable_dir = minifi::utils::file::FileUtils::get_executable_dir();
   REQUIRE(controller.plan->setProperty(ssl_context_service, controllers::SSLContextService::CACertificate.getName(),
-    minifi::utils::file::FileUtils::get_executable_dir() + "/resources/ca_cert.crt"));
+    minifi::utils::file::concat_path(executable_dir, "resources/ca_cert.crt")));

Review Comment:
   We need `concat_path` between `resources` and `ca_cert.crt`, too.  Maybe it's time for a 3-parameter (or variadic) `concat_path`?



-- 
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] martinzink commented on a diff in pull request #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r925329712


##########
extensions/standard-processors/processors/ListenSyslog.h:
##########
@@ -45,13 +45,15 @@ class ListenSyslog : public NetworkListenerProcessor {
   EXTENSIONAPI static const core::Property MaxBatchSize;
   EXTENSIONAPI static const core::Property ParseMessages;
   EXTENSIONAPI static const core::Property MaxQueueSize;
+  EXTENSIONAPI static const core::Property SSLContextService;

Review Comment:
   I think we could  `EXTENSIONAPI static const core::Property ClientAuth;` here aswell.
   



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948126571


##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -434,4 +434,35 @@ TEST_CASE("ListenSyslog max queue and max batch size test", "[ListenSyslog]") {
   CHECK(controller.trigger().at(ListenSyslog::Success).empty());
 }
 
-}  // namespace org::apache::nifi::minifi::processors::testing
+TEST_CASE("Test ListenSyslog via TCP with SSL connection", "[ListenSyslog]") {
+  const auto listen_syslog = std::make_shared<ListenSyslog>("ListenSyslog");
+
+  SingleProcessorTestController controller{listen_syslog};
+  auto ssl_context_service = controller.plan->addController("SSLContextService", "SSLContextService");
+  REQUIRE(controller.plan->setProperty(ssl_context_service, controllers::SSLContextService::CACertificate.getName(),
+    minifi::utils::file::FileUtils::get_executable_dir() + "/resources/ca_cert.crt"));

Review Comment:
   Good point, updated in f3fb107052af05c62635ba3d4a9c1906c714d6fe



##########
libminifi/src/utils/net/SslServer.cpp:
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 "utils/net/SslServer.h"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+SslSession::SslSession(asio::io_context& io_context, asio::ssl::context& context, utils::ConcurrentQueue<Message>& concurrent_queue,
+    std::optional<size_t> max_queue_size, std::shared_ptr<core::logging::Logger> logger)
+  : concurrent_queue_(concurrent_queue),
+    max_queue_size_(max_queue_size),

Review Comment:
   Updated in f3fb107052af05c62635ba3d4a9c1906c714d6fe



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
fgerlits closed pull request #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370


-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
szaszm commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r923297637


##########
PROCESSORS.md:
##########
@@ -1292,11 +1293,13 @@ Listens for incoming TCP connections and reads data from each connection using a
 
 In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
 
-| Name                          | Default Value | Allowable Values | Description                                                                                                                                                                                   |
-|-------------------------------|---------------|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| **Listening Port**            |               |                  | The port to listen on for communication.                                                                                                                                                      |
-| **Max Batch Size**            | 500           |                  | The maximum number of messages to process at a time.                                                                                                                                          |
-| **Max Size of Message Queue** | 10000         |                  | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| Name                          | Default Value | Allowable Values           | Description                                                                                                                                                                                      |
+|-------------------------------|---------------|----------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| **Listening Port**            |               |                            | The port to listen on for communication.                                                                                                                                                         |
+| **Max Batch Size**            | 500           |                            | The maximum number of messages to process at a time.                                                                                                                                             |
+| **Max Size of Message Queue** | 10000         |                            | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| SSL Context Service           |               |                            | The Controller Service to use in order to obtain an SSL Context. If this property is set, messages will be received over a secure connection.                                                    |
+| Client Auth                   | NONE          | NONE<br/>WANT<br/>REQUIRED | The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.                                                                                    |

Review Comment:
   Wouldn't it make sense to use client auth whenever available in the SSL Context Service?



-- 
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] martinzink commented on a diff in pull request #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
martinzink commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r925346167


##########
PROCESSORS.md:
##########
@@ -1292,11 +1293,13 @@ Listens for incoming TCP connections and reads data from each connection using a
 
 In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
 
-| Name                          | Default Value | Allowable Values | Description                                                                                                                                                                                   |
-|-------------------------------|---------------|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| **Listening Port**            |               |                  | The port to listen on for communication.                                                                                                                                                      |
-| **Max Batch Size**            | 500           |                  | The maximum number of messages to process at a time.                                                                                                                                          |
-| **Max Size of Message Queue** | 10000         |                  | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| Name                          | Default Value | Allowable Values           | Description                                                                                                                                                                                      |
+|-------------------------------|---------------|----------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| **Listening Port**            |               |                            | The port to listen on for communication.                                                                                                                                                         |
+| **Max Batch Size**            | 500           |                            | The maximum number of messages to process at a time.                                                                                                                                             |
+| **Max Size of Message Queue** | 10000         |                            | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| SSL Context Service           |               |                            | The Controller Service to use in order to obtain an SSL Context. If this property is set, messages will be received over a secure connection.                                                    |
+| Client Auth                   | NONE          | NONE<br/>WANT<br/>REQUIRED | The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.                                                                                    |
 

Review Comment:
   I know this is not part of this PR, but I think we should list the output attributes (tcp.port, tcp.sender) here aswell similar to ListenSyslog. So the feature is apparent to the user.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948040417


##########
libminifi/src/utils/net/SslServer.cpp:
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 "utils/net/SslServer.h"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+SslSession::SslSession(asio::io_context& io_context, asio::ssl::context& context, utils::ConcurrentQueue<Message>& concurrent_queue,
+    std::optional<size_t> max_queue_size, std::shared_ptr<core::logging::Logger> logger)
+  : concurrent_queue_(concurrent_queue),
+    max_queue_size_(max_queue_size),
+    logger_(std::move(logger)),
+    socket_(io_context, context) {
+}
+
+ssl_socket::lowest_layer_type& SslSession::getSocket() {
+  return socket_.lowest_layer();
+}
+
+void SslSession::start() {
+  socket_.async_handshake(asio::ssl::stream_base::server,
+    [this, self = shared_from_this()](const std::error_code& error_code) {
+      if (error_code) {
+        logger_->log_error("Error occured during SSL handshake: (%d) %s", error_code.value(), error_code.message());
+        return;
+      }
+      asio::async_read_until(socket_,
+                             buffer_,
+                             '\n',
+                             [self](const auto& error_code, size_t) -> void {
+                               self->handleReadUntilNewLine(error_code);

Review Comment:
   I'm not sure as this was an older code, but I think shared_from_this is used due to lifetime considerations, that the session might still be alive when server is being destroyed, so we need to keep it alive until the session is finished, but this is just a guess.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948126207


##########
PROCESSORS.md:
##########
@@ -1303,11 +1305,26 @@ Listens for incoming TCP connections and reads data from each connection using a
 
 In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
 
-| Name                          | Default Value | Allowable Values | Description                                                                                                                                                                                   |
-|-------------------------------|---------------|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| **Listening Port**            |               |                  | The port to listen on for communication.                                                                                                                                                      |
-| **Max Batch Size**            | 500           |                  | The maximum number of messages to process at a time.                                                                                                                                          |
-| **Max Size of Message Queue** | 10000         |                  | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| Name                          | Default Value | Allowable Values           | Description                                                                                                                                                                                      |
+|-------------------------------|---------------|----------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| **Listening Port**            |               |                            | The port to listen on for communication.                                                                                                                                                         |
+| **Max Batch Size**            | 500           |                            | The maximum number of messages to process at a time.                                                                                                                                             |
+| **Max Size of Message Queue** | 10000         |                            | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| SSL Context Service           |               |                            | The Controller Service to use in order to obtain an SSL Context. If this property is set, messages will be received over a secure connection.                                                    |
+| Client Auth                   | NONE          | NONE<br/>WANT<br/>REQUIRED | The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.                                                                                    |
+
+### Relationships
+
+| Name    | Description                                                        |
+|---------|--------------------------------------------------------------------|
+| success | Messages received successfully will be sent out this relationship. |
+
+### Output Attributes
+
+| Attribute                | Description                                                        | Requirements           |
+|--------------------------|--------------------------------------------------------------------|------------------------|
+| _tcp.port_               | The sending host of the messages.                                  | -                      |
+| _tcp.sender_             | 	The sending port the messages were received.                      | -                      |

Review Comment:
   Good catch, updated in f3fb107052af05c62635ba3d4a9c1906c714d6fe



##########
PROCESSORS.md:
##########
@@ -1254,13 +1254,15 @@ With parsing disabled all message will be routed to the success relationship, bu
 
 In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
 
-| Name                      | Default Value | Allowable Values | Description                                                                                                                                                                                          |
-|---------------------------|---------------|------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| Listening Port            | 514           |                  | The port for Syslog communication. (Well-known ports (0-1023) require root access)                                                                                                                   |
-| Protocol                  | UDP           | UDP<br>TCP<br>   | The protocol for Syslog communication.                                                                                                                                                               |
-| Parse Messages            | false         | false<br>true    | Indicates if the processor should parse the Syslog messages. If set to false, each outgoing FlowFile will only contain the sender, protocol, and port, and no additional attributes.                 |
-| Max Batch Size            | 500           |                  | The maximum number of Syslog events to process at a time.                                                                                                                                            |
-| Max Size of Message Queue | 10000         |                  | Maximum number of Syslog messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| Name                      | Default Value | Allowable Values           | Description                                                                                                                                                                                             |
+|---------------------------|---------------|----------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| Listening Port            | 514           |                            | The port for Syslog communication. (Well-known ports (0-1023) require root access)                                                                                                                      |
+| Protocol                  | UDP           | UDP<br>TCP<br>             | The protocol for Syslog communication.                                                                                                                                                                  |
+| Parse Messages            | false         | false<br>true              | Indicates if the processor should parse the Syslog messages. If set to false, each outgoing FlowFile will only contain the sender, protocol, and port, and no additional attributes.                    |
+| Max Batch Size            | 500           |                            | The maximum number of Syslog events to process at a time.                                                                                                                                               |
+| Max Size of Message Queue | 10000         |                            | Maximum number of Syslog messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| SSL Context Service       |               |                            | The Controller Service to use in order to obtain an SSL Context. If this property is set, messages will be received over a secure connection.                                                           |

Review Comment:
   Added in f3fb107052af05c62635ba3d4a9c1906c714d6fe



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
fgerlits commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r947938125


##########
libminifi/src/utils/net/SslServer.cpp:
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 "utils/net/SslServer.h"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+SslSession::SslSession(asio::io_context& io_context, asio::ssl::context& context, utils::ConcurrentQueue<Message>& concurrent_queue,
+    std::optional<size_t> max_queue_size, std::shared_ptr<core::logging::Logger> logger)
+  : concurrent_queue_(concurrent_queue),
+    max_queue_size_(max_queue_size),

Review Comment:
   minor, but `max_queue_size` could be moved instead of copied



##########
PROCESSORS.md:
##########
@@ -1254,13 +1254,15 @@ With parsing disabled all message will be routed to the success relationship, bu
 
 In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
 
-| Name                      | Default Value | Allowable Values | Description                                                                                                                                                                                          |
-|---------------------------|---------------|------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| Listening Port            | 514           |                  | The port for Syslog communication. (Well-known ports (0-1023) require root access)                                                                                                                   |
-| Protocol                  | UDP           | UDP<br>TCP<br>   | The protocol for Syslog communication.                                                                                                                                                               |
-| Parse Messages            | false         | false<br>true    | Indicates if the processor should parse the Syslog messages. If set to false, each outgoing FlowFile will only contain the sender, protocol, and port, and no additional attributes.                 |
-| Max Batch Size            | 500           |                  | The maximum number of Syslog events to process at a time.                                                                                                                                            |
-| Max Size of Message Queue | 10000         |                  | Maximum number of Syslog messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| Name                      | Default Value | Allowable Values           | Description                                                                                                                                                                                             |
+|---------------------------|---------------|----------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| Listening Port            | 514           |                            | The port for Syslog communication. (Well-known ports (0-1023) require root access)                                                                                                                      |
+| Protocol                  | UDP           | UDP<br>TCP<br>             | The protocol for Syslog communication.                                                                                                                                                                  |
+| Parse Messages            | false         | false<br>true              | Indicates if the processor should parse the Syslog messages. If set to false, each outgoing FlowFile will only contain the sender, protocol, and port, and no additional attributes.                    |
+| Max Batch Size            | 500           |                            | The maximum number of Syslog events to process at a time.                                                                                                                                               |
+| Max Size of Message Queue | 10000         |                            | Maximum number of Syslog messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| SSL Context Service       |               |                            | The Controller Service to use in order to obtain an SSL Context. If this property is set, messages will be received over a secure connection.                                                           |

Review Comment:
   In `ListenSyslog.cpp`, there is one more sentence in the description.  Should that be included here?



##########
libminifi/test/Utils.h:
##########
@@ -135,3 +142,37 @@ struct FlowFileQueueTestAccessor {
   FIELD_ACCESSOR(load_task_);
   FIELD_ACCESSOR(queue_);
 };
+
+bool sendMessagesViaSSL(const std::vector<std::string_view>& contents, uint64_t port, const std::string& ca_cert_path, const std::optional<minifi::utils::net::SslData>& ssl_data = std::nullopt) {
+  asio::ssl::context ctx(asio::ssl::context::sslv23);
+  ctx.load_verify_file(ca_cert_path);
+  if (ssl_data) {
+    ctx.set_verify_mode(asio::ssl::verify_peer);
+    ctx.use_certificate_file(ssl_data->cert_loc, asio::ssl::context::pem);
+    ctx.use_private_key_file(ssl_data->key_loc, asio::ssl::context::pem);

Review Comment:
   Do we also need to set the password callback?



##########
libminifi/src/utils/net/SslServer.cpp:
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 "utils/net/SslServer.h"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+SslSession::SslSession(asio::io_context& io_context, asio::ssl::context& context, utils::ConcurrentQueue<Message>& concurrent_queue,
+    std::optional<size_t> max_queue_size, std::shared_ptr<core::logging::Logger> logger)
+  : concurrent_queue_(concurrent_queue),
+    max_queue_size_(max_queue_size),
+    logger_(std::move(logger)),
+    socket_(io_context, context) {
+}
+
+ssl_socket::lowest_layer_type& SslSession::getSocket() {
+  return socket_.lowest_layer();
+}
+
+void SslSession::start() {
+  socket_.async_handshake(asio::ssl::stream_base::server,
+    [this, self = shared_from_this()](const std::error_code& error_code) {
+      if (error_code) {
+        logger_->log_error("Error occured during SSL handshake: (%d) %s", error_code.value(), error_code.message());
+        return;
+      }
+      asio::async_read_until(socket_,
+                             buffer_,
+                             '\n',
+                             [self](const auto& error_code, size_t) -> void {
+                               self->handleReadUntilNewLine(error_code);

Review Comment:
   Why do we need `self` as a capture?  Could we use `this` here instead?



##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -434,4 +434,35 @@ TEST_CASE("ListenSyslog max queue and max batch size test", "[ListenSyslog]") {
   CHECK(controller.trigger().at(ListenSyslog::Success).empty());
 }
 
-}  // namespace org::apache::nifi::minifi::processors::testing
+TEST_CASE("Test ListenSyslog via TCP with SSL connection", "[ListenSyslog]") {
+  const auto listen_syslog = std::make_shared<ListenSyslog>("ListenSyslog");
+
+  SingleProcessorTestController controller{listen_syslog};
+  auto ssl_context_service = controller.plan->addController("SSLContextService", "SSLContextService");
+  REQUIRE(controller.plan->setProperty(ssl_context_service, controllers::SSLContextService::CACertificate.getName(),
+    minifi::utils::file::FileUtils::get_executable_dir() + "/resources/ca_cert.crt"));

Review Comment:
   Does this work on Windows?  Probably better to use `utils::file::concat_path()`.



##########
PROCESSORS.md:
##########
@@ -1303,11 +1305,26 @@ Listens for incoming TCP connections and reads data from each connection using a
 
 In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered optional. The table also indicates any default values, and whether a property supports the NiFi Expression Language.
 
-| Name                          | Default Value | Allowable Values | Description                                                                                                                                                                                   |
-|-------------------------------|---------------|------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| **Listening Port**            |               |                  | The port to listen on for communication.                                                                                                                                                      |
-| **Max Batch Size**            | 500           |                  | The maximum number of messages to process at a time.                                                                                                                                          |
-| **Max Size of Message Queue** | 10000         |                  | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| Name                          | Default Value | Allowable Values           | Description                                                                                                                                                                                      |
+|-------------------------------|---------------|----------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| **Listening Port**            |               |                            | The port to listen on for communication.                                                                                                                                                         |
+| **Max Batch Size**            | 500           |                            | The maximum number of messages to process at a time.                                                                                                                                             |
+| **Max Size of Message Queue** | 10000         |                            | Maximum number of messages allowed to be buffered before processing them when the processor is triggered. If the buffer is full, the message is ignored. If set to zero the buffer is unlimited. |
+| SSL Context Service           |               |                            | The Controller Service to use in order to obtain an SSL Context. If this property is set, messages will be received over a secure connection.                                                    |
+| Client Auth                   | NONE          | NONE<br/>WANT<br/>REQUIRED | The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.                                                                                    |
+
+### Relationships
+
+| Name    | Description                                                        |
+|---------|--------------------------------------------------------------------|
+| success | Messages received successfully will be sent out this relationship. |
+
+### Output Attributes
+
+| Attribute                | Description                                                        | Requirements           |
+|--------------------------|--------------------------------------------------------------------|------------------------|
+| _tcp.port_               | The sending host of the messages.                                  | -                      |
+| _tcp.sender_             | 	The sending port the messages were received.                      | -                      |

Review Comment:
   The descriptions look like they should be switched.  Also, there are some whitespace issues in line 1327.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948129229


##########
libminifi/include/utils/net/SessionHandlingServer.h:
##########
@@ -0,0 +1,67 @@
+/**
+ * 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.
+ */
+#pragma once
+
+#include <utility>
+#include <memory>
+
+#include "Server.h"
+#include "asio/ssl.hpp"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+template<typename SessionType>
+class SessionHandlingServer : public Server {
+ public:
+  SessionHandlingServer(std::optional<size_t> max_queue_size, uint16_t port, std::shared_ptr<core::logging::Logger> logger)
+      : Server(max_queue_size, std::move(logger)),
+        acceptor_(io_context_, asio::ip::tcp::endpoint(asio::ip::tcp::v4(), port)) {
+  }
+
+  void run() override {
+    startAccept();
+    Server::run();
+  }
+
+ protected:
+  void startAccept() {
+    auto new_session = createSession();
+    acceptor_.async_accept(new_session->getSocket(),
+                           [this, new_session](const auto& error_code) -> void {
+                             handleAccept(new_session, error_code);
+                           });

Review Comment:
   Sounds logical, but for some reason changing it to unique_ptr causes a segmentation fault in the asio library. I wasn't able to figure out what caused it so far.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r930969826


##########
libminifi/test/Utils.h:
##########
@@ -115,6 +119,43 @@ void sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t
     tcp_message += '\n';
     socket.send(asio::buffer(tcp_message, tcp_message.size()), 0, err);
   }
-  REQUIRE(!err);
+  if (err) {
+    return false;
+  }
   socket.close();
+  return true;
 }
+
+bool sendMessagesViaSSL(const std::vector<std::string_view>& contents, uint64_t port, const std::string& ca_cert_path, const std::optional<minifi::utils::net::SslData>& ssl_data = std::nullopt) {
+  asio::ssl::context ctx(asio::ssl::context::sslv23);
+  ctx.load_verify_file(ca_cert_path);
+  if (ssl_data) {
+    ctx.set_verify_mode(asio::ssl::verify_peer);
+    ctx.use_certificate_file(ssl_data->cert_loc, asio::ssl::context::pem);
+    ctx.use_private_key_file(ssl_data->key_loc, asio::ssl::context::pem);
+  }
+  asio::io_context io_context;
+  asio::ssl::stream<asio::ip::tcp::socket> socket(io_context, ctx);
+  asio::ip::tcp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
+  asio::error_code err;
+  socket.lowest_layer().connect(remote_endpoint, err);
+  if (err) {
+    return false;
+  }
+  socket.handshake(asio::ssl::stream_base::client, err);
+  if (err) {
+    return false;
+  }
+  for (auto& content : contents) {
+    std::string tcp_message(content);
+    tcp_message += '\n';
+    socket.write_some(asio::buffer(tcp_message, tcp_message.size()));

Review Comment:
   `write_some` does not seem to guarantee that all bytes are written (it returns the number of written bytes), the docs mention a `write` function to achieve complete write (could not find 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] lordgamez commented on a diff in pull request #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r931052677


##########
libminifi/test/Utils.h:
##########
@@ -115,6 +119,43 @@ void sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t
     tcp_message += '\n';
     socket.send(asio::buffer(tcp_message, tcp_message.size()), 0, err);
   }
-  REQUIRE(!err);
+  if (err) {
+    return false;
+  }
   socket.close();
+  return true;
 }
+
+bool sendMessagesViaSSL(const std::vector<std::string_view>& contents, uint64_t port, const std::string& ca_cert_path, const std::optional<minifi::utils::net::SslData>& ssl_data = std::nullopt) {
+  asio::ssl::context ctx(asio::ssl::context::sslv23);
+  ctx.load_verify_file(ca_cert_path);
+  if (ssl_data) {
+    ctx.set_verify_mode(asio::ssl::verify_peer);
+    ctx.use_certificate_file(ssl_data->cert_loc, asio::ssl::context::pem);
+    ctx.use_private_key_file(ssl_data->key_loc, asio::ssl::context::pem);
+  }
+  asio::io_context io_context;
+  asio::ssl::stream<asio::ip::tcp::socket> socket(io_context, ctx);
+  asio::ip::tcp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
+  asio::error_code err;
+  socket.lowest_layer().connect(remote_endpoint, err);
+  if (err) {
+    return false;
+  }
+  socket.handshake(asio::ssl::stream_base::client, err);
+  if (err) {
+    return false;
+  }
+  for (auto& content : contents) {
+    std::string tcp_message(content);
+    tcp_message += '\n';
+    socket.write_some(asio::buffer(tcp_message, tcp_message.size()));

Review Comment:
   Updated in 5d55facd7c7f853ed099d3bef89d0a0329b23043



##########
libminifi/test/Utils.h:
##########
@@ -115,6 +119,43 @@ void sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t
     tcp_message += '\n';
     socket.send(asio::buffer(tcp_message, tcp_message.size()), 0, err);

Review Comment:
   Updated in 5d55facd7c7f853ed099d3bef89d0a0329b23043



##########
libminifi/src/utils/net/Ssl.cpp:
##########
@@ -0,0 +1,53 @@
+/**
+ * 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 "utils/net/Ssl.h"
+#include "controllers/SSLContextService.h"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+std::optional<utils::net::SslData> getSslData(const core::ProcessContext& context, const core::Property& ssl_prop, const std::shared_ptr<core::logging::Logger>& logger) {
+  auto getSslContextService = [&]() -> std::shared_ptr<minifi::controllers::SSLContextService> {
+    if (auto ssl_service_name = context.getProperty(ssl_prop); ssl_service_name && !ssl_service_name->empty()) {
+      if (auto service = context.getControllerService(*ssl_service_name)) {
+        if (auto ssl_service = std::dynamic_pointer_cast<org::apache::nifi::minifi::controllers::SSLContextService>(service)) {
+          return ssl_service;
+        } else {
+          logger->log_warn("SSL Context Service property is set to '%s', but it is not a valid SSLContextService.", *ssl_service_name);
+        }
+      } else {
+        logger->log_warn("SSL Context Service property is set to '%s', but the controller service could not be found.", *ssl_service_name);
+      }
+    } else {
+      logger->log_warn("No valid SSL Context Service property is set.");
+    }
+    return nullptr;
+  };
+
+  if (auto service = getSslContextService()) {
+    auto ssl_service = std::static_pointer_cast<minifi::controllers::SSLContextService>(service);

Review Comment:
   Updated in 5d55facd7c7f853ed099d3bef89d0a0329b23043



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r947909188


##########
extensions/civetweb/tests/ListenHTTPTests.cpp:
##########
@@ -60,15 +60,15 @@ class ListenHTTPTestsFixture {
     LogTestController::getInstance().setTrace<minifi::processors::ListenHTTP>();
     LogTestController::getInstance().setTrace<minifi::processors::ListenHTTP::Handler>();
     LogTestController::getInstance().setDebug<minifi::processors::LogAttribute>();
-    LogTestController::getInstance().setDebug<utils::HTTPClient>();

Review Comment:
   Those changes were actually needed, as we now have a header using `minifi::test::utils` namespace included and the tests are in the `minifi::test` namespace. Due to this the compiler expects HTTPClient to be in the `minifi::test::utils` namespace if we only specify it as `utils::HTTPClient`. It returns a compiler error: `error: ‘HTTPClient’ is not a member of ‘org::apache::nifi::minifi::test::utils’; did you mean ‘org::apache::nifi::minifi::utils::HTTPClient’`



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948798024


##########
libminifi/include/utils/net/SessionHandlingServer.h:
##########
@@ -0,0 +1,67 @@
+/**
+ * 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.
+ */
+#pragma once
+
+#include <utility>
+#include <memory>
+
+#include "Server.h"
+#include "asio/ssl.hpp"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+template<typename SessionType>
+class SessionHandlingServer : public Server {
+ public:
+  SessionHandlingServer(std::optional<size_t> max_queue_size, uint16_t port, std::shared_ptr<core::logging::Logger> logger)
+      : Server(max_queue_size, std::move(logger)),
+        acceptor_(io_context_, asio::ip::tcp::endpoint(asio::ip::tcp::v4(), port)) {
+  }
+
+  void run() override {
+    startAccept();
+    Server::run();
+  }
+
+ protected:
+  void startAccept() {
+    auto new_session = createSession();
+    acceptor_.async_accept(new_session->getSocket(),
+                           [this, new_session](const auto& error_code) -> void {
+                             handleAccept(new_session, error_code);
+                           });

Review Comment:
   You are right, that caused this issue. Unfortunately there is another problem with not using `shared_ptr` here. If we use `unique_ptr` instead `TcpSession::start` fails with `bad_weak_ptr` when it tries to use `shared_from_this()`.



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
lordgamez commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r948822004


##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -439,12 +439,13 @@ TEST_CASE("Test ListenSyslog via TCP with SSL connection", "[ListenSyslog]") {
 
   SingleProcessorTestController controller{listen_syslog};
   auto ssl_context_service = controller.plan->addController("SSLContextService", "SSLContextService");
+  const auto executable_dir = minifi::utils::file::FileUtils::get_executable_dir();
   REQUIRE(controller.plan->setProperty(ssl_context_service, controllers::SSLContextService::CACertificate.getName(),
-    minifi::utils::file::FileUtils::get_executable_dir() + "/resources/ca_cert.crt"));
+    minifi::utils::file::concat_path(executable_dir, "resources/ca_cert.crt")));

Review Comment:
   I checked on Windows and it works with the current solution, it can evaluate the path this way as well, it is similarly done in the SFTP tests. It is a good idea to change the concat_path to have variadic arguments, but I think it should be done outside the scope of this PR. I created a separate Jira ticket for this task https://issues.apache.org/jira/browse/MINIFICPP-1911



-- 
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 #1370: MINIFICPP-1866 Secure connection for ListenSyslog, ListenTCP

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on code in PR #1370:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1370#discussion_r931015557


##########
libminifi/src/utils/net/Ssl.cpp:
##########
@@ -0,0 +1,53 @@
+/**
+ * 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 "utils/net/Ssl.h"
+#include "controllers/SSLContextService.h"
+
+namespace org::apache::nifi::minifi::utils::net {
+
+std::optional<utils::net::SslData> getSslData(const core::ProcessContext& context, const core::Property& ssl_prop, const std::shared_ptr<core::logging::Logger>& logger) {
+  auto getSslContextService = [&]() -> std::shared_ptr<minifi::controllers::SSLContextService> {
+    if (auto ssl_service_name = context.getProperty(ssl_prop); ssl_service_name && !ssl_service_name->empty()) {
+      if (auto service = context.getControllerService(*ssl_service_name)) {
+        if (auto ssl_service = std::dynamic_pointer_cast<org::apache::nifi::minifi::controllers::SSLContextService>(service)) {
+          return ssl_service;
+        } else {
+          logger->log_warn("SSL Context Service property is set to '%s', but it is not a valid SSLContextService.", *ssl_service_name);
+        }
+      } else {
+        logger->log_warn("SSL Context Service property is set to '%s', but the controller service could not be found.", *ssl_service_name);
+      }
+    } else {
+      logger->log_warn("No valid SSL Context Service property is set.");
+    }
+    return nullptr;
+  };
+
+  if (auto service = getSslContextService()) {
+    auto ssl_service = std::static_pointer_cast<minifi::controllers::SSLContextService>(service);

Review Comment:
   `getSslContextService` seems to return a `std::shared_ptr<SSLContextService>`, the static cast might not be necessary 



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