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 2020/08/19 18:03:24 UTC

[GitHub] [nifi-minifi-cpp] szaszm opened a new pull request #875: WIP: Disk space watchdog prototype

szaszm opened a new pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875


   Stop the FlowController if available disk space is <15MB, restart when it goes above 20MB.
   
   ___
   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 travis-ci 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r498035883



##########
File path: libminifi/include/DiskSpaceWatchdog.h
##########
@@ -0,0 +1,60 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <chrono>
+#include <cinttypes>
+#include <string>
+#include <vector>
+
+#include "utils/IntervalSwitch.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+class Configure;
+namespace core {
+namespace logging {
+class Logger;
+}  // namespace logging
+}  // namespace core
+
+namespace disk_space_watchdog {
+struct Config {
+  std::chrono::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+Config read_config(const Configure&);
+
+inline utils::IntervalSwitch<std::uintmax_t> disk_space_interval_switch(Config config) {

Review comment:
       It would be better to pass config by const ref.

##########
File path: libminifi/src/utils/file/PathUtils.cpp
##########
@@ -84,13 +86,58 @@ std::string PathUtils::getFullPath(const std::string& path) {
 #endif
 }
 
-std::string PathUtils::globToRegex(std::string glob) {
+std::string globToRegex(std::string glob) {
   utils::StringUtils::replaceAll(glob, ".", "\\.");
   utils::StringUtils::replaceAll(glob, "*", ".*");
   utils::StringUtils::replaceAll(glob, "?", ".");
   return glob;
 }
 
+space_info space(const path p, std::error_code& ec) noexcept {

Review comment:
       Please rename 'p' to something more searchable as it has a bit larger scope. If it clashes with the type name it is usually recommended to use the '_' suffix in these cases.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r478526342



##########
File path: libminifi/include/utils/file/PathUtils.h
##########
@@ -47,7 +53,52 @@ std::string getFullPath(const std::string& path);
 
 std::string globToRegex(std::string glob);
 
-}  // namespace PathUtils
+inline bool isAbsolutePath(const char* const path) noexcept {
+#ifdef _WIN32
+  return path && std::isalpha(path[0]) && path[1] == ':' && (path[2] == '\\' || path[2] == '/');
+#else
+  return path && path[0] == '/';
+#endif
+}
+
+
+/**
+ * Represents filesystem space information in bytes
+ */
+struct space_info {
+  std::uintmax_t capacity;
+  std::uintmax_t free;
+  std::uintmax_t available;
+
+  friend bool operator==(const space_info& a, const space_info& b) noexcept {
+    return a.capacity == b.capacity && a.free == b.free && a.available == b.available;
+  }
+};
+
+class filesystem_error : public std::system_error {
+  std::shared_ptr<const std::pair<std::string, std::string>> paths_involved_;

Review comment:
       fixed in 13116f0




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r476314573



##########
File path: libminifi/include/utils/OptionalUtils.h
##########
@@ -37,11 +39,47 @@ optional<typename gsl_lite::remove_cvref<T>::type> optional_from_ptr(T&& obj) {
   return obj == nullptr ? nullopt : optional<typename gsl_lite::remove_cvref<T>::type>{ std::forward<T>(obj) };
 }
 
+template<typename>
+struct is_optional : std::false_type {};
+
+template<typename T>
+struct is_optional<optional<T>> : std::true_type {};
+
 }  // namespace utils
 }  // namespace minifi
 }  // namespace nifi
 }  // namespace apache
 }  // namespace org
 
+namespace nonstd {
+namespace optional_lite {
+
+namespace minifiutils = org::apache::nifi::minifi::utils;
+
+// map
+template<typename SourceType, typename F>
+auto operator|(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))
+    -> optional<typename std::decay<decltype(minifiutils::invoke(std::forward<F>(f), *o))>::type> {
+  if (o.has_value()) {
+    return make_optional(minifiutils::invoke(std::forward<F>(f), *o));
+  } else {
+    return nullopt;
+  }
+}
+
+// bind
+template<typename SourceType, typename F>
+auto operator>>=(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))

Review comment:
       I'd rather we use the conventional `map` and `flatMap` names, could we create a custom `optional` class (with `map` and `flatMap`) extending the `nonstd::optional`, and make that available as `utils::optional`?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r478525506



