You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by sz...@apache.org on 2022/05/13 14:37:17 UTC

[nifi-minifi-cpp] branch main updated (4a98b0f62 -> 0b9c6bea0)

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

szaszm pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git


    from 4a98b0f62 MINIFICPP-1823 Fix absolute.path output attribute in ListFile
     new e41fc9e09 MINIFICPP-1806 Use boyer_moore for extension verification
     new c3906edfa MINIFICPP-1749 Validate C2 property update values
     new 0b9c6bea0 MINIFICPP-1827 - Verify SSL connection

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 extensions/http-curl/processors/InvokeHTTP.cpp     |  9 ++++-
 .../http-curl/tests/C2PropertiesUpdateTests.cpp    | 31 +++++++++++++--
 extensions/http-curl/tests/TestServer.h            |  6 +--
 .../tests/unit/ConfigurationTests.cpp              | 11 ++++++
 libminifi/include/properties/Configuration.h       | 11 ++----
 libminifi/src/Configuration.cpp                    |  9 +++++
 libminifi/src/c2/C2Agent.cpp                       | 13 ++++---
 libminifi/src/utils/file/FileUtils.cpp             | 45 ++++++++--------------
 .../test/resources/TestInvokeHTTPPostSecure.yml    |  1 -
 9 files changed, 83 insertions(+), 53 deletions(-)


[nifi-minifi-cpp] 01/03: MINIFICPP-1806 Use boyer_moore for extension verification

Posted by sz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

szaszm pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git

commit e41fc9e09033639ceab1a5d1992dbb3fc71a1ad7
Author: Adam Debreceni <ad...@apache.org>
AuthorDate: Fri May 13 16:30:41 2022 +0200

    MINIFICPP-1806 Use boyer_moore for extension verification
    
    For a debug build the extension verification for libcore-minifi.so took
    ~1400 ms (which together with other dynamic libraries adds seconds to
    the startup time during development), with this change it takes ~100 ms.
    
    Alternatives considered:
    - boyer_moore + custom span concat: ~150 ms
    - boyer_moore + ranges::concat: ~1700 ms
    - boyer_moore + mmap: ~100 ms (the mmap wrapper would add significant
      complexity)
    
    Closes #1310
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 libminifi/src/utils/file/FileUtils.cpp | 45 ++++++++++++----------------------
 1 file changed, 15 insertions(+), 30 deletions(-)

diff --git a/libminifi/src/utils/file/FileUtils.cpp b/libminifi/src/utils/file/FileUtils.cpp
index 33602a9fe..d141da61c 100644
--- a/libminifi/src/utils/file/FileUtils.cpp
+++ b/libminifi/src/utils/file/FileUtils.cpp
@@ -22,6 +22,9 @@
 #include <algorithm>
 #include <iostream>
 
