You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "fgerlits (via GitHub)" <gi...@apache.org> on 2023/03/30 13:41:59 UTC

[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1543: MINIFICPP-2074 Fix time-period/integer validated properties during lo…

fgerlits commented on code in PR #1543:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1543#discussion_r1153018041


##########
extensions/standard-processors/tests/unit/ConfigurationTests.cpp:
##########
@@ -53,4 +54,130 @@ TEST_CASE("Configuration can validate values to be assigned to specific properti
   REQUIRE(Configuration::validatePropertyValue("random.property", "random_value"));
 }
 
+TEST_CASE("Configuration can fix misconfigured timeperiod<->integer validated properties") {
+  LogTestController::getInstance().setInfo<minifi::Configure>();
+  LogTestController::getInstance().setInfo<minifi::Properties>();
+
+  auto properties_path = std::filesystem::temp_directory_path() /  "test.properties";
+
+  {
+    std::ofstream properties_file(properties_path);
+    properties_file << "nifi.c2.agent.heartbeat.period=1min" << std::endl;
+    properties_file << "nifi.administrative.yield.duration=30000" << std::endl;
+    properties_file.close();
+  }
+  auto properties_file_time_after_creation = std::filesystem::last_write_time(properties_path);
+  const std::shared_ptr<minifi::Configure> configure = std::make_shared<minifi::Configure>();
+
+  configure->loadConfigureFile(properties_path);
+  CHECK(configure->get("nifi.c2.agent.heartbeat.period") == "60000");
+  CHECK(configure->get("nifi.administrative.yield.duration") == "30000 ms");
+
+  {
+    CHECK(properties_file_time_after_creation == std::filesystem::last_write_time(properties_path));
+    std::ifstream properties_file(properties_path);
+    std::string first_line;
+    std::string second_line;
+    CHECK(std::getline(properties_file, first_line));
+    CHECK(std::getline(properties_file, second_line));
+    CHECK(first_line == "nifi.c2.agent.heartbeat.period=1min");
+    CHECK(second_line == "nifi.administrative.yield.duration=30000");
+  }
+
+  CHECK(configure->commitChanges());
+
+  {
+    CHECK(properties_file_time_after_creation <= std::filesystem::last_write_time(properties_path));
+    std::ifstream properties_file(properties_path);
+    std::string first_line;
+    std::string second_line;
+    CHECK(std::getline(properties_file, first_line));
+    CHECK(std::getline(properties_file, second_line));
+    CHECK(first_line == "nifi.c2.agent.heartbeat.period=60000");
+    CHECK(second_line == "nifi.administrative.yield.duration=30000 ms");
+  }
+}
+
+TEST_CASE("Configuration can fix misconfigured datasize<->integer validated properties") {
+  LogTestController::getInstance().setInfo<minifi::Configure>();
+  LogTestController::getInstance().setInfo<minifi::Properties>();
+
+  auto properties_path = std::filesystem::temp_directory_path() /  "test.properties";
+
+  {
+    std::ofstream properties_file(properties_path);
+    properties_file << "appender.rolling.max_file_size=6000" << std::endl;
+    properties_file.close();
+  }
+  auto properties_file_time_after_creation = std::filesystem::last_write_time(properties_path);
+  const std::shared_ptr<minifi::Configure> configure = std::make_shared<minifi::Configure>();
+
+  configure->loadConfigureFile(properties_path, "nifi.log.");
+  CHECK(configure->get("appender.rolling.max_file_size") == "6000 B");
+
+  {
+    CHECK(properties_file_time_after_creation <= std::filesystem::last_write_time(properties_path));
+    std::ifstream properties_file(properties_path);
+    std::string first_line;
+    CHECK(std::getline(properties_file, first_line));
+    CHECK(first_line == "appender.rolling.max_file_size=6000");
+  }
+
+  CHECK(configure->commitChanges());
+
+  {
+    CHECK(properties_file_time_after_creation <= std::filesystem::last_write_time(properties_path));
+    std::ifstream properties_file(properties_path);
+    std::string first_line;
+    CHECK(std::getline(properties_file, first_line));
+    CHECK(first_line == "appender.rolling.max_file_size=6000 B");
+  }
+}
+
+
+TEST_CASE("Configuration misconfigured validated properties with in environmental variables") {

Review Comment:
   ```suggestion
   TEST_CASE("Configuration can fix misconfigured validated properties within environmental variables") {
   ```



##########
extensions/http-curl/tests/HTTPHandlers.h:
##########
@@ -556,18 +556,18 @@ class HeartbeatHandler : public ServerAwareHandler {
         std::vector<std::unordered_map<std::string, std::string>> config_properties;
         const auto prop_reader = [this](const std::string& sensitive_props) { return configuration_->getString(sensitive_props); };
         const auto sensitive_props = minifi::Configuration::getSensitiveProperties(prop_reader);
-        for (const auto& property : minifi::Configuration::CONFIGURATION_PROPERTIES) {
-          if (ranges::find(sensitive_props, property.name) != ranges::end(sensitive_props)) {
+        for (const auto& [property_name, property_validator] : minifi::Configuration::CONFIGURATION_PROPERTIES) {
+          if (ranges::find(sensitive_props, property_name) != ranges::end(sensitive_props)) {

Review Comment:
   minor improvement, but this could be written as
   ```suggestion
             if (ranges::contains(sensitive_props, property_name)) {
   ```



##########
libminifi/include/properties/Properties.h:
##########
@@ -113,7 +113,7 @@ class Properties {
    * Load configure file
    * @param fileName path of the configuration file RELATIVE to MINIFI_HOME set by setHome()
    */
-  void loadConfigureFile(const std::filesystem::path& configuration_file);
+  void loadConfigureFile(const std::filesystem::path& configuration_file, const std::string_view prefix = "");

Review Comment:
   the `const` doesn't do anything (except triggering a clang-tidy warning)



##########
libminifi/include/EventDrivenSchedulingAgent.h:
##########
@@ -38,11 +38,15 @@ class EventDrivenSchedulingAgent : public ThreadedSchedulingAgent {
                              std::shared_ptr<core::Repository> flow_repo, std::shared_ptr<core::ContentRepository> content_repo, std::shared_ptr<Configure> configuration,
                              utils::ThreadPool<utils::TaskRescheduleInfo> &thread_pool)
       : ThreadedSchedulingAgent(controller_service_provider, repo, flow_repo, content_repo, configuration, thread_pool) {
-    int slice = configuration->getInt(Configure::nifi_flow_engine_event_driven_time_slice, DEFAULT_TIME_SLICE_MS);
-    if (slice < 10 || 1000 < slice) {
+    using namespace std::literals::chrono_literals;
+
+    time_slice_ = configuration->get(Configure::nifi_flow_engine_event_driven_time_slice)
+        | utils::flatMap(utils::timeutils::StringToDuration<std::chrono::milliseconds>)
+        | utils::valueOrElse([] { return std::chrono::milliseconds(DEFAULT_TIME_SLICE_MS);});

Review Comment:
   `DEFAULT_TIME_SLICE_MS` could be a `constexpr std::chrono::milliseconds` instead of a macro
   
   (also `SCHEDULING_WATCHDOG_DEFAULT_ALERT_PERIOD_MS` below)



##########
libminifi/include/utils/TimeUtil.h:
##########
@@ -201,38 +203,17 @@ std::optional<TargetDuration> cast_to_matching_unit(std::string& unit, const int
   ((result = cast_if_unit_matches<TargetDuration, T>(unit, value)) || ...);
   return result;
 }
-
-inline bool get_unit_and_value(const std::string& input, std::string& unit, int64_t& value) {
-  const char* begin = input.c_str();
-  char *end;
-  errno = 0;
-  value = std::strtoll(begin, &end, 0);
-  if (end == begin || errno == ERANGE) {
-    return false;
-  }
-
-  if (end[0] == '\0') {
-    return false;
-  }
-
-  while (*end == ' ') {
-    // Skip the spaces
-    end++;
-  }
-  unit = std::string(end);
-  std::transform(unit.begin(), unit.end(), unit.begin(), ::tolower);
-  return true;
-}
-
 }  // namespace details
 
 template<class TargetDuration>
 std::optional<TargetDuration> StringToDuration(const std::string& input) {
   std::string unit;
   int64_t value;
-  if (!details::get_unit_and_value(input, unit, value))
+  if (!StringUtils::splitToUnitAndValue(input, unit, value))
     return std::nullopt;
 
+  std::transform(unit.begin(), unit.end(), unit.begin(), ::tolower);

Review Comment:
   we could use `StringUtils::toLower()` here



##########
libminifi/src/properties/Properties.cpp:
##########
@@ -62,8 +63,100 @@ int Properties::getInt(const std::string &key, int default_value) const {
   return it != properties_.end() ? std::stoi(it->second.active_value) : default_value;
 }
 
+namespace {
+const core::PropertyValidator* getValidator(const std::string& lookup_value) {
+  auto configuration_property = Configuration::CONFIGURATION_PROPERTIES.find(lookup_value);
+
+  if (configuration_property != Configuration::CONFIGURATION_PROPERTIES.end())
+    return configuration_property->second;
+  return nullptr;
+}
+
+std::optional<std::string> ensureTimePeriodValidatedPropertyHasExplicitUnit(const core::PropertyValidator* const validator, std::string& value) {
+  if (validator != core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())
+    return std::nullopt;
+  if (value.empty() || !std::all_of(value.begin(), value.end(), ::isdigit))

Review Comment:
   the same `unsigned char` thingie applies to `isdigit` as to `toupper`/`tolower`



##########
libminifi/include/utils/StringUtils.h:
##########
@@ -505,7 +505,7 @@ class StringUtils {
    */
   static bool matchesSequence(std::string_view str, const std::vector<std::string>& patterns);
 
- private:
+  static bool splitToUnitAndValue(std::string_view input, std::string& unit, int64_t& value);

Review Comment:
   can we also switch the arguments, please?
   ```suggestion
     static bool splitToUnitAndValue(std::string_view input, int64_t& value, std::string& unit);
   ```
   that would look more logical to me



##########
libminifi/include/core/TypedValues.h:
##########
@@ -110,20 +110,26 @@ class DataSizeValue : public TransformableValue, public state::response::UInt64V
       : state::response::UInt64Value(0) {
   }
 
-
-  // Convert String to Integer
-  template<typename T, typename std::enable_if<
-      std::is_integral<T>::value>::type* = nullptr>
-  static bool StringToInt(const std::string &input, T &output) {
+  static std::optional<uint64_t> getUnitMultiplier(std::string unit_str) {
     // TODO(adebreceni): this mapping is to preserve backwards compatibility,
     //  we should entertain the idea of moving to standardized units in
     //  the configuration (i.e. K = 1000, Ki = 1024)
     static std::map<std::string, int64_t> unit_map{
-      {"B", 1},
-      {"K", 1_KB}, {"M", 1_MB}, {"G", 1_GB}, {"T", 1_TB}, {"P", 1_PB},
-      {"KB", 1_KiB}, {"MB", 1_MiB}, {"GB", 1_GiB}, {"TB", 1_TiB}, {"PB", 1_PiB},
+        {"B", 1},
+        {"K", 1_KB}, {"M", 1_MB}, {"G", 1_GB}, {"T", 1_TB}, {"P", 1_PB},
+        {"KB", 1_KiB}, {"MB", 1_MiB}, {"GB", 1_GiB}, {"TB", 1_TiB}, {"PB", 1_PiB},
     };
+    std::transform(unit_str.begin(), unit_str.end(), unit_str.begin(), ::toupper);

Review Comment:
   I know it's old code, but this should be
   ```suggestion
       std::transform(unit_str.begin(), unit_str.end(), unit_str.begin(), [](unsigned char c){ return ::toupper(c); });
   ```
   (see https://en.cppreference.com/w/cpp/string/byte/toupper)
   
   EDIT: we have `StringUtils::toLower()` which does this correctly; maybe we should add `StringUtils::toUpper()`, as well, and use that instead of `transform` + `::toupper` directly?
   
   EDIT2: and possibly move both to the `StringUtils` header, so they can be inlined



##########
libminifi/include/core/TypedValues.h:
##########
@@ -110,20 +110,26 @@ class DataSizeValue : public TransformableValue, public state::response::UInt64V
       : state::response::UInt64Value(0) {
   }
 
-
-  // Convert String to Integer
-  template<typename T, typename std::enable_if<
-      std::is_integral<T>::value>::type* = nullptr>
-  static bool StringToInt(const std::string &input, T &output) {
+  static std::optional<uint64_t> getUnitMultiplier(std::string unit_str) {
     // TODO(adebreceni): this mapping is to preserve backwards compatibility,
     //  we should entertain the idea of moving to standardized units in
     //  the configuration (i.e. K = 1000, Ki = 1024)
     static std::map<std::string, int64_t> unit_map{
-      {"B", 1},
-      {"K", 1_KB}, {"M", 1_MB}, {"G", 1_GB}, {"T", 1_TB}, {"P", 1_PB},
-      {"KB", 1_KiB}, {"MB", 1_MiB}, {"GB", 1_GiB}, {"TB", 1_TiB}, {"PB", 1_PiB},
+        {"B", 1},
+        {"K", 1_KB}, {"M", 1_MB}, {"G", 1_GB}, {"T", 1_TB}, {"P", 1_PB},
+        {"KB", 1_KiB}, {"MB", 1_MiB}, {"GB", 1_GiB}, {"TB", 1_TiB}, {"PB", 1_PiB},
     };
+    std::transform(unit_str.begin(), unit_str.end(), unit_str.begin(), ::toupper);
+    auto unit_multiplier_it = unit_map.find(unit_str);
+    if (unit_multiplier_it == unit_map.end())
+      return std::nullopt;
+    return unit_multiplier_it->second;
+  }
 
+  // Convert String to Integer
+  template<typename T, typename std::enable_if<
+      std::is_integral<T>::value>::type* = nullptr>

Review Comment:
   I can't remember if we can use the `<concepts>` header yet, but
   ```suggestion
     template<std::integral T>
   ```
   would be much nicer



##########
libminifi/include/core/TypedValues.h:
##########
@@ -110,20 +110,26 @@ class DataSizeValue : public TransformableValue, public state::response::UInt64V
       : state::response::UInt64Value(0) {
   }
 
-
-  // Convert String to Integer
-  template<typename T, typename std::enable_if<
-      std::is_integral<T>::value>::type* = nullptr>
-  static bool StringToInt(const std::string &input, T &output) {
+  static std::optional<uint64_t> getUnitMultiplier(std::string unit_str) {

Review Comment:
   there is no point in converting the return value to `uint64_t`, as it will be converted back to `int64_t` at line 143 (with a "loss of precision" clang-tidy warning)
   ```suggestion
     static std::optional<int64_t> getUnitMultiplier(std::string unit_str) {
   ```



##########
libminifi/src/properties/Properties.cpp:
##########
@@ -62,8 +63,100 @@ int Properties::getInt(const std::string &key, int default_value) const {
   return it != properties_.end() ? std::stoi(it->second.active_value) : default_value;
 }
 
+namespace {
+const core::PropertyValidator* getValidator(const std::string& lookup_value) {
+  auto configuration_property = Configuration::CONFIGURATION_PROPERTIES.find(lookup_value);
+
+  if (configuration_property != Configuration::CONFIGURATION_PROPERTIES.end())
+    return configuration_property->second;
+  return nullptr;
+}
+
+std::optional<std::string> ensureTimePeriodValidatedPropertyHasExplicitUnit(const core::PropertyValidator* const validator, std::string& value) {
+  if (validator != core::StandardValidators::get().TIME_PERIOD_VALIDATOR.get())
+    return std::nullopt;
+  if (value.empty() || !std::all_of(value.begin(), value.end(), ::isdigit))
+    return std::nullopt;
+
+  return value + " ms";
+}
+
+std::optional<std::string> ensureDataSizeValidatedPropertyHasExplicitUnit(const core::PropertyValidator* const validator, std::string& value) {
+  if (validator != core::StandardValidators::get().DATA_SIZE_VALIDATOR.get())
+    return std::nullopt;
+  if (value.empty() || !std::all_of(value.begin(), value.end(), ::isdigit))
+    return std::nullopt;
+
+  return value + " B";
+}
+
+bool integerValidatedProperty(const core::PropertyValidator* const validator) {
+  return validator == core::StandardValidators::get().INTEGER_VALIDATOR.get()
+      || validator == core::StandardValidators::get().UNSIGNED_INT_VALIDATOR.get()
+      || validator == core::StandardValidators::get().LONG_VALIDATOR.get()
+      || validator == core::StandardValidators::get().UNSIGNED_LONG_VALIDATOR.get();
+}
+
+std::optional<uint64_t> stringToDataSize(std::string_view input) {

Review Comment:
   if we change `getUnitMultiplier()` to return `int64_t`, then this should be changed to `int64_t`, too
   
   otherwise, some `gsl::narrow`s can be added, that is fine, too



##########
libminifi/src/c2/C2Agent.cpp:
##########
@@ -180,17 +180,18 @@ void C2Agent::configure(const std::shared_ptr<Configure> &configure, bool reconf
     try {
       if (auto heartbeat_period_ms = utils::timeutils::StringToDuration<std::chrono::milliseconds>(heartbeat_period)) {
         heart_beat_period_ = *heartbeat_period_ms;
-        logger_->log_debug("Using %u ms as the heartbeat period", heart_beat_period_.count());
       } else {
         heart_beat_period_ = std::chrono::milliseconds(std::stoi(heartbeat_period));
       }
     } catch (const std::invalid_argument &) {
+      logger_->log_error("Invalid heartbeat period: %s", heartbeat_period);
       heart_beat_period_ = 3s;
     }
   } else {
     if (!reconfigure)
       heart_beat_period_ = 3s;
   }
+  logger_->log_debug("Using %u ms as the heartbeat period", heart_beat_period_.count());

Review Comment:
   old code + nitpicking, but
   ```suggestion
     logger_->log_debug("Using %" PRId64 " ms as the heartbeat period", heart_beat_period_.count());
   ```



##########
libminifi/src/utils/StringUtils.cpp:
##########
@@ -510,4 +511,20 @@ bool StringUtils::matchesSequence(std::string_view str, const std::vector<std::s
   return true;
 }
 
+bool StringUtils::splitToUnitAndValue(std::string_view input, std::string& unit, int64_t& value) {
+  const char* begin = input.data();
+  const char* end = begin + input.size();

Review Comment:
   minor improvement, but I think
   ```suggestion
     const char* begin = std::begin(input);
     const char* end = std::end(input);
   ```
   would be nicer



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