##########
File path: libminifi/src/DiskSpaceWatchdog.cpp
##########
@@ -0,0 +1,133 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "DiskSpaceWatchdog.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <type_traits>
+
+#include "core/Property.h"
+#include "FlowController.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
+#include "utils/file/PathUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace {
+namespace chr = std::chrono;
+struct Config {
+  chr::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+utils::optional<chr::milliseconds> string_to_milliseconds(const std::string& str) {
+  uint64_t millisec_value{};
+  const bool success = core::Property::getTimeMSFromString(str, millisec_value);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(chr::milliseconds{millisec_value});
+}
+
+template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
+utils::optional<T> string_to_int(const std::string& str) {
+  T result{};
+  const bool success = core::Property::StringToInt(str, result);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(result);
+}
+
+Config read_config(const Configure& conf) {
+  const auto interval_ms = conf.get(Configure::minifi_disk_space_watchdog_interval_ms) >>= string_to_milliseconds;
+  const auto stop_bytes = conf.get(Configure::minifi_disk_space_watchdog_stop_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  const auto restart_bytes = conf.get(Configure::minifi_disk_space_watchdog_restart_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  if (restart_bytes < stop_bytes) { throw std::runtime_error{"disk space watchdog stop threshold must be <= restart threshold"}; }
+  constexpr auto mebibytes = 1024 * 1024;
+  return {
+      interval_ms.value_or(chr::seconds{1}),
+      stop_bytes.value_or(15 * mebibytes),
+      restart_bytes.value_or(20 * mebibytes)
+  };
+}
+
+struct callback {
+  Config cfg;
+  gsl::not_null<FlowController*> flow_controller;
+  bool stopped;

Review comment:
       Thanks, I really appreciate the info, I did not know about this C++11 specific restriction.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r476331404



##########
File path: libminifi/include/utils/OptionalUtils.h
##########
@@ -37,11 +39,47 @@ optional<typename gsl_lite::remove_cvref<T>::type> optional_from_ptr(T&& obj) {
   return obj == nullptr ? nullopt : optional<typename gsl_lite::remove_cvref<T>::type>{ std::forward<T>(obj) };
 }
 
+template<typename>
+struct is_optional : std::false_type {};
+
+template<typename T>
+struct is_optional<optional<T>> : std::true_type {};
+
 }  // namespace utils
 }  // namespace minifi
 }  // namespace nifi
 }  // namespace apache
 }  // namespace org
 
+namespace nonstd {
+namespace optional_lite {
+
+namespace minifiutils = org::apache::nifi::minifi::utils;
+
+// map
+template<typename SourceType, typename F>
+auto operator|(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))
+    -> optional<typename std::decay<decltype(minifiutils::invoke(std::forward<F>(f), *o))>::type> {
+  if (o.has_value()) {
+    return make_optional(minifiutils::invoke(std::forward<F>(f), *o));
+  } else {
+    return nullopt;
+  }
+}
+
+// bind
+template<typename SourceType, typename F>
+auto operator>>=(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))

Review comment:
       I'd prefer to avoid modifying the optional class. What do you think about this syntax?
   ```
   utils::optional<T> opt{...};
   utils::optional<U> map_result = opt | utils::map([](T) -> U {...});
   utils::optional<U> flat_map_result = opt | utils::flatMap([](T) -> utils::optional<U>{...});
   ```




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r497621072



##########
File path: libminifi/include/utils/IntervalSwitch.h
##########
@@ -0,0 +1,80 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <functional>
+#include <utility>
+
+#include "utils/gsl.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+enum class IntervalSwitchState {
+  LOWER,
+  UPPER,
+};
+
+namespace detail {
+struct SwitchReturn {
+  IntervalSwitchState state;
+  bool switched;
+};
+}  // namespace detail
+
+template<typename T, typename Comp = std::less<T>>
+class IntervalSwitch {
+ public:
+  IntervalSwitch(T lower_threshold, T upper_threshold, const IntervalSwitchState initial_state = IntervalSwitchState::UPPER)
+      :lower_threshold_{std::move(lower_threshold)}, upper_threshold_{std::move(upper_threshold)}, state_{initial_state} {
+    gsl_Expects(!less_(upper_threshold_, lower_threshold_));
+  }
+
+  detail::SwitchReturn operator()(const T& value) {

Review comment:
       My intention was to write code that describes the intent as much as possible, but you're right, I wrote code that's describing trivialities. I'll change it to be shorter.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r497617784



##########
File path: libminifi/include/properties/Properties.h
##########
@@ -82,6 +84,16 @@ class Properties {
    */
   int getInt(const std::string &key, int default_value) const;
 
+  utils::optional<std::string> get(const std::string& key) const {
+    std::string result;
+    const bool found = get(key, result);

Review comment:
       I totally agree but they need to be of the same type to be able to use the ternary operator and in this case I rely on conversions from different types to construct the return 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r498129745



##########
File path: libminifi/include/DiskSpaceWatchdog.h
##########
@@ -0,0 +1,60 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <chrono>
+#include <cinttypes>
+#include <string>
+#include <vector>
+
+#include "utils/IntervalSwitch.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+class Configure;
+namespace core {
+namespace logging {
+class Logger;
+}  // namespace logging
+}  // namespace core
+
+namespace disk_space_watchdog {
+struct Config {
+  std::chrono::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+Config read_config(const Configure&);
+
+inline utils::IntervalSwitch<std::uintmax_t> disk_space_interval_switch(Config config) {

Review comment:
       You can leave it as it is, I do not insist on it if it's still cheap to copy. My only concern was if this config may be expanded in the future, but I suppose the watchdog configuration will not really 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r478528737



##########
File path: libminifi/include/utils/OptionalUtils.h
##########
@@ -37,11 +39,47 @@ optional<typename gsl_lite::remove_cvref<T>::type> optional_from_ptr(T&& obj) {
   return obj == nullptr ? nullopt : optional<typename gsl_lite::remove_cvref<T>::type>{ std::forward<T>(obj) };
 }
 
+template<typename>
+struct is_optional : std::false_type {};
+
+template<typename T>
+struct is_optional<optional<T>> : std::true_type {};
+
 }  // namespace utils
 }  // namespace minifi
 }  // namespace nifi
 }  // namespace apache
 }  // namespace org
 
+namespace nonstd {
+namespace optional_lite {
+
+namespace minifiutils = org::apache::nifi::minifi::utils;
+
+// map
+template<typename SourceType, typename F>
+auto operator|(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))
+    -> optional<typename std::decay<decltype(minifiutils::invoke(std::forward<F>(f), *o))>::type> {
+  if (o.has_value()) {
+    return make_optional(minifiutils::invoke(std::forward<F>(f), *o));
+  } else {
+    return nullopt;
+  }
+}
+
+// bind
+template<typename SourceType, typename F>
+auto operator>>=(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))

Review comment:
       fixed in 13116f0 and fdcec88




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r474207680



##########
File path: libminifi/include/properties/Properties.h
##########
@@ -82,6 +84,16 @@ class Properties {
    */
   int getInt(const std::string &key, int default_value) const;
 
+  utils::optional<std::string> get(const std::string& key) const noexcept {

Review comment:
       TODO: this shouldn't be `noexcept`




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r476344817



##########
File path: libminifi/include/utils/OptionalUtils.h
##########
@@ -37,11 +39,47 @@ optional<typename gsl_lite::remove_cvref<T>::type> optional_from_ptr(T&& obj) {
   return obj == nullptr ? nullopt : optional<typename gsl_lite::remove_cvref<T>::type>{ std::forward<T>(obj) };
 }
 
+template<typename>
+struct is_optional : std::false_type {};
+
+template<typename T>
+struct is_optional<optional<T>> : std::true_type {};
+
 }  // namespace utils
 }  // namespace minifi
 }  // namespace nifi
 }  // namespace apache
 }  // namespace org
 