+#include "utils/Literals.h"
+#include "utils/Searcher.h"
+
 namespace org {
 namespace apache {
 namespace nifi {
@@ -49,41 +52,23 @@ uint64_t computeChecksum(const std::string &file_name, uint64_t up_to_position)
 }
 
 bool contains(const std::filesystem::path& file_path, std::string_view text_to_search) {
-  gsl_Expects(text_to_search.size() <= 8192);
+  gsl_Expects(text_to_search.size() <= 8_KiB);
   gsl_ExpectsAudit(std::filesystem::exists(file_path));
-  std::array<char, 8192> buf1{};
-  std::array<char, 8192> buf2{};
-  gsl::span<char> left = buf1;
-  gsl::span<char> right = buf2;
-
-  const auto charat = [&](size_t idx) {
-    if (idx < left.size()) {
-      return left[idx];
-    } else if (idx < left.size() + right.size()) {
-      return right[idx - left.size()];
-    } else {
-      return '\0';
-    }
-  };
-  const auto check_range = [&](size_t start, size_t end) -> size_t {
-    for (size_t i = start; i < end; ++i) {
-      size_t j{};
-      for (j = 0; j < text_to_search.size(); ++j) {
-        if (charat(i + j) != text_to_search[j]) break;
-      }
-      if (j == text_to_search.size()) return true;
-    }
-    return false;
-  };
+  std::array<char, 16_KiB> buf{};
+  gsl::span<char> view;
+
+  Searcher searcher(text_to_search.begin(), text_to_search.end());
 
   std::ifstream ifs{file_path, std::ios::binary};
-  ifs.read(right.data(), gsl::narrow<std::streamsize>(right.size()));
   do {
-    std::swap(left, right);
-    ifs.read(right.data(), gsl::narrow<std::streamsize>(right.size()));
-    if (check_range(0, left.size())) return true;
+    std::copy(buf.end() - text_to_search.size(), buf.end(), buf.begin());
+    ifs.read(buf.data() + text_to_search.size(), buf.size() - text_to_search.size());
+    view = gsl::span<char>(buf.data(), text_to_search.size() + gsl::narrow<size_t>(ifs.gcount()));
+    if (std::search(view.begin(), view.end(), searcher) != view.end()) {
+      return true;
+    }
   } while (ifs);
-  return check_range(left.size(), left.size() + right.size());
+  return std::search(view.begin(), view.end(), searcher) != view.end();
 }
 
 time_t to_time_t(std::filesystem::file_time_type file_time) {


[nifi-minifi-cpp] 02/03: MINIFICPP-1749 Validate C2 property update values

Posted by sz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

szaszm pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git

commit c3906edfa830f65738c3562f5f288bff76911dc6
Author: Gabor Gyimesi <ga...@gmail.com>
AuthorDate: Fri May 13 16:32:27 2022 +0200

    MINIFICPP-1749 Validate C2 property update values
    
    Added verification of C2 property values while updating them, also
    changed the C2 response value in case no value could be updated due to
    validation failure or if the update is denied by the
    UpdatePolicyController to NOT_APPLIED instead of PARTIALLY_APPLIED. The
    PARTIALLY_APPLIED response is still sent if any of the requested updates
    could be finished.
    
    Closes #1324
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 .../http-curl/tests/C2PropertiesUpdateTests.cpp    | 31 +++++++++++++++++++---
 .../tests/unit/ConfigurationTests.cpp              | 11 ++++++++
 libminifi/include/properties/Configuration.h       | 11 +++-----
 libminifi/src/Configuration.cpp                    |  9 +++++++
 libminifi/src/c2/C2Agent.cpp                       | 13 +++++----
 5 files changed, 58 insertions(+), 17 deletions(-)

diff --git a/extensions/http-curl/tests/C2PropertiesUpdateTests.cpp b/extensions/http-curl/tests/C2PropertiesUpdateTests.cpp
index 27773ddd6..870604bcc 100644
--- a/extensions/http-curl/tests/C2PropertiesUpdateTests.cpp
+++ b/extensions/http-curl/tests/C2PropertiesUpdateTests.cpp
@@ -17,6 +17,8 @@
  */
 
 #undef NDEBUG
+#include <mutex>
+
 #include "HTTPIntegrationBase.h"
 #include "HTTPHandlers.h"
 #include "utils/gsl.h"
@@ -39,6 +41,7 @@ struct PropertyChange {
 class C2HeartbeatHandler : public ServerAwareHandler {
  public:
   bool handlePost(CivetServer* /*server*/, struct mg_connection *conn) override {
+    std::lock_guard<std::mutex> lock(mutex_);
     if (response_) {
       mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
                       "text/plain\r\nContent-Length: %lu\r\nConnection: close\r\n\r\n",
@@ -61,6 +64,8 @@ class C2HeartbeatHandler : public ServerAwareHandler {
         fields.push_back(fmt::format(R"("{}": "{}")", change.name, change.value));
       }
     }
+
+    std::lock_guard<std::mutex> lock(mutex_);
     response_ =
         R"({
         "operation" : "heartbeat",
@@ -76,6 +81,7 @@ class C2HeartbeatHandler : public ServerAwareHandler {
   }
 
  private:
+  std::mutex mutex_;
   std::optional<std::string> response_;
 };
 
@@ -158,13 +164,28 @@ int main() {
   harness = VerifyPropertyUpdate([&] {
     assert(utils::verifyEventHappenedInPollTime(3s, [&] {return ack_handler.isAcknowledged("79");}));
     assert(utils::verifyEventHappenedInPollTime(3s, [&] {
-      return ack_handler.getApplyCount("FULLY_APPLIED") == 1
-          && harness.getRestartRequestedCount() == 1;
+      return ack_handler.getApplyCount("FULLY_APPLIED") == 1;
     }));
+
+    // Updating the same property will result in a no operation response
     assert(utils::verifyEventHappenedInPollTime(3s, [&] {
-      return ack_handler.getApplyCount("NO_OPERATION") > 0
-          && harness.getRestartRequestedCount() == 1;  // only one, i.e. no additional restart requests compared to the previous update.
+      return ack_handler.getApplyCount("NO_OPERATION") > 0;
     }));
+
+    // Change the update response to 1 invalid and 1 valid value update
+    hb_handler.setProperties({{minifi::Configuration::nifi_c2_rest_heartbeat_minimize_updates, "banana", true}, {minifi::Configuration::minifi_disk_space_watchdog_enable, "true", true}});
+
+    // Due to 1 invalid value the result will be partially applied
+    assert(utils::verifyEventHappenedInPollTime(3s, [&] {
+      return ack_handler.getApplyCount("PARTIALLY_APPLIED") == 1;
+    }));
+
+    // Repeating the previous update request results in 1 no operation and 1 failure which results in not applied response
+    assert(utils::verifyEventHappenedInPollTime(3s, [&] {
+      return ack_handler.getApplyCount("NOT_APPLIED") > 0
+        && harness.getRestartRequestedCount() == 2;
+    }));
+
     // update operation acknowledged
     {
       // verify final log levels
@@ -181,6 +202,8 @@ int main() {
       assert(!minifi_properties.hasValue("nifi.dummy.property"));
       assert(minifi_properties.getValue("nifi.property.one") == "bush");
       assert(minifi_properties.getValue("nifi.property.two") == "ring");
+      assert(!minifi_properties.hasValue(minifi::Configuration::nifi_c2_rest_heartbeat_minimize_updates));
+      assert(minifi_properties.getValue(minifi::Configuration::minifi_disk_space_watchdog_enable) == "true");
     }
 
     {
diff --git a/extensions/standard-processors/tests/unit/ConfigurationTests.cpp b/extensions/standard-processors/tests/unit/ConfigurationTests.cpp
index eb8d95dc9..d395905c5 100644
--- a/extensions/standard-processors/tests/unit/ConfigurationTests.cpp
+++ b/extensions/standard-processors/tests/unit/ConfigurationTests.cpp
@@ -22,6 +22,8 @@
 
 using org::apache::nifi::minifi::Configuration;
 
+namespace org::apache::nifi::minifi::test {
+
 TEST_CASE("Configuration can merge lists of property names", "[mergeProperties]") {
   using vector = std::vector<std::string>;
 
@@ -45,3 +47,12 @@ TEST_CASE("Configuration can merge lists of property names", "[mergeProperties]"
   REQUIRE(Configuration::mergeProperties(vector{"a", "b"}, vector{"a", "c\r\n"}) == (vector{"a", "b", "c"}));
   REQUIRE(Configuration::mergeProperties(vector{"a", "b"}, vector{"b\n", "\t c"}) == (vector{"a", "b", "c"}));
 }
+
+TEST_CASE("Configuration can validate values to be assigned to specific properties", "[validatePropertyValue]") {
+  REQUIRE(Configuration::validatePropertyValue(Configuration::nifi_server_name, "anything is valid"));
+  REQUIRE_FALSE(Configuration::validatePropertyValue(Configuration::nifi_flow_configuration_encrypt, "invalid.value"));
+  REQUIRE(Configuration::validatePropertyValue(Configuration::nifi_flow_configuration_encrypt, "true"));
+  REQUIRE(Configuration::validatePropertyValue("random.property", "random_value"));
+}
+
+}  // namespace org::apache::nifi::minifi::test
diff --git a/libminifi/include/properties/Configuration.h b/libminifi/include/properties/Configuration.h
index f7db69cd4..301bdafd2 100644
--- a/libminifi/include/properties/Configuration.h
+++ b/libminifi/include/properties/Configuration.h
@@ -24,10 +24,7 @@
 #include "utils/OptionalUtils.h"
 #include "utils/Export.h"
 
-namespace org {
-namespace apache {
-namespace nifi {
-namespace minifi {
+namespace org::apache::nifi::minifi {
 
 namespace core {
   struct ConfigurationProperty;
@@ -162,9 +159,7 @@ class Configuration : public Properties {
   static std::vector<std::string> mergeProperties(std::vector<std::string> properties,
                                                   const std::vector<std::string>& additional_properties);
   static std::vector<std::string> getSensitiveProperties(std::function<std::optional<std::string>(const std::string&)> reader);
+  static bool validatePropertyValue(const std::string& property_name, const std::string& property_value);
 };
 
-}  // namespace minifi
-}  // namespace nifi
-}  // namespace apache
-}  // namespace org
+}  // namespace org::apache::nifi::minifi
diff --git a/libminifi/src/Configuration.cpp b/libminifi/src/Configuration.cpp
index 4e50b2447..c8bc7e445 100644
--- a/libminifi/src/Configuration.cpp
+++ b/libminifi/src/Configuration.cpp
@@ -155,4 +155,13 @@ std::vector<std::string> Configuration::getSensitiveProperties(std::function<std
   return sensitive_properties;
 }
 
+bool Configuration::validatePropertyValue(const std::string& property_name, const std::string& property_value) {
+  for (const auto& config_property: Configuration::CONFIGURATION_PROPERTIES) {
+    if (config_property.name == property_name) {
+      return config_property.validator->validate(property_name, property_value).valid();
+    }
+  }
+  return true;
+}
+
 }  // namespace org::apache::nifi::minifi
diff --git a/libminifi/src/c2/C2Agent.cpp b/libminifi/src/c2/C2Agent.cpp
index d3c635a1d..3e6fc92a3 100644
--- a/libminifi/src/c2/C2Agent.cpp
+++ b/libminifi/src/c2/C2Agent.cpp
@@ -611,9 +611,10 @@ void C2Agent::handlePropertyUpdate(const C2ContentResponse &resp) {
       if (update_result == UpdateResult::UPDATE_SUCCESSFUL) {
         result = state::UpdateState::FULLY_APPLIED;
       } else if (update_result == UpdateResult::UPDATE_FAILED) {
-        result = state::UpdateState::PARTIALLY_APPLIED;
+        result = state::UpdateState::NOT_APPLIED;
       }
-    } else if (result == state::UpdateState::FULLY_APPLIED && update_result == UpdateResult::UPDATE_FAILED) {
+    } else if ((result == state::UpdateState::FULLY_APPLIED && update_result == UpdateResult::UPDATE_FAILED) ||
+               (result == state::UpdateState::NOT_APPLIED && update_result == UpdateResult::UPDATE_SUCCESSFUL)) {
       result = state::UpdateState::PARTIALLY_APPLIED;
     }
   };
@@ -627,19 +628,21 @@ void C2Agent::handlePropertyUpdate(const C2ContentResponse &resp) {
     changeUpdateState(update_property(entry.first, entry.second.to_string(), lifetime));
   }
   // apply changes and persist properties requested to be persisted
-  if (result != state::UpdateState::NO_OPERATION && !configuration_->commitChanges()) {
+  const bool propertyWasUpdated = result == state::UpdateState::FULLY_APPLIED || result == state::UpdateState::PARTIALLY_APPLIED;
+  if (propertyWasUpdated && !configuration_->commitChanges()) {
     result = state::UpdateState::PARTIALLY_APPLIED;
   }
   C2Payload response(Operation::ACKNOWLEDGE, result, resp.ident, true);
   enqueue_c2_response(std::move(response));
-  if (result != state::UpdateState::NO_OPERATION) { restart_needed_ = true; }
+  if (propertyWasUpdated) { restart_needed_ = true; }
 }
 
 /**
  * Updates a property
  */
 C2Agent::UpdateResult C2Agent::update_property(const std::string &property_name, const std::string &property_value, PropertyChangeLifetime lifetime) {
-  if (update_service_ && !update_service_->canUpdate(property_name)) {
+  if (!Configuration::validatePropertyValue(property_name, property_value) ||
+      (update_service_ && !update_service_->canUpdate(property_name))) {
     return UpdateResult::UPDATE_FAILED;
   }
 


[nifi-minifi-cpp] 03/03: MINIFICPP-1827 - Verify SSL connection

Posted by sz...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

szaszm pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-minifi-cpp.git

commit 0b9c6bea098c1e602e5efe6b292d99c5cfc2ac9e
Author: Adam Debreceni <ad...@apache.org>
AuthorDate: Fri May 13 16:33:26 2022 +0200

    MINIFICPP-1827 - Verify SSL connection
    
    Closes #1329
    Signed-off-by: Marton Szasz <sz...@apache.org>
---
 extensions/http-curl/processors/InvokeHTTP.cpp        | 9 +++++++--
 extensions/http-curl/tests/TestServer.h               | 6 +++---
 libminifi/test/resources/TestInvokeHTTPPostSecure.yml | 1 -
 3 files changed, 10 insertions(+), 6 deletions(-)

diff --git a/extensions/http-curl/processors/InvokeHTTP.cpp b/extensions/http-curl/processors/InvokeHTTP.cpp
index aea10f730..a0a7f9048 100644
--- a/extensions/http-curl/processors/InvokeHTTP.cpp
+++ b/extensions/http-curl/processors/InvokeHTTP.cpp
@@ -231,8 +231,13 @@ void InvokeHTTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context
   std::string context_name;
   if (context->getProperty(SSLContext.getName(), context_name) && !IsNullOrEmpty(context_name)) {
     std::shared_ptr<core::controller::ControllerService> service = context->getControllerService(context_name);
-    if (nullptr != service) {
-      ssl_context_service_ = std::static_pointer_cast<minifi::controllers::SSLContextService>(service);
+    if (!service) {
+      logger_->log_error("Couldn't find controller service with name '%s'", context_name);
+    } else {
+      ssl_context_service_ = std::dynamic_pointer_cast<minifi::controllers::SSLContextService>(service);
+      if (!ssl_context_service_) {
+        logger_->log_error("Controller service '%s' is not an SSLContextService", context_name);
+      }
     }
   }
 
diff --git a/extensions/http-curl/tests/TestServer.h b/extensions/http-curl/tests/TestServer.h
index 3ee3c977c..12f959129 100644
--- a/extensions/http-curl/tests/TestServer.h
+++ b/extensions/http-curl/tests/TestServer.h
@@ -50,7 +50,7 @@ class TestServer{
   };
 
  public:
-  TestServer(std::string &port, std::string &rooturi, CivetHandler *handler, CivetCallbacks *callbacks, std::string& /*cert*/, std::string &ca_cert) {
+  TestServer(std::string &port, std::string &rooturi, CivetHandler *handler, CivetCallbacks *callbacks, std::string& cert, std::string &ca_cert) {
     if (!mg_check_feature(2)) {
       throw std::runtime_error("Error: Embedded example built with SSL support, "
                                "but civetweb library build without.\n");
@@ -59,8 +59,8 @@ class TestServer{
 
     // ECDH+AESGCM+AES256:!aNULL:!MD5:!DSS
     const std::vector<std::string> cpp_options{ "document_root", ".", "listening_ports", port, "error_log_file",
-                              "error.log", "ssl_certificate", ca_cert, "ssl_protocol_version", "4", "ssl_cipher_list",
-                              "ALL", "request_timeout_ms", "10000", "enable_auth_domain_check", "no", "ssl_verify_peer", "no"};
+                              "error.log", "ssl_certificate", cert, "ssl_ca_file", ca_cert, "ssl_protocol_version", "4", "ssl_cipher_list",
+                              "ALL", "request_timeout_ms", "10000", "enable_auth_domain_check", "no", "ssl_verify_peer", "yes"};
     server_ = std::make_unique<CivetServer>(cpp_options, callbacks);
     addHandler(rooturi, handler);
   }
diff --git a/libminifi/test/resources/TestInvokeHTTPPostSecure.yml b/libminifi/test/resources/TestInvokeHTTPPostSecure.yml
index 5f9237207..45718396a 100644
--- a/libminifi/test/resources/TestInvokeHTTPPostSecure.yml
+++ b/libminifi/test/resources/TestInvokeHTTPPostSecure.yml
@@ -108,7 +108,6 @@ Processors:
   - failure
   Properties:
     invoke_http: failure
-Controller Services: []
 Process Groups: []
 Input Ports: []
 Output Ports: []