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/08/30 07:44:57 UTC

[GitHub] [nifi-minifi-cpp] martinzink opened a new pull request, #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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

   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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,48 +98,48 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(udp::v4(), hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
+  };
+
+  const auto debug_log_resolved_endpoint = [&hostname, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> udp::endpoint {
+    if (logger->should_log(core::logging::LOG_LEVEL::debug))
+      core::logging::LOG_DEBUG(logger) << "resolved " << hostname << " to: " << resolved_query->endpoint();
+    return resolved_query->endpoint();

Review Comment:
   I've reworked this part (and also ListenTCP, ListenSyslog) in https://github.com/apache/nifi-minifi-cpp/pull/1412/commits/aba9c5bb58f317143a6301e1552087c8f6d4a5fa



##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,48 +98,48 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(udp::v4(), hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
+  };
+
+  const auto debug_log_resolved_endpoint = [&hostname, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> udp::endpoint {
+    if (logger->should_log(core::logging::LOG_LEVEL::debug))
+      core::logging::LOG_DEBUG(logger) << "resolved " << hostname << " to: " << resolved_query->endpoint();
+    return resolved_query->endpoint();
   };
 
-  const auto debug_log_resolved_names = [&, this](const addrinfo& names) -> decltype(auto) {
-    if (logger_->should_log(core::logging::LOG_LEVEL::debug)) {
-      std::vector<std::string> names_vector;
-      for (const addrinfo* it = &names; it; it = it->ai_next) {
-        names_vector.push_back(nonthrowing_sockaddr_ntop(it->ai_addr));
-      }
-      logger_->log_debug("resolved \'%s\' to: %s",
-          hostname,
-          names_vector | ranges::views::join(',') | ranges::to<std::string>());
-    }
-    return names;
+  const auto send_data_to_endpoint = [&io_context, &data](const udp::endpoint& endpoint) -> nonstd::expected<void, std::error_code> {
+    std::error_code send_error;
+    udp::socket socket(io_context);
+    socket.open(udp::v4());

Review Comment:
   I've reworked this part (and also ListenTCP, ListenSyslog) in https://github.com/apache/nifi-minifi-cpp/pull/1412/commits/aba9c5bb58f317143a6301e1552087c8f6d4a5fa



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -197,21 +197,10 @@ constexpr std::string_view rfc5424_logger_example_1 = R"(<13>1 2022-03-17T10:10:
 
 constexpr std::string_view invalid_syslog = "not syslog";
 
-void sendUDPPacket(const std::string_view content, uint64_t port) {
-  asio::io_context io_context;
-  asio::ip::udp::socket socket(io_context);
-  asio::ip::udp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
-  socket.open(asio::ip::udp::v4());
-  std::error_code err;
-  socket.send_to(asio::buffer(content, content.size()), remote_endpoint, 0, err);
-  REQUIRE(!err);
-  socket.close();
-}
-
 void check_for_only_basic_attributes(core::FlowFile& flow_file, uint16_t port, std::string_view protocol) {
   CHECK(std::to_string(port) == flow_file.getAttribute("syslog.port"));
   CHECK(protocol == flow_file.getAttribute("syslog.protocol"));
-  CHECK("127.0.0.1" == flow_file.getAttribute("syslog.sender"));
+  CHECK(("::ffff:127.0.0.1" == flow_file.getAttribute("syslog.sender") || "::1" == flow_file.getAttribute("syslog.sender")));

Review Comment:
   the `::ffff:` is a subnet prefix for IPv4 addresses that are placed inside IPv6.
   The UDP/TCP Servers now listen on IPv6 with dual stacking enabled, this means that the OS listens on ipv4 and ipv6 simultaneously and any request that comes through ipv4 will be transferred to ipv6's ipv4 subnet, thus `127.0.0.1` (original sender) will become `::ffff:127.0.0.1`



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -197,21 +198,11 @@ constexpr std::string_view rfc5424_logger_example_1 = R"(<13>1 2022-03-17T10:10:
 
 constexpr std::string_view invalid_syslog = "not syslog";
 
-void sendUDPPacket(const std::string_view content, uint64_t port) {
-  asio::io_context io_context;
-  asio::ip::udp::socket socket(io_context);
-  asio::ip::udp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
-  socket.open(asio::ip::udp::v4());
-  std::error_code err;
-  socket.send_to(asio::buffer(content, content.size()), remote_endpoint, 0, err);
-  REQUIRE(!err);
-  socket.close();
-}
-
 void check_for_only_basic_attributes(core::FlowFile& flow_file, uint16_t port, std::string_view protocol) {
   CHECK(std::to_string(port) == flow_file.getAttribute("syslog.port"));
   CHECK(protocol == flow_file.getAttribute("syslog.protocol"));
-  CHECK("127.0.0.1" == flow_file.getAttribute("syslog.sender"));
+  const auto local_addresses = {"127.0.0.1", "::ffff:127.0.0.1", "::1"};

Review Comment:
   Would you mind extracting this to some global constant? In the global scope, even `constexpr` seems to work, and it would reduce duplication.



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/tests/unit/PutUDPTests.cpp:
##########
@@ -24,91 +24,87 @@
 #include "Catch.h"
 #include "PutUDP.h"
 #include "core/ProcessContext.h"
-#include "utils/net/DNS.h"
-#include "utils/net/Socket.h"
+#include "utils/net/UdpServer.h"
 #include "utils/expected.h"
 #include "utils/StringUtils.h"
 
+using namespace std::literals::chrono_literals;
+
 namespace org::apache::nifi::minifi::processors {
 
 namespace {
-struct DatagramListener {
-  DatagramListener(const char* const hostname, const char* const port)
-    :resolved_names_{utils::net::resolveHost(hostname, port, utils::net::IpProtocol::UDP).value()},
-     open_socket_{utils::net::open_socket(*resolved_names_)
-        | utils::valueOrElse([=]() -> utils::net::OpenSocketResult { throw std::runtime_error{utils::StringUtils::join_pack("Failed to connect to ", hostname, " on port ", port)}; })}
-  {
-    const auto bind_result = bind(open_socket_.socket_.get(), open_socket_.selected_name->ai_addr, open_socket_.selected_name->ai_addrlen);
-    if (bind_result == utils::net::SocketError) {
-      throw std::runtime_error{utils::StringUtils::join_pack("bind: ", utils::net::get_last_socket_error().message())};
-    }
+std::optional<utils::net::Message> tryDequeueWithTimeout(utils::net::UdpServer& listener, std::chrono::milliseconds timeout = 200ms, std::chrono::milliseconds interval = 10ms) {
+  auto start_time = std::chrono::system_clock::now();
+  utils::net::Message result;
+  while (start_time + timeout > std::chrono::system_clock::now()) {
+    if (listener.tryDequeue(result))
+      return result;
+    std::this_thread::sleep_for(interval);
   }
-
-  struct ReceiveResult {
-    std::string remote_address;
-    std::string message;
-  };
-
-  [[nodiscard]] ReceiveResult receive(const size_t max_message_size = 8192) const {
-    ReceiveResult result;
-    result.message.resize(max_message_size);
-    sockaddr_storage remote_address{};
-    socklen_t addrlen = sizeof(remote_address);
-    const auto recv_result = recvfrom(open_socket_.socket_.get(), result.message.data(), result.message.size(), 0, std::launder(reinterpret_cast<sockaddr*>(&remote_address)), &addrlen);
-    if (recv_result == utils::net::SocketError) {
-      throw std::runtime_error{utils::StringUtils::join_pack("recvfrom: ", utils::net::get_last_socket_error().message())};
-    }
-    result.message.resize(gsl::narrow<size_t>(recv_result));
-    result.remote_address = utils::net::sockaddr_ntop(std::launder(reinterpret_cast<sockaddr*>(&remote_address)));
-    return result;
-  }
-
-  std::unique_ptr<addrinfo, utils::net::addrinfo_deleter> resolved_names_;
-  utils::net::OpenSocketResult open_socket_;
-};
+  return std::nullopt;
+}
 }  // namespace
 
-// Testing the failure relationship is not required, because since UDP in general without guarantees, flow files are always routed to success, unless there is
-// some weird IO error with the content repo.
 TEST_CASE("PutUDP", "[putudp]") {
-  const auto putudp = std::make_shared<PutUDP>("PutUDP");
+  const auto put_udp = std::make_shared<PutUDP>("PutUDP");
   auto random_engine = std::mt19937{std::random_device{}()};  // NOLINT: "Missing space before {  [whitespace/braces] [5]"
   // most systems use ports 32768 - 65535 as ephemeral ports, so avoid binding to those
   const auto port = std::uniform_int_distribution<uint16_t>{10000, 32768 - 1}(random_engine);
-  const auto port_str = std::to_string(port);
 
-  test::SingleProcessorTestController controller{putudp};
+  test::SingleProcessorTestController controller{put_udp};
   LogTestController::getInstance().setTrace<PutUDP>();
   LogTestController::getInstance().setTrace<core::ProcessContext>();
   LogTestController::getInstance().setLevelByClassName(spdlog::level::trace, "org::apache::nifi::minifi::core::ProcessContextExpr");
-  putudp->setProperty(PutUDP::Hostname, "${literal('localhost')}");
-  putudp->setProperty(PutUDP::Port, utils::StringUtils::join_pack("${literal('", port_str, "')}"));
+  put_udp->setProperty(PutUDP::Hostname, "${literal('localhost')}");
+  put_udp->setProperty(PutUDP::Port, utils::StringUtils::join_pack("${literal('", std::to_string(port), "')}"));
 
-  DatagramListener listener{"localhost", port_str.c_str()};
+  utils::net::UdpServer listener{std::nullopt, port, core::logging::LoggerFactory<utils::net::UdpServer>().getLogger()};
+
+  auto server_thread = std::thread([&listener]() { listener.run(); });
+  auto cleanup_server = gsl::finally([&]{
+    listener.stop();
+    server_thread.join();
+  });
 
   {
     const char* const message = "first message: hello";
     const auto result = controller.trigger(message);
     const auto& success_flow_files = result.at(PutUDP::Success);
     REQUIRE(success_flow_files.size() == 1);
-    REQUIRE(result.at(PutUDP::Failure).empty());
-    REQUIRE(controller.plan->getContent(success_flow_files[0]) == message);
-    auto receive_result = listener.receive();
-    REQUIRE(receive_result.message == message);
-    REQUIRE(!receive_result.remote_address.empty());
+    CHECK(result.at(PutUDP::Failure).empty());
+    CHECK(controller.plan->getContent(success_flow_files[0]) == message);
+    auto received_message = tryDequeueWithTimeout(listener);
+    REQUIRE(received_message);
+    CHECK(received_message->message_data == message);
+    CHECK(received_message->protocol == utils::net::IpProtocol::UDP);
+    CHECK(!received_message->sender_address.to_string().empty());
   }
 
   {
     const char* const message = "longer message AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA";  // NOLINT
     const auto result = controller.trigger(message);
     const auto& success_flow_files = result.at(PutUDP::Success);
     REQUIRE(success_flow_files.size() == 1);
-    REQUIRE(result.at(PutUDP::Failure).empty());
-    REQUIRE(controller.plan->getContent(success_flow_files[0]) == message);
-    auto receive_result = listener.receive();
-    REQUIRE(receive_result.message == message);
-    REQUIRE(!receive_result.remote_address.empty());
+    CHECK(result.at(PutUDP::Failure).empty());
+    CHECK(controller.plan->getContent(success_flow_files[0]) == message);
+    auto received_message = tryDequeueWithTimeout(listener);
+    REQUIRE(received_message);
+    CHECK(received_message->message_data == message);
+    CHECK(received_message->protocol == utils::net::IpProtocol::UDP);
+    CHECK(!received_message->sender_address.to_string().empty());
   }
-}
 
+  {
+    const char* const message = "message for invalid host";
+    controller.plan->setProperty(put_udp, PutUDP::Hostname.getName(), "invalid_hostname");
+    const auto result = controller.trigger(message);
+    const auto& failure_flow_files = result.at(PutUDP::Failure);
+    auto received_message = tryDequeueWithTimeout(listener);
+    CHECK(!received_message);
+    REQUIRE(failure_flow_files.size() == 1);
+    CHECK(result.at(PutUDP::Success).empty());
+    CHECK(controller.plan->getContent(failure_flow_files[0]) == message);
+    CHECK((LogTestController::getInstance().contains("Host not found") || LogTestController::getInstance().contains("No such host is known")));

Review Comment:
   please call `LogTestController::getInstance().clear()` at the start of this section to make sure the message was logged in this section and not earlier



##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,48 +98,48 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(udp::v4(), hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
+  };
+
+  const auto debug_log_resolved_endpoint = [&hostname, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> udp::endpoint {
+    if (logger->should_log(core::logging::LOG_LEVEL::debug))
+      core::logging::LOG_WARN(logger) << "resolved " << hostname << " to: " << resolved_query->endpoint();
+    return resolved_query->endpoint();
   };
 
-  const auto debug_log_resolved_names = [&, this](const addrinfo& names) -> decltype(auto) {
-    if (logger_->should_log(core::logging::LOG_LEVEL::debug)) {
-      std::vector<std::string> names_vector;
-      for (const addrinfo* it = &names; it; it = it->ai_next) {
-        names_vector.push_back(nonthrowing_sockaddr_ntop(it->ai_addr));
-      }
-      logger_->log_debug("resolved \'%s\' to: %s",
-          hostname,
-          names_vector | ranges::views::join(',') | ranges::to<std::string>());
-    }
-    return names;
+  const auto send_data_to_endpoint = [&io_context, &data](const udp::endpoint& endpoint) -> nonstd::expected<void, std::error_code> {
+    std::error_code send_error;
+    udp::socket socket(io_context);
+    socket.open(udp::v4());
+    socket.send_to(asio::buffer(data.buffer), endpoint, udp::socket::message_flags{}, send_error);
+    if (send_error)
+      return nonstd::make_unexpected(send_error);
+    return {};
+  };
+
+  const auto transfer_to_success = [&session, &flow_file]() -> void {
+    session->transfer(flow_file, Success);
+  };
+
+  const auto transfer_to_failure = [&session, &flow_file, &logger = this->logger_](std::error_code ec) -> void {
+    gsl_Expects(ec);
+    logger->log_error("%s", ec.message());
+    session->transfer(flow_file, Failure);
   };
 
-  utils::net::resolveHost(hostname.c_str(), port.c_str(), utils::net::IpProtocol::UDP)
-      | utils::map(utils::dereference)
-      | utils::map(debug_log_resolved_names)
-      | utils::flatMap([](const auto& names) { return utils::net::open_socket(names); })
-      | utils::flatMap([&, this](utils::net::OpenSocketResult socket_handle_and_selected_name) -> nonstd::expected<void, std::error_code> {
-        const auto& [socket_handle, selected_name] = socket_handle_and_selected_name;
-        logger_->log_debug("connected to %s", nonthrowing_sockaddr_ntop(selected_name->ai_addr));
-#ifdef WIN32
-        const char* const buffer_ptr = reinterpret_cast<const char*>(data.buffer.data());
-#else
-        const void* const buffer_ptr = data.buffer.data();
-#endif
-        const auto send_result = ::sendto(socket_handle.get(), buffer_ptr, data.buffer.size(), 0, selected_name->ai_addr, selected_name->ai_addrlen);
-        logger_->log_trace("sendto returned %ld", static_cast<long>(send_result));  // NOLINT: sendto
-        if (send_result == utils::net::SocketError) {
-          return nonstd::make_unexpected(utils::net::get_last_socket_error());
-        }
-        session->transfer(flow_file, Success);
-        return {};
-      })
-      | utils::orElse([&, this](std::error_code ec) {
-        gsl_Expects(ec);
-        logger_->log_error("%s", ec.message());
-        session->transfer(flow_file, Failure);
-      });
+  resolve_hostname()
+      | utils::map(debug_log_resolved_endpoint)
+      | utils::flatMap(send_data_to_endpoint)
+      | utils::map(transfer_to_success)
+      | utils::orElse(transfer_to_failure);

Review Comment:
   👍 it's much more readable this way



##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,48 +98,48 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(udp::v4(), hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
+  };
+
+  const auto debug_log_resolved_endpoint = [&hostname, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> udp::endpoint {
+    if (logger->should_log(core::logging::LOG_LEVEL::debug))
+      core::logging::LOG_WARN(logger) << "resolved " << hostname << " to: " << resolved_query->endpoint();

Review Comment:
   should this be `LOG_DEBUG`?



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
libminifi/test/Utils.h:
##########
@@ -111,24 +111,42 @@ bool countLogOccurrencesUntil(const std::string& pattern,
   return false;
 }
 
-bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t port) {
+bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, const asio::ip::tcp::endpoint& remote_endpoint) {
   asio::io_context io_context;
   asio::ip::tcp::socket socket(io_context);
-  asio::ip::tcp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
   socket.connect(remote_endpoint);
   std::error_code err;
   for (auto& content : contents) {
     std::string tcp_message(content);
     tcp_message += '\n';
     asio::write(socket, asio::buffer(tcp_message, tcp_message.size()), err);

Review Comment:
   Won't this overwrite the previous value of `err`?  I.e., if there are 2 writes, the first fails and the second succeeds, then we'll return `true` as if everything was A-OK.



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,48 +98,48 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(udp::v4(), hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
+  };
+
+  const auto debug_log_resolved_endpoint = [&hostname, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> udp::endpoint {
+    if (logger->should_log(core::logging::LOG_LEVEL::debug))
+      core::logging::LOG_DEBUG(logger) << "resolved " << hostname << " to: " << resolved_query->endpoint();
+    return resolved_query->endpoint();

Review Comment:
   How does this select the best endpoint from the list of returned matches? The old code iterated over the list until it could open a socket.



##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,48 +98,48 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(udp::v4(), hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
+  };
+
+  const auto debug_log_resolved_endpoint = [&hostname, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> udp::endpoint {
+    if (logger->should_log(core::logging::LOG_LEVEL::debug))
+      core::logging::LOG_DEBUG(logger) << "resolved " << hostname << " to: " << resolved_query->endpoint();
+    return resolved_query->endpoint();
   };
 
-  const auto debug_log_resolved_names = [&, this](const addrinfo& names) -> decltype(auto) {
-    if (logger_->should_log(core::logging::LOG_LEVEL::debug)) {
-      std::vector<std::string> names_vector;
-      for (const addrinfo* it = &names; it; it = it->ai_next) {
-        names_vector.push_back(nonthrowing_sockaddr_ntop(it->ai_addr));
-      }
-      logger_->log_debug("resolved \'%s\' to: %s",
-          hostname,
-          names_vector | ranges::views::join(',') | ranges::to<std::string>());
-    }
-    return names;
+  const auto send_data_to_endpoint = [&io_context, &data](const udp::endpoint& endpoint) -> nonstd::expected<void, std::error_code> {
+    std::error_code send_error;
+    udp::socket socket(io_context);
+    socket.open(udp::v4());

Review Comment:
   This looks like limited to IPv4. Can we do the same without the limitation?



-- 
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 closed pull request #1412: MINIFICPP-1923 Refactor PutUDP to use asio

Posted by GitBox <gi...@apache.org>.
szaszm closed pull request #1412: MINIFICPP-1923 Refactor PutUDP to use asio
URL: https://github.com/apache/nifi-minifi-cpp/pull/1412


-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,48 +98,48 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(udp::v4(), hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
+  };
+
+  const auto debug_log_resolved_endpoint = [&hostname, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> udp::endpoint {
+    if (logger->should_log(core::logging::LOG_LEVEL::debug))
+      core::logging::LOG_DEBUG(logger) << "resolved " << hostname << " to: " << resolved_query->endpoint();
+    return resolved_query->endpoint();

Review Comment:
   How does this select the best endpoint from the list of returned matches? The old code iterated over the list until it could open a socket.
   
   I thought this was the best we can do with a connectionless protocol, but now that I think about it, iterating until it's able to send would be even better. Consider doing it if it's easy, but I'm not asking you to extend the scope with this if it's a difficult change.



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/tests/unit/PutUDPTests.cpp:
##########
@@ -24,91 +24,86 @@
 #include "Catch.h"
 #include "PutUDP.h"
 #include "core/ProcessContext.h"
-#include "utils/net/DNS.h"
-#include "utils/net/Socket.h"
+#include "utils/net/UdpServer.h"
 #include "utils/expected.h"
 #include "utils/StringUtils.h"
 
+using namespace std::literals::chrono_literals;
+
 namespace org::apache::nifi::minifi::processors {
 
 namespace {
-struct DatagramListener {
-  DatagramListener(const char* const hostname, const char* const port)
-    :resolved_names_{utils::net::resolveHost(hostname, port, utils::net::IpProtocol::UDP).value()},
-     open_socket_{utils::net::open_socket(*resolved_names_)
-        | utils::valueOrElse([=]() -> utils::net::OpenSocketResult { throw std::runtime_error{utils::StringUtils::join_pack("Failed to connect to ", hostname, " on port ", port)}; })}
-  {
-    const auto bind_result = bind(open_socket_.socket_.get(), open_socket_.selected_name->ai_addr, open_socket_.selected_name->ai_addrlen);
-    if (bind_result == utils::net::SocketError) {
-      throw std::runtime_error{utils::StringUtils::join_pack("bind: ", utils::net::get_last_socket_error().message())};
-    }
-  }
-
-  struct ReceiveResult {
-    std::string remote_address;
-    std::string message;
-  };
-
-  [[nodiscard]] ReceiveResult receive(const size_t max_message_size = 8192) const {
-    ReceiveResult result;
-    result.message.resize(max_message_size);
-    sockaddr_storage remote_address{};
-    socklen_t addrlen = sizeof(remote_address);
-    const auto recv_result = recvfrom(open_socket_.socket_.get(), result.message.data(), result.message.size(), 0, std::launder(reinterpret_cast<sockaddr*>(&remote_address)), &addrlen);
-    if (recv_result == utils::net::SocketError) {
-      throw std::runtime_error{utils::StringUtils::join_pack("recvfrom: ", utils::net::get_last_socket_error().message())};
-    }
-    result.message.resize(gsl::narrow<size_t>(recv_result));
-    result.remote_address = utils::net::sockaddr_ntop(std::launder(reinterpret_cast<sockaddr*>(&remote_address)));
-    return result;
+std::optional<utils::net::Message> tryDequeueWithTimeout(utils::net::UdpServer& listener, std::chrono::milliseconds timeout = 200ms, std::chrono::milliseconds interval = 10ms) {
+  auto start_time = std::chrono::system_clock::now();
+  utils::net::Message result;
+  while (start_time + timeout > std::chrono::system_clock::now()) {
+    if (listener.tryDequeue(result))
+      return result;
+    std::this_thread::sleep_for(interval);
   }
-
-  std::unique_ptr<addrinfo, utils::net::addrinfo_deleter> resolved_names_;
-  utils::net::OpenSocketResult open_socket_;
-};
+  return std::nullopt;
+}
 }  // namespace
 
-// Testing the failure relationship is not required, because since UDP in general without guarantees, flow files are always routed to success, unless there is
-// some weird IO error with the content repo.

Review Comment:
   This was not strictly true, because the hostname resolution could fail (added a test-case for that).



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

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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/tests/unit/PutUDPTests.cpp:
##########
@@ -24,91 +24,87 @@
 #include "Catch.h"
 #include "PutUDP.h"
 #include "core/ProcessContext.h"
-#include "utils/net/DNS.h"
-#include "utils/net/Socket.h"
+#include "utils/net/UdpServer.h"
 #include "utils/expected.h"
 #include "utils/StringUtils.h"
 
+using namespace std::literals::chrono_literals;
+
 namespace org::apache::nifi::minifi::processors {
 
 namespace {
-struct DatagramListener {
-  DatagramListener(const char* const hostname, const char* const port)
-    :resolved_names_{utils::net::resolveHost(hostname, port, utils::net::IpProtocol::UDP).value()},
-     open_socket_{utils::net::open_socket(*resolved_names_)
-        | utils::valueOrElse([=]() -> utils::net::OpenSocketResult { throw std::runtime_error{utils::StringUtils::join_pack("Failed to connect to ", hostname, " on port ", port)}; })}
-  {
-    const auto bind_result = bind(open_socket_.socket_.get(), open_socket_.selected_name->ai_addr, open_socket_.selected_name->ai_addrlen);
-    if (bind_result == utils::net::SocketError) {
-      throw std::runtime_error{utils::StringUtils::join_pack("bind: ", utils::net::get_last_socket_error().message())};
-    }
+std::optional<utils::net::Message> tryDequeueWithTimeout(utils::net::UdpServer& listener, std::chrono::milliseconds timeout = 200ms, std::chrono::milliseconds interval = 10ms) {
+  auto start_time = std::chrono::system_clock::now();
+  utils::net::Message result;
+  while (start_time + timeout > std::chrono::system_clock::now()) {
+    if (listener.tryDequeue(result))
+      return result;
+    std::this_thread::sleep_for(interval);
   }
-
-  struct ReceiveResult {
-    std::string remote_address;
-    std::string message;
-  };
-
-  [[nodiscard]] ReceiveResult receive(const size_t max_message_size = 8192) const {
-    ReceiveResult result;
-    result.message.resize(max_message_size);
-    sockaddr_storage remote_address{};
-    socklen_t addrlen = sizeof(remote_address);
-    const auto recv_result = recvfrom(open_socket_.socket_.get(), result.message.data(), result.message.size(), 0, std::launder(reinterpret_cast<sockaddr*>(&remote_address)), &addrlen);
-    if (recv_result == utils::net::SocketError) {
-      throw std::runtime_error{utils::StringUtils::join_pack("recvfrom: ", utils::net::get_last_socket_error().message())};
-    }
-    result.message.resize(gsl::narrow<size_t>(recv_result));
-    result.remote_address = utils::net::sockaddr_ntop(std::launder(reinterpret_cast<sockaddr*>(&remote_address)));
-    return result;
-  }
-
-  std::unique_ptr<addrinfo, utils::net::addrinfo_deleter> resolved_names_;
-  utils::net::OpenSocketResult open_socket_;
-};
+  return std::nullopt;
+}
 }  // namespace
 
-// Testing the failure relationship is not required, because since UDP in general without guarantees, flow files are always routed to success, unless there is
-// some weird IO error with the content repo.
 TEST_CASE("PutUDP", "[putudp]") {
-  const auto putudp = std::make_shared<PutUDP>("PutUDP");
+  const auto put_udp = std::make_shared<PutUDP>("PutUDP");
   auto random_engine = std::mt19937{std::random_device{}()};  // NOLINT: "Missing space before {  [whitespace/braces] [5]"
   // most systems use ports 32768 - 65535 as ephemeral ports, so avoid binding to those
   const auto port = std::uniform_int_distribution<uint16_t>{10000, 32768 - 1}(random_engine);
-  const auto port_str = std::to_string(port);
 
-  test::SingleProcessorTestController controller{putudp};
+  test::SingleProcessorTestController controller{put_udp};
   LogTestController::getInstance().setTrace<PutUDP>();
   LogTestController::getInstance().setTrace<core::ProcessContext>();
   LogTestController::getInstance().setLevelByClassName(spdlog::level::trace, "org::apache::nifi::minifi::core::ProcessContextExpr");
-  putudp->setProperty(PutUDP::Hostname, "${literal('localhost')}");
-  putudp->setProperty(PutUDP::Port, utils::StringUtils::join_pack("${literal('", port_str, "')}"));
+  put_udp->setProperty(PutUDP::Hostname, "${literal('localhost')}");
+  put_udp->setProperty(PutUDP::Port, utils::StringUtils::join_pack("${literal('", std::to_string(port), "')}"));
 
-  DatagramListener listener{"localhost", port_str.c_str()};
+  utils::net::UdpServer listener{std::nullopt, port, core::logging::LoggerFactory<utils::net::UdpServer>().getLogger()};
+
+  auto server_thread = std::thread([&listener]() { listener.run(); });
+  auto cleanup_server = gsl::finally([&]{
+    listener.stop();
+    server_thread.join();
+  });
 
   {
     const char* const message = "first message: hello";
     const auto result = controller.trigger(message);
     const auto& success_flow_files = result.at(PutUDP::Success);
     REQUIRE(success_flow_files.size() == 1);
-    REQUIRE(result.at(PutUDP::Failure).empty());
-    REQUIRE(controller.plan->getContent(success_flow_files[0]) == message);
-    auto receive_result = listener.receive();
-    REQUIRE(receive_result.message == message);
-    REQUIRE(!receive_result.remote_address.empty());
+    CHECK(result.at(PutUDP::Failure).empty());
+    CHECK(controller.plan->getContent(success_flow_files[0]) == message);
+    auto received_message = tryDequeueWithTimeout(listener);
+    REQUIRE(received_message);
+    CHECK(received_message->message_data == message);
+    CHECK(received_message->protocol == utils::net::IpProtocol::UDP);
+    CHECK(!received_message->sender_address.to_string().empty());
   }
 
   {
     const char* const message = "longer message AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA";  // NOLINT
     const auto result = controller.trigger(message);
     const auto& success_flow_files = result.at(PutUDP::Success);
     REQUIRE(success_flow_files.size() == 1);
-    REQUIRE(result.at(PutUDP::Failure).empty());
-    REQUIRE(controller.plan->getContent(success_flow_files[0]) == message);
-    auto receive_result = listener.receive();
-    REQUIRE(receive_result.message == message);
-    REQUIRE(!receive_result.remote_address.empty());
+    CHECK(result.at(PutUDP::Failure).empty());
+    CHECK(controller.plan->getContent(success_flow_files[0]) == message);
+    auto received_message = tryDequeueWithTimeout(listener);
+    REQUIRE(received_message);
+    CHECK(received_message->message_data == message);
+    CHECK(received_message->protocol == utils::net::IpProtocol::UDP);
+    CHECK(!received_message->sender_address.to_string().empty());
   }
-}
 
+  {
+    const char* const message = "message for invalid host";
+    controller.plan->setProperty(put_udp, PutUDP::Hostname.getName(), "invalid_hostname");
+    const auto result = controller.trigger(message);
+    const auto& failure_flow_files = result.at(PutUDP::Failure);
+    auto received_message = tryDequeueWithTimeout(listener);
+    CHECK(!received_message);
+    REQUIRE(failure_flow_files.size() == 1);
+    CHECK(result.at(PutUDP::Success).empty());
+    CHECK(controller.plan->getContent(failure_flow_files[0]) == message);
+    CHECK((LogTestController::getInstance().contains("Host not found") || LogTestController::getInstance().contains("No such host is known")));

Review Comment:
   good idea, changed it in https://github.com/apache/nifi-minifi-cpp/pull/1412/commits/45948030831865d2a71d6a47135a2f6deac823d0#diff-f690439939269c49549e72231c05edf2e8ba912ec3fe206031d6465b7d161779R98



##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,48 +98,48 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(udp::v4(), hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
+  };
+
+  const auto debug_log_resolved_endpoint = [&hostname, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> udp::endpoint {
+    if (logger->should_log(core::logging::LOG_LEVEL::debug))
+      core::logging::LOG_WARN(logger) << "resolved " << hostname << " to: " << resolved_query->endpoint();

Review Comment:
   you are right, fixed it https://github.com/apache/nifi-minifi-cpp/pull/1412/commits/45948030831865d2a71d6a47135a2f6deac823d0#diff-e126536bed52648bd8b876e5a90622a3ebfc00207b7c0b10ca7f1f9605e67c0eR114



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
libminifi/test/Utils.h:
##########
@@ -111,24 +111,42 @@ bool countLogOccurrencesUntil(const std::string& pattern,
   return false;
 }
 
-bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t port) {
+bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, const asio::ip::tcp::endpoint& remote_endpoint) {
   asio::io_context io_context;
   asio::ip::tcp::socket socket(io_context);
-  asio::ip::tcp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
   socket.connect(remote_endpoint);
   std::error_code err;
   for (auto& content : contents) {
     std::string tcp_message(content);
     tcp_message += '\n';
     asio::write(socket, asio::buffer(tcp_message, tcp_message.size()), err);
   }
-  if (err) {
+  if (err)
+    return false;
+  socket.close();
+  return true;

Review Comment:
   Just checked and fortunetly RAII will take of everything so we dont need to manually close it anywhere.
   I moved the error checking inside the loop in https://github.com/apache/nifi-minifi-cpp/pull/1412/commits/3caa5a47fc16dc44dddc298c7f3d20b33fefe6f7



##########
libminifi/test/Utils.h:
##########
@@ -111,24 +111,42 @@ bool countLogOccurrencesUntil(const std::string& pattern,
   return false;
 }
 
-bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t port) {
+bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, const asio::ip::tcp::endpoint& remote_endpoint) {
   asio::io_context io_context;
   asio::ip::tcp::socket socket(io_context);
-  asio::ip::tcp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
   socket.connect(remote_endpoint);
   std::error_code err;
   for (auto& content : contents) {
     std::string tcp_message(content);
     tcp_message += '\n';
     asio::write(socket, asio::buffer(tcp_message, tcp_message.size()), err);

Review Comment:
   You are right :+1: I moved the error checking inside the loop in https://github.com/apache/nifi-minifi-cpp/pull/1412/commits/3caa5a47fc16dc44dddc298c7f3d20b33fefe6f7



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -197,21 +197,10 @@ constexpr std::string_view rfc5424_logger_example_1 = R"(<13>1 2022-03-17T10:10:
 
 constexpr std::string_view invalid_syslog = "not syslog";
 
-void sendUDPPacket(const std::string_view content, uint64_t port) {
-  asio::io_context io_context;
-  asio::ip::udp::socket socket(io_context);
-  asio::ip::udp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
-  socket.open(asio::ip::udp::v4());
-  std::error_code err;
-  socket.send_to(asio::buffer(content, content.size()), remote_endpoint, 0, err);
-  REQUIRE(!err);
-  socket.close();
-}
-
 void check_for_only_basic_attributes(core::FlowFile& flow_file, uint16_t port, std::string_view protocol) {
   CHECK(std::to_string(port) == flow_file.getAttribute("syslog.port"));
   CHECK(protocol == flow_file.getAttribute("syslog.protocol"));
-  CHECK("127.0.0.1" == flow_file.getAttribute("syslog.sender"));
+  CHECK(("::ffff:127.0.0.1" == flow_file.getAttribute("syslog.sender") || "::1" == flow_file.getAttribute("syslog.sender")));

Review Comment:
   this seems to be only IPv6 now; how does this work in the IPv4 case?



##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,51 +98,53 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
   };
 
-  const auto debug_log_resolved_names = [&, this](const addrinfo& names) -> decltype(auto) {
-    if (logger_->should_log(core::logging::LOG_LEVEL::debug)) {
-      std::vector<std::string> names_vector;
-      for (const addrinfo* it = &names; it; it = it->ai_next) {
-        names_vector.push_back(nonthrowing_sockaddr_ntop(it->ai_addr));
+  const auto send_data_to_endpoint = [&io_context, &data, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> nonstd::expected<void, std::error_code> {
+    std::error_code error;
+    for (const auto& resolver_entry : resolved_query) {
+      error.clear();
+      udp::socket socket(io_context);
+      socket.open(resolver_entry.endpoint().protocol(), error);
+      if (error) {
+        logger->log_debug("opening %s socket failed due to %s ", resolver_entry.endpoint().protocol() == udp::v4() ? "IPv4" : "IPv6", error.message());
+        continue;
       }
-      logger_->log_debug("resolved \'%s\' to: %s",
-          hostname,
-          names_vector | ranges::views::join(',') | ranges::to<std::string>());
+      socket.send_to(asio::buffer(data.buffer), resolver_entry.endpoint(), udp::socket::message_flags{}, error);
+      if (error) {
+        core::logging::LOG_DEBUG(logger) << "sending to endpoint " << resolver_entry.endpoint() << " failed due to " << error.message();

Review Comment:
   Are these errors expected and harmless?  If not, we should log it on error (or warning) level; also at line 119.
   
   Also, I think it would be useful to add a debug-level log in the successful case, so we can see that "sending to X failed", "sending to Y failed", "sending to Z succeeded".



##########
libminifi/test/Utils.h:
##########
@@ -111,24 +111,42 @@ bool countLogOccurrencesUntil(const std::string& pattern,
   return false;
 }
 
-bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t port) {
+bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, const asio::ip::tcp::endpoint& remote_endpoint) {
   asio::io_context io_context;
   asio::ip::tcp::socket socket(io_context);
-  asio::ip::tcp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
   socket.connect(remote_endpoint);
   std::error_code err;
   for (auto& content : contents) {
     std::string tcp_message(content);
     tcp_message += '\n';
     asio::write(socket, asio::buffer(tcp_message, tcp_message.size()), err);
   }
-  if (err) {
+  if (err)
+    return false;
+  socket.close();
+  return true;

Review Comment:
   I can see this was like this before, but why don't we (try to) close the socket in the error case?  Maybe the `connect()` and the first `write()` were successful, only (say) the second `write()` failed.



##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -269,19 +258,37 @@ TEST_CASE("ListenSyslog without parsing test", "[ListenSyslog]") {
   std::string protocol;
 
   SECTION("UDP") {
+    asio::ip::udp::endpoint endpoint;
+    SECTION("sending through IPv4", "[IPv4]") {
+      endpoint = asio::ip::udp::endpoint(asio::ip::address_v4::loopback(), SYSLOG_PORT);
+    }
+    SECTION("sending through IPv6", "[IPv6]") {
+      if (utils::isIPv6Disabled())
+        return;

Review Comment:
   this will exit the test case, so the "TCP" section won't be run



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -197,21 +197,10 @@ constexpr std::string_view rfc5424_logger_example_1 = R"(<13>1 2022-03-17T10:10:
 
 constexpr std::string_view invalid_syslog = "not syslog";
 
-void sendUDPPacket(const std::string_view content, uint64_t port) {
-  asio::io_context io_context;
-  asio::ip::udp::socket socket(io_context);
-  asio::ip::udp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
-  socket.open(asio::ip::udp::v4());
-  std::error_code err;
-  socket.send_to(asio::buffer(content, content.size()), remote_endpoint, 0, err);
-  REQUIRE(!err);
-  socket.close();
-}
-
 void check_for_only_basic_attributes(core::FlowFile& flow_file, uint16_t port, std::string_view protocol) {
   CHECK(std::to_string(port) == flow_file.getAttribute("syslog.port"));
   CHECK(protocol == flow_file.getAttribute("syslog.protocol"));
-  CHECK("127.0.0.1" == flow_file.getAttribute("syslog.sender"));
+  CHECK(("::ffff:127.0.0.1" == flow_file.getAttribute("syslog.sender") || "::1" == flow_file.getAttribute("syslog.sender")));

Review Comment:
   I would add back the IPv4 non-mapped address, so that the test can run on boxes with IPv6 disabled as well.
   
   ```suggestion
     const auto local_addresses = {"127.0.0.1", "::ffff:127.0.0.1", "::1"};
     CHECK(std::find(std::begin(local_addresses), std::end(local_addresses), flow_file.getAttribute("syslog.sender") != std::end(local_addresses));
   ```
   or
   
   ```suggestion
     const auto local_addresses = {"127.0.0.1", "::ffff:127.0.0.1", "::1"};
     CHECK(ranges::contains(local_addresses, flow_file.getAttribute("syslog.sender")));
   ```



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,51 +98,53 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;

Review Comment:
   Just a naming nitpick: a "resolved query" is not a query anymore, but a list of names. They are called "results" in the return type, which is also valid.



##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -197,21 +197,10 @@ constexpr std::string_view rfc5424_logger_example_1 = R"(<13>1 2022-03-17T10:10:
 
 constexpr std::string_view invalid_syslog = "not syslog";
 
-void sendUDPPacket(const std::string_view content, uint64_t port) {
-  asio::io_context io_context;
-  asio::ip::udp::socket socket(io_context);
-  asio::ip::udp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
-  socket.open(asio::ip::udp::v4());
-  std::error_code err;
-  socket.send_to(asio::buffer(content, content.size()), remote_endpoint, 0, err);
-  REQUIRE(!err);
-  socket.close();
-}
-
 void check_for_only_basic_attributes(core::FlowFile& flow_file, uint16_t port, std::string_view protocol) {
   CHECK(std::to_string(port) == flow_file.getAttribute("syslog.port"));
   CHECK(protocol == flow_file.getAttribute("syslog.protocol"));
-  CHECK("127.0.0.1" == flow_file.getAttribute("syslog.sender"));
+  CHECK(("::ffff:127.0.0.1" == flow_file.getAttribute("syslog.sender") || "::1" == flow_file.getAttribute("syslog.sender")));

Review Comment:
   I would add back the IPv4 non-mapped address as well, so that the test can run on boxes with IPv6 disabled as well.
   
   ```suggestion
     const auto local_addresses = {"127.0.0.1", "::ffff:127.0.0.1", "::1"};
     CHECK(std::find(std::begin(local_addresses), std::end(local_addresses), flow_file.getAttribute("syslog.sender") != std::end(local_addresses));
   ```
   or
   
   ```suggestion
     const auto local_addresses = {"127.0.0.1", "::ffff:127.0.0.1", "::1"};
     CHECK(ranges::contains(local_addresses, flow_file.getAttribute("syslog.sender")));
   ```



##########
libminifi/test/Utils.h:
##########
@@ -111,24 +111,39 @@ bool countLogOccurrencesUntil(const std::string& pattern,
   return false;
 }
 
-bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t port) {
+bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, const asio::ip::tcp::endpoint& remote_endpoint) {
   asio::io_context io_context;
   asio::ip::tcp::socket socket(io_context);
-  asio::ip::tcp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
   socket.connect(remote_endpoint);
   std::error_code err;
   for (auto& content : contents) {
     std::string tcp_message(content);
     tcp_message += '\n';
     asio::write(socket, asio::buffer(tcp_message, tcp_message.size()), err);
+    if (err) {
+      return false;
+    }
   }
-  if (err) {
-    return false;
-  }
-  socket.close();
   return true;
 }
 
+bool sendUDPPacket(const std::string_view content, const asio::ip::udp::endpoint& remote_endpoint) {

Review Comment:
   I would rename this to `sendUdpDatagram`, since the PDU of UDP is called datagram. Also, the content is perfect candidate for a byte span, since it doesn't necessarily have to be text. I would add a delegating overload, but keep the string_view version as well, for convenience when used with text.



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
libminifi/test/Utils.h:
##########
@@ -111,24 +111,47 @@ bool countLogOccurrencesUntil(const std::string& pattern,
   return false;
 }
 
-bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, uint64_t port) {
+bool sendMessagesViaTCP(const std::vector<std::string_view>& contents, const asio::ip::tcp::endpoint& remote_endpoint) {
   asio::io_context io_context;
   asio::ip::tcp::socket socket(io_context);
-  asio::ip::tcp::endpoint remote_endpoint(asio::ip::address::from_string("127.0.0.1"), port);
   socket.connect(remote_endpoint);
   std::error_code err;
   for (auto& content : contents) {
     std::string tcp_message(content);
     tcp_message += '\n';
     asio::write(socket, asio::buffer(tcp_message, tcp_message.size()), err);
+    if (err) {
+      return false;
+    }
   }
-  if (err) {
-    return false;
-  }
-  socket.close();
   return true;
 }
 
+bool sendUdpDatagram(const asio::const_buffer& content, const asio::ip::udp::endpoint& remote_endpoint) {

Review Comment:
   `asio::const_buffer` is cheap to copy, similarly to `gsl::span` and `std::string_view`. I would pass it by value.



-- 
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 pull request #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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

   I've moved from only IPv4 only mode to IPv6, I also modified ListenTCP and ListenSyslog processors aswell.
   This is effectively fixes a separate ticket aswell. So could you guys review the latest [commit](https://github.com/apache/nifi-minifi-cpp/pull/1412/commits/aba9c5bb58f317143a6301e1552087c8f6d4a5fa) aswell? @fgerlits @lordgamez
   


-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/processors/PutUDP.cpp:
##########
@@ -107,51 +98,53 @@ void PutUDP::onTrigger(core::ProcessContext* context, core::ProcessSession* cons
     return;
   }
 
-  const auto nonthrowing_sockaddr_ntop = [](const sockaddr* const sa) -> std::string {
-    return utils::try_expression([sa] { return utils::net::sockaddr_ntop(sa); }).value_or("(n/a)");
+  asio::io_context io_context;
+
+  const auto resolve_hostname = [&io_context, &hostname, &port]() -> nonstd::expected<udp::resolver::results_type, std::error_code> {
+    udp::resolver resolver(io_context);
+    std::error_code error_code;
+    auto resolved_query = resolver.resolve(hostname, port, error_code);
+    if (error_code)
+      return nonstd::make_unexpected(error_code);
+    return resolved_query;
   };
 
-  const auto debug_log_resolved_names = [&, this](const addrinfo& names) -> decltype(auto) {
-    if (logger_->should_log(core::logging::LOG_LEVEL::debug)) {
-      std::vector<std::string> names_vector;
-      for (const addrinfo* it = &names; it; it = it->ai_next) {
-        names_vector.push_back(nonthrowing_sockaddr_ntop(it->ai_addr));
+  const auto send_data_to_endpoint = [&io_context, &data, &logger = this->logger_](const udp::resolver::results_type& resolved_query) -> nonstd::expected<void, std::error_code> {
+    std::error_code error;
+    for (const auto& resolver_entry : resolved_query) {
+      error.clear();
+      udp::socket socket(io_context);
+      socket.open(resolver_entry.endpoint().protocol(), error);
+      if (error) {
+        logger->log_debug("opening %s socket failed due to %s ", resolver_entry.endpoint().protocol() == udp::v4() ? "IPv4" : "IPv6", error.message());
+        continue;
       }
-      logger_->log_debug("resolved \'%s\' to: %s",
-          hostname,
-          names_vector | ranges::views::join(',') | ranges::to<std::string>());
+      socket.send_to(asio::buffer(data.buffer), resolver_entry.endpoint(), udp::socket::message_flags{}, error);
+      if (error) {
+        core::logging::LOG_DEBUG(logger) << "sending to endpoint " << resolver_entry.endpoint() << " failed due to " << error.message();

Review Comment:
   As long as we succeed to send to one resolved endpoint, I don't think we need to handle these as warnings or errors, since this can happen during normal operation. (and the last error will be logged as error)
   
   I've added the success logging in https://github.com/apache/nifi-minifi-cpp/pull/1412/commits/3caa5a47fc16dc44dddc298c7f3d20b33fefe6f7 and https://github.com/apache/nifi-minifi-cpp/pull/1412/commits/64dfce35ea2b477ecd8695bd868b184da9696512



-- 
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 #1412: MINIFICPP-1923 Refactor PutUDP to use asio

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


##########
extensions/standard-processors/tests/unit/ListenSyslogTests.cpp:
##########
@@ -269,19 +258,37 @@ TEST_CASE("ListenSyslog without parsing test", "[ListenSyslog]") {
   std::string protocol;
 
   SECTION("UDP") {
+    asio::ip::udp::endpoint endpoint;
+    SECTION("sending through IPv4", "[IPv4]") {
+      endpoint = asio::ip::udp::endpoint(asio::ip::address_v4::loopback(), SYSLOG_PORT);
+    }
+    SECTION("sending through IPv6", "[IPv6]") {
+      if (utils::isIPv6Disabled())
+        return;

Review Comment:
   this will exit the test case, so the "TCP" section won't be run
   
   EDIT: no, I'm wrong, this is fine.



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