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/05/27 09:06:26 UTC

[GitHub] [nifi-minifi-cpp] adamdebreceni opened a new pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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


   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 master)?
   
   - [ ] 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] hunyadi-dev commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -138,26 +150,28 @@ class PropertyValue : public state::response::ValueNode {
    */
   template<typename T>
   auto operator=(const T ref) -> typename std::enable_if<std::is_same<T, std::string>::value,PropertyValue&>::type {

Review comment:
       Yes, that is correct, `T&&` should bind to `T`, and is_same handles `volatile`/`const`. Still, the simple overload should be simpler?




----------------------------------------------------------------
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] hunyadi-dev commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/Property.h
##########
@@ -129,41 +133,32 @@ class Property {
 
   template<typename T = std::string>
   void setValue(const T &value) {
-    PropertyValue vn = default_value_;
-    vn = value;
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     } else {
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);
+    vn = value;
+    ValidationResult result = vn.validate(name_);
+    if(!result.valid())
+      throw utils::InvalidValueException(name_ + " value validation failed");
   }
 
-  void setValue(PropertyValue &vn) {
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
+  void setValue(PropertyValue &newValue) {
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(newValue);
     } else {
-      values_.push_back(vn);
+      values_.push_back(newValue);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);

Review comment:
       I think it would be cleaner to have either one or the other. We could set the validator true by default and never expect it missing, OR handle the no-validator case as a valid option (both literally and implementationwise). 




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), utils::InvalidValueException);
+}
+
+TEST_CASE("Parsing int") {

Review comment:
       done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,115 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+#define LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+
+#include <string>
+
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+
+} /* namespace core */
+
+namespace utils {
+
+class ValueException: public Exception{
+ private:
+  explicit ValueException(const std::string& err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit ValueException(const char* err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+
+  friend class ParseException;
+  friend class ConversionException;
+  friend class InvalidValueException;
+};

Review comment:
       I wanted the API as tight as possible, introducing new classes that are up for grabs is rarely beneficial, especially with the new major release coming up we should be extra careful what unremovable stuff we introduce, this clearly indicates that only we should be extending these classes




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+        throw ParseException("Cannot convert long to int");
+      }
+      out = {static_cast<int>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse int");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");
+    try {
+      char *end;
+      long long result{std::strtoll(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long result{std::strtoul(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result > (std::numeric_limits<uint32_t>::max)()) {
+        throw ParseException("Cannot convert unsigned long to uint32_t");
+      }
+      out = {static_cast<uint32_t>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long long result{std::strtoull(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out){
+    const char* options[] = {"false", "true"};
+    const bool values[] = {false, true};
+    auto index = parseAny(options);
+    if(index == -1)throw ParseException("Couldn't parse bool");
+    out = values[index];
+    return *this;
+  }
+
+  int parseAny(const std::vector<std::string> &options) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < options.size(); ++optionIdx) {
+      const auto &option = options[optionIdx];
+      if (offset + option.length() <= str.length()) {
+        if (std::equal(option.begin(), option.end(), str.begin() + offset)) {
+          offset += option.length();
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  template<std::size_t N>
+  int parseAny(const char* (&options)[N]) {

Review comment:
       done

##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+        throw ParseException("Cannot convert long to int");
+      }
+      out = {static_cast<int>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse int");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");
+    try {
+      char *end;
+      long long result{std::strtoll(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long result{std::strtoul(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result > (std::numeric_limits<uint32_t>::max)()) {
+        throw ParseException("Cannot convert unsigned long to uint32_t");
+      }
+      out = {static_cast<uint32_t>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long long result{std::strtoull(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out){
+    const char* options[] = {"false", "true"};
+    const bool values[] = {false, true};
+    auto index = parseAny(options);
+    if(index == -1)throw ParseException("Couldn't parse bool");
+    out = values[index];
+    return *this;
+  }
+
+  int parseAny(const std::vector<std::string> &options) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < options.size(); ++optionIdx) {
+      const auto &option = options[optionIdx];
+      if (offset + option.length() <= str.length()) {
+        if (std::equal(option.begin(), option.end(), str.begin() + offset)) {
+          offset += option.length();
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  template<std::size_t N>
+  int parseAny(const char* (&options)[N]) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < N; ++optionIdx) {
+      const auto &option = options[optionIdx];
+      auto len = std::strlen(option);
+      if (offset + len <= str.length()) {
+        if (std::equal(option, option + len, str.begin() + offset)) {
+          offset += len;
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  void parseEnd(){
+    parseSpace();
+    if(offset < str.length()){
+      throw ParseException("Expected to parse till the end");
+    }
+  }
+
+ private:
+  void parseSpace() {
+    while (offset < str.length() && std::isspace(str[offset]))++offset;

Review comment:
       done

##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,118 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_PROPERTYERRORS_H
+#define NIFI_MINIFI_CPP_PROPERTYERRORS_H
+
+#include "Exception.h"
+
+namespace org{
+namespace apache{
+namespace nifi{
+namespace minifi{
+namespace core{
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+}

Review comment:
       done

##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -199,15 +214,40 @@ class PropertyValue : public state::response::ValueNode {
   auto operator=(const std::string &ref) -> typename std::enable_if<
   std::is_same<T, DataSizeValue >::value ||
   std::is_same<T, TimePeriodValue >::value,PropertyValue&>::type {
-    value_ = std::make_shared<T>(ref);
-    type_id = value_->getTypeIndex();
-    return *this;
+    validator_.clearValidationResult();
+    return WithAssignmentGuard(ref, [&] () -> PropertyValue& {
+      value_ = std::make_shared<T>(ref);
+      type_id = value_->getTypeIndex();
+      return *this;
+    });
+  }
+
+ private:
+
+  bool isValueUsable() const {
+    if (!value_) return false;
+    if (validator_.isValid() == CachedValueValidator::Result::FAILURE) return false;
+    if (validator_.isValid() == CachedValueValidator::Result::SUCCESS) return true;
+    return validate("__unknown__").valid();
+  }
+
+  template<typename Fn>
+  auto WithAssignmentGuard(const std::string& ref, Fn&& functor) -> decltype(std::forward<Fn>(functor)()) {
+    // TODO: as soon as c++17 comes jump to a RAII implementation

Review comment:
       done

##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;

Review comment:
       done

##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,118 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_PROPERTYERRORS_H
+#define NIFI_MINIFI_CPP_PROPERTYERRORS_H
+
+#include "Exception.h"
+
+namespace org{
+namespace apache{
+namespace nifi{
+namespace minifi{

Review comment:
       done

##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -138,26 +150,28 @@ class PropertyValue : public state::response::ValueNode {
    */
   template<typename T>
   auto operator=(const T ref) -> typename std::enable_if<std::is_same<T, std::string>::value,PropertyValue&>::type {

Review comment:
       I would not touch this now, as it is part of the API, after 0.8.0 we should revisit this, as well as the whole codebase in general

##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+  CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator->() const {
+    return validator_;
+  }
+
+  operator bool() const {
+    return (bool)validator_;

Review comment:
       done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/Property.h
##########
@@ -129,41 +133,32 @@ class Property {
 
   template<typename T = std::string>
   void setValue(const T &value) {
-    PropertyValue vn = default_value_;
-    vn = value;
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     } else {
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);
+    vn = value;
+    ValidationResult result = vn.validate(name_);
+    if(!result.valid())
+      throw utils::InvalidValueException(name_ + " value validation failed");
   }
 
-  void setValue(PropertyValue &vn) {
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
+  void setValue(PropertyValue &newValue) {
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(newValue);
     } else {
-      values_.push_back(vn);
+      values_.push_back(newValue);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);

Review comment:
       propagated `gsl::not_null` all around, had to remove `StandardValidator::VALID` as multiple statically initialized `Property`s use it 




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -216,16 +216,20 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const{
 
    auto &&it = properties_.find(name);

Review comment:
       no idea, removed it, but it is an universal reference and will bind everything, useful when the expression returns a proxy reference, e.g. when accessing elements of `std::vector<bool>`, I mostly use them in range-based loops, but generally makes people think that you are doing something funny, so I would use them with great care




----------------------------------------------------------------
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 pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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


   @adamdebreceni : please check the CI error as well:
   On Linux
   ```
   CoapC2VerifyHeartbeat:
    what():  General Operation: Required property is empty: Remote Server
   ```
   
   On Win:
   ```
   C:\projects\nifi-minifi-cpp\libminifi\test\unit\PropertyValidationTests.cpp(77): FAILED:
     REQUIRE_THROWS_AS( prop.setValue("  5000000000  ") )
   because no exception was thrown where one was expected:
   ```
   CoapC2VerifyHeartBeat 


----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}

Review comment:
       the only owner of a `CachedValueValidator` instance should be a `PropertyValue`, ideally the move-constructor (and move-assignment) should be deleted, but since the `PropertyValue` has those defaulted that's not an option, when you move a `CachedValueValidator` into a `PropertyValue::validator_` you change the validator object in a value thus the cached validation result cannot be trusted anymore, the most important part is that such a `CachedValueValidator` should only be executed on its owner 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] szaszm commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,115 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+#define LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+
+#include <string>
+
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+
+} /* namespace core */
+
+namespace utils {
+
+class ValueException: public Exception{
+ private:
+  explicit ValueException(const std::string& err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit ValueException(const char* err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+
+  friend class ParseException;
+  friend class ConversionException;
+  friend class InvalidValueException;
+};

Review comment:
       Not that I'm aware of. I think we can assume that if something is inside an `impl` or `internal` or similar namespace, then people won't depend on it, but sadly no clear conventions in this regard.




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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


   


----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+        throw ParseException("Cannot convert long to int");
+      }
+      out = {static_cast<int>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse int");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");
+    try {
+      char *end;
+      long long result{std::strtoll(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long result{std::strtoul(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result > (std::numeric_limits<uint32_t>::max)()) {
+        throw ParseException("Cannot convert unsigned long to uint32_t");
+      }
+      out = {static_cast<uint32_t>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long long result{std::strtoull(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out){
+    const char* options[] = {"false", "true"};
+    const bool values[] = {false, true};
+    auto index = parseAny(options);
+    if(index == -1)throw ParseException("Couldn't parse bool");
+    out = values[index];
+    return *this;
+  }
+
+  int parseAny(const std::vector<std::string> &options) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < options.size(); ++optionIdx) {
+      const auto &option = options[optionIdx];
+      if (offset + option.length() <= str.length()) {
+        if (std::equal(option.begin(), option.end(), str.begin() + offset)) {
+          offset += option.length();
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  template<std::size_t N>
+  int parseAny(const char* (&options)[N]) {

Review comment:
       indeed, it does seem convoluted for its current use case, I was probably planning on using this to parse enums, will remove both




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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


   This is my reply to a resolved issue that I intended as a new point to the discussion. I can't reopen or open a new issue on the same line. Please have a quick look.
   https://github.com/apache/nifi-minifi-cpp/pull/797#discussion_r445487228


----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,238 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+using namespace utils::internal;
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();

Review comment:
       done

##########
File path: libminifi/include/utils/ValueParser.h
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+#define LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include <type_traits>
+#include <limits>
+
+#include "PropertyErrors.h"
+#include "GeneralUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+namespace internal {
+
+class ValueParser {
+ private:
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible : std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>> : std::true_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+ public:
+  explicit ValueParser(const std::string& str, std::size_t offset = 0) : str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse int");
+    }
+    if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+      throw ParseException("Cannot convert long to int");
+    }
+    offset += len;
+    out = {static_cast<int>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");  // NOLINT
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");  // NOLINT
+    long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoll, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    skipWhitespace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoul, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse uint32_t");
+    }
+    if (result > (std::numeric_limits<uint32_t>::max)()) {
+      throw ParseException("Cannot convert unsigned long to uint32_t");
+    }
+    offset += len;
+    out = {static_cast<uint32_t>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");  // NOLINT
+    skipWhitespace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoull, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse unsigned long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out) {

Review comment:
       done

##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,101 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+#define LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+
+#include <string>
+
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+
+} /* namespace core */
+
+namespace utils {
+namespace internal {
+
+class ValueException : public Exception {
+ protected:
+  explicit ValueException(const std::string& err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit ValueException(const char* err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+};
+
+class PropertyException : public Exception {
+ protected:
+  explicit PropertyException(const std::string& err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit PropertyException(const char* err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+};
+
+/**
+ * Thrown during converting from and to Value
+ */
+class ConversionException : public ValueException {
+ public:
+  explicit ConversionException(const std::string& err) : ValueException(err) {}
+  explicit ConversionException(const char* err) : ValueException(err) {}

Review comment:
       done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -202,14 +196,50 @@ class PropertyValue : public state::response::ValueNode {
   auto operator=(const std::string &ref) -> typename std::enable_if<
   std::is_same<T, DataSizeValue >::value ||
   std::is_same<T, TimePeriodValue >::value, PropertyValue&>::type {
-    value_ = std::make_shared<T>(ref);
-    type_id = value_->getTypeIndex();
-    return *this;
+    validator_.clearValidationResult();
+    return WithAssignmentGuard(ref, [&] () -> PropertyValue& {
+      value_ = std::make_shared<T>(ref);
+      type_id = value_->getTypeIndex();
+      return *this;
+    });
+  }
+
+ private:
+  template<typename T>
+  T convertImpl(const char* const type_name) const {
+    if (!isValueUsable()) {
+      throw utils::InvalidValueException("Cannot convert invalid value");
+    }
+    T res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw utils::ConversionException(std::string("Invalid conversion to ") + type_name + " for " + value_->getStringValue());
+  }

Review comment:
       This could use `core::getClassName<T>`. I would also make this public and encourage its use over the implicit conversion operators, with a long term goal of moving to explicit conversions.

##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -75,64 +77,52 @@ class PropertyValue : public state::response::ValueNode {
   }
 
   std::shared_ptr<PropertyValidator> getValidator() const {
-    return validator_;
+    return *validator_;
   }
 
   ValidationResult validate(const std::string &subject) const {
-    if (validator_) {
-      return validator_->validate(subject, getValue());
-    } else {
+    auto cachedResult = validator_.isValid();
+    if (cachedResult == CachedValueValidator::Result::SUCCESS) {
       return ValidationResult::Builder::createBuilder().isValid(true).build();
     }
+    if (cachedResult == CachedValueValidator::Result::FAILURE) {
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(getValue()->getStringValue()).isValid(false).build();
+    }
+    auto result = validator_->validate(subject, getValue());
+    validator_.setValidationResult(result.valid());
+    return result;

Review comment:
       This logic should move to `CachedValueValidator` and the caching behavior should be transparent IMO.




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), utils::InvalidValueException);
+}
+
+TEST_CASE("Parsing int") {

Review comment:
       indeed it is, removing duplicate




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueParser.h
##########
@@ -0,0 +1,183 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+#define LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include <type_traits>
+#include <limits>
+
+#include "PropertyErrors.h"
+#include "GeneralUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+namespace internal {
+
+class ValueParser {
+ private:
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible : std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>> : std::true_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };

Review comment:
       With the new approach, these are no longer used and can be removed.
   
   edit: That is, if you leave the new approach. It's currently failing on the CI because there is no `parse(uint64_t&)` overload




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,115 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+#define LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+
+#include <string>
+
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+
+} /* namespace core */
+
+namespace utils {
+
+class ValueException: public Exception{
+ private:
+  explicit ValueException(const std::string& err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit ValueException(const char* err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+
+  friend class ParseException;
+  friend class ConversionException;
+  friend class InvalidValueException;
+};

Review comment:
       This approach seems to violate the Open-closed principle. I don't mind it, as I'm not a big fan of its "open" part, but I'm interested why did you choose this approach over `protected` constructors.

##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}

Review comment:
       Instead of taking over the `validator_` but leaving the `validation_result_` in place, why don't we take over both and reset `other.validation_result_` to the default `RECOMPUTE`?

##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {

Review comment:
       Self-assignment should be handled in both copy and move assignment.

##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+  CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator->() const {
+    return validator_;
+  }
+
+  operator bool() const {

Review comment:
       I'd recommend an `explicit` conversion operator instead of an implicit one, if really needed. Explicit conversion happens inside an `if` statement.

##########
File path: extensions/coap/controllerservice/CoapConnector.cpp
##########
@@ -39,7 +39,7 @@ static core::Property RemoteServer;
 static core::Property Port;
 static core::Property MaxQueueSize;
 
-core::Property CoapConnectorService::RemoteServer(core::PropertyBuilder::createProperty("Remote Server")->withDescription("Remote CoAP server")->isRequired(true)->build());
+core::Property CoapConnectorService::RemoteServer(core::PropertyBuilder::createProperty("Remote Server")->withDescription("Remote CoAP server")->isRequired(false)->build());

Review comment:
       Why is this change? I don't think it makes much sense to set up coap without a server address.

##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+  CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator->() const {
+    return validator_;
+  }
+
+  operator bool() const {
+    return (bool)validator_;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator*() const {
+    return validator_;
+  }
+
+  void setValidationResult(bool success) const {
+    validation_result_ = success ? Result::SUCCESS : Result::FAILURE;
+  }
+
+  void clearValidationResult() const {
+    validation_result_ = Result::RECOMPUTE;
+  }

Review comment:
       These are not logically `const` and `validation_result_` should not be `mutable`.

##########
File path: libminifi/include/core/state/Value.h
##########
@@ -86,35 +87,57 @@ class Value {
   }
 
   virtual bool getValue(uint32_t &ref) {
-    const auto negative = string_value.find_first_of('-') != std::string::npos;
-     if (negative) {
-       return false;
-     }
-    ref = std::stoul(string_value);
+    try {
+      uint32_t value;
+      utils::ValueParser(string_value).parseUInt32(value).parseEnd();
+      ref = value;
+    } catch(const utils::ParseException&) {
+      return false;
+    }

Review comment:
       To me, the changes look like they are not more general, just moving the logic elsewhere, with an increase of required boilerplate. Not sure if this is an improvement, but maybe I'm missing a point.

##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,193 @@
+/**

Review comment:
       I suggest renaming the file to `ValueParser.h` to reflect the class name. -Utils is usually used for potentially related, but largely independent set of utilities. This is a stateful parser with its own invariants

##########
File path: libminifi/src/core/ConfigurableComponent.cpp
##########
@@ -103,11 +105,12 @@ bool ConfigurableComponent::updateProperty(const std::string &name, const std::s
 
   if (it != properties_.end()) {
     Property orig_property = it->second;
-    Property new_property = orig_property;
+    Property& new_property = it->second;
+    utils::ScopeGuard onExit([&] {

Review comment:
       Now with `gsl::finally`, we have `finally` in C++. :slightly_smiling_face: 

##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -202,14 +196,50 @@ class PropertyValue : public state::response::ValueNode {
   auto operator=(const std::string &ref) -> typename std::enable_if<
   std::is_same<T, DataSizeValue >::value ||
   std::is_same<T, TimePeriodValue >::value, PropertyValue&>::type {
-    value_ = std::make_shared<T>(ref);
-    type_id = value_->getTypeIndex();
-    return *this;
+    validator_.clearValidationResult();
+    return WithAssignmentGuard(ref, [&] () -> PropertyValue& {
+      value_ = std::make_shared<T>(ref);
+      type_id = value_->getTypeIndex();
+      return *this;
+    });
+  }
+
+ private:
+  template<typename T>
+  T convertImpl(const char* const type_name) const {
+    if (!isValueUsable()) {
+      throw utils::InvalidValueException("Cannot convert invalid value");
+    }
+    T res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw utils::ConversionException(std::string("Invalid conversion to ") + type_name + " for " + value_->getStringValue());
+  }
+
+  bool isValueUsable() const {
+    if (!value_) return false;
+    if (validator_.isValid() == CachedValueValidator::Result::FAILURE) return false;
+    if (validator_.isValid() == CachedValueValidator::Result::SUCCESS) return true;
+    return validate("__unknown__").valid();
+  }
+
+  template<typename Fn>
+  auto WithAssignmentGuard(const std::string& ref, Fn&& functor) -> decltype(std::forward<Fn>(functor)()) {
+    // TODO(adebreceni): as soon as c++17 comes jump to a RAII implementation
+    // as we will need std::uncaught_exceptions()
+    try {
+      return std::forward<Fn>(functor)();
+    } catch(const utils::ValueException&) {
+      type_id = std::type_index(typeid(std::string));
+      value_ = minifi::state::response::createValue(ref);
+      throw;
+    }
   }
 
  protected:
   std::type_index type_id;
-  std::shared_ptr<PropertyValidator> validator_;
+  CachedValueValidator validator_;

Review comment:
       :+1: 
   I hope this API breakage is not significant enough to prevent merging before the next release. @arpadboda may be able to provide more insight.




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/Property.h
##########
@@ -129,41 +133,32 @@ class Property {
 
   template<typename T = std::string>
   void setValue(const T &value) {
-    PropertyValue vn = default_value_;
-    vn = value;
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     } else {
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);
+    vn = value;
+    ValidationResult result = vn.validate(name_);
+    if(!result.valid())
+      throw utils::InvalidValueException(name_ + " value validation failed");
   }
 
-  void setValue(PropertyValue &vn) {
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
+  void setValue(PropertyValue &newValue) {
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(newValue);
     } else {
-      values_.push_back(vn);
+      values_.push_back(newValue);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);

Review comment:
       it sets the validator in the PropertyValue, if we didn't the incoming `newValue`'s validator would remain in the copy, yes we could also just clear the validator as most places interpret the absence of a validator as being valid, but I find it unappealing, imo we should always have a non-empty validator instance




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), utils::InvalidValueException);
+}
+
+TEST_CASE("Parsing int") {

Review comment:
       indeed it is, removed duplicate




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+  CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator->() const {
+    return validator_;
+  }
+
+  operator bool() const {
+    return (bool)validator_;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator*() const {
+    return validator_;
+  }
+
+  void setValidationResult(bool success) const {
+    validation_result_ = success ? Result::SUCCESS : Result::FAILURE;
+  }
+
+  void clearValidationResult() const {
+    validation_result_ = Result::RECOMPUTE;
+  }
+
+  Result isValid() const {
+    if(!validator_ || validation_result_ == Result::SUCCESS){

Review comment:
       done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}

Review comment:
       In my understanding, when I move construct something, I want the other object's content in the new object and preferably leave the other object in a somewhat valid state. (There is no clear consensus in the community about the last part as far as I'm aware.)
   What happens here is we take the validator, but not the cached result. This way after a move, we are forced to redo the validation, and I assume this is somewhat expensive, otherwise we wouldn't need a cache. In the other object, we leave a moved-from validator with a validation result that no longer corresponds to the validator, and is therefore invalid.
   
   Given the above, this seems less than ideal, but it's possible that I didn't fully understand your point and the code.




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+  CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator->() const {
+    return validator_;
+  }
+
+  operator bool() const {
+    return (bool)validator_;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator*() const {
+    return validator_;
+  }
+
+  void setValidationResult(bool success) const {
+    validation_result_ = success ? Result::SUCCESS : Result::FAILURE;
+  }
+
+  void clearValidationResult() const {
+    validation_result_ = Result::RECOMPUTE;
+  }

Review comment:
       clearValidationResult can be not const, but setValidationResult must be const are `PropertyValue::validate` is const, so the cached result must be mutable




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), utils::InvalidValueException);
+}
+
+TEST_CASE("Parsing int") {

Review comment:
       indeed it is removing duplicate




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -71,64 +72,75 @@ class PropertyValue : public state::response::ValueNode {
   }
 
   std::shared_ptr<PropertyValidator> getValidator() const {
-    return validator_;
+    return *validator_;
   }
 
   ValidationResult validate(const std::string &subject) const {
-    if (validator_) {
-      return validator_->validate(subject, getValue());
-    } else {
+    auto cachedResult = validator_.isValid();
+    if(cachedResult == CachedValueValidator::Result::SUCCESS){
       return ValidationResult::Builder::createBuilder().isValid(true).build();
     }
+    if(cachedResult == CachedValueValidator::Result::FAILURE){
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(getValue()->getStringValue()).isValid(false).build();
+    }
+    auto result = validator_->validate(subject, getValue());
+    validator_.setValidationResult(result.valid());
+    return result;
   }
 
   operator uint64_t() const {
+    if(!isValueUsable())throw utils::InvalidValueException("Cannot convert invalid value");

Review comment:
       done




----------------------------------------------------------------
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 pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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


   in the nanofi lib, there was a macro called `SUCCESS` which collides with the `CachedValueValidator::Result::SUCCESS`, and this only came out when building with python enabled


----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -202,14 +196,50 @@ class PropertyValue : public state::response::ValueNode {
   auto operator=(const std::string &ref) -> typename std::enable_if<
   std::is_same<T, DataSizeValue >::value ||
   std::is_same<T, TimePeriodValue >::value, PropertyValue&>::type {
-    value_ = std::make_shared<T>(ref);
-    type_id = value_->getTypeIndex();
-    return *this;
+    validator_.clearValidationResult();
+    return WithAssignmentGuard(ref, [&] () -> PropertyValue& {
+      value_ = std::make_shared<T>(ref);
+      type_id = value_->getTypeIndex();
+      return *this;
+    });
+  }
+
+ private:
+  template<typename T>
+  T convertImpl(const char* const type_name) const {
+    if (!isValueUsable()) {
+      throw utils::InvalidValueException("Cannot convert invalid value");
+    }
+    T res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw utils::ConversionException(std::string("Invalid conversion to ") + type_name + " for " + value_->getStringValue());
+  }
+
+  bool isValueUsable() const {
+    if (!value_) return false;
+    if (validator_.isValid() == CachedValueValidator::Result::FAILURE) return false;
+    if (validator_.isValid() == CachedValueValidator::Result::SUCCESS) return true;
+    return validate("__unknown__").valid();
+  }
+
+  template<typename Fn>
+  auto WithAssignmentGuard(const std::string& ref, Fn&& functor) -> decltype(std::forward<Fn>(functor)()) {
+    // TODO(adebreceni): as soon as c++17 comes jump to a RAII implementation
+    // as we will need std::uncaught_exceptions()
+    try {
+      return std::forward<Fn>(functor)();
+    } catch(const utils::ValueException&) {
+      type_id = std::type_index(typeid(std::string));
+      value_ = minifi::state::response::createValue(ref);
+      throw;
+    }
   }
 
  protected:
   std::type_index type_id;
-  std::shared_ptr<PropertyValidator> validator_;
+  CachedValueValidator validator_;

Review comment:
       I can definitely live with this :)
   Property validation has always been lagging behind, seems to be properly done finally.




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,101 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+#define LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+
+#include <string>
+
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+
+} /* namespace core */
+
+namespace utils {
+namespace internal {
+
+class ValueException : public Exception {
+ protected:
+  explicit ValueException(const std::string& err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit ValueException(const char* err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+};
+
+class PropertyException : public Exception {
+ protected:
+  explicit PropertyException(const std::string& err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit PropertyException(const char* err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+};
+
+/**
+ * Thrown during converting from and to Value
+ */
+class ConversionException : public ValueException {
+ public:
+  explicit ConversionException(const std::string& err) : ValueException(err) {}
+  explicit ConversionException(const char* err) : ValueException(err) {}

Review comment:
       could you expand on what problem this would solve?




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,238 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+using namespace utils::internal;
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();

Review comment:
       This particular case is not specified by the style guide but it specifies most cases of continuation indentation to be 4 spaces wide, so I suggest making this one (and the rest of the tests in this file) so as well.

##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,101 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+#define LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+
+#include <string>
+
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+
+} /* namespace core */
+
+namespace utils {
+namespace internal {
+
+class ValueException : public Exception {
+ protected:
+  explicit ValueException(const std::string& err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit ValueException(const char* err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+};
+
+class PropertyException : public Exception {
+ protected:
+  explicit PropertyException(const std::string& err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit PropertyException(const char* err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+};
+
+/**
+ * Thrown during converting from and to Value
+ */
+class ConversionException : public ValueException {
+ public:
+  explicit ConversionException(const std::string& err) : ValueException(err) {}
+  explicit ConversionException(const char* err) : ValueException(err) {}

Review comment:
       Tip: `using ValueException::ValueException;`

##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -215,18 +215,23 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const
 
   auto &&it = properties_.find(name);
   if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr) {
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     } else {
-       logger_->log_warn("Component %s property name %s, empty value", name, item.getName());
-       return false;
-     }
+    const Property& item = it->second;
+    if (item.getValue().getValue() == nullptr) {
+      // empty value
+      if (item.getRequired()) {
+        logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+        throw utils::internal::RequiredPropertyMissingException("Required property is empty: " + item.getName());

Review comment:
       1. A required property is missing. This should be a warning or error rather than a debug level log message.
   
   2. This is core API, the behavior change here may cause existing setups to break. It may also be fine, I'm interested in the input of @arpadboda here.

##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,238 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+using namespace utils::internal;
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), InvalidValueException);
+}
+
+TEST_CASE("Parsing int has baggage after") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("55almost int"), ParseException);
+}
+
+TEST_CASE("Parsing int has spaces") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  prop.setValue("  55  ");
+  REQUIRE(static_cast<int>(prop.getValue()) == 55);
+}
+
+TEST_CASE("Parsing int out of range") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  REQUIRE_THROWS_AS(prop.setValue("  5000000000  "), ParseException);

Review comment:
       The standard permits `int` to be larger than 32 bits. We may want to
   1. ignore those platforms if they exist and assume that all our supported platforms have 32 bit wide `int`s,
   2. use an even larger number to cover some future architectures, or 
   3. generate a string with a large number based on `sizeof(int)`.
   
   I'm fine with the current version (option 1), just wanted to raise this question.

##########
File path: libminifi/include/core/Property.h
##########
@@ -129,41 +133,32 @@ class Property {
 
   template<typename T = std::string>
   void setValue(const T &value) {
-    PropertyValue vn = default_value_;
-    vn = value;
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     } else {
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);
+    vn = value;
+    ValidationResult result = vn.validate(name_);
+    if(!result.valid())
+      throw utils::InvalidValueException(name_ + " value validation failed");
   }
 
-  void setValue(PropertyValue &vn) {
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
+  void setValue(PropertyValue &newValue) {
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(newValue);
     } else {
-      values_.push_back(vn);
+      values_.push_back(newValue);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);

Review comment:
       Would wrapping the validator in `gsl::not_null` be a viable option? It could keep compatibility with APIs that expect `shared_ptr` and still be guaranteed to be present.

##########
File path: libminifi/include/utils/ValueParser.h
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+#define LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include <type_traits>
+#include <limits>
+
+#include "PropertyErrors.h"
+#include "GeneralUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+namespace internal {
+
+class ValueParser {
+ private:
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible : std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>> : std::true_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+ public:
+  explicit ValueParser(const std::string& str, std::size_t offset = 0) : str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse int");
+    }
+    if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+      throw ParseException("Cannot convert long to int");
+    }
+    offset += len;
+    out = {static_cast<int>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");  // NOLINT
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");  // NOLINT
+    long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoll, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    skipWhitespace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoul, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse uint32_t");
+    }
+    if (result > (std::numeric_limits<uint32_t>::max)()) {
+      throw ParseException("Cannot convert unsigned long to uint32_t");
+    }
+    offset += len;
+    out = {static_cast<uint32_t>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");  // NOLINT
+    skipWhitespace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoull, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse unsigned long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out) {

Review comment:
       Thinking about these signatures, I got the idea that I would really love to see:
   ```
   bool result;
   ValueParser{string}.parse(result).parseEnd();
   ```
   `parse` could be templated on the parsed type and let it be deduced if it is an exact match with the out type. If it's not an exact match, an interface like this could work: `.parse<bool>(result)`, where `result` can be a different type where lossless conversion is possible, like now.
   
   This interface would probably make future uses of the class in generic code much easier.
   
   Untested implementation idea:
   ```
   template<typename T, typename Out>
   ValueParser& parse(Out&); // unimplemented
   
   template<typename T>
   ValueParser& parse<T, T>(T& out) { return parse<T>(out); }
   
   template<typename Out>
   ValueParser& parse<bool, Out>(Out& out) { return parseBool(out); }
   
   template<typename Out>
   ValueParser& parse<int, Out>(Out& out) { return parseInt(out); }
   
   // ...
   ```
   
   Let me know if you would like to work on a similar interface, if there are problems with the idea, or you just don't want to spend time on this.

##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -215,18 +215,23 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const
 
   auto &&it = properties_.find(name);
   if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr) {
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     } else {
-       logger_->log_warn("Component %s property name %s, empty value", name, item.getName());
-       return false;
-     }
+    const Property& item = it->second;
+    if (item.getValue().getValue() == nullptr) {
+      // empty value
+      if (item.getRequired()) {
+        logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+        throw utils::internal::RequiredPropertyMissingException("Required property is empty: " + item.getName());
+      }
+      logger_->log_warn("Component %s property name %s, empty value", name, item.getName());

Review comment:
       related issue: https://issues.apache.org/jira/browse/MINIFICPP-1213
   
   I had a brief attempt at fixing this in the past but the required/not required difference got me a bit confused, so I moved on. Now with the above change, the fix looks trivial: change from warn to debug. Do you agree? Would you mind doing it in context of this PR or rather suggest doing it separately?

##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,129 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_CORE_CACHEDVALUEVALIDATOR_H_
+#define LIBMINIFI_INCLUDE_CORE_CACHEDVALUEVALIDATOR_H_
+
+#include <utility>
+#include <memory>
+#include <string>
+#include "PropertyValidation.h"
+#include "state/Value.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class PropertyValue;
+
+namespace internal {
+
+class CachedValueValidator {
+  friend class core::PropertyValue;
+
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}
+
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    if (this == &other) {
+      return *this;
+    }
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    if (this == &other) {
+      return *this;
+    }
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  explicit CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+
+  explicit CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;

Review comment:
       Calling `invalidateCachedResult()` even in members would be more human-readable IMO.




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/Property.h
##########
@@ -129,41 +133,32 @@ class Property {
 
   template<typename T = std::string>
   void setValue(const T &value) {
-    PropertyValue vn = default_value_;
-    vn = value;
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     } else {
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);
+    vn = value;
+    ValidationResult result = vn.validate(name_);
+    if(!result.valid())
+      throw utils::InvalidValueException(name_ + " value validation failed");
   }
 
-  void setValue(PropertyValue &vn) {
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
+  void setValue(PropertyValue &newValue) {
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(newValue);
     } else {
-      values_.push_back(vn);
+      values_.push_back(newValue);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);

Review comment:
       yes, replacing `std::shared_ptr<PropertyValidator>` with a suitable wrapper that guarantees a `PropertyValidator` instance is viable, but since most API functions expect/return a `shared_ptr` I would wait for after `0.8.0` to make the 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] adamdebreceni commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,129 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_CORE_CACHEDVALUEVALIDATOR_H_
+#define LIBMINIFI_INCLUDE_CORE_CACHEDVALUEVALIDATOR_H_
+
+#include <utility>
+#include <memory>
+#include <string>
+#include "PropertyValidation.h"
+#include "state/Value.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class PropertyValue;
+
+namespace internal {
+
+class CachedValueValidator {
+  friend class core::PropertyValue;
+
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}
+
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    if (this == &other) {
+      return *this;
+    }
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    if (this == &other) {
+      return *this;
+    }
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  explicit CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+
+  explicit CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;

Review comment:
       done

##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -215,18 +215,23 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const
 
   auto &&it = properties_.find(name);
   if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr) {
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     } else {
-       logger_->log_warn("Component %s property name %s, empty value", name, item.getName());
-       return false;
-     }
+    const Property& item = it->second;
+    if (item.getValue().getValue() == nullptr) {
+      // empty value
+      if (item.getRequired()) {
+        logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+        throw utils::internal::RequiredPropertyMissingException("Required property is empty: " + item.getName());
+      }
+      logger_->log_warn("Component %s property name %s, empty value", name, item.getName());

Review comment:
       done

##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -215,18 +215,23 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const
 
   auto &&it = properties_.find(name);
   if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr) {
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     } else {
-       logger_->log_warn("Component %s property name %s, empty value", name, item.getName());
-       return false;
-     }
+    const Property& item = it->second;
+    if (item.getValue().getValue() == nullptr) {
+      // empty value
+      if (item.getRequired()) {
+        logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+        throw utils::internal::RequiredPropertyMissingException("Required property is empty: " + item.getName());

Review comment:
       logging error instead




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}

Review comment:
       I generally like to take only what I use and making it as explicit as possible, instead of adjusting the effects after the fact, but if you think this would improve the readability I could make the 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] adamdebreceni commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -202,14 +196,50 @@ class PropertyValue : public state::response::ValueNode {
   auto operator=(const std::string &ref) -> typename std::enable_if<
   std::is_same<T, DataSizeValue >::value ||
   std::is_same<T, TimePeriodValue >::value, PropertyValue&>::type {
-    value_ = std::make_shared<T>(ref);
-    type_id = value_->getTypeIndex();
-    return *this;
+    validator_.clearValidationResult();
+    return WithAssignmentGuard(ref, [&] () -> PropertyValue& {
+      value_ = std::make_shared<T>(ref);
+      type_id = value_->getTypeIndex();
+      return *this;
+    });
+  }
+
+ private:
+  template<typename T>
+  T convertImpl(const char* const type_name) const {
+    if (!isValueUsable()) {
+      throw utils::InvalidValueException("Cannot convert invalid value");
+    }
+    T res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw utils::ConversionException(std::string("Invalid conversion to ") + type_name + " for " + value_->getStringValue());
+  }

Review comment:
       if we can live with `uint64_t` being printed as `unsigned long` on some platforms and `unsigned __int64` on others (windows) I can make the change, but then the error message would be invalid, as we do not want to convert to `unsigned long` we want to convert to `uint64_t` that the two coincide is a platform dependent implementation detail




----------------------------------------------------------------
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] hunyadi-dev commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+        throw ParseException("Cannot convert long to int");
+      }
+      out = {static_cast<int>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse int");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");
+    try {
+      char *end;
+      long long result{std::strtoll(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long result{std::strtoul(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result > (std::numeric_limits<uint32_t>::max)()) {
+        throw ParseException("Cannot convert unsigned long to uint32_t");
+      }
+      out = {static_cast<uint32_t>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long long result{std::strtoull(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out){
+    const char* options[] = {"false", "true"};
+    const bool values[] = {false, true};
+    auto index = parseAny(options);
+    if(index == -1)throw ParseException("Couldn't parse bool");
+    out = values[index];
+    return *this;
+  }
+
+  int parseAny(const std::vector<std::string> &options) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < options.size(); ++optionIdx) {
+      const auto &option = options[optionIdx];
+      if (offset + option.length() <= str.length()) {
+        if (std::equal(option.begin(), option.end(), str.begin() + offset)) {
+          offset += option.length();
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  template<std::size_t N>
+  int parseAny(const char* (&options)[N]) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < N; ++optionIdx) {
+      const auto &option = options[optionIdx];
+      auto len = std::strlen(option);
+      if (offset + len <= str.length()) {
+        if (std::equal(option, option + len, str.begin() + offset)) {
+          offset += len;
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  void parseEnd(){
+    parseSpace();
+    if(offset < str.length()){
+      throw ParseException("Expected to parse till the end");
+    }
+  }
+
+ private:
+  void parseSpace() {
+    while (offset < str.length() && std::isspace(str[offset]))++offset;

Review comment:
       Should be:
   ```c++
   while(...) {
       ...
   }
   ```




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueParser.h
##########
@@ -0,0 +1,193 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_VALUEUTILS_H_
+#define LIBMINIFI_INCLUDE_UTILS_VALUEUTILS_H_
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include <type_traits>
+#include <limits>
+
+#include "PropertyErrors.h"
+#include "GeneralUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+ public:
+  explicit ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse int");
+    }
+    if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+      throw ParseException("Cannot convert long to int");
+    }
+    offset += len;
+    out = {static_cast<int>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");  // NOLINT
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");  // NOLINT
+    long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoll, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    parseSpace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoul, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse uint32_t");
+    }
+    if (result > (std::numeric_limits<uint32_t>::max)()) {
+      throw ParseException("Cannot convert unsigned long to uint32_t");
+    }
+    offset += len;
+    out = {static_cast<uint32_t>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");  // NOLINT
+    parseSpace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoull, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse unsigned long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out) {
+    parseSpace();
+    if (std::strncmp(str.c_str() + offset, "false", std::strlen("false")) == 0) {
+      offset += std::strlen("false");
+      out = false;
+    } else if (std::strncmp(str.c_str() + offset, "true", std::strlen("true")) == 0) {
+      offset += std::strlen("true");
+      out = true;
+    } else {
+      throw ParseException("Couldn't parse bool");
+    }
+    return *this;
+  }
+
+  void parseEnd() {
+    parseSpace();
+    if (offset < str.length()) {
+      throw ParseException("Expected to parse till the end");
+    }
+  }
+
+ private:
+  /**
+   *
+   * @tparam T
+   * @param converter
+   * @param out
+   * @return the number of characters used during conversion, 0 for error
+   */
+  template<typename T>
+  std::size_t safeCallConverter(T (*converter)(const char* begin, char** end, int base), T& out) {
+    const char* const begin = str.c_str() + offset;
+    char* end;
+    errno = 0;
+    T result = converter(begin, &end, 10);
+    if (end == begin || errno == ERANGE) {
+      return 0;
+    }
+    out = result;
+    return end - begin;
+  }
+
+  void parseSpace() {

Review comment:
       done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+        throw ParseException("Cannot convert long to int");
+      }
+      out = {static_cast<int>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse int");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");
+    try {
+      char *end;
+      long long result{std::strtoll(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long result{std::strtoul(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result > (std::numeric_limits<uint32_t>::max)()) {
+        throw ParseException("Cannot convert unsigned long to uint32_t");
+      }
+      out = {static_cast<uint32_t>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long long result{std::strtoull(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out){
+    const char* options[] = {"false", "true"};
+    const bool values[] = {false, true};
+    auto index = parseAny(options);
+    if(index == -1)throw ParseException("Couldn't parse bool");
+    out = values[index];
+    return *this;
+  }
+
+  int parseAny(const std::vector<std::string> &options) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < options.size(); ++optionIdx) {
+      const auto &option = options[optionIdx];
+      if (offset + option.length() <= str.length()) {
+        if (std::equal(option.begin(), option.end(), str.begin() + offset)) {
+          offset += option.length();
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  template<std::size_t N>
+  int parseAny(const char* (&options)[N]) {

Review comment:
       it is used in `ValueParser::parseBool`




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+        throw ParseException("Cannot convert long to int");
+      }
+      out = {static_cast<int>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse int");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");
+    try {
+      char *end;
+      long long result{std::strtoll(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long result{std::strtoul(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result > (std::numeric_limits<uint32_t>::max)()) {
+        throw ParseException("Cannot convert unsigned long to uint32_t");
+      }
+      out = {static_cast<uint32_t>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long long result{std::strtoull(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out){
+    const char* options[] = {"false", "true"};
+    const bool values[] = {false, true};
+    auto index = parseAny(options);
+    if(index == -1)throw ParseException("Couldn't parse bool");
+    out = values[index];
+    return *this;
+  }
+
+  int parseAny(const std::vector<std::string> &options) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < options.size(); ++optionIdx) {
+      const auto &option = options[optionIdx];
+      if (offset + option.length() <= str.length()) {
+        if (std::equal(option.begin(), option.end(), str.begin() + offset)) {
+          offset += option.length();
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  template<std::size_t N>
+  int parseAny(const char* (&options)[N]) {

Review comment:
       indeed, it does seem convoluted, I was probably planning on using this to parse enums, will remove both




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,238 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+using namespace utils::internal;
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), InvalidValueException);
+}
+
+TEST_CASE("Parsing int has baggage after") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("55almost int"), ParseException);
+}
+
+TEST_CASE("Parsing int has spaces") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  prop.setValue("  55  ");
+  REQUIRE(static_cast<int>(prop.getValue()) == 55);
+}
+
+TEST_CASE("Parsing int out of range") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  REQUIRE_THROWS_AS(prop.setValue("  5000000000  "), ParseException);

Review comment:
       (not only can it bigger, but it can also be as small as 16 bits)
   I'll leave this for now as is, if this test fails due to the platform, it should be ease to fix, and also warn us, that we should be more mindful about `int` at other parts of the code
   
   I would actually go as far as to remove the `int` (and other non-fixed with integers) as a possible value type, as these are user facing and we can't have a property validation fail on one agent but succeed on an other just because they are running on different platforms




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/state/Value.h
##########
@@ -86,35 +87,57 @@ class Value {
   }
 
   virtual bool getValue(uint32_t &ref) {
-    const auto negative = string_value.find_first_of('-') != std::string::npos;
-     if (negative) {
-       return false;
-     }
-    ref = std::stoul(string_value);
+    try {
+      uint32_t value;
+      utils::ValueParser(string_value).parseUInt32(value).parseEnd();
+      ref = value;
+    } catch(const utils::ParseException&) {
+      return false;
+    }

Review comment:
       they are not ment to be more general, they are ment to be correct, all these conversion methods are incorrect as they all happily convert `"1banana"` to `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] adamdebreceni commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), utils::InvalidValueException);
+}
+
+TEST_CASE("Parsing int") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+}
+
+TEST_CASE("Parsing int has baggage after") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("55almost int"), utils::ParseException);
+}
+
+TEST_CASE("Parsing int has spaces") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  prop.setValue("  55  ");
+  REQUIRE(static_cast<int>(prop.getValue()) == 55);
+}
+
+TEST_CASE("Parsing int out of range") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  REQUIRE_THROWS_AS(prop.setValue("  5000000000  "), utils::ParseException);
+}
+
+TEST_CASE("Parsing bool has baggage after") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<bool>(true)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("false almost bool"), utils::ParseException);
+}
+
+class TestConfigurableComponent : public ConfigurableComponent {
+ public:
+  bool supportsDynamicProperties() override {
+    return true;
+  }
+
+  bool canEdit() override {
+    return true;
+  }
+
+  void onPropertyModified(const Property &old_property, const Property &new_property) override {
+    if (onPropertyModifiedCallback) onPropertyModifiedCallback(old_property, new_property);
+  }
+
+  void onDynamicPropertyModified(const Property &old_property, const Property &new_property) override {
+    if (onDynamicPropertyModifiedCallback) onDynamicPropertyModifiedCallback(old_property, new_property);
+  }
+
+  template<typename Fn>
+  void setPropertyModifiedCallback(Fn&& functor) {
+    onPropertyModifiedCallback = std::forward<Fn>(functor);
+  }
+
+  template<typename Fn>
+  void setDynamicPropertyModifiedCallback(Fn&& functor) {
+    onDynamicPropertyModifiedCallback = std::forward<Fn>(functor);
+  }
+
+ private:
+  std::function<void(const Property&, const Property&)> onPropertyModifiedCallback;
+  std::function<void(const Property&, const Property&)> onDynamicPropertyModifiedCallback;
+};
+
+TEST_CASE("Missing Required With Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("default")
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  std::string value;
+  REQUIRE(component.getProperty(prop.getName(), value));
+  REQUIRE(value == "default");
+}
+
+TEST_CASE("Missing Required Without Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->isRequired(true)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  std::string value;
+  REQUIRE_THROWS_AS(component.getProperty(prop.getName(), value), utils::RequiredPropertyMissingException);
+}
+
+TEST_CASE("Missing Optional Without Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->isRequired(false)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  std::string value;
+  REQUIRE_FALSE(component.getProperty(prop.getName(), value));
+}
+
+TEST_CASE("Valid Optional Without Default") {
+  // without a default the value will be stored as a string
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->isRequired(false)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  component.setProperty(prop.getName(), "some data");
+  std::string value;
+  REQUIRE(component.getProperty(prop.getName(), value));
+  REQUIRE(value == "some data");
+}
+
+TEST_CASE("Invalid With Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<bool>(true)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  REQUIRE_THROWS_AS(component.setProperty("prop", "banana"), utils::ParseException);
+  std::string value;
+  REQUIRE_THROWS_AS(component.getProperty(prop.getName(), value), utils::InvalidValueException);
+}
+
+TEST_CASE("Valid With Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(55)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  REQUIRE(component.setProperty("prop", "23"));
+  int value;
+  REQUIRE(component.getProperty(prop.getName(), value));
+  REQUIRE(value == 23);
+}
+
+TEST_CASE("Invalid conversion") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<std::string>("banana")
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  bool value;
+  REQUIRE_THROWS_AS(component.getProperty(prop.getName(), value), utils::ConversionException);
+}
+
+TEST_CASE("Write Invalid Then Override With Valid") {
+  // we always base the assignment on the default value
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(55)
+  ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  REQUIRE_THROWS_AS(component.setProperty(prop.getName(), "banana"), utils::ConversionException);
+  component.setProperty(prop.getName(), "98");
+  int value;
+  REQUIRE(component.getProperty(prop.getName(), value));
+  REQUIRE(value == 98);
+}
+
+TEST_CASE("Property Change notification gets called even on erroneous assignment") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<bool>(true)
+  ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  int callbackCount = 0;
+  component.setPropertyModifiedCallback([&] (const Property&, const Property&) {
+    ++callbackCount;
+  });
+  REQUIRE_THROWS_AS(component.setProperty(prop.getName(), "banana"), utils::ConversionException);
+  REQUIRE(callbackCount == 1);
+}
+
+TEST_CASE("Correctly Typed Property With Invalid Validation") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int64_t>(5, std::make_shared<LongValidator>("myValidator", 0, 10))
+  ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  int callbackCount = 0;
+  component.setPropertyModifiedCallback([&] (const Property&, const Property&) {
+    ++callbackCount;
+  });
+  REQUIRE_THROWS_AS(component.setProperty(prop.getName(), "20"), utils::InvalidValueException);
+  REQUIRE(callbackCount == 1);
+}
+

Review comment:
       this crossed my mind, but as properties have different kinds of configurations, I believe we would lose more in terms of coupling than gain in legibility 




----------------------------------------------------------------
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] hunyadi-dev commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+        throw ParseException("Cannot convert long to int");
+      }
+      out = {static_cast<int>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse int");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");
+    try {
+      char *end;
+      long long result{std::strtoll(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long result{std::strtoul(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result > (std::numeric_limits<uint32_t>::max)()) {
+        throw ParseException("Cannot convert unsigned long to uint32_t");
+      }
+      out = {static_cast<uint32_t>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long long result{std::strtoull(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out){
+    const char* options[] = {"false", "true"};
+    const bool values[] = {false, true};
+    auto index = parseAny(options);
+    if(index == -1)throw ParseException("Couldn't parse bool");
+    out = values[index];
+    return *this;
+  }
+
+  int parseAny(const std::vector<std::string> &options) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < options.size(); ++optionIdx) {
+      const auto &option = options[optionIdx];
+      if (offset + option.length() <= str.length()) {
+        if (std::equal(option.begin(), option.end(), str.begin() + offset)) {
+          offset += option.length();
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  template<std::size_t N>
+  int parseAny(const char* (&options)[N]) {

Review comment:
       Looks awfully complicated for something that should be an `std::strncmp`. What exactly is `offset` and why do we add the match length to it?




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -216,16 +216,20 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const{
 
    auto &&it = properties_.find(name);
    if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr){
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     }
-     else{
+     const Property& item = it->second;
+     if (item.getValue().getValue() == nullptr) {
+       // empty value
+       if (item.getRequired()) {
+         logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+         throw utils::RequiredPropertyMissingException("Required property is empty: " + item.getName());

Review comment:
       the thing is, that you as a processor implementer should not care if `getProperty` throws or not, returning `false` means: "hey some error occurred that you should handle", throwing an exception means that somebody upstream messed up, e.g. called `onSchedule` without validating if all required properties are present or not checking invalid properties, if I mark a property required or add a validator I do not want to be have to handle if it is missing or 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.

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



[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -39,14 +39,29 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-core::Property MergeContent::MergeStrategy("Merge Strategy", "Defragment or Bin-Packing Algorithm", MERGE_STRATEGY_DEFRAGMENT);
-core::Property MergeContent::MergeFormat("Merge Format", "Merge Format", MERGE_FORMAT_CONCAT_VALUE);
+core::Property MergeContent::MergeStrategy(
+  core::PropertyBuilder::createProperty("Merge Strategy")
+  ->withDescription("Defragment or Bin-Packing Algorithm")
+  ->withAllowableValues<std::string>({MERGE_STRATEGY_DEFRAGMENT, MERGE_STRATEGY_BIN_PACK})
+  ->withDefaultValue(MERGE_STRATEGY_DEFRAGMENT)->build());
+core::Property MergeContent::MergeFormat(
+  core::PropertyBuilder::createProperty("Merge Format")
+  ->withDescription("Merge Format")
+  ->withAllowableValues<std::string>({MERGE_FORMAT_CONCAT_VALUE, MERGE_FORMAT_TAR_VALUE, MERGE_FORMAT_ZIP_VALUE})
+  ->withDefaultValue(MERGE_FORMAT_CONCAT_VALUE)->build());
 core::Property MergeContent::CorrelationAttributeName("Correlation Attribute Name", "Correlation Attribute Name", "");
-core::Property MergeContent::DelimiterStratgey("Delimiter Strategy", "Determines if Header, Footer, and Demarcator should point to files", DELIMITER_STRATEGY_FILENAME);
+core::Property MergeContent::DelimiterStratgey(

Review comment:
       typo: stratgey -> strategy -- this was in the old code, but we might as well fix it now




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {

Review comment:
       this wouldn't work for two reasons: 
   - as you said the return expression prevents narrowing conversions
   - casting to signed integrals, if the source value cannot be represented in the target type, is implementation defined (might even raise a signal)




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -216,16 +216,20 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const{
 
    auto &&it = properties_.find(name);
    if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr){
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     }
-     else{
+     const Property& item = it->second;
+     if (item.getValue().getValue() == nullptr) {
+       // empty value
+       if (item.getRequired()) {
+         logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+         throw utils::RequiredPropertyMissingException("Required property is empty: " + item.getName());

Review comment:
       definitely, if I create a processor and mark a property required and it is not there, it is the fault of the processor scheduler and I should not handle its absence (should not catch the exception but let it up to the scheduler), on the other hand if I have an optional property and it is not there, tough luck but I'll have to manage its absence




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,115 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+#define LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+
+#include <string>
+
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+
+} /* namespace core */
+
+namespace utils {
+
+class ValueException: public Exception{
+ private:
+  explicit ValueException(const std::string& err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit ValueException(const char* err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+
+  friend class ParseException;
+  friend class ConversionException;
+  friend class InvalidValueException;
+};

Review comment:
       do we have some `implementation_detail` directory where we could stash it, so it doesn't become part of the API?




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -75,64 +77,52 @@ class PropertyValue : public state::response::ValueNode {
   }
 
   std::shared_ptr<PropertyValidator> getValidator() const {
-    return validator_;
+    return *validator_;
   }
 
   ValidationResult validate(const std::string &subject) const {
-    if (validator_) {
-      return validator_->validate(subject, getValue());
-    } else {
+    auto cachedResult = validator_.isValid();
+    if (cachedResult == CachedValueValidator::Result::SUCCESS) {
       return ValidationResult::Builder::createBuilder().isValid(true).build();
     }
+    if (cachedResult == CachedValueValidator::Result::FAILURE) {
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(getValue()->getStringValue()).isValid(false).build();
+    }
+    auto result = validator_->validate(subject, getValue());
+    validator_.setValidationResult(result.valid());
+    return result;

Review comment:
       done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: extensions/libarchive/MergeContent.cpp
##########
@@ -39,14 +39,29 @@ namespace nifi {
 namespace minifi {
 namespace processors {
 
-core::Property MergeContent::MergeStrategy("Merge Strategy", "Defragment or Bin-Packing Algorithm", MERGE_STRATEGY_DEFRAGMENT);
-core::Property MergeContent::MergeFormat("Merge Format", "Merge Format", MERGE_FORMAT_CONCAT_VALUE);
+core::Property MergeContent::MergeStrategy(
+  core::PropertyBuilder::createProperty("Merge Strategy")
+  ->withDescription("Defragment or Bin-Packing Algorithm")
+  ->withAllowableValues<std::string>({MERGE_STRATEGY_DEFRAGMENT, MERGE_STRATEGY_BIN_PACK})
+  ->withDefaultValue(MERGE_STRATEGY_DEFRAGMENT)->build());
+core::Property MergeContent::MergeFormat(
+  core::PropertyBuilder::createProperty("Merge Format")
+  ->withDescription("Merge Format")
+  ->withAllowableValues<std::string>({MERGE_FORMAT_CONCAT_VALUE, MERGE_FORMAT_TAR_VALUE, MERGE_FORMAT_ZIP_VALUE})
+  ->withDefaultValue(MERGE_FORMAT_CONCAT_VALUE)->build());
 core::Property MergeContent::CorrelationAttributeName("Correlation Attribute Name", "Correlation Attribute Name", "");
-core::Property MergeContent::DelimiterStratgey("Delimiter Strategy", "Determines if Header, Footer, and Demarcator should point to files", DELIMITER_STRATEGY_FILENAME);
+core::Property MergeContent::DelimiterStratgey(

Review comment:
       done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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


   What was the failure that made d5a4bb4 necessary? I didn't see failures on travis, and I don't see how it changes behavior in a way that could fix a build failure.


----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+  CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator->() const {
+    return validator_;
+  }
+
+  operator bool() const {

Review comment:
       done, and the self-assignment check is also done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) noexcept : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+  CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator->() const {
+    return validator_;
+  }
+
+  operator bool() const {
+    return (bool)validator_;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator*() const {
+    return validator_;
+  }
+
+  void setValidationResult(bool success) const {
+    validation_result_ = success ? Result::SUCCESS : Result::FAILURE;
+  }
+
+  void clearValidationResult() const {
+    validation_result_ = Result::RECOMPUTE;
+  }

Review comment:
       clearValidationResult can be not const, but setValidationResult must be const as `PropertyValue::validate` is const, so the cached result must be mutable




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {

Review comment:
       I think for these simple cases it would be an overkill, also when we change `std::shared_ptr<PropertyValidator>` to something else I do not yet see if we want to allow it to be default constructible, also after a `swap` I would expect the two objects to exchange all states, this is not the case for the `validation_result_`




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueParser.h
##########
@@ -0,0 +1,194 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+#define LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include <type_traits>
+#include <limits>
+
+#include "PropertyErrors.h"
+#include "GeneralUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+namespace internal {
+
+class ValueParser {
+ private:
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible : std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>> : std::true_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+ public:
+  explicit ValueParser(const std::string& str, std::size_t offset = 0) : str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse int");
+    }
+    if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+      throw ParseException("Cannot convert long to int");
+    }
+    offset += len;
+    out = {static_cast<int>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");  // NOLINT
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");  // NOLINT
+    long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoll, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    skipWhitespace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoul, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse uint32_t");
+    }
+    if (result > (std::numeric_limits<uint32_t>::max)()) {
+      throw ParseException("Cannot convert unsigned long to uint32_t");
+    }
+    offset += len;
+    out = {static_cast<uint32_t>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");  // NOLINT
+    skipWhitespace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoull, result);
+    if (len == 0) {
+      throw ParseException("Couldn't parse unsigned long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out) {

Review comment:
       I dig it, the only problem with this specific approach is that, template function partial specialization is a no-go, but will think about something like this




----------------------------------------------------------------
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] hunyadi-dev commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -216,16 +216,20 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const{
 
    auto &&it = properties_.find(name);
    if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr){
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     }
-     else{
+     const Property& item = it->second;
+     if (item.getValue().getValue() == nullptr) {
+       // empty value
+       if (item.getRequired()) {
+         logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+         throw utils::RequiredPropertyMissingException("Required property is empty: " + item.getName());

Review comment:
       I am not saying that the logic is bad, just that this requires knowledge of the implementation details of `getProperty`. My expectation on calling a function of `bool getProperty(std::string)` is that I get a false returned if there is an error, and I might miss the fact that this throws. It would either be a good idea to show that this throws-if-required-missing in the name of the function or move the whole is-required check up to the scope of the caller.




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {

Review comment:
       I would argue that a value of type `CachedValueValidator::Result` carries more intention than that of a `std::optional<bool>`




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -215,18 +215,23 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const
 
   auto &&it = properties_.find(name);
   if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr) {
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     } else {
-       logger_->log_warn("Component %s property name %s, empty value", name, item.getName());
-       return false;
-     }
+    const Property& item = it->second;
+    if (item.getValue().getValue() == nullptr) {
+      // empty value
+      if (item.getRequired()) {
+        logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+        throw utils::internal::RequiredPropertyMissingException("Required property is empty: " + item.getName());

Review comment:
       I think the it just adds a secondary safety net, processors are already prepared to handle configuration errors (and throwing exceptions in onSchedule), so I don't mind this 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] adamdebreceni commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";

Review comment:
       I was torn between adding a comment like "mind the leading space", or this




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -202,14 +196,50 @@ class PropertyValue : public state::response::ValueNode {
   auto operator=(const std::string &ref) -> typename std::enable_if<
   std::is_same<T, DataSizeValue >::value ||
   std::is_same<T, TimePeriodValue >::value, PropertyValue&>::type {
-    value_ = std::make_shared<T>(ref);
-    type_id = value_->getTypeIndex();
-    return *this;
+    validator_.clearValidationResult();
+    return WithAssignmentGuard(ref, [&] () -> PropertyValue& {
+      value_ = std::make_shared<T>(ref);
+      type_id = value_->getTypeIndex();
+      return *this;
+    });
+  }
+
+ private:
+  template<typename T>
+  T convertImpl(const char* const type_name) const {
+    if (!isValueUsable()) {
+      throw utils::InvalidValueException("Cannot convert invalid value");
+    }
+    T res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw utils::ConversionException(std::string("Invalid conversion to ") + type_name + " for " + value_->getStringValue());
+  }

Review comment:
       Good point, I take my suggestions back.




----------------------------------------------------------------
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] fgerlits commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueParser.h
##########
@@ -0,0 +1,193 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_VALUEUTILS_H_
+#define LIBMINIFI_INCLUDE_UTILS_VALUEUTILS_H_
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include <type_traits>
+#include <limits>
+
+#include "PropertyErrors.h"
+#include "GeneralUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+ public:
+  explicit ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse int");
+    }
+    if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+      throw ParseException("Cannot convert long to int");
+    }
+    offset += len;
+    out = {static_cast<int>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");  // NOLINT
+    long result;  // NOLINT
+    auto len = safeCallConverter(std::strtol, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");  // NOLINT
+    long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoll, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    parseSpace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoul, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse uint32_t");
+    }
+    if (result > (std::numeric_limits<uint32_t>::max)()) {
+      throw ParseException("Cannot convert unsigned long to uint32_t");
+    }
+    offset += len;
+    out = {static_cast<uint32_t>(result)};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");  // NOLINT
+    parseSpace();
+    if (offset < str.length() && str[offset] == '-') {
+      throw ParseException("Not an unsigned long");
+    }
+    unsigned long long result;  // NOLINT
+    auto len = safeCallConverter(std::strtoull, result);
+    if ( len == 0 ) {
+      throw ParseException("Couldn't parse unsigned long long");
+    }
+    offset += len;
+    out = {result};
+    return *this;
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out) {
+    parseSpace();
+    if (std::strncmp(str.c_str() + offset, "false", std::strlen("false")) == 0) {
+      offset += std::strlen("false");
+      out = false;
+    } else if (std::strncmp(str.c_str() + offset, "true", std::strlen("true")) == 0) {
+      offset += std::strlen("true");
+      out = true;
+    } else {
+      throw ParseException("Couldn't parse bool");
+    }
+    return *this;
+  }
+
+  void parseEnd() {
+    parseSpace();
+    if (offset < str.length()) {
+      throw ParseException("Expected to parse till the end");
+    }
+  }
+
+ private:
+  /**
+   *
+   * @tparam T
+   * @param converter
+   * @param out
+   * @return the number of characters used during conversion, 0 for error
+   */
+  template<typename T>
+  std::size_t safeCallConverter(T (*converter)(const char* begin, char** end, int base), T& out) {
+    const char* const begin = str.c_str() + offset;
+    char* end;
+    errno = 0;
+    T result = converter(begin, &end, 10);
+    if (end == begin || errno == ERANGE) {
+      return 0;
+    }
+    out = result;
+    return end - begin;
+  }
+
+  void parseSpace() {

Review comment:
       very minor, but I would call this `skipWhitespace()`




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,115 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+#define LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+
+#include <string>
+
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+
+} /* namespace core */
+
+namespace utils {
+
+class ValueException: public Exception{
+ private:
+  explicit ValueException(const std::string& err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit ValueException(const char* err): Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+
+  friend class ParseException;
+  friend class ConversionException;
+  friend class InvalidValueException;
+};

Review comment:
       moved most stuff into the namespace `internal`




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueParser.h
##########
@@ -0,0 +1,183 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+#define LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include <type_traits>
+#include <limits>
+
+#include "PropertyErrors.h"
+#include "GeneralUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+namespace internal {
+
+class ValueParser {
+ private:
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible : std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>> : std::true_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };

Review comment:
       With the new approach, these are no longer used and can be removed.




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -202,14 +196,50 @@ class PropertyValue : public state::response::ValueNode {
   auto operator=(const std::string &ref) -> typename std::enable_if<
   std::is_same<T, DataSizeValue >::value ||
   std::is_same<T, TimePeriodValue >::value, PropertyValue&>::type {
-    value_ = std::make_shared<T>(ref);
-    type_id = value_->getTypeIndex();
-    return *this;
+    validator_.clearValidationResult();
+    return WithAssignmentGuard(ref, [&] () -> PropertyValue& {
+      value_ = std::make_shared<T>(ref);
+      type_id = value_->getTypeIndex();
+      return *this;
+    });
+  }
+
+ private:
+  template<typename T>
+  T convertImpl(const char* const type_name) const {
+    if (!isValueUsable()) {
+      throw utils::InvalidValueException("Cannot convert invalid value");
+    }
+    T res;
+    if (value_->convertValue(res)) {
+      return res;
+    }
+    throw utils::ConversionException(std::string("Invalid conversion to ") + type_name + " for " + value_->getStringValue());
+  }

Review comment:
       we could always slap a nice macro on it




----------------------------------------------------------------
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] hunyadi-dev commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {

Review comment:
       Basically, this class has the implementation of a basic `optional<bool>`. Shouldn't it be cleaner to have it extracted out to a helper class with `get` `set` and `clear` methods, and just call those here?

##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+  CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator->() const {
+    return validator_;
+  }
+
+  operator bool() const {
+    return (bool)validator_;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator*() const {
+    return validator_;
+  }
+
+  void setValidationResult(bool success) const {
+    validation_result_ = success ? Result::SUCCESS : Result::FAILURE;
+  }
+
+  void clearValidationResult() const {
+    validation_result_ = Result::RECOMPUTE;
+  }
+
+  Result isValid() const {
+    if(!validator_ || validation_result_ == Result::SUCCESS){

Review comment:
       The right side of `||` is redundant here
   ```c++
   if(!validator)
   {
       return Result::SUCCESS;
   }
   return validation_result_;
   ```

##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";

Review comment:
       What is the point of defining a constant for " "?

##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {
+    validator_ = other.validator_;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(CachedValueValidator&& other) {
+    validator_ = std::move(other.validator_);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  CachedValueValidator(const std::shared_ptr<PropertyValidator>& other) : validator_(other) {}
+  CachedValueValidator(std::shared_ptr<PropertyValidator>&& other) : validator_(std::move(other)) {}
+  CachedValueValidator& operator=(const std::shared_ptr<PropertyValidator>& new_validator) {
+    validator_ = new_validator;
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+  CachedValueValidator& operator=(std::shared_ptr<PropertyValidator>&& new_validator) {
+    validator_ = std::move(new_validator);
+    validation_result_ = Result::RECOMPUTE;
+    return *this;
+  }
+
+  const std::shared_ptr<PropertyValidator>& operator->() const {
+    return validator_;
+  }
+
+  operator bool() const {
+    return (bool)validator_;

Review comment:
       C style cast.

##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -199,15 +214,40 @@ class PropertyValue : public state::response::ValueNode {
   auto operator=(const std::string &ref) -> typename std::enable_if<
   std::is_same<T, DataSizeValue >::value ||
   std::is_same<T, TimePeriodValue >::value,PropertyValue&>::type {
-    value_ = std::make_shared<T>(ref);
-    type_id = value_->getTypeIndex();
-    return *this;
+    validator_.clearValidationResult();
+    return WithAssignmentGuard(ref, [&] () -> PropertyValue& {
+      value_ = std::make_shared<T>(ref);
+      type_id = value_->getTypeIndex();
+      return *this;
+    });
+  }
+
+ private:
+
+  bool isValueUsable() const {
+    if (!value_) return false;
+    if (validator_.isValid() == CachedValueValidator::Result::FAILURE) return false;
+    if (validator_.isValid() == CachedValueValidator::Result::SUCCESS) return true;
+    return validate("__unknown__").valid();
+  }
+
+  template<typename Fn>
+  auto WithAssignmentGuard(const std::string& ref, Fn&& functor) -> decltype(std::forward<Fn>(functor)()) {
+    // TODO: as soon as c++17 comes jump to a RAII implementation

Review comment:
       Minor, but once this is linter checked, the expected format for TODO-s would become `// TODO(username): ...`

##########
File path: libminifi/include/core/CachedValueValidator.h
##########
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+#define NIFI_MINIFI_CPP_CACHEDVALUEVALIDATOR_H
+
+#include "PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class CachedValueValidator{
+ public:
+  enum class Result {
+    FAILURE,
+    SUCCESS,
+    RECOMPUTE
+  };
+
+  CachedValueValidator() = default;
+  CachedValueValidator(const CachedValueValidator& other) : validator_(other.validator_) {}
+  CachedValueValidator(CachedValueValidator&& other) : validator_(std::move(other.validator_)) {}
+  CachedValueValidator& operator=(const CachedValueValidator& other) {

Review comment:
       Might be simpler using [copy-and-swap](https://stackoverflow.com/a/3279550/13369902).

##########
File path: libminifi/include/core/Property.h
##########
@@ -129,41 +133,32 @@ class Property {
 
   template<typename T = std::string>
   void setValue(const T &value) {
-    PropertyValue vn = default_value_;
-    vn = value;
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     } else {
-      values_.push_back(vn);
+      values_.push_back(default_value_);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);
+    vn = value;
+    ValidationResult result = vn.validate(name_);
+    if(!result.valid())
+      throw utils::InvalidValueException(name_ + " value validation failed");
   }
 
-  void setValue(PropertyValue &vn) {
-    if (validator_) {
-      vn.setValidator(validator_);
-      ValidationResult result = validator_->validate(name_, vn.getValue());
-      if (!result.valid()) {
-        // throw some exception?
-      }
-    } else {
-      vn.setValidator(core::StandardValidators::VALID);
-    }
+  void setValue(PropertyValue &newValue) {
     if (!is_collection_) {
       values_.clear();
-      values_.push_back(vn);
+      values_.push_back(newValue);
     } else {
-      values_.push_back(vn);
+      values_.push_back(newValue);
     }
+    PropertyValue& vn = values_.back();
+    vn.setValidator(validator_ ? validator_ : core::StandardValidators::VALID);

Review comment:
       Is there any point in setting the validator if it is not set? I find it a bit confusing. We now have a `CachedValidator` in `PropertyValue` which would return VALID if no validator is set.

##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), utils::InvalidValueException);
+}
+
+TEST_CASE("Parsing int") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+}
+
+TEST_CASE("Parsing int has baggage after") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("55almost int"), utils::ParseException);
+}
+
+TEST_CASE("Parsing int has spaces") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  prop.setValue("  55  ");
+  REQUIRE(static_cast<int>(prop.getValue()) == 55);
+}
+
+TEST_CASE("Parsing int out of range") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  REQUIRE_THROWS_AS(prop.setValue("  5000000000  "), utils::ParseException);
+}
+
+TEST_CASE("Parsing bool has baggage after") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<bool>(true)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("false almost bool"), utils::ParseException);
+}
+
+class TestConfigurableComponent : public ConfigurableComponent {
+ public:
+  bool supportsDynamicProperties() override {
+    return true;
+  }
+
+  bool canEdit() override {
+    return true;
+  }
+
+  void onPropertyModified(const Property &old_property, const Property &new_property) override {
+    if (onPropertyModifiedCallback) onPropertyModifiedCallback(old_property, new_property);
+  }
+
+  void onDynamicPropertyModified(const Property &old_property, const Property &new_property) override {
+    if (onDynamicPropertyModifiedCallback) onDynamicPropertyModifiedCallback(old_property, new_property);
+  }
+
+  template<typename Fn>
+  void setPropertyModifiedCallback(Fn&& functor) {
+    onPropertyModifiedCallback = std::forward<Fn>(functor);
+  }
+
+  template<typename Fn>
+  void setDynamicPropertyModifiedCallback(Fn&& functor) {
+    onDynamicPropertyModifiedCallback = std::forward<Fn>(functor);
+  }
+
+ private:
+  std::function<void(const Property&, const Property&)> onPropertyModifiedCallback;
+  std::function<void(const Property&, const Property&)> onDynamicPropertyModifiedCallback;
+};
+
+TEST_CASE("Missing Required With Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->isRequired(true)
+    ->withDefaultValue<std::string>("default")
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  std::string value;
+  REQUIRE(component.getProperty(prop.getName(), value));
+  REQUIRE(value == "default");
+}
+
+TEST_CASE("Missing Required Without Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->isRequired(true)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  std::string value;
+  REQUIRE_THROWS_AS(component.getProperty(prop.getName(), value), utils::RequiredPropertyMissingException);
+}
+
+TEST_CASE("Missing Optional Without Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->isRequired(false)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  std::string value;
+  REQUIRE_FALSE(component.getProperty(prop.getName(), value));
+}
+
+TEST_CASE("Valid Optional Without Default") {
+  // without a default the value will be stored as a string
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->isRequired(false)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  component.setProperty(prop.getName(), "some data");
+  std::string value;
+  REQUIRE(component.getProperty(prop.getName(), value));
+  REQUIRE(value == "some data");
+}
+
+TEST_CASE("Invalid With Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<bool>(true)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  REQUIRE_THROWS_AS(component.setProperty("prop", "banana"), utils::ParseException);
+  std::string value;
+  REQUIRE_THROWS_AS(component.getProperty(prop.getName(), value), utils::InvalidValueException);
+}
+
+TEST_CASE("Valid With Default") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(55)
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  REQUIRE(component.setProperty("prop", "23"));
+  int value;
+  REQUIRE(component.getProperty(prop.getName(), value));
+  REQUIRE(value == 23);
+}
+
+TEST_CASE("Invalid conversion") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<std::string>("banana")
+    ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  bool value;
+  REQUIRE_THROWS_AS(component.getProperty(prop.getName(), value), utils::ConversionException);
+}
+
+TEST_CASE("Write Invalid Then Override With Valid") {
+  // we always base the assignment on the default value
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(55)
+  ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  REQUIRE_THROWS_AS(component.setProperty(prop.getName(), "banana"), utils::ConversionException);
+  component.setProperty(prop.getName(), "98");
+  int value;
+  REQUIRE(component.getProperty(prop.getName(), value));
+  REQUIRE(value == 98);
+}
+
+TEST_CASE("Property Change notification gets called even on erroneous assignment") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<bool>(true)
+  ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  int callbackCount = 0;
+  component.setPropertyModifiedCallback([&] (const Property&, const Property&) {
+    ++callbackCount;
+  });
+  REQUIRE_THROWS_AS(component.setProperty(prop.getName(), "banana"), utils::ConversionException);
+  REQUIRE(callbackCount == 1);
+}
+
+TEST_CASE("Correctly Typed Property With Invalid Validation") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int64_t>(5, std::make_shared<LongValidator>("myValidator", 0, 10))
+  ->build();
+  TestConfigurableComponent component;
+  component.setSupportedProperties({prop});
+  int callbackCount = 0;
+  component.setPropertyModifiedCallback([&] (const Property&, const Property&) {
+    ++callbackCount;
+  });
+  REQUIRE_THROWS_AS(component.setProperty(prop.getName(), "20"), utils::InvalidValueException);
+  REQUIRE(callbackCount == 1);
+}
+

Review comment:
       Minor, but maybe it would make it a bit more organized/decrease the ammount of cluttering, if the tests were separated in sections.

##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,118 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_PROPERTYERRORS_H
+#define NIFI_MINIFI_CPP_PROPERTYERRORS_H
+
+#include "Exception.h"
+
+namespace org{
+namespace apache{
+namespace nifi{
+namespace minifi{

Review comment:
       The namespaces until this point could be combined with the ones below.

##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,118 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_PROPERTYERRORS_H
+#define NIFI_MINIFI_CPP_PROPERTYERRORS_H
+
+#include "Exception.h"
+
+namespace org{
+namespace apache{
+namespace nifi{
+namespace minifi{
+namespace core{
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+}

Review comment:
       No comments on closing tags for namespaces.

##########
File path: libminifi/src/core/ConfigurableComponent.cpp
##########
@@ -103,11 +105,12 @@ bool ConfigurableComponent::updateProperty(const std::string &name, const std::s
 
   if (it != properties_.end()) {
     Property orig_property = it->second;
-    Property new_property = orig_property;
+    Property& new_property = it->second;
+    utils::ScopeGuard onExit([&] {

Review comment:
       Is this better than the original? What problem does it solve? If it is throwing in `addValue`, then maybe we should fix that possibility by catching potential exceptions there.

##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -216,16 +216,20 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const{
 
    auto &&it = properties_.find(name);
    if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr){
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     }
-     else{
+     const Property& item = it->second;
+     if (item.getValue().getValue() == nullptr) {
+       // empty value
+       if (item.getRequired()) {
+         logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+         throw utils::RequiredPropertyMissingException("Required property is empty: " + item.getName());

Review comment:
       Minor, but do we really want to force the caller to handle required and non required properties differently?
   If we really want that functionality implemented here, maybe a better design would be to have two functions, this with the original behaviour, and one that explicitly has "throws if missing required" in its name.

##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;

Review comment:
       This is already defined in `GeneralUtils.h`

##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -216,16 +216,20 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const{
 
    auto &&it = properties_.find(name);

Review comment:
       Why do we have an rvalue reference here?

##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -71,64 +72,75 @@ class PropertyValue : public state::response::ValueNode {
   }
 
   std::shared_ptr<PropertyValidator> getValidator() const {
-    return validator_;
+    return *validator_;
   }
 
   ValidationResult validate(const std::string &subject) const {
-    if (validator_) {
-      return validator_->validate(subject, getValue());
-    } else {
+    auto cachedResult = validator_.isValid();
+    if(cachedResult == CachedValueValidator::Result::SUCCESS){
       return ValidationResult::Builder::createBuilder().isValid(true).build();
     }
+    if(cachedResult == CachedValueValidator::Result::FAILURE){
+      return ValidationResult::Builder::createBuilder().withSubject(subject).withInput(getValue()->getStringValue()).isValid(false).build();
+    }
+    auto result = validator_->validate(subject, getValue());
+    validator_.setValidationResult(result.valid());
+    return result;
   }
 
   operator uint64_t() const {
+    if(!isValueUsable())throw utils::InvalidValueException("Cannot convert invalid value");

Review comment:
       How about refactoring these functions to something like:
   ```c++
   template <typename T>
   T convert()
   {
       if(!isValueUsable())throw utils::InvalidValueException("Cannot convert invalid value");
       T res;
       if (value_->convertValue(res)) {
         return res;
       }
       // __FUNCSIG__ on WIN32
       throw utils::ConversionException(std::string("Invalid ") + __PRETTY_FUNCTION__ + value_->getStringValue());
   }
   
   operator uint64_t() const {
       return convert<uint64_t>();
   }
   
   operator int64_t() const {
       return convert<int64_t>();
   }
   
   // ...

##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {
+        throw ParseException("Cannot convert long to int");
+      }
+      out = {static_cast<int>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse int");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long, Out>::value, "Expected lossless conversion from long");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<long long, Out>::value, "Expected lossless conversion from long long");
+    try {
+      char *end;
+      long long result{std::strtoll(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUInt32(Out& out) {
+    static_assert(is_non_narrowing_convertible<uint32_t, Out>::value, "Expected lossless conversion from uint32_t");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long result{std::strtoul(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result > (std::numeric_limits<uint32_t>::max)()) {
+        throw ParseException("Cannot convert unsigned long to uint32_t");
+      }
+      out = {static_cast<uint32_t>(result)};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseUnsignedLongLong(Out& out) {
+    static_assert(is_non_narrowing_convertible<unsigned long long, Out>::value, "Expected lossless conversion from unsigned long long");
+    try {
+      parseSpace();
+      if (offset < str.length() && str[offset] == '-') {
+        throw ParseException("Not an unsigned long");
+      }
+      char *end;
+      unsigned long long result{std::strtoull(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      out = {result};
+      return *this;
+    }catch(...){
+      throw ParseException("Could not parse unsigned long long");
+    }
+  }
+
+  template<typename Out>
+  ValueParser& parseBool(Out& out){
+    const char* options[] = {"false", "true"};
+    const bool values[] = {false, true};
+    auto index = parseAny(options);
+    if(index == -1)throw ParseException("Couldn't parse bool");
+    out = values[index];
+    return *this;
+  }
+
+  int parseAny(const std::vector<std::string> &options) {
+    parseSpace();
+    for (std::size_t optionIdx = 0; optionIdx < options.size(); ++optionIdx) {
+      const auto &option = options[optionIdx];
+      if (offset + option.length() <= str.length()) {
+        if (std::equal(option.begin(), option.end(), str.begin() + offset)) {
+          offset += option.length();
+          return optionIdx;
+        }
+      }
+    }
+    return -1;
+  }
+
+  template<std::size_t N>
+  int parseAny(const char* (&options)[N]) {

Review comment:
       Is this used? Do we expect to know `N` in compile time?

##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -138,26 +150,28 @@ class PropertyValue : public state::response::ValueNode {
    */
   template<typename T>
   auto operator=(const T ref) -> typename std::enable_if<std::is_same<T, std::string>::value,PropertyValue&>::type {

Review comment:
       Maybe I am missing the obvious, but what is the advantage of templating this over just having
   ```c++
   PropertyValue& operator=(const std::string& ref) { ... }
   ```
   Would this for example not fail in case we pass `std::string&&` or `const std::string` as `T`?

##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), utils::InvalidValueException);
+}
+
+TEST_CASE("Parsing int") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+}
+
+TEST_CASE("Parsing int has baggage after") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("55almost int"), utils::ParseException);
+}
+
+TEST_CASE("Parsing int has spaces") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  prop.setValue("  55  ");
+  REQUIRE(static_cast<int>(prop.getValue()) == 55);
+}
+
+TEST_CASE("Parsing int out of range") {

Review comment:
       👍 It is a good idea to test this.

##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {

Review comment:
       Maybe you could simplify the asserts by pulling in [this](https://stackoverflow.com/a/29603857/13369902).
   
   [[Demo]](https://godbolt.org/#g:!((g:!((k:0.7001909611712285,l:'2',m:100,n:'0',o:'',t:'0'),(g:!((g:!((h:codeEditor,i:(fontScale:14,j:2,lang:c%2B%2B,selection:(endColumn:1,endLineNumber:22,positionColumn:1,positionLineNumber:22,selectionStartColumn:1,selectionStartLineNumber:22,startColumn:1,startLineNumber:22),source:'%23include+%3Ciostream%3E%0A%23include+%3Cstring%3E%0A%23include+%3Cutility%3E%0A%23include+%3Cmemory%3E%0A%23include+%3Ctype_traits%3E%0A%0Atemplate%3Ctypename...+Conds%3E%0A++struct+and_%0A++:+std::true_type%0A++%7B+%7D%3B%0A%0Atemplate%3Ctypename+Cond,+typename...+Conds%3E%0A++struct+and_%3CCond,+Conds...%3E%0A++:+std::conditional%3CCond::value,+and_%3CConds...%3E,+std::false_type%3E::type%0A++%7B+%7D%3B%0A%0Atemplate%3Ctypename...+T%3E%0Ausing+all_integer+%3D+and_%3Cstd::is_integral%3CT%3E...%3E%3B%0A%0Astatic_assert(all_integer%3Cint,+uint64_t,+bool,+int%3E::value,+%22All+integers%22)%3B%0A//+static_assert(all_integer%3Cint,+uint64_t,+float,+bool%3E::value,+%22Not+all+integers%22)%3B%0A%0Aint+main()%0A%7B%0A++++return+0%3B%0A%7D'),l:'5',n:'0',o:'C%2B%2B+source+%232',t:'0')),k:50,l:'4',n:'0',o:'',s:0,t:'0'),(g:!((g:!((h:compiler,i:(compiler:clang1000,filters:(b:'0',binary:'1',commentOnly:'0',demangle:'0',directives:'0',execute:'0',intel:'0',libraryCode:'1',trim:'1'),fontScale:14,j:1,lang:c%2B%2B,libs:!(),options:'',selection:(endColumn:1,endLineNumber:1,positionColumn:1,positionLineNumber:1,selectionStartColumn:1,selectionStartLineNumber:1,startColumn:1,startLineNumber:1),source:2),l:'5',n:'0',o:'x86-64+clang+10.0.0+(Editor+%232,+Compiler+%231)+C%2B%2B',t:'0')),header:(),k:50,l:'4',m:50,n:'0',o:'',s:0,t:'0'),(g:!((h:output,i:(compiler:1,editor:2,fontScale:8,wrap:'1'),l:'5',n:'0',o:'%231+with+x86-64+clang+10.0.0',t:'0')),header:(),l:'4',m:50,n:'0',o:'',s:0,t:'0')),k:50,l:'3',n:'0',o:'',t:'0')),k:99.29980903882876,l:'2',m:100,n:'0',o:'',t:'0')),l:'2',n:'0',o:'',t:'0')),version:4)

##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,203 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef NIFI_MINIFI_CPP_VALUEUTILS_H
+#define NIFI_MINIFI_CPP_VALUEUTILS_H
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include "PropertyErrors.h"
+#include <type_traits>
+#include <limits>
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+
+class ValueParser {
+ private:
+  template< class... >
+  using void_t = void;
+
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible: std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>>: std::true_type{
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+  
+ public:
+  ValueParser(const std::string& str, std::size_t offset = 0): str(str), offset(offset) {}
+
+  template<typename Out>
+  ValueParser& parseInt(Out& out) {
+    static_assert(is_non_narrowing_convertible<int, Out>::value, "Expected lossless conversion from int");
+    try {
+      char *end;
+      long result{std::strtol(str.c_str() + offset, &end, 10)};
+      offset = end - str.c_str();
+      if (result < (std::numeric_limits<int>::min)() || result > (std::numeric_limits<int>::max)()) {

Review comment:
       Maybe this wouldn't work (due to `-Wnarrow`), but wouldn't this simplify things?
   ```c++
   if(out != static_cast<int>(out))
   {
       throw ...;
   }
   return { result };
   ```

##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,244 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), utils::ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), utils::ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), utils::InvalidValueException);
+}
+
+TEST_CASE("Parsing int") {

Review comment:
       Isn't this the same test case as the one 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] adamdebreceni commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/PropertyValue.h
##########
@@ -138,26 +150,28 @@ class PropertyValue : public state::response::ValueNode {
    */
   template<typename T>
   auto operator=(const T ref) -> typename std::enable_if<std::is_same<T, std::string>::value,PropertyValue&>::type {

Review comment:
       I believe `T` if not explicitly provided will always be deduced to a non-reference type, good question what the author wanted to prevent (or facilitate) with this solution




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/src/core/ConfigurableComponent.cpp
##########
@@ -103,11 +105,12 @@ bool ConfigurableComponent::updateProperty(const std::string &name, const std::s
 
   if (it != properties_.end()) {
     Property orig_property = it->second;
-    Property new_property = orig_property;
+    Property& new_property = it->second;
+    utils::ScopeGuard onExit([&] {

Review comment:
       if `addValue` throws this still notifies the component of the changed property, as we do not have `finally` in c++ it is cleaner than calling it both in a catch and outside




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueParser.h
##########
@@ -0,0 +1,183 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+#define LIBMINIFI_INCLUDE_UTILS_VALUEPARSER_H_
+
+#include <exception>
+#include <string>
+#include <cstring>
+#include <vector>
+#include <cstdlib>
+#include <type_traits>
+#include <limits>
+
+#include "PropertyErrors.h"
+#include "GeneralUtils.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace utils {
+namespace internal {
+
+class ValueParser {
+ private:
+  template<typename From, typename To, typename = void>
+  struct is_non_narrowing_convertible : std::false_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };
+
+  template<typename From, typename To>
+  struct is_non_narrowing_convertible<From, To, void_t<decltype(To{std::declval<From>()})>> : std::true_type {
+    static_assert(std::is_integral<From>::value && std::is_integral<To>::value, "Checks only integral values");
+  };

Review comment:
       done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/core/ConfigurableComponent.h
##########
@@ -215,18 +215,23 @@ bool ConfigurableComponent::getProperty(const std::string name, T &value) const
 
   auto &&it = properties_.find(name);
   if (it != properties_.end()) {
-     Property item = it->second;
-     value = static_cast<T>(item.getValue());
-     if (item.getValue().getValue() != nullptr) {
-       logger_->log_debug("Component %s property name %s value %s", name, item.getName(), item.getValue().to_string());
-       return true;
-     } else {
-       logger_->log_warn("Component %s property name %s, empty value", name, item.getName());
-       return false;
-     }
+    const Property& item = it->second;
+    if (item.getValue().getValue() == nullptr) {
+      // empty value
+      if (item.getRequired()) {
+        logger_->log_debug("Component %s required property %s is empty", name, item.getName());
+        throw utils::internal::RequiredPropertyMissingException("Required property is empty: " + item.getName());

Review comment:
       I think this just adds a secondary safety net, processors are already prepared to handle configuration errors (and throwing exceptions in onSchedule), so I don't mind this 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] adamdebreceni commented on a change in pull request #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/test/unit/PropertyValidationTests.cpp
##########
@@ -0,0 +1,238 @@
+/**
+ *
+ * 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 "../TestBase.h"
+#include "core/ConfigurableComponent.h"
+#include "utils/PropertyErrors.h"
+#include "core/PropertyValidation.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+using namespace utils::internal;
+/**
+ * This Tests checks a deprecated behavior that should be removed
+ * in the next major release.
+ */
+TEST_CASE("Some default values get coerced to typed variants") {
+  auto prop = Property("prop", "d", "true");
+  REQUIRE_THROWS_AS(prop.setValue("banana"), ConversionException);
+
+  const std::string SPACE = " ";
+  auto prop2 = Property("prop", "d", SPACE + "true");
+  prop2.setValue("banana");
+}
+
+TEST_CASE("Converting invalid PropertyValue") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("not int"), ParseException);
+  REQUIRE_THROWS_AS(static_cast<int>(prop.getValue()), InvalidValueException);
+}
+
+TEST_CASE("Parsing int has baggage after") {
+  auto prop = PropertyBuilder::createProperty("prop")
+    ->withDefaultValue<int>(0)
+    ->build();
+  REQUIRE_THROWS_AS(prop.setValue("55almost int"), ParseException);
+}
+
+TEST_CASE("Parsing int has spaces") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  prop.setValue("  55  ");
+  REQUIRE(static_cast<int>(prop.getValue()) == 55);
+}
+
+TEST_CASE("Parsing int out of range") {
+  auto prop = PropertyBuilder::createProperty("prop")
+  ->withDefaultValue<int>(0)
+  ->build();
+  REQUIRE_THROWS_AS(prop.setValue("  5000000000  "), ParseException);

Review comment:
       (not only can it be bigger, but it can also be as small as 16 bits)
   I'll leave this for now as is, if this test fails due to the platform, it should be ease to fix, and also warn us, that we should be more mindful about `int` at other parts of the code
   
   I would actually go as far as to remove the `int` (and other non-fixed with integers) as a possible value type, as these are user facing and we can't have a property validation fail on one agent but succeed on an other just because they are running on different platforms




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/ValueUtils.h
##########
@@ -0,0 +1,193 @@
+/**

Review comment:
       done




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: extensions/coap/controllerservice/CoapConnector.cpp
##########
@@ -39,7 +39,7 @@ static core::Property RemoteServer;
 static core::Property Port;
 static core::Property MaxQueueSize;
 
-core::Property CoapConnectorService::RemoteServer(core::PropertyBuilder::createProperty("Remote Server")->withDescription("Remote CoAP server")->isRequired(true)->build());
+core::Property CoapConnectorService::RemoteServer(core::PropertyBuilder::createProperty("Remote Server")->withDescription("Remote CoAP server")->isRequired(false)->build());

Review comment:
       one of the tests did not provide the property and failed, from the code it seems like this service can handle it not being set
   
   ```
   void CoapConnectorService::onEnable() {
     std::string port_str;
     if (getProperty(RemoteServer.getName(), host_) && !host_.empty() && getProperty(Port.getName(), port_str) && !port_str.empty()) {
       core::Property::StringToInt(port_str, port_);
     } else {
       // this is the case where we aren't being used in the context of a single controller service.
       if (configuration_->get("nifi.c2.agent.coap.host", host_) && configuration_->get("nifi.c2.agent.coap.port", port_str)) {
         core::Property::StringToInt(port_str, port_);
       }
   
     }
   }
   ```




----------------------------------------------------------------
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 #797: MINIFICPP-1231 - General property validation + use them in MergeContent.

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



##########
File path: libminifi/include/utils/PropertyErrors.h
##########
@@ -0,0 +1,101 @@
+/**
+ *
+ * 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/licenseas/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.
+ */
+
+#ifndef LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+#define LIBMINIFI_INCLUDE_UTILS_PROPERTYERRORS_H_
+
+#include <string>
+
+#include "Exception.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+
+namespace core {
+
+class PropertyValue;
+class ConfigurableComponent;
+class Property;
+
+} /* namespace core */
+
+namespace utils {
+namespace internal {
+
+class ValueException : public Exception {
+ protected:
+  explicit ValueException(const std::string& err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit ValueException(const char* err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+};
+
+class PropertyException : public Exception {
+ protected:
+  explicit PropertyException(const std::string& err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+  explicit PropertyException(const char* err) : Exception(ExceptionType::GENERAL_EXCEPTION, err) {}
+
+  // base class already has a virtual destructor
+};
+
+/**
+ * Thrown during converting from and to Value
+ */
+class ConversionException : public ValueException {
+ public:
+  explicit ConversionException(const std::string& err) : ValueException(err) {}
+  explicit ConversionException(const char* err) : ValueException(err) {}

Review comment:
       could you expand on what problem this would solve?




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