+namespace nonstd {
+namespace optional_lite {
+
+namespace minifiutils = org::apache::nifi::minifi::utils;
+
+// map
+template<typename SourceType, typename F>
+auto operator|(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))
+    -> optional<typename std::decay<decltype(minifiutils::invoke(std::forward<F>(f), *o))>::type> {
+  if (o.has_value()) {
+    return make_optional(minifiutils::invoke(std::forward<F>(f), *o));
+  } else {
+    return nullopt;
+  }
+}
+
+// bind
+template<typename SourceType, typename F>
+auto operator>>=(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))

Review comment:
       yes, that could work




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r498129867



##########
File path: libminifi/src/utils/file/PathUtils.cpp
##########
@@ -84,13 +86,58 @@ std::string PathUtils::getFullPath(const std::string& path) {
 #endif
 }
 
-std::string PathUtils::globToRegex(std::string glob) {
+std::string globToRegex(std::string glob) {
   utils::StringUtils::replaceAll(glob, ".", "\\.");
   utils::StringUtils::replaceAll(glob, "*", ".*");
   utils::StringUtils::replaceAll(glob, "?", ".");
   return glob;
 }
 
+space_info space(const path p, std::error_code& ec) noexcept {

Review comment:
       I'm fine with that as well.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r476369441



##########
File path: libminifi/src/DiskSpaceWatchdog.cpp
##########
@@ -0,0 +1,133 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "DiskSpaceWatchdog.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <type_traits>
+
+#include "core/Property.h"
+#include "FlowController.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
+#include "utils/file/PathUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace {
+namespace chr = std::chrono;
+struct Config {
+  chr::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+utils::optional<chr::milliseconds> string_to_milliseconds(const std::string& str) {
+  uint64_t millisec_value{};
+  const bool success = core::Property::getTimeMSFromString(str, millisec_value);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(chr::milliseconds{millisec_value});
+}
+
+template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
+utils::optional<T> string_to_int(const std::string& str) {
+  T result{};
+  const bool success = core::Property::StringToInt(str, result);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(result);
+}
+
+Config read_config(const Configure& conf) {
+  const auto interval_ms = conf.get(Configure::minifi_disk_space_watchdog_interval_ms) >>= string_to_milliseconds;
+  const auto stop_bytes = conf.get(Configure::minifi_disk_space_watchdog_stop_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  const auto restart_bytes = conf.get(Configure::minifi_disk_space_watchdog_restart_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  if (restart_bytes < stop_bytes) { throw std::runtime_error{"disk space watchdog stop threshold must be <= restart threshold"}; }
+  constexpr auto mebibytes = 1024 * 1024;
+  return {
+      interval_ms.value_or(chr::seconds{1}),
+      stop_bytes.value_or(15 * mebibytes),
+      restart_bytes.value_or(20 * mebibytes)
+  };
+}
+
+struct callback {

Review comment:
       Even for a small scope I think it would be better to use a more distinguished name and capitalized.

##########
File path: libminifi/include/utils/file/PathUtils.h
##########
@@ -47,7 +53,52 @@ std::string getFullPath(const std::string& path);
 
 std::string globToRegex(std::string glob);
 
-}  // namespace PathUtils
+inline bool isAbsolutePath(const char* const path) noexcept {
+#ifdef _WIN32
+  return path && std::isalpha(path[0]) && path[1] == ':' && (path[2] == '\\' || path[2] == '/');
+#else
+  return path && path[0] == '/';
+#endif
+}
+
+
+/**
+ * Represents filesystem space information in bytes
+ */
+struct space_info {
+  std::uintmax_t capacity;
+  std::uintmax_t free;
+  std::uintmax_t available;
+
+  friend bool operator==(const space_info& a, const space_info& b) noexcept {
+    return a.capacity == b.capacity && a.free == b.free && a.available == b.available;
+  }
+};
+
+class filesystem_error : public std::system_error {
+  std::shared_ptr<const std::pair<std::string, std::string>> paths_involved_;

Review comment:
       Minor: Move private members at the end of the class https://github.com/isocpp/CppCoreGuidelines/blob/master/CppCoreGuidelines.md#Rl-order

##########
File path: libminifi/src/DiskSpaceWatchdog.cpp
##########
@@ -0,0 +1,133 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "DiskSpaceWatchdog.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <type_traits>
+
+#include "core/Property.h"
+#include "FlowController.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
+#include "utils/file/PathUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace {
+namespace chr = std::chrono;
+struct Config {
+  chr::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+utils::optional<chr::milliseconds> string_to_milliseconds(const std::string& str) {
+  uint64_t millisec_value{};
+  const bool success = core::Property::getTimeMSFromString(str, millisec_value);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(chr::milliseconds{millisec_value});
+}
+
+template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
+utils::optional<T> string_to_int(const std::string& str) {
+  T result{};
+  const bool success = core::Property::StringToInt(str, result);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(result);
+}
+
+Config read_config(const Configure& conf) {
+  const auto interval_ms = conf.get(Configure::minifi_disk_space_watchdog_interval_ms) >>= string_to_milliseconds;
+  const auto stop_bytes = conf.get(Configure::minifi_disk_space_watchdog_stop_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  const auto restart_bytes = conf.get(Configure::minifi_disk_space_watchdog_restart_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  if (restart_bytes < stop_bytes) { throw std::runtime_error{"disk space watchdog stop threshold must be <= restart threshold"}; }
+  constexpr auto mebibytes = 1024 * 1024;
+  return {
+      interval_ms.value_or(chr::seconds{1}),
+      stop_bytes.value_or(15 * mebibytes),
+      restart_bytes.value_or(20 * mebibytes)
+  };
+}
+
+struct callback {
+  Config cfg;
+  gsl::not_null<FlowController*> flow_controller;
+  bool stopped;

Review comment:
       Initialize bool

##########
File path: libminifi/test/unit/GeneralUtilsTest.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <functional>
+#include <string>
+#include <type_traits>
+
+#include "../TestBase.h"
+#include "utils/GeneralUtils.h"
+
+namespace utils = org::apache::nifi::minifi::utils;
+
+static_assert(std::is_same<decltype(utils::make_unique<char16_t>()), std::unique_ptr<char16_t>>::value, "utils::make_unique type is correct");

Review comment:
       Shouldn't be the message "type is incorrect"? The message appears if the assertion fails. Same for the static_asserts below.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r498119240



##########
File path: libminifi/include/DiskSpaceWatchdog.h
##########
@@ -0,0 +1,60 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <chrono>
+#include <cinttypes>
+#include <string>
+#include <vector>
+
+#include "utils/IntervalSwitch.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+class Configure;
+namespace core {
+namespace logging {
+class Logger;
+}  // namespace logging
+}  // namespace core
+
+namespace disk_space_watchdog {
+struct Config {
+  std::chrono::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+Config read_config(const Configure&);
+
+inline utils::IntervalSwitch<std::uintmax_t> disk_space_interval_switch(Config config) {

Review comment:
       It has the size of 3 doubles/pointers on my machine and is trivially copiable, so I thought it's better to copy it than pass by const reference.
   https://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#Rf-in
   The text says that 2-3 doubles/pointers is cheap to copy, yet the enforcement section suggests flagging anything over `2*sizeof(void*)`. I'm still leaning towards copy, so let me know if you insist on the change given the above.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r473275694



##########
File path: libminifi/src/DiskSpaceWatchdog.cpp
##########
@@ -0,0 +1,96 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "DiskSpaceWatchdog.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <type_traits>
+
+#include "core/Property.h"
+#include "FlowController.h"
+#include "properties/Configure.h"
+#include "utils/file/PathUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace {
+namespace chr = std::chrono;
+struct Config {
+  chr::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+utils::optional<chr::milliseconds> string_to_milliseconds(const std::string& str) {
+  uint64_t millisec_value{};
+  const bool success = core::Property::getTimeMSFromString(str, millisec_value);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(chr::milliseconds{millisec_value});
+}
+
+template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
+utils::optional<T> string_to_int(const std::string& str) {
+  T result{};
+  const bool success = core::Property::StringToInt(str, result);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(result);
+}
+
+Config read_config(const Configure* const conf) {
+  const auto interval_ms = conf->get(Configure::minifi_disk_space_watchdog_interval_ms) >>= string_to_milliseconds;
+  const auto stop_bytes = conf->get(Configure::minifi_disk_space_watchdog_stop_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  const auto restart_bytes = conf->get(Configure::minifi_disk_space_watchdog_restart_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  if (restart_bytes < stop_bytes) { throw std::runtime_error{"disk space watchdog stop threshold must be <= restart threshold"}; }
+  constexpr auto mebibytes = 1024 * 1024;
+  return {
+      interval_ms.value_or(chr::milliseconds{100}),
+      stop_bytes.value_or(15 * mebibytes),
+      restart_bytes.value_or(20 * mebibytes)
+  };
+}
+
+struct callback {
+  Config cfg;
+  FlowController* flow_controller;
+  bool stopped;
+
+  void operator()() {
+    const auto spaceinfo = utils::file::space(".");

Review comment:
       TODO: check all repositories and the log directory separately
   TODO: handle exceptions




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r478532755



##########
File path: libminifi/src/DiskSpaceWatchdog.cpp
##########
@@ -0,0 +1,133 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "DiskSpaceWatchdog.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <type_traits>
+
+#include "core/Property.h"
+#include "FlowController.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
+#include "utils/file/PathUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace {
+namespace chr = std::chrono;
+struct Config {
+  chr::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+utils::optional<chr::milliseconds> string_to_milliseconds(const std::string& str) {
+  uint64_t millisec_value{};
+  const bool success = core::Property::getTimeMSFromString(str, millisec_value);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(chr::milliseconds{millisec_value});
+}
+
+template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
+utils::optional<T> string_to_int(const std::string& str) {
+  T result{};
+  const bool success = core::Property::StringToInt(str, result);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(result);
+}
+
+Config read_config(const Configure& conf) {
+  const auto interval_ms = conf.get(Configure::minifi_disk_space_watchdog_interval_ms) >>= string_to_milliseconds;
+  const auto stop_bytes = conf.get(Configure::minifi_disk_space_watchdog_stop_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  const auto restart_bytes = conf.get(Configure::minifi_disk_space_watchdog_restart_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  if (restart_bytes < stop_bytes) { throw std::runtime_error{"disk space watchdog stop threshold must be <= restart threshold"}; }
+  constexpr auto mebibytes = 1024 * 1024;
+  return {
+      interval_ms.value_or(chr::seconds{1}),
+      stop_bytes.value_or(15 * mebibytes),
+      restart_bytes.value_or(20 * mebibytes)
+  };
+}
+
+struct callback {
+  Config cfg;
+  gsl::not_null<FlowController*> flow_controller;
+  bool stopped;

Review comment:
       I didn't know it either until I ran into the compilation error. :smile: 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r498122682



##########
File path: libminifi/src/utils/file/PathUtils.cpp
##########
@@ -84,13 +86,58 @@ std::string PathUtils::getFullPath(const std::string& path) {
 #endif
 }
 
-std::string PathUtils::globToRegex(std::string glob) {
+std::string globToRegex(std::string glob) {
   utils::StringUtils::replaceAll(glob, ".", "\\.");
   utils::StringUtils::replaceAll(glob, "*", ".*");
   utils::StringUtils::replaceAll(glob, "?", ".");
   return glob;
 }
 
+space_info space(const path p, std::error_code& ec) noexcept {

Review comment:
       I'm not a fan of trailing underscores unless really necessary, so I went with `path` and let them clash. Let me know if you're fine with this approach.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r476387922



##########
File path: libminifi/test/unit/GeneralUtilsTest.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <functional>
+#include <string>
+#include <type_traits>
+
+#include "../TestBase.h"
+#include "utils/GeneralUtils.h"
+
+namespace utils = org::apache::nifi::minifi::utils;
+
+static_assert(std::is_same<decltype(utils::make_unique<char16_t>()), std::unique_ptr<char16_t>>::value, "utils::make_unique type is correct");

Review comment:
       How about "type must be correct"? That would result in something like "assertion failure: ... type must be correct". 
   
   I avoid negative message because I perceive assertion to be a positive check and its message should be about the check, not the error IMHO.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r497632424



##########
File path: libminifi/include/properties/Properties.h
##########
@@ -82,6 +84,16 @@ class Properties {
    */
   int getInt(const std::string &key, int default_value) const;
 
+  utils::optional<std::string> get(const std::string& key) const {
+    std::string result;
+    const bool found = get(key, result);

Review comment:
       ```
   /home/szaszm/nifi-minifi-cpp-2/libminifi/include/properties/Properties.h: In member function ‘nonstd::optional_lite::optional<std::__cxx11::basic_string<char> > org::apache::nifi::minifi::Properties::get(const string&) const’:
   /home/szaszm/nifi-minifi-cpp-2/libminifi/include/properties/Properties.h:89:29: error: operands to ‘?:’ have different types ‘std::string’ {aka ‘std::__cxx11::basic_string<char>’} and ‘const nonstd::optional_lite::nullopt_t’
      89 |     return get(key, result) ? result : utils::nullopt;
         |            ~~~~~~~~~~~~~~~~~^~~~~~~~~~~~~~~~~~~~~~~~~
   ```




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r478518109



##########
File path: libminifi/src/DiskSpaceWatchdog.cpp
##########
@@ -0,0 +1,133 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "DiskSpaceWatchdog.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <type_traits>
+
+#include "core/Property.h"
+#include "FlowController.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
+#include "utils/file/PathUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace {
+namespace chr = std::chrono;
+struct Config {
+  chr::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+utils::optional<chr::milliseconds> string_to_milliseconds(const std::string& str) {
+  uint64_t millisec_value{};
+  const bool success = core::Property::getTimeMSFromString(str, millisec_value);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(chr::milliseconds{millisec_value});
+}
+
+template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
+utils::optional<T> string_to_int(const std::string& str) {
+  T result{};
+  const bool success = core::Property::StringToInt(str, result);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(result);
+}
+
+Config read_config(const Configure& conf) {
+  const auto interval_ms = conf.get(Configure::minifi_disk_space_watchdog_interval_ms) >>= string_to_milliseconds;
+  const auto stop_bytes = conf.get(Configure::minifi_disk_space_watchdog_stop_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  const auto restart_bytes = conf.get(Configure::minifi_disk_space_watchdog_restart_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  if (restart_bytes < stop_bytes) { throw std::runtime_error{"disk space watchdog stop threshold must be <= restart threshold"}; }
+  constexpr auto mebibytes = 1024 * 1024;
+  return {
+      interval_ms.value_or(chr::seconds{1}),
+      stop_bytes.value_or(15 * mebibytes),
+      restart_bytes.value_or(20 * mebibytes)
+  };
+}
+
+struct callback {
+  Config cfg;
+  gsl::not_null<FlowController*> flow_controller;
+  bool stopped;

Review comment:
       It would cause the callback type to no longer be an aggregate in C++11 (fixed in C++14).
   https://godbolt.org/z/MGo5cf
   https://en.cppreference.com/w/cpp/language/aggregate_initialization
   
   in particular:
   > no default member initializers (since C++11) (until C++14)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r478526483



##########
File path: libminifi/src/DiskSpaceWatchdog.cpp
##########
@@ -0,0 +1,133 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "DiskSpaceWatchdog.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <type_traits>
+
+#include "core/Property.h"
+#include "FlowController.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
+#include "utils/file/PathUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace {
+namespace chr = std::chrono;
+struct Config {
+  chr::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+utils::optional<chr::milliseconds> string_to_milliseconds(const std::string& str) {
+  uint64_t millisec_value{};
+  const bool success = core::Property::getTimeMSFromString(str, millisec_value);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(chr::milliseconds{millisec_value});
+}
+
+template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
+utils::optional<T> string_to_int(const std::string& str) {
+  T result{};
+  const bool success = core::Property::StringToInt(str, result);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(result);
+}
+
+Config read_config(const Configure& conf) {
+  const auto interval_ms = conf.get(Configure::minifi_disk_space_watchdog_interval_ms) >>= string_to_milliseconds;
+  const auto stop_bytes = conf.get(Configure::minifi_disk_space_watchdog_stop_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  const auto restart_bytes = conf.get(Configure::minifi_disk_space_watchdog_restart_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  if (restart_bytes < stop_bytes) { throw std::runtime_error{"disk space watchdog stop threshold must be <= restart threshold"}; }
+  constexpr auto mebibytes = 1024 * 1024;
+  return {
+      interval_ms.value_or(chr::seconds{1}),
+      stop_bytes.value_or(15 * mebibytes),
+      restart_bytes.value_or(20 * mebibytes)
+  };
+}
+
+struct callback {

Review comment:
       renamed in 13116f0




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r497600016



##########
File path: libminifi/include/properties/Properties.h
##########
@@ -82,6 +84,16 @@ class Properties {
    */
   int getInt(const std::string &key, int default_value) const;
 
+  utils::optional<std::string> get(const std::string& key) const {
+    std::string result;
+    const bool found = get(key, result);

Review comment:
       Just fancy, for me this is the case when ternary operator simplifies the code:
   ```
   return get(key, result) ? result : utils::nullopt;
   ```
   
   Feel free to ignore in case you doesn't share this opinion. 

##########
File path: libminifi/include/utils/IntervalSwitch.h
##########
@@ -0,0 +1,80 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <functional>
+#include <utility>
+
+#include "utils/gsl.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+enum class IntervalSwitchState {
+  LOWER,
+  UPPER,
+};
+
+namespace detail {
+struct SwitchReturn {
+  IntervalSwitchState state;
+  bool switched;
+};
+}  // namespace detail
+
+template<typename T, typename Comp = std::less<T>>
+class IntervalSwitch {
+ public:
+  IntervalSwitch(T lower_threshold, T upper_threshold, const IntervalSwitchState initial_state = IntervalSwitchState::UPPER)
+      :lower_threshold_{std::move(lower_threshold)}, upper_threshold_{std::move(upper_threshold)}, state_{initial_state} {
+    gsl_Expects(!less_(upper_threshold_, lower_threshold_));
+  }
+
+  detail::SwitchReturn operator()(const T& value) {

Review comment:
       This seems a bit overcomplicated to be with multiple local members and return.
   
   Suggestion:
   ```
       const auto old_state = state_;
       if (less_(value, lower_threshold_)) {
         state_ = IntervalSwitchState::LOWER;
       } else if (!less_(value, upper_threshold_)) {
         state_ = IntervalSwitchState::UPPER;
       } 
      return {state_, state_ != old_state};
   ```
   
   Az the "else if" branch was actually doing nothing, inverting the condition could help removing it completely. 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r478520677



##########
File path: libminifi/src/DiskSpaceWatchdog.cpp
##########
@@ -0,0 +1,133 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "DiskSpaceWatchdog.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <type_traits>
+
+#include "core/Property.h"
+#include "FlowController.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
+#include "utils/file/PathUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace {
+namespace chr = std::chrono;
+struct Config {
+  chr::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+utils::optional<chr::milliseconds> string_to_milliseconds(const std::string& str) {
+  uint64_t millisec_value{};
+  const bool success = core::Property::getTimeMSFromString(str, millisec_value);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(chr::milliseconds{millisec_value});
+}
+
+template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
+utils::optional<T> string_to_int(const std::string& str) {
+  T result{};
+  const bool success = core::Property::StringToInt(str, result);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(result);
+}
+
+Config read_config(const Configure& conf) {
+  const auto interval_ms = conf.get(Configure::minifi_disk_space_watchdog_interval_ms) >>= string_to_milliseconds;
+  const auto stop_bytes = conf.get(Configure::minifi_disk_space_watchdog_stop_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  const auto restart_bytes = conf.get(Configure::minifi_disk_space_watchdog_restart_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  if (restart_bytes < stop_bytes) { throw std::runtime_error{"disk space watchdog stop threshold must be <= restart threshold"}; }
+  constexpr auto mebibytes = 1024 * 1024;
+  return {
+      interval_ms.value_or(chr::seconds{1}),
+      stop_bytes.value_or(15 * mebibytes),
+      restart_bytes.value_or(20 * mebibytes)
+  };
+}
+
+struct callback {

Review comment:
       I can't give it a name that any more distinguished than "DiskSpaceWatchdogCallback", but the "DiskSpaceWatchdog" part should be obvious from the context (DiskSpaceWatchdog implementation file). Renaming anyway, because it doesn't hurt. :smile: 
   
   If I misunderstood you in some way, please correct me.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
szaszm commented on pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#issuecomment-679914981


   In case you want to review, these are the problems I'm planning to fix:
   - don't use exceptions for control flow
   - nuke DiskSpaceWatchdog class to multiple parts because it's doing too much on its own


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r498139911



##########
File path: libminifi/include/DiskSpaceWatchdog.h
##########
@@ -0,0 +1,60 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <chrono>
+#include <cinttypes>
+#include <string>
+#include <vector>
+
+#include "utils/IntervalSwitch.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+class Configure;
+namespace core {
+namespace logging {
+class Logger;
+}  // namespace logging
+}  // namespace core
+
+namespace disk_space_watchdog {
+struct Config {
+  std::chrono::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+Config read_config(const Configure&);
+
+inline utils::IntervalSwitch<std::uintmax_t> disk_space_interval_switch(Config config) {

Review comment:
       I would be really surprised if it made any difference performance-wise, this function is most probably going to get inlined either way with the optimization level we are using




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] arpadboda closed pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
arpadboda closed pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r476331404



##########
File path: libminifi/include/utils/OptionalUtils.h
##########
@@ -37,11 +39,47 @@ optional<typename gsl_lite::remove_cvref<T>::type> optional_from_ptr(T&& obj) {
   return obj == nullptr ? nullopt : optional<typename gsl_lite::remove_cvref<T>::type>{ std::forward<T>(obj) };
 }
 
+template<typename>
+struct is_optional : std::false_type {};
+
+template<typename T>
+struct is_optional<optional<T>> : std::true_type {};
+
 }  // namespace utils
 }  // namespace minifi
 }  // namespace nifi
 }  // namespace apache
 }  // namespace org
 
+namespace nonstd {
+namespace optional_lite {
+
+namespace minifiutils = org::apache::nifi::minifi::utils;
+
+// map
+template<typename SourceType, typename F>
+auto operator|(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))
+    -> optional<typename std::decay<decltype(minifiutils::invoke(std::forward<F>(f), *o))>::type> {
+  if (o.has_value()) {
+    return make_optional(minifiutils::invoke(std::forward<F>(f), *o));
+  } else {
+    return nullopt;
+  }
+}
+
+// bind
+template<typename SourceType, typename F>
+auto operator>>=(const optional<SourceType>& o, F&& f) noexcept(noexcept(minifiutils::invoke(std::forward<F>(f), *o)))

Review comment:
       I'd prefer to avoid modifying the optional class. What do you think about this syntax?
   ```
   utils::optional<T> opt{...};
   auto map_result = opt | utils::map([](T) -> U {...});
   auto flat_map_result = opt | utils::flatMap([](T) -> utils::optional<U>{...});
   ```




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r498111891



##########
File path: libminifi/include/properties/Properties.h
##########
@@ -82,6 +84,16 @@ class Properties {
    */
   int getInt(const std::string &key, int default_value) const;
 
+  utils::optional<std::string> get(const std::string& key) const {
+    std::string result;
+    const bool found = get(key, result);

Review comment:
       😞 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r478526718



##########
File path: libminifi/test/unit/GeneralUtilsTest.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <functional>
+#include <string>
+#include <type_traits>
+
+#include "../TestBase.h"
+#include "utils/GeneralUtils.h"
+
+namespace utils = org::apache::nifi::minifi::utils;
+
+static_assert(std::is_same<decltype(utils::make_unique<char16_t>()), std::unique_ptr<char16_t>>::value, "utils::make_unique type is correct");

Review comment:
       changed in 13116f0




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r498148335



##########
File path: libminifi/include/utils/GeneralUtils.h
##########
@@ -43,6 +43,7 @@ using std::make_unique;
 
 template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
 T intdiv_ceil(T numerator, T denominator) {
+  gsl_Expects(denominator != 0);
   // note: division and remainder is 1 instruction on x86
   return numerator / denominator + (numerator % denominator > 0);

Review comment:
       I know this is old code, but unless we restrict the domain of this function to non-negative arguments, it might give incorrect results, e.g. `intdiv_ceil(-5, -3) == 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #875: WIP: Disk space watchdog prototype

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r476391715



##########
File path: libminifi/test/unit/GeneralUtilsTest.cpp
##########
@@ -0,0 +1,117 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <functional>
+#include <string>
+#include <type_traits>
+
+#include "../TestBase.h"
+#include "utils/GeneralUtils.h"
+
+namespace utils = org::apache::nifi::minifi::utils;
+
+static_assert(std::is_same<decltype(utils::make_unique<char16_t>()), std::unique_ptr<char16_t>>::value, "utils::make_unique type is correct");

Review comment:
       I see, yes that should be okay.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #875: MINIFICPP-1332 Prevent errneous behavior by stopping FlowController in low disk space conditions

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #875:
URL: https://github.com/apache/nifi-minifi-cpp/pull/875#discussion_r478876052



##########
File path: libminifi/src/DiskSpaceWatchdog.cpp
##########
@@ -0,0 +1,133 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#include "DiskSpaceWatchdog.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <type_traits>
+
+#include "core/Property.h"
+#include "FlowController.h"
+#include "core/logging/Logger.h"
+#include "properties/Configure.h"
+#include "utils/file/PathUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace {
+namespace chr = std::chrono;
+struct Config {
+  chr::milliseconds interval;
+  std::uintmax_t stop_threshold_bytes;
+  std::uintmax_t restart_threshold_bytes;
+};
+
+utils::optional<chr::milliseconds> string_to_milliseconds(const std::string& str) {
+  uint64_t millisec_value{};
+  const bool success = core::Property::getTimeMSFromString(str, millisec_value);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(chr::milliseconds{millisec_value});
+}
+
+template<typename T, typename = typename std::enable_if<std::is_integral<T>::value>::type>
+utils::optional<T> string_to_int(const std::string& str) {
+  T result{};
+  const bool success = core::Property::StringToInt(str, result);
+  if (!success) return utils::nullopt;
+  return utils::make_optional(result);
+}
+
+Config read_config(const Configure& conf) {
+  const auto interval_ms = conf.get(Configure::minifi_disk_space_watchdog_interval_ms) >>= string_to_milliseconds;
+  const auto stop_bytes = conf.get(Configure::minifi_disk_space_watchdog_stop_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  const auto restart_bytes = conf.get(Configure::minifi_disk_space_watchdog_restart_threshold_bytes) >>= string_to_int<std::uintmax_t>;
+  if (restart_bytes < stop_bytes) { throw std::runtime_error{"disk space watchdog stop threshold must be <= restart threshold"}; }
+  constexpr auto mebibytes = 1024 * 1024;
+  return {
+      interval_ms.value_or(chr::seconds{1}),
+      stop_bytes.value_or(15 * mebibytes),
+      restart_bytes.value_or(20 * mebibytes)
+  };
+}
+
+struct callback {

Review comment:
       Thanks, it's just for making it easier to search for its references.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org