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 2021/12/13 12:42:01 UTC

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1225: MINIFICPP-1688: When storing time durations we should use std::chrono…

szaszm commented on a change in pull request #1225:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1225#discussion_r767636169



##########
File path: extensions/mqtt/processors/AbstractMQTTProcessor.h
##########
@@ -130,8 +128,8 @@ class AbstractMQTTProcessor : public core::Processor {
   MQTTClient_deliveryToken delivered_token_;
   std::string uri_;
   std::string topic_;
-  int64_t keepAliveInterval_;
-  int64_t connectionTimeOut_;
+  std::chrono::seconds keepAliveInterval_{60};
+  std::chrono::seconds connectionTimeOut_{30};

Review comment:
       There is a bug here. [`MQTTClient_disconnect`](https://www.eclipse.org/paho/files/mqttdoc/MQTTClient/html/_m_q_t_t_client_8h.html#a1e4d90c13a3c0705bc4a13bfe64e6525) expects milliseconds, while [`MQTTClient_connectOptions`](https://www.eclipse.org/paho/files/mqttdoc/MQTTClient/html/struct_m_q_t_t_client__connect_options.html#ac8dd0930672a9c7d71fc645aa1f0521d) expects timeouts in seconds. (Madness, but it is what it is.)
   I think we should store at least `connectionTimeOut_` as milliseconds and check all usages of these to make sure that they are appropriately converted.

##########
File path: extensions/rocksdb-repos/FlowFileRepository.h
##########
@@ -51,9 +51,9 @@ namespace repository {
 #define FLOWFILE_CHECKPOINT_DIRECTORY "./flowfile_checkpoint"
 #endif
 #define MAX_FLOWFILE_REPOSITORY_STORAGE_SIZE (10*1024*1024)  // 10M
-#define MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME (600000)  // 10 minute
-#define FLOWFILE_REPOSITORY_PURGE_PERIOD (2000)  // 2000 msec
-#define FLOWFILE_REPOSITORY_RETRY_INTERVAL_INCREMENTS (500)  // msec
+constexpr std::chrono::milliseconds MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME = std::chrono::minutes(10);
+constexpr std::chrono::milliseconds FLOWFILE_REPOSITORY_PURGE_PERIOD = std::chrono::seconds(2);
+constexpr std::chrono::milliseconds FLOWFILE_REPOSITORY_RETRY_INTERVAL_INCREMENTS = std::chrono::milliseconds(500);

Review comment:
       Why convert them to milliseconds prematurely? I would prefer to keep their normal type and convert when it's actually needed. This happens to be the same behavior as with `auto`.
   ```suggestion
   using namespace std::literals;
   constexpr auto MAX_FLOWFILE_REPOSITORY_ENTRY_LIFE_TIME = 10m;
   constexpr auto FLOWFILE_REPOSITORY_PURGE_PERIOD = 2s;
   constexpr auto FLOWFILE_REPOSITORY_RETRY_INTERVAL_INCREMENTS = 500ms;
   ```

##########
File path: extensions/sftp/processors/ListSFTP.cpp
##########
@@ -359,21 +360,21 @@ bool ListSFTP::filterFile(const std::string& parent_path, const std::string& fil
 
   /* Age */
   time_t now = time(nullptr);
-  uint64_t file_age = (now - attrs.mtime) * 1000;
+  std::chrono::milliseconds file_age = std::chrono::seconds(now - attrs.mtime);

Review comment:
       ```c++
     std::chrono::milliseconds file_age = std::chrono::file_clock::now() - std::chrono::seconds(attrs.mtime);
   ```

##########
File path: libminifi/include/utils/ValueParser.h
##########
@@ -27,6 +27,7 @@
 #include <type_traits>
 #include <limits>
 #include <algorithm>
+#include <chrono>

Review comment:
       This is not used.

##########
File path: libminifi/include/utils/TimeUtil.h
##########
@@ -197,6 +193,100 @@ inline bool getDateTimeStr(int64_t unix_timestamp, std::string& date_time_str) {
   return true;
 }
 
+namespace details {
+
+template<class Duration>
+bool unit_matches(const std::string&) {
+  return false;
+}
+
+template<>
+inline bool unit_matches<std::chrono::nanoseconds>(const std::string& unit) {
+  return unit == "ns" || unit == "nano" || unit == "nanos" || unit == "nanoseconds";
+}
+
+template<>
+inline bool unit_matches<std::chrono::microseconds>(const std::string& unit) {
+  return unit == "us" || unit == "micro" || unit == "micros" || unit == "microseconds" || unit == "microsecond";
+}
+
+template<>
+inline bool unit_matches<std::chrono::milliseconds>(const std::string& unit) {
+  return unit == "msec" || unit == "ms" || unit == "millisecond" || unit == "milliseconds" || unit == "msecs" || unit == "millis" || unit == "milli";
+}
+
+template<>
+inline bool unit_matches<std::chrono::seconds>(const std::string& unit) {
+  return unit == "sec" || unit == "s" || unit == "second" || unit == "seconds" || unit == "secs";
+}
+
+template<>
+inline bool unit_matches<std::chrono::minutes>(const std::string& unit) {
+  return unit == "min" || unit == "m" || unit == "mins" || unit == "minute" || unit == "minutes";
+}
+
+template<>
+inline bool unit_matches<std::chrono::hours>(const std::string& unit) {
+  return unit == "h" || unit == "hr" || unit == "hour" || unit == "hrs" || unit == "hours";
+}
+
+template<>
+inline bool unit_matches<std::chrono::days>(const std::string& unit) {
+  return unit == "d" || unit == "day" || unit == "days";
+}
+
+
+template<class TargetDuration, class SourceDuration>
+std::optional<TargetDuration> cast_if_unit_matches(const std::string& unit, const int64_t value) {
+  if (unit_matches<SourceDuration>(unit)) {
+    return std::chrono::duration_cast<TargetDuration>(SourceDuration(value));
+  } else {
+    return std::nullopt;
+  }
+}
+
+template<class TargetDuration, typename... T>
+std::optional<TargetDuration> cast_to_matching_unit(std::string& unit, const int64_t value) {
+  std::optional<TargetDuration> result;
+  ((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) {
+  std::stringstream input_stream(input);
+  input_stream >> value >> unit;
+  std::transform(unit.begin(), unit.end(), unit.begin(), ::tolower);
+  return !input_stream.fail();
+}
+
+}  // 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))
+    return std::nullopt;
+
+  return details::cast_to_matching_unit<TargetDuration,
+    std::chrono::nanoseconds,
+    std::chrono::microseconds,
+    std::chrono::milliseconds,
+    std::chrono::seconds,
+    std::chrono::minutes,
+    std::chrono::hours,
+    std::chrono::days>(unit, value);
+}
+
+
+inline std::chrono::milliseconds getMillisecondsSinceUnixEpoch(const std::chrono::time_point<std::chrono::system_clock> time_point) {
+  return std::chrono::duration_cast<std::chrono::milliseconds>(time_point.time_since_epoch());
+}
+
+inline std::chrono::time_point<std::chrono::system_clock> fromMillisecondsSinceUnixEpoch(const std::chrono::milliseconds milliseconds_since_unix_epoch) {
+  return std::chrono::time_point<std::chrono::system_clock>() + milliseconds_since_unix_epoch;
+}

Review comment:
       I prefer to have these inlined manually. They are longer only because of long names, but there is not much extra value added by these function over just calling the constructor or the `.time_since_epoch()` member function and a cast directly IMO. I'm not hard set on this, just an opinion.

##########
File path: extensions/sftp/processors/ListSFTP.cpp
##########
@@ -965,25 +966,18 @@ void ListSFTP::onTrigger(const std::shared_ptr<core::ProcessContext> &context, c
 
   /* Parse processor-specific properties */
   std::string remote_path;
-  uint64_t entity_tracking_time_window = 0U;
+  std::chrono::milliseconds entity_tracking_time_window = 3h;  /* The default is 3 hours */
 
   std::string value;
   context->getProperty(RemotePath.getName(), remote_path);
   /* Remove trailing slashes */
   while (remote_path.size() > 1U && remote_path.back() == '/') {
     remote_path.resize(remote_path.size() - 1);
   }
-  if (context->getProperty(EntityTrackingTimeWindow.getName(), value)) {
-    core::TimeUnit unit;
-    if (!core::Property::StringToTime(value, entity_tracking_time_window, unit) ||
-        !core::Property::ConvertTimeUnitToMS(entity_tracking_time_window, unit, entity_tracking_time_window)) {
-      /* The default is 3 hours */
-      entity_tracking_time_window = 3 * 3600 * 1000;
-      logger_->log_error("Entity Tracking Time Window attribute is invalid");
-    }
+  if (auto entity_tracking_window = context->getProperty<core::TimePeriodValue>(EntityTrackingTimeWindow)) {
+    entity_tracking_time_window = entity_tracking_window->getMilliseconds();
   } else {
-    /* The default is 3 hours */
-    entity_tracking_time_window = 3 * 3600 * 1000;
+    logger_->log_error("Entity Tracking Time Window attribute is invalid");

Review comment:
       I think `getProperty` returns `false` when the property is not required and missing. Invalid values cause exceptions. This log message seems to be invalid, because this property is not required, so it's not an error to have it be empty and thus `getProperty` returning `false` is valid.

##########
File path: extensions/rocksdb-repos/ProvenanceRepository.h
##########
@@ -38,8 +38,8 @@ namespace provenance {
 
 #define PROVENANCE_DIRECTORY "./provenance_repository"
 #define MAX_PROVENANCE_STORAGE_SIZE (10*1024*1024)  // 10M
-#define MAX_PROVENANCE_ENTRY_LIFE_TIME (60000)  // 1 minute
-#define PROVENANCE_PURGE_PERIOD (2500)  // 2500 msec
+constexpr std::chrono::milliseconds MAX_PROVENANCE_ENTRY_LIFE_TIME = std::chrono::minutes(1);
+constexpr std::chrono::milliseconds PROVENANCE_PURGE_PERIOD = std::chrono::milliseconds(2500);

Review comment:
       ```suggestion
   using namespace std::literals;
   constexpr auto MAX_PROVENANCE_ENTRY_LIFE_TIME = 1m;
   constexpr auto PROVENANCE_PURGE_PERIOD = 2500ms;
   ```

##########
File path: libminifi/include/core/Repository.h
##########
@@ -56,17 +56,19 @@ namespace core {
 
 #define REPOSITORY_DIRECTORY "./repo"
 #define MAX_REPOSITORY_STORAGE_SIZE (10*1024*1024)  // 10M
-#define MAX_REPOSITORY_ENTRY_LIFE_TIME (600000)  // 10 minute
-#define REPOSITORY_PURGE_PERIOD (2500)  // 2500 msec
+constexpr std::chrono::milliseconds MAX_REPOSITORY_ENTRY_LIFE_TIME = std::chrono::minutes(10);
+constexpr std::chrono::milliseconds REPOSITORY_PURGE_PERIOD = std::chrono::milliseconds(2500);

Review comment:
       ```suggestion
   using namespace std::literals;
   constexpr auto MAX_REPOSITORY_ENTRY_LIFE_TIME = 10m;
   constexpr auto REPOSITORY_PURGE_PERIOD = 2500ms;
   ```

##########
File path: extensions/sftp/processors/ListSFTP.cpp
##########
@@ -261,20 +263,19 @@ void ListSFTP::onSchedule(const std::shared_ptr<core::ProcessContext> &context,
   }
   context->getProperty(TargetSystemTimestampPrecision.getName(), target_system_timestamp_precision_);
   context->getProperty(EntityTrackingInitialListingTarget.getName(), entity_tracking_initial_listing_target_);
-  if (!context->getProperty(MinimumFileAge.getName(), value)) {
-    logger_->log_error("Minimum File Age attribute is missing or invalid");
+
+  if (auto minimum_file_age = context->getProperty<core::TimePeriodValue>(MinimumFileAge)) {
+    minimum_file_age_ = minimum_file_age->getMilliseconds();
   } else {
-    core::TimeUnit unit;
-    if (!core::Property::StringToTime(value, minimum_file_age_, unit) || !core::Property::ConvertTimeUnitToMS(minimum_file_age_, unit, minimum_file_age_)) {
-      logger_->log_error("Minimum File Age attribute is invalid");
-    }
+    logger_->log_error("Minimum File Age attribute is missing or invalid");
   }
-  if (context->getProperty(MaximumFileAge.getName(), value)) {
-    core::TimeUnit unit;
-    if (!core::Property::StringToTime(value, maximum_file_age_, unit) || !core::Property::ConvertTimeUnitToMS(maximum_file_age_, unit, maximum_file_age_)) {
-      logger_->log_error("Maximum File Age attribute is invalid");
-    }
+
+  if (auto maximum_file_age = context->getProperty<core::TimePeriodValue>(MaximumFileAge)) {
+    maximum_file_age_ = maximum_file_age->getMilliseconds();
+  } else {
+    logger_->log_error("Maximum File Age attribute is missing or invalid");

Review comment:
       An exception could happen that couldn't happen before: `getProperty` throws on invalid value, and the conversion got pushed into `getProperty`. An exception thrown from `onSchedule` prevents the processor from starting, while the old behavior was to log an error and proceed with whatever default was set in the member. Please keep the old behavior if the property is invalid.




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