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

[GitHub] [nifi-minifi-cpp] fgerlits opened a new pull request, #1589: MINIFICPP-1825 Create Properties at compile time

fgerlits opened a new pull request, #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589

   Create Properties at compile time; also Relationships, Output Attributes and Dynamic Properties.
   
   This only applies to Processor and Controller Service metadata used in generating the manifest and PROCESSORS.md.  I did not change Properties and Relationships used later on during runtime, as those are not always const and sometimes they need to own their own strings.
   
   There are a few changes in the generated PROCESSORS.md, mainly because the list of allowed values for a Property is no longer alphabetized; now the allowed values are listed in the order they appear in either the property definition or in the corresponding smart enum.
   
   ---
   
   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:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [x] 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.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [x] 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 GitHub Actions CI results 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1247978755


##########
PROCESSORS.md:
##########
@@ -469,12 +471,12 @@ In the list below, the names of required properties appear in bold. Any other pr
 | Resolve Metadata in Attributes     | true                                                                                                                                                                                                                        |                                     | If true, any metadata that is resolved ( such as IDs or keyword metadata ) will be placed into attributes, otherwise it will be replaced in the XML or text output                                                                                                                                                                                                    |
 | Event Header Delimiter             |                                                                                                                                                                                                                             |                                     | If set, the chosen delimiter will be used in the Event output header. Otherwise, a colon followed by spaces will be used.                                                                                                                                                                                                                                             |
 | Event Header                       | LOG_NAME=Log Name, SOURCE = Source, TIME_CREATED = Date,EVENT_RECORDID=Record ID,EVENTID = Event ID,TASK_CATEGORY = Task Category,LEVEL = Level,KEYWORDS = Keywords,USER = User,COMPUTER = Computer, EVENT_TYPE = EventType |                                     | Comma seperated list of key/value pairs with the following keys LOG_NAME, SOURCE, TIME_CREATED,EVENT_RECORDID,EVENTID,TASK_CATEGORY,LEVEL,KEYWORDS,USER,COMPUTER, and EVENT_TYPE. Eliminating fields will remove them from the header.                                                                                                                                |
-| **Output Format**                  | Both                                                                                                                                                                                                                        | Both<br/>JSON<br/>Plaintext<br/>XML | Set the output format type. In case 'Both' is selected the processor generates two flow files for every event captured in format XML and Plaintext                                                                                                                                                                                                                    |
-| **JSON Format**                    | Simple                                                                                                                                                                                                                      | Simple<br/>Raw<br/>Flattened        | Set the json format type. Only applicable if Output Format is set to 'JSON'                                                                                                                                                                                                                                                                                           |
+| **Output Format**                  | Both                                                                                                                                                                                                                        | XML<br/>Both<br/>Plaintext<br/>JSON | Set the output format type. In case 'Both' is selected the processor generates two flow files for every event captured in format XML and Plaintext                                                                                                                                                                                                                    |
+| **JSON Format**                    | Simple                                                                                                                                                                                                                      | Raw<br/>Simple<br/>Flattened        | Set the json format type. Only applicable if Output Format is set to 'JSON'                                                                                                                                                                                                                                                                                           |
 | Batch Commit Size                  | 1000                                                                                                                                                                                                                        |                                     | Maximum number of Events to consume and create to Flow Files from before committing.                                                                                                                                                                                                                                                                                  |
 | State Directory                    | CWELState                                                                                                                                                                                                                   |                                     | DEPRECATED. Only use it for state migration from the state file, supplying the legacy state directory.                                                                                                                                                                                                                                                                |
 | **Process Old Events**             | false                                                                                                                                                                                                                       |                                     | This property defines if old events (which are created before first time server is started) should be processed.                                                                                                                                                                                                                                                      |
-| Cache SID Lookups                  | true                                                                                                                                                                                                                        | true<br>false                       | Determines whether SID to name lookups are cached in memory                                                                                                                                                                                                                                                                                                           |
+| Cache SID Lookups                  | true                                                                                                                                                                                                                        |                                     | Determines whether SID to name lookups are cached in memory                                                                                                                                                                                                                                                                                                           |

Review Comment:
   This was added manually, and only for a small minority of boolean properties, and I wanted `PROCESSORS.md` to be auto-generated, as much as possible.
   
   We could add `true<br>false` in the `Allowable Values` column for all boolean properties in the docs generator, but I'd prefer to do that in a separate pull request.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259363743


##########
libminifi/test/unit/ArrayUtilsTests.cpp:
##########


Review Comment:
   These tests could be done using `static_assert`s. I think that would be better: earlier failure, with more diagnostics, and doesn't depend on Catch2.



##########
libminifi/test/unit/CoreTests.cpp:
##########


Review Comment:
   same here



##########
libminifi/include/agent/agent_docs.h:
##########
@@ -61,9 +63,12 @@ struct Components {
 };
 
 namespace detail {
-template<typename Container>
-auto toVector(const Container& container) {
-  return std::vector<typename Container::value_type>(container.begin(), container.end());
+inline auto toVector(std::span<const core::PropertyReference> properties) {
+  return std::vector<core::Property>(properties.begin(), properties.end());
+}
+
+inline auto toVector(std::span<const core::RelationshipDefinition> relationships) {
+  return std::vector<core::Relationship>(relationships.begin(), relationships.end());

Review Comment:
   I'd use `span_to` here, instead of the local helpers. Before #1584, it's in utils/gsl.h, afterwards in utils/span.h



##########
libminifi/include/core/OutputAttributeDefinition.h:
##########
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <array>
+#include <string_view>
+
+#include "utils/gsl.h"
+#include "RelationshipDefinition.h"
+
+namespace org::apache::nifi::minifi::core {
+
+template<size_t NumRelationships = 1>
+struct OutputAttributeDefinition {
+  constexpr OutputAttributeDefinition(std::string_view name, std::array<RelationshipDefinition, NumRelationships> relationships, std::string_view description)
+      : name(name),
+        relationships(relationships),
+        description(description) {
+  }
+
+  std::string_view name;
+  std::array<RelationshipDefinition, NumRelationships> relationships;
+  std::string_view description;
+};
+
+struct OutputAttributeReference {
+  constexpr OutputAttributeReference() = default;  // TODO(fgerlits): is this needed?

Review Comment:
   Is it needed? 



##########
libminifi/test/unit/EnumTests.cpp:
##########


Review Comment:
   same here



##########
libminifi/include/core/Core.h:
##########
@@ -95,6 +98,62 @@ static inline std::string getClassName() {
 #endif
 }
 
+constexpr std::string_view removeStructOrClassPrefix(std::string_view input) {
+  constexpr std::string_view STRUCT = "struct ";  // should be static constexpr, but that is only allowed inside a constexpr function with std >= c++23
+  constexpr std::string_view CLASS = "class ";

Review Comment:
   Would `constinit const` work to circumvent the limitation?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1267693191


##########
libminifi/include/core/PropertyType.h:
##########
@@ -90,12 +47,19 @@ class PropertyValidator {
   [[nodiscard]] virtual ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const = 0;
 
   [[nodiscard]] virtual ValidationResult validate(const std::string &subject, const std::string &input) const = 0;
+};
+
+class PropertyType : public PropertyParser, public PropertyValidator {

Review Comment:
   as discussed offline, I have renamed `getName()` to `getValidatorName()` in b052e55ef42515d9ef441975cf1188e95c9625f0



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259929160


##########
libminifi/include/controllers/UpdatePolicyControllerService.h:
##########
@@ -50,18 +53,32 @@ class UpdatePolicyControllerService : public core::controller::ControllerService
   MINIFIAPI static constexpr const char* Description = "UpdatePolicyControllerService allows a flow specific policy on allowing or disallowing updates. "
       "Since the flow dictates the purpose of a device it will also be used to dictate updates to specific components.";
 
-  MINIFIAPI static const core::Property AllowAllProperties;
-  MINIFIAPI static const core::Property PersistUpdates;
-  MINIFIAPI static const core::Property AllowedProperties;
-  MINIFIAPI static const core::Property DisallowedProperties;
-  static auto properties() {
-    return std::array{
+  MINIFIAPI static constexpr auto AllowAllProperties = core::PropertyDefinitionBuilder<>::createProperty("Allow All Properties")
+      .withDescription("Allows all properties, which are also not disallowed, to be updated")
+      .withPropertyType(core::StandardPropertyTypes::BOOLEAN_TYPE)
+      .withDefaultValue("false")
+      .build();
+  MINIFIAPI static constexpr auto PersistUpdates = core::PropertyDefinitionBuilder<>::createProperty("Persist Updates")
+      .withDescription("Property that dictates whether updates should persist after a restart")
+      .isRequired(false)
+      .withPropertyType(core::StandardPropertyTypes::BOOLEAN_TYPE)
+      .withDefaultValue("false")
+      .build();
+  MINIFIAPI static constexpr auto AllowedProperties = core::PropertyDefinitionBuilder<>::createProperty("Allowed Properties")
+      .withDescription("Properties for which we will allow updates")
+      .isRequired(false)
+      .build();
+  MINIFIAPI static constexpr auto DisallowedProperties = core::PropertyDefinitionBuilder<>::createProperty("Disallowed Properties")
+      .withDescription("Properties for which we will not allow updates")
+      .isRequired(false)
+      .build();
+  MINIFIAPI static constexpr auto Properties = std::array<core::PropertyReference, 4>{
       AllowAllProperties,
       PersistUpdates,
       AllowedProperties,
       DisallowedProperties
-    };
-  }
+  };
+

Review Comment:
   fixed in b06c107ed2ce67957cac421f4442fecfab0799db



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259935350


##########
libminifi/include/core/ProcessContext.h:
##########
@@ -90,6 +91,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return processor_node_;
   }
 
+  // TODO(fgerlits): remove if possible

Review Comment:
   Not at the moment, unfortunately.  I have removed the TODOs, for 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265642290


##########
libminifi/include/core/PropertyDefinition.h:
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <array>
+#include <optional>
+#include <string_view>
+#include <utility>
+
+#include "core/PropertyType.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::core {
+
+template<size_t NumAllowedValues = 0, size_t NumAllowedTypes = 0, size_t NumDependentProperties = 0, size_t NumExclusiveOfProperties = 0>
+struct PropertyDefinition {
+  std::string_view name;
+  std::string_view display_name;
+  std::string_view description;
+  bool is_required = false;
+  std::string_view valid_regex;
+  std::array<std::string_view, NumAllowedValues> allowed_values;
+  std::array<std::string_view, NumAllowedTypes> allowed_types;

Review Comment:
   Since this was a list of strings before, as well, I would prefer to do this separately.  I have created a Jira for it: https://issues.apache.org/jira/browse/MINIFICPP-2164.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1267693191


##########
libminifi/include/core/PropertyType.h:
##########
@@ -90,12 +47,19 @@ class PropertyValidator {
   [[nodiscard]] virtual ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const = 0;
 
   [[nodiscard]] virtual ValidationResult validate(const std::string &subject, const std::string &input) const = 0;
+};
+
+class PropertyType : public PropertyParser, public PropertyValidator {

Review Comment:
   as discussed offline, I have renamed `getName()` to `getValidatorName()` in b052e55ef42515d9ef441975cf1188e95c9625f0 (675e2a662b66d1330690873e6fe9b006eee51670 after the rebase)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1247998015


##########
extensions/mqtt/processors/AbstractMQTTProcessor.h:
##########
@@ -44,47 +59,76 @@ class AbstractMQTTProcessor : public core::Processor {
     freeResources();
   }
 
-  SMART_ENUM(MqttVersions,
-    (V_3X_AUTO, "3.x AUTO"),
-    (V_3_1_0, "3.1.0"),
-    (V_3_1_1, "3.1.1"),
-    (V_5_0, "5.0"));
-
-  SMART_ENUM(MqttQoS,
-    (LEVEL_0, "0"),
-    (LEVEL_1, "1"),
-    (LEVEL_2, "2"));
-
-  EXTENSIONAPI static const core::Property BrokerURI;
-  EXTENSIONAPI static const core::Property ClientID;
-  EXTENSIONAPI static const core::Property QoS;
-  EXTENSIONAPI static const core::Property MqttVersion;
-  EXTENSIONAPI static const core::Property ConnectionTimeout;
-  EXTENSIONAPI static const core::Property KeepAliveInterval;
-  EXTENSIONAPI static const core::Property LastWillTopic;
-  EXTENSIONAPI static const core::Property LastWillMessage;
-  EXTENSIONAPI static const core::Property LastWillQoS;
-  EXTENSIONAPI static const core::Property LastWillRetain;
-  EXTENSIONAPI static const core::Property LastWillContentType;
-  EXTENSIONAPI static const core::Property Username;
-  EXTENSIONAPI static const core::Property Password;
-  EXTENSIONAPI static const core::Property SecurityProtocol;
-  EXTENSIONAPI static const core::Property SecurityCA;
-  EXTENSIONAPI static const core::Property SecurityCert;
-  EXTENSIONAPI static const core::Property SecurityPrivateKey;
-  EXTENSIONAPI static const core::Property SecurityPrivateKeyPassword;
-
-
-  static auto basicProperties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto BrokerURI = core::PropertyDefinitionBuilder<>::createProperty("Broker URI")
+      .withDescription("The URI to use to connect to the MQTT broker")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ClientID =       core::PropertyDefinitionBuilder<>::createProperty("Client ID")

Review Comment:
   fixed in bde74b220d3a6755b91e5104648b0a2b0f57fafc



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1247633966


##########
extensions/gcp/processors/ListGCSBucket.h:
##########
@@ -21,12 +21,26 @@
 #include <string>
 #include <utility>
 
+#include "../GCPAttributes.h"
 #include "GCSProcessor.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "core/OutputAttributeDefinition.h"
+#include "core/PropertyDefinition.h"
+#include "core/PropertyDefinitionBuilder.h"
+#include "core/PropertyType.h"
+#include "core/RelationshipDefinition.h"
 #include "utils/ArrayUtils.h"
 
 namespace org::apache::nifi::minifi::extensions::gcp {
 
+namespace detail {
+inline constexpr std::string_view FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_PART_1{"Same as "};
+inline constexpr auto FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_ARRAY = utils::array_cat(
+    utils::string_view_to_array<FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_PART_1.size()>(FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_PART_1),
+    utils::string_view_to_array<GCS_OBJECT_NAME_ATTR.size()>(GCS_OBJECT_NAME_ATTR));
+inline constexpr auto FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION = utils::array_to_string_view(FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_ARRAY);

Review Comment:
   All caps are better reserved for macros IMO, I'd use something else for typed constants. I don't think we have a consistent naming scheme for this, but both CamelCase and snake_case works for me. I think CamelCase is a bit more common in the codebase for constants, especially now that they're used for property definitions as well.
   The Google C++ style guide recommends kPrefixedCamelCase, but I don't really like that, and we don't follow this anywhere at the moment. https://google.github.io/styleguide/cppguide.html#Constant_Names



##########
extensions/mqtt/processors/AbstractMQTTProcessor.h:
##########
@@ -44,47 +59,76 @@ class AbstractMQTTProcessor : public core::Processor {
     freeResources();
   }
 
-  SMART_ENUM(MqttVersions,
-    (V_3X_AUTO, "3.x AUTO"),
-    (V_3_1_0, "3.1.0"),
-    (V_3_1_1, "3.1.1"),
-    (V_5_0, "5.0"));
-
-  SMART_ENUM(MqttQoS,
-    (LEVEL_0, "0"),
-    (LEVEL_1, "1"),
-    (LEVEL_2, "2"));
-
-  EXTENSIONAPI static const core::Property BrokerURI;
-  EXTENSIONAPI static const core::Property ClientID;
-  EXTENSIONAPI static const core::Property QoS;
-  EXTENSIONAPI static const core::Property MqttVersion;
-  EXTENSIONAPI static const core::Property ConnectionTimeout;
-  EXTENSIONAPI static const core::Property KeepAliveInterval;
-  EXTENSIONAPI static const core::Property LastWillTopic;
-  EXTENSIONAPI static const core::Property LastWillMessage;
-  EXTENSIONAPI static const core::Property LastWillQoS;
-  EXTENSIONAPI static const core::Property LastWillRetain;
-  EXTENSIONAPI static const core::Property LastWillContentType;
-  EXTENSIONAPI static const core::Property Username;
-  EXTENSIONAPI static const core::Property Password;
-  EXTENSIONAPI static const core::Property SecurityProtocol;
-  EXTENSIONAPI static const core::Property SecurityCA;
-  EXTENSIONAPI static const core::Property SecurityCert;
-  EXTENSIONAPI static const core::Property SecurityPrivateKey;
-  EXTENSIONAPI static const core::Property SecurityPrivateKeyPassword;
-
-
-  static auto basicProperties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto BrokerURI = core::PropertyDefinitionBuilder<>::createProperty("Broker URI")
+      .withDescription("The URI to use to connect to the MQTT broker")
+      .isRequired(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ClientID =       core::PropertyDefinitionBuilder<>::createProperty("Client ID")

Review Comment:
   Extra spaces



##########
PROCESSORS.md:
##########
@@ -469,12 +471,12 @@ In the list below, the names of required properties appear in bold. Any other pr
 | Resolve Metadata in Attributes     | true                                                                                                                                                                                                                        |                                     | If true, any metadata that is resolved ( such as IDs or keyword metadata ) will be placed into attributes, otherwise it will be replaced in the XML or text output                                                                                                                                                                                                    |
 | Event Header Delimiter             |                                                                                                                                                                                                                             |                                     | If set, the chosen delimiter will be used in the Event output header. Otherwise, a colon followed by spaces will be used.                                                                                                                                                                                                                                             |
 | Event Header                       | LOG_NAME=Log Name, SOURCE = Source, TIME_CREATED = Date,EVENT_RECORDID=Record ID,EVENTID = Event ID,TASK_CATEGORY = Task Category,LEVEL = Level,KEYWORDS = Keywords,USER = User,COMPUTER = Computer, EVENT_TYPE = EventType |                                     | Comma seperated list of key/value pairs with the following keys LOG_NAME, SOURCE, TIME_CREATED,EVENT_RECORDID,EVENTID,TASK_CATEGORY,LEVEL,KEYWORDS,USER,COMPUTER, and EVENT_TYPE. Eliminating fields will remove them from the header.                                                                                                                                |
-| **Output Format**                  | Both                                                                                                                                                                                                                        | Both<br/>JSON<br/>Plaintext<br/>XML | Set the output format type. In case 'Both' is selected the processor generates two flow files for every event captured in format XML and Plaintext                                                                                                                                                                                                                    |
-| **JSON Format**                    | Simple                                                                                                                                                                                                                      | Simple<br/>Raw<br/>Flattened        | Set the json format type. Only applicable if Output Format is set to 'JSON'                                                                                                                                                                                                                                                                                           |
+| **Output Format**                  | Both                                                                                                                                                                                                                        | XML<br/>Both<br/>Plaintext<br/>JSON | Set the output format type. In case 'Both' is selected the processor generates two flow files for every event captured in format XML and Plaintext                                                                                                                                                                                                                    |
+| **JSON Format**                    | Simple                                                                                                                                                                                                                      | Raw<br/>Simple<br/>Flattened        | Set the json format type. Only applicable if Output Format is set to 'JSON'                                                                                                                                                                                                                                                                                           |
 | Batch Commit Size                  | 1000                                                                                                                                                                                                                        |                                     | Maximum number of Events to consume and create to Flow Files from before committing.                                                                                                                                                                                                                                                                                  |
 | State Directory                    | CWELState                                                                                                                                                                                                                   |                                     | DEPRECATED. Only use it for state migration from the state file, supplying the legacy state directory.                                                                                                                                                                                                                                                                |
 | **Process Old Events**             | false                                                                                                                                                                                                                       |                                     | This property defines if old events (which are created before first time server is started) should be processed.                                                                                                                                                                                                                                                      |
-| Cache SID Lookups                  | true                                                                                                                                                                                                                        | true<br>false                       | Determines whether SID to name lookups are cached in memory                                                                                                                                                                                                                                                                                                           |
+| Cache SID Lookups                  | true                                                                                                                                                                                                                        |                                     | Determines whether SID to name lookups are cached in memory                                                                                                                                                                                                                                                                                                           |

Review Comment:
   true/false was more helpful than nothing



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1258396345


##########
extensions/azure/processors/ListAzureBlobStorage.h:
##########
@@ -27,31 +27,42 @@
 #include <vector>
 
 #include "core/Property.h"
+#include "core/PropertyDefinition.h"
 #include "AzureBlobStorageProcessorBase.h"
 #include "core/logging/LoggerConfiguration.h"
 
 namespace org::apache::nifi::minifi::azure::processors {
 
+namespace azure {
+SMART_ENUM(EntityTracking,
+  (NONE, "none"),
+  (TIMESTAMPS, "timestamps")
+)
+}  // namespace azure

Review Comment:
   This definition makes the `EntityTracking` part of namespace `org::apache::nifi::minifi::azure::processors::azure` where `azure` is duplicated, I'm not sure that's a good idea. I think it would be better to have it in the `org::apache::nifi::minifi::azure` namespace like:
   
   ```
   namespace org::apache::nifi::minifi::azure {
   SMART_ENUM(EntityTracking,
     (NONE, "none"),
     (TIMESTAMPS, "timestamps")
   )
   namespace processors {
   ...
   }  // namespace processors
   }  // namespace org::apache::nifi::minifi::azure
   
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259929960


##########
libminifi/test/unit/CoreTests.cpp:
##########


Review Comment:
   fixed in 0352bb48727b839c9a4288193ea72348a70dc57e



##########
libminifi/test/unit/EnumTests.cpp:
##########


Review Comment:
   fixed in 0352bb48727b839c9a4288193ea72348a70dc57e



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265144838


##########
libminifi/src/core/Property.cpp:
##########
@@ -90,4 +94,41 @@ std::vector<std::pair<std::string, std::string>> Property::getExclusiveOfPropert
   return exclusive_of_properties_;
 }
 
+namespace {
+std::vector<PropertyValue> createPropertyValues(gsl::span<const std::string_view> values, const core::PropertyParser& property_parser) {
+  return ranges::views::transform(values, [&property_parser](const auto& value) {
+    return property_parser.parse(value);
+  }) | ranges::to<std::vector>;
+}
+
+inline std::vector<std::string> createStrings(gsl::span<const std::string_view> string_views) {
+  return ranges::views::transform(string_views, [](const auto& string_view) { return std::string{string_view}; })
+      | ranges::to<std::vector>;
+}
+
+inline std::vector<std::pair<std::string, std::string>> createStrings(gsl::span<const std::pair<std::string_view, std::string_view>> pairs_of_string_views) {
+  return ranges::views::transform(pairs_of_string_views, [](const auto& pair_of_string_views) { return std::pair<std::string, std::string>(pair_of_string_views); })
+      | ranges::to<std::vector>;
+}
+}  // namespace
+
+Property::Property(const PropertyReference& compile_time_property)
+    : name_(compile_time_property.name),
+      description_(compile_time_property.description),
+      is_required_(compile_time_property.is_required),
+      valid_regex_(compile_time_property.valid_regex),

Review Comment:
   You're right, it isn't used.  We do include it in `AgentInformation.h` as `validRegex` (although I can't see it in the manifest, which is confusing, but there is a unit test for it).  Should I remove it from there, too, or should we keep it as an always-empty string, as it is 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259931609


##########
libminifi/include/core/Core.h:
##########
@@ -95,6 +98,62 @@ static inline std::string getClassName() {
 #endif
 }
 
+constexpr std::string_view removeStructOrClassPrefix(std::string_view input) {
+  constexpr std::string_view STRUCT = "struct ";  // should be static constexpr, but that is only allowed inside a constexpr function with std >= c++23
+  constexpr std::string_view CLASS = "class ";

Review Comment:
   I like this; changed to this version in 441b998f4fa682fa71345e1be8fafe8073c0cd32.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1258454601


##########
extensions/azure/processors/ListAzureBlobStorage.h:
##########
@@ -27,31 +27,42 @@
 #include <vector>
 
 #include "core/Property.h"
+#include "core/PropertyDefinition.h"
 #include "AzureBlobStorageProcessorBase.h"
 #include "core/logging/LoggerConfiguration.h"
 
 namespace org::apache::nifi::minifi::azure::processors {
 
+namespace azure {
+SMART_ENUM(EntityTracking,
+  (NONE, "none"),
+  (TIMESTAMPS, "timestamps")
+)
+}  // namespace azure

Review Comment:
   good catch, thanks!  fixed in 843c7db76370ace9e42ba9aca07c96bbc705fd8a



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1263887566


##########
libminifi/include/core/PropertyDefinition.h:
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <array>
+#include <optional>
+#include <string_view>
+#include <utility>
+
+#include "core/PropertyType.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::core {
+
+template<size_t NumAllowedValues = 0, size_t NumAllowedTypes = 0, size_t NumDependentProperties = 0, size_t NumExclusiveOfProperties = 0>
+struct PropertyDefinition {
+  std::string_view name;
+  std::string_view display_name;
+  std::string_view description;
+  bool is_required = false;
+  std::string_view valid_regex;
+  std::array<std::string_view, NumAllowedValues> allowed_values;
+  std::array<std::string_view, NumAllowedTypes> allowed_types;

Review Comment:
   I think we could store these as actual types, instead of type name strings, and defer the string conversion to later. `utils/meta/type_list.h` could optionally be used for one kind of implementation.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1247987780


##########
extensions/gcp/processors/ListGCSBucket.h:
##########
@@ -21,12 +21,26 @@
 #include <string>
 #include <utility>
 
+#include "../GCPAttributes.h"
 #include "GCSProcessor.h"
 #include "core/logging/LoggerConfiguration.h"
+#include "core/OutputAttributeDefinition.h"
+#include "core/PropertyDefinition.h"
+#include "core/PropertyDefinitionBuilder.h"
+#include "core/PropertyType.h"
+#include "core/RelationshipDefinition.h"
 #include "utils/ArrayUtils.h"
 
 namespace org::apache::nifi::minifi::extensions::gcp {
 
+namespace detail {
+inline constexpr std::string_view FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_PART_1{"Same as "};
+inline constexpr auto FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_ARRAY = utils::array_cat(
+    utils::string_view_to_array<FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_PART_1.size()>(FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_PART_1),
+    utils::string_view_to_array<GCS_OBJECT_NAME_ATTR.size()>(GCS_OBJECT_NAME_ATTR));
+inline constexpr auto FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION = utils::array_to_string_view(FILENAME_OUTPUT_ATTRIBUTE_DESCRIPTION_ARRAY);

Review Comment:
   I'm not sure about this, since we do use ALL_CAPS for many typed constants (e.g. `GCPAttributes.h`); although some of these probably used to be macros.  I think ALL_CAPS could be extended from just macros to compile-time constant numbers and strings.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265636500


##########
libminifi/include/core/PropertyType.h:
##########
@@ -90,12 +47,19 @@ class PropertyValidator {
   [[nodiscard]] virtual ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const = 0;
 
   [[nodiscard]] virtual ValidationResult validate(const std::string &subject, const std::string &input) const = 0;
+};
+
+class PropertyType : public PropertyParser, public PropertyValidator {

Review Comment:
   On the 2nd part: good catch, thanks!  We can't change the validator names we send to C2.  I have reverted them in 1f9cd0bf0dd802b61175e670dba59d4e0ca2fb25.
   
   On the 1st part: this is not really multiple inheritance, just a separation of two interfaces.  Are you sure we want to replace 12 classes, each implementing two interfaces, with 36 classes using composition?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1268204386


##########
extensions/standard-processors/tests/unit/ManifestTests.cpp:
##########
@@ -74,15 +74,13 @@ TEST_CASE("Test Valid Regex", "[validRegex]") {
   const auto &prop_descriptors = proc_0.children[0];
   REQUIRE_FALSE(prop_descriptors.children.empty());
   const auto &prop_0 = prop_descriptors.children[0];
-  REQUIRE(prop_0.children.size() >= 3);
+  REQUIRE(prop_0.children.size() >= 6);

Review Comment:
   We were dereferencing `children[5]` and `children[6]`, but only checked that `children` has at least 3 elements.  I have removed the check of `children[6]`, but we still use `children[5]`, so we need at least 6 elements.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259656414


##########
libminifi/include/agent/agent_docs.h:
##########
@@ -61,9 +63,12 @@ struct Components {
 };
 
 namespace detail {
-template<typename Container>
-auto toVector(const Container& container) {
-  return std::vector<typename Container::value_type>(container.begin(), container.end());
+inline auto toVector(std::span<const core::PropertyReference> properties) {
+  return std::vector<core::Property>(properties.begin(), properties.end());
+}
+
+inline auto toVector(std::span<const core::RelationshipDefinition> relationships) {
+  return std::vector<core::Relationship>(relationships.begin(), relationships.end());

Review Comment:
   This would mean changing
   ```c++
   .class_properties_ = detail::toVector(Class::Properties)
   ```
   to
   ```c++
   .class_properties_ = utils::span_to<std::vector<core::Property>>(std::span<const core::PropertyReference>(Class::Properties))
   ```
   in three places (plus one more for Relationships).  I'll do it if you think it's nicer, but I'm not sure it's nicer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259937407


##########
libminifi/include/utils/ArrayUtils.h:
##########
@@ -37,4 +39,35 @@ constexpr auto array_cat(const std::array<Type, sizes>&... arrays) {
   return result;
 }
 
+template<std::size_t size>
+constexpr auto string_view_to_array(std::string_view input) {
+  std::array<char, size> result;
+  std::copy_n(input.begin(), size, result.begin());
+  return result;
+}
+
+template<std::size_t size>
+constexpr std::string_view array_to_string_view(const std::array<char, size>& input) {
+  return {input.data(), input.size()};
+}
+
+template<typename Key, typename Value, size_t Size>
+constexpr std::array<Key, Size> getKeys(const std::array<std::pair<Key, Value>, Size>& mapping) {
+  std::array<Key, Size> result;
+  for (size_t idx = 0; idx < Size; ++idx) {
+    result.at(idx) = mapping.at(idx).first;
+  }
+  return result;

Review Comment:
   fixed in 22c4a25e3707770977ff57d34fd77e59d6ce3b84



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] martinzink commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "martinzink (via GitHub)" <gi...@apache.org>.
martinzink commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1247874034


##########
extensions/aws/processors/S3Processor.h:
##########
@@ -83,23 +90,67 @@ struct CommonProperties {
 
 class S3Processor : public core::Processor {
  public:
-  static const std::set<std::string> REGIONS;
-
-  EXTENSIONAPI static const core::Property Bucket;
-  EXTENSIONAPI static const core::Property AccessKey;
-  EXTENSIONAPI static const core::Property SecretKey;
-  EXTENSIONAPI static const core::Property CredentialsFile;
-  EXTENSIONAPI static const core::Property AWSCredentialsProviderService;
-  EXTENSIONAPI static const core::Property Region;
-  EXTENSIONAPI static const core::Property CommunicationsTimeout;
-  EXTENSIONAPI static const core::Property EndpointOverrideURL;
-  EXTENSIONAPI static const core::Property ProxyHost;
-  EXTENSIONAPI static const core::Property ProxyPort;
-  EXTENSIONAPI static const core::Property ProxyUsername;
-  EXTENSIONAPI static const core::Property ProxyPassword;
-  EXTENSIONAPI static const core::Property UseDefaultCredentials;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto Bucket = core::PropertyDefinitionBuilder<>::createProperty("Bucket")
+      .withDescription("The S3 bucket")
+      .isRequired(true)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto AccessKey = core::PropertyDefinitionBuilder<>::createProperty("Access Key")
+      .withDescription("AWS account access key")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto SecretKey = core::PropertyDefinitionBuilder<>::createProperty("Secret Key")
+      .withDescription("AWS account secret key")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto CredentialsFile = core::PropertyDefinitionBuilder<>::createProperty("Credentials File")
+      .withDescription("Path to a file containing AWS access key and secret key in properties file format. Properties used: accessKey and secretKey")
+      .build();
+  EXTENSIONAPI static constexpr auto AWSCredentialsProviderService = core::PropertyDefinitionBuilder<>::createProperty("AWS Credentials Provider service")
+      .withDescription("The name of the AWS Credentials Provider controller service that is used to obtain AWS credentials.")
+      .build();
+  EXTENSIONAPI static constexpr auto Region = core::PropertyDefinitionBuilder<region::REGIONS.size()>::createProperty("Region")
+      .isRequired(true)
+      .withDefaultValue(region::US_WEST_2)
+      .withAllowedValues(region::REGIONS)
+      .withDescription("AWS Region")
+      .build();
+  EXTENSIONAPI static constexpr auto CommunicationsTimeout = core::PropertyDefinitionBuilder<>::createProperty("Communications Timeout")
+      .isRequired(true)
+      .withPropertyType(core::StandardPropertyTypes::TIME_PERIOD_TYPE)
+      .withDefaultValue("30 sec")
+      .withDescription("Sets the timeout of the communication between the AWS server and the client")
+      .build();
+  EXTENSIONAPI static constexpr auto EndpointOverrideURL = core::PropertyDefinitionBuilder<>::createProperty("Endpoint Override URL")
+      .withDescription("Endpoint URL to use instead of the AWS default including scheme, host, "
+          "port, and path. The AWS libraries select an endpoint URL based on the AWS "
+          "region, but this property overrides the selected endpoint URL, allowing use "
+          "with other S3-compatible endpoints.")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ProxyHost = core::PropertyDefinitionBuilder<>::createProperty("Proxy Host")
+      .withDescription("Proxy host name or IP")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ProxyPort = core::PropertyDefinitionBuilder<>::createProperty("Proxy Port")
+      .withDescription("The port number of the proxy host")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ProxyUsername =   core::PropertyDefinitionBuilder<>::createProperty("Proxy Username")

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto ProxyUsername = core::PropertyDefinitionBuilder<>::createProperty("Proxy Username")
   ```



##########
extensions/gcp/processors/ListGCSBucket.h:
##########
@@ -37,66 +51,71 @@ class ListGCSBucket : public GCSProcessor {
   EXTENSIONAPI static constexpr const char* Description = "Retrieves a listing of objects from an GCS bucket. "
       "For each object that is listed, creates a FlowFile that represents the object so that it can be fetched in conjunction with FetchGCSObject.";
 
-  EXTENSIONAPI static const core::Property Bucket;
-  EXTENSIONAPI static const core::Property ListAllVersions;
-  static auto properties() {
-    return utils::array_cat(GCSProcessor::properties(), std::array{
+  EXTENSIONAPI static constexpr auto Bucket = core::PropertyDefinitionBuilder<>::createProperty("Bucket")
+      .withDescription("Bucket of the object.")
+      .isRequired(true)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ListAllVersions = core::PropertyDefinitionBuilder<>::createProperty("List all versions")
+      .withDescription("Set this option to `true` to get all the previous versions separately.")
+      .withPropertyType(core::StandardPropertyTypes::BOOLEAN_TYPE)
+      .withDefaultValue("false")
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = utils::array_cat(GCSProcessor::Properties, std::array<core::PropertyReference, 2>{
       Bucket,
       ListAllVersions
-    });
-  }
+  });
 
-  EXTENSIONAPI static const core::Relationship Success;
-  static auto relationships() { return std::array{Success}; }
-
-  EXTENSIONAPI static const core::OutputAttribute BucketOutputAttribute;
-  EXTENSIONAPI static const core::OutputAttribute Key;
-  EXTENSIONAPI static const core::OutputAttribute Filename;
-  EXTENSIONAPI static const core::OutputAttribute Size;
-  EXTENSIONAPI static const core::OutputAttribute Crc32c;
-  EXTENSIONAPI static const core::OutputAttribute Md5;
-  EXTENSIONAPI static const core::OutputAttribute OwnerEntity;
-  EXTENSIONAPI static const core::OutputAttribute OwnerEntityId;
-  EXTENSIONAPI static const core::OutputAttribute ContentEncoding;
-  EXTENSIONAPI static const core::OutputAttribute ContentLanguage;
-  EXTENSIONAPI static const core::OutputAttribute ContentDisposition;
-  EXTENSIONAPI static const core::OutputAttribute MediaLink;
-  EXTENSIONAPI static const core::OutputAttribute SelfLink;
-  EXTENSIONAPI static const core::OutputAttribute Etag;
-  EXTENSIONAPI static const core::OutputAttribute GeneratedId;
-  EXTENSIONAPI static const core::OutputAttribute Generation;
-  EXTENSIONAPI static const core::OutputAttribute Metageneration;
-  EXTENSIONAPI static const core::OutputAttribute CreateTime;
-  EXTENSIONAPI static const core::OutputAttribute UpdateTime;
-  EXTENSIONAPI static const core::OutputAttribute DeleteTime;
-  EXTENSIONAPI static const core::OutputAttribute EncryptionAlgorithm;
-  EXTENSIONAPI static const core::OutputAttribute EncryptionSha256;
-  static auto outputAttributes() {
-    return std::array{
-        BucketOutputAttribute,
-        Key,
-        Filename,
-        Size,
-        Crc32c,
-        Md5,
-        OwnerEntity,
-        OwnerEntityId,
-        ContentEncoding,
-        ContentLanguage,
-        ContentDisposition,
-        MediaLink,
-        SelfLink,
-        Etag,
-        GeneratedId,
-        Generation,
-        Metageneration,
-        CreateTime,
-        UpdateTime,
-        DeleteTime,
-        EncryptionAlgorithm,
-        EncryptionSha256
-    };
-  }
+
+  EXTENSIONAPI static constexpr auto Success =  core::RelationshipDefinition{"success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation."};

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Success = core::RelationshipDefinition{"success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation."};
   ```



##########
extensions/pcap/CapturePacket.h:
##########
@@ -99,21 +103,34 @@ class CapturePacket : public core::Processor {
     " Configuration options exist to adjust the batching of PCAP files. PCAP batching will place a single PCAP into a flow file. "
     "A regular expression selects network interfaces. Bluetooth network interfaces can be selected through a separate option.";
 
-  static const core::Property BatchSize;
-  static const core::Property NetworkControllers;
-  static const core::Property BaseDir;
-  static const core::Property CaptureBluetooth;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto BatchSize = core::PropertyDefinitionBuilder<>::createProperty("Batch Size")
+      .withDescription("The number of packets to combine within a given PCAP")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("50")
+      .build();
+  EXTENSIONAPI static constexpr auto NetworkControllers = core::PropertyDefinitionBuilder<>::createProperty("Network Controllers")
+      .withDescription("Regular expression of the network controller(s) to which we will attach")
+      .withDefaultValue(".*")
+      .build();
+  EXTENSIONAPI static constexpr auto BaseDir = core::PropertyDefinitionBuilder<>::createProperty("Base Directory")
+      .withDescription("Scratch directory for PCAP files")
+      .withDefaultValue("/tmp/")
+      .build();
+  EXTENSIONAPI static constexpr auto CaptureBluetooth = core::PropertyDefinitionBuilder<>::createProperty("Capture Bluetooth")
+      .withDescription("True indicates that we support bluetooth interfaces")
+      .withPropertyType(core::StandardPropertyTypes::BOOLEAN_TYPE)
+      .withDefaultValue("false")
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = std::array<core::PropertyReference, 4>{
       BatchSize,
       NetworkControllers,
       BaseDir,
       CaptureBluetooth
-    };
-  }
+  };
+
 
-  EXTENSIONAPI static const core::Relationship Success;
-  static auto relationships() { return std::array{Success}; }
+  EXTENSIONAPI static constexpr auto Success =  core::RelationshipDefinition{"success", "All files are routed to success"};

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Success = core::RelationshipDefinition{"success", "All files are routed to success"};
   ```



##########
extensions/sql/processors/ExecuteSQL.h:
##########
@@ -41,14 +43,19 @@ class ExecuteSQL : public SQLProcessor, public FlowFileSource {
       "This processor can be scheduled to run using the standard timer-based scheduling methods, or it can be triggered by an incoming FlowFile. "
       "If it is triggered by an incoming FlowFile, then attributes of that FlowFile will be available when evaluating the query.";
 
-  EXTENSIONAPI static const core::Property SQLSelectQuery;
-  static auto properties() {
-    return utils::array_cat(SQLProcessor::properties(), FlowFileSource::properties(), std::array{SQLSelectQuery});
-  }
+  EXTENSIONAPI static constexpr auto SQLSelectQuery = core::PropertyDefinitionBuilder<>::createProperty("SQL select query")
+      .withDescription(
+        "The SQL select query to execute. The query can be empty, a constant value, or built from attributes using Expression Language. "
+        "If this property is specified, it will be used regardless of the content of incoming flowfiles. "
+        "If this property is empty, the content of the incoming flow file is expected to contain a valid SQL select query, to be issued by the processor to the database. "
+        "Note that Expression Language is not evaluated for flow file contents.")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = utils::array_cat(SQLProcessor::Properties, FlowFileSource::Properties, std::array<core::PropertyReference, 1>{SQLSelectQuery});
 
-  EXTENSIONAPI static const core::Relationship Success;
-  EXTENSIONAPI static const core::Relationship Failure;
-  static auto relationships() { return std::array{Success, Failure}; }
+  EXTENSIONAPI static constexpr auto Success =  core::RelationshipDefinition{"success", "Successfully created FlowFile from SQL query result set."};
+  EXTENSIONAPI static constexpr auto Failure =  core::RelationshipDefinition{"failure", "Flow files containing malformed sql statements"};

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Success = core::RelationshipDefinition{"success", "Successfully created FlowFile from SQL query result set."};
     EXTENSIONAPI static constexpr auto Failure = core::RelationshipDefinition{"failure", "Flow files containing malformed sql statements"};
   ```



##########
extensions/standard-processors/tests/unit/ProcessorTests.cpp:
##########
@@ -496,9 +494,9 @@ class TestProcessorNoContent : public minifi::core::Processor {
   }
 
   static constexpr const char* Description = "test resource";
-  static auto properties() { return std::array<core::Property, 0>{}; }
-  static const core::Relationship Success;
-  static auto relationships() { return std::array{Success}; }
+  static constexpr auto Properties = std::array<core::PropertyReference, 0>{};
+  static constexpr auto Success =  core::RelationshipDefinition{"success", "All files are routed to success"};

Review Comment:
   ```suggestion
     static constexpr auto Success = core::RelationshipDefinition{"success", "All files are routed to success"};
   ```



##########
extensions/standard-processors/processors/GenerateFlowFile.h:
##########
@@ -47,23 +51,45 @@ class GenerateFlowFile : public core::Processor {
   EXTENSIONAPI static constexpr const char* Description = "This processor creates FlowFiles with random data or custom content. "
       "GenerateFlowFile is useful for load testing, configuration, and simulation.";
 
-  EXTENSIONAPI static const core::Property FileSize;
-  EXTENSIONAPI static const core::Property BatchSize;
-  EXTENSIONAPI static const core::Property DataFormat;
-  EXTENSIONAPI static const core::Property UniqueFlowFiles;
-  EXTENSIONAPI static const core::Property CustomText;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto FileSize = core::PropertyDefinitionBuilder<>::createProperty("File Size")
+      .withDescription("The size of the file that will be used")
+      .isRequired(false)
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("1 kB")
+      .build();
+  EXTENSIONAPI static constexpr auto BatchSize = core::PropertyDefinitionBuilder<>::createProperty("Batch Size")
+      .withDescription("The number of FlowFiles to be transferred in each invocation")
+      .isRequired(false)
+      .withPropertyType(core::StandardPropertyTypes::INTEGER_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto DataFormat =  core::PropertyDefinitionBuilder<2>::createProperty("Data Format")

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto DataFormat = core::PropertyDefinitionBuilder<2>::createProperty("Data Format")
   ```



##########
extensions/libarchive/BinFiles.h:
##########
@@ -199,33 +202,57 @@ class BinFiles : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* Description = "Bins flow files into buckets based on the number of entries or size of entries";
 
-  EXTENSIONAPI static const core::Property MinSize;
-  EXTENSIONAPI static const core::Property MaxSize;
-  EXTENSIONAPI static const core::Property MinEntries;
-  EXTENSIONAPI static const core::Property MaxEntries;
-  EXTENSIONAPI static const core::Property MaxBinCount;
-  EXTENSIONAPI static const core::Property MaxBinAge;
-  EXTENSIONAPI static const core::Property BatchSize;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto MinSize = core::PropertyDefinitionBuilder<>::createProperty("Minimum Group Size")
+      .withDescription("The minimum size of for the bundle")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_LONG_TYPE)
+      .withDefaultValue("0")
+      .build();
+  EXTENSIONAPI static constexpr auto MaxSize = core::PropertyDefinitionBuilder<>::createProperty("Maximum Group Size")
+      .withDescription("The maximum size for the bundle. If not specified, there is no maximum.")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_LONG_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MinEntries = core::PropertyDefinitionBuilder<>::createProperty("Minimum Number of Entries")
+      .withDescription("The minimum number of files to include in a bundle")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto MaxEntries = core::PropertyDefinitionBuilder<>::createProperty("Maximum Number of Entries")
+      .withDescription("The maximum number of files to include in a bundle. If not specified, there is no maximum.")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxBinAge = core::PropertyDefinitionBuilder<>::createProperty("Max Bin Age")
+      .withDescription("The maximum age of a Bin that will trigger a Bin to be complete. Expected format is <duration> <time unit>")
+      .withPropertyType(core::StandardPropertyTypes::TIME_PERIOD_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxBinCount = core::PropertyDefinitionBuilder<>::createProperty("Maximum number of Bins")
+      .withDescription("Specifies the maximum number of bins that can be held in memory at any one time")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("100")
+      .build();
+  EXTENSIONAPI static constexpr auto BatchSize = core::PropertyDefinitionBuilder<>::createProperty("Batch Size")
+      .withDescription("Maximum number of FlowFiles processed in a single session")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = std::array<core::PropertyReference, 7>{
       MinSize,
       MaxSize,
       MinEntries,
       MaxEntries,
       MaxBinCount,
       MaxBinAge,
       BatchSize
-    };
-  }
+  };
+
 
-  EXTENSIONAPI static const core::Relationship Failure;
-  EXTENSIONAPI static const core::Relationship Original;
-  static auto relationships() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto Failure =  core::RelationshipDefinition{"failure",

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Failure = core::RelationshipDefinition{"failure",
   ```



##########
extensions/libarchive/BinFiles.h:
##########
@@ -199,33 +202,57 @@ class BinFiles : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* Description = "Bins flow files into buckets based on the number of entries or size of entries";
 
-  EXTENSIONAPI static const core::Property MinSize;
-  EXTENSIONAPI static const core::Property MaxSize;
-  EXTENSIONAPI static const core::Property MinEntries;
-  EXTENSIONAPI static const core::Property MaxEntries;
-  EXTENSIONAPI static const core::Property MaxBinCount;
-  EXTENSIONAPI static const core::Property MaxBinAge;
-  EXTENSIONAPI static const core::Property BatchSize;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto MinSize = core::PropertyDefinitionBuilder<>::createProperty("Minimum Group Size")
+      .withDescription("The minimum size of for the bundle")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_LONG_TYPE)
+      .withDefaultValue("0")
+      .build();
+  EXTENSIONAPI static constexpr auto MaxSize = core::PropertyDefinitionBuilder<>::createProperty("Maximum Group Size")
+      .withDescription("The maximum size for the bundle. If not specified, there is no maximum.")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_LONG_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MinEntries = core::PropertyDefinitionBuilder<>::createProperty("Minimum Number of Entries")
+      .withDescription("The minimum number of files to include in a bundle")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto MaxEntries = core::PropertyDefinitionBuilder<>::createProperty("Maximum Number of Entries")
+      .withDescription("The maximum number of files to include in a bundle. If not specified, there is no maximum.")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxBinAge = core::PropertyDefinitionBuilder<>::createProperty("Max Bin Age")
+      .withDescription("The maximum age of a Bin that will trigger a Bin to be complete. Expected format is <duration> <time unit>")
+      .withPropertyType(core::StandardPropertyTypes::TIME_PERIOD_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxBinCount = core::PropertyDefinitionBuilder<>::createProperty("Maximum number of Bins")
+      .withDescription("Specifies the maximum number of bins that can be held in memory at any one time")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("100")
+      .build();
+  EXTENSIONAPI static constexpr auto BatchSize = core::PropertyDefinitionBuilder<>::createProperty("Batch Size")
+      .withDescription("Maximum number of FlowFiles processed in a single session")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = std::array<core::PropertyReference, 7>{
       MinSize,
       MaxSize,
       MinEntries,
       MaxEntries,
       MaxBinCount,
       MaxBinAge,
       BatchSize
-    };
-  }
+  };
+
 
-  EXTENSIONAPI static const core::Relationship Failure;
-  EXTENSIONAPI static const core::Relationship Original;
-  static auto relationships() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto Failure =  core::RelationshipDefinition{"failure",
+      "If the bundle cannot be created, all FlowFiles that would have been used to create the bundle will be transferred to failure"};
+  EXTENSIONAPI static constexpr auto Original =  core::RelationshipDefinition{"original",

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Original = core::RelationshipDefinition{"original",
   ```



##########
extensions/sql/processors/QueryDatabaseTable.h:
##########
@@ -48,25 +50,56 @@ class QueryDatabaseTable: public SQLProcessor, public FlowFileSource {
       "Fetches all rows of a table, whose values in the specified Maximum-value Columns are larger than the previously-seen maxima. "
       "If that property is not provided, all rows are returned. The rows are grouped according to the value of Max Rows Per Flow File property and formatted as JSON.";
 
-  EXTENSIONAPI static const core::Property TableName;
-  EXTENSIONAPI static const core::Property ColumnNames;
-  EXTENSIONAPI static const core::Property MaxValueColumnNames;
-  EXTENSIONAPI static const core::Property WhereClause;
-  static auto properties() {
-    return utils::array_cat(SQLProcessor::properties(), FlowFileSource::properties(), std::array{
+  EXTENSIONAPI static constexpr auto TableName = core::PropertyDefinitionBuilder<>::createProperty("Table Name")
+      .withDescription("The name of the database table to be queried.")
+      .isRequired(true)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ColumnNames = core::PropertyDefinitionBuilder<>::createProperty("Columns to Return")
+      .withDescription(
+        "A comma-separated list of column names to be used in the query. If your database requires special treatment of the names (quoting, e.g.), each name should include such treatment. "
+        "If no column names are supplied, all columns in the specified table will be returned. "
+        "NOTE: It is important to use consistent column names for a given table for incremental fetch to work properly.")
+      .isRequired(false)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxValueColumnNames = core::PropertyDefinitionBuilder<>::createProperty("Maximum-value Columns")
+      .withDescription(
+        "A comma-separated list of column names. The processor will keep track of the maximum value for each column that has been returned since the processor started running. "
+        "Using multiple columns implies an order to the column list, and each column's values are expected to increase more slowly than the previous columns' values. "
+        "Thus, using multiple columns implies a hierarchical structure of columns, which is usually used for partitioning tables. "
+        "This processor can be used to retrieve only those rows that have been added/updated since the last retrieval. "
+        "Note that some ODBC types such as bit/boolean are not conducive to maintaining maximum value, so columns of these types should not be listed in this property, and will result in error(s) during processing. "
+        "If no columns are provided, all rows from the table will be considered, which could have a performance impact. "
+        "NOTE: It is important to use consistent max-value column names for a given table for incremental fetch to work properly. "
+        "NOTE: Because of a limitation of database access library 'soci', which doesn't support milliseconds in it's 'dt_date', "
+        "there is a possibility that flowfiles might have duplicated records, if a max-value column with 'dt_date' type has value with milliseconds.")
+      .isRequired(false)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto WhereClause = core::PropertyDefinitionBuilder<>::createProperty("Where Clause")
+      .withDescription("A custom clause to be added in the WHERE condition when building SQL queries.")
+      .isRequired(false)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = utils::array_cat(SQLProcessor::Properties, FlowFileSource::Properties, std::array<core::PropertyReference, 4>{
       TableName,
       ColumnNames,
       MaxValueColumnNames,
       WhereClause
-    });
-  }
+  });
 
-  EXTENSIONAPI static const core::Relationship Success;
-  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr auto Success =  core::RelationshipDefinition{"success", "Successfully created FlowFile from SQL query result set."};

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Success = core::RelationshipDefinition{"success", "Successfully created FlowFile from SQL query result set."};
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "adamdebreceni (via GitHub)" <gi...@apache.org>.
adamdebreceni commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1237128037


##########
extensions/gcp/processors/GCSProcessor.h:
##########
@@ -34,19 +38,33 @@ class GCSProcessor : public core::Processor {
  public:
   GCSProcessor(std::string name, const minifi::utils::Identifier& uuid, std::shared_ptr<core::logging::Logger> logger)
       : core::Processor(std::move(name), uuid),
-        logger_(std::move(logger)) {
+     #include "PropertyDefinitionBuilder.h"

Review Comment:
   is this here by accident?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259937618


##########
libminifi/include/utils/ArrayUtils.h:
##########
@@ -37,4 +39,35 @@ constexpr auto array_cat(const std::array<Type, sizes>&... arrays) {
   return result;
 }
 
+template<std::size_t size>
+constexpr auto string_view_to_array(std::string_view input) {
+  std::array<char, size> result;
+  std::copy_n(input.begin(), size, result.begin());
+  return result;
+}
+
+template<std::size_t size>
+constexpr std::string_view array_to_string_view(const std::array<char, size>& input) {
+  return {input.data(), input.size()};
+}
+
+template<typename Key, typename Value, size_t Size>
+constexpr std::array<Key, Size> getKeys(const std::array<std::pair<Key, Value>, Size>& mapping) {
+  std::array<Key, Size> result;
+  for (size_t idx = 0; idx < Size; ++idx) {
+    result.at(idx) = mapping.at(idx).first;
+  }
+  return result;
+}
+
+template<typename Container, typename ComparableToKeyType>
+constexpr auto at(const Container& mapping, ComparableToKeyType key) {
+  for (const auto& kv : mapping) {
+    if (kv.first == key) {
+      return kv.second;
+    }
+  }

Review Comment:
   fixed in 22c4a25e3707770977ff57d34fd77e59d6ce3b84



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259935350


##########
libminifi/include/core/ProcessContext.h:
##########
@@ -90,6 +91,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return processor_node_;
   }
 
+  // TODO(fgerlits): remove if possible

Review Comment:
   Not at the moment, unfortunately.  I have removed the TODOs, for now, in 655a8e73959314f2df0807f3bb157f12e1ba0048.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259300239


##########
extensions/azure/processors/ListAzureDataLakeStorage.h:
##########
@@ -25,36 +25,53 @@
 #include <memory>
 
 #include "AzureDataLakeStorageProcessorBase.h"
+#include "core/PropertyDefinitionBuilder.h"
+#include "core/PropertyType.h"
 #include "utils/ArrayUtils.h"
 
 class ListAzureDataLakeStorageTestsFixture;
 
 namespace org::apache::nifi::minifi::azure::processors {
 
+namespace azure {
+SMART_ENUM(EntityTracking,
+  (NONE, "none"),
+  (TIMESTAMPS, "timestamps")
+)
+}  // namespace azure

Review Comment:
   This should also be moved to `org::apache::nifi::minifi::azure` namespace



##########
extensions/azure/processors/PutAzureDataLakeStorage.h:
##########
@@ -35,18 +37,29 @@ class AzureDataLakeStorageTestsFixture;
 
 namespace org::apache::nifi::minifi::azure::processors {
 
+namespace azure {
+SMART_ENUM(FileExistsResolutionStrategy,
+  (FAIL_FLOW, "fail"),
+  (REPLACE_FILE, "replace"),
+  (IGNORE_REQUEST, "ignore")
+)
+}  // namespace azure

Review Comment:
   Same namespace issue here



##########
extensions/standard-processors/tests/unit/ReplaceTextTests.cpp:
##########
@@ -93,7 +93,7 @@ TEST_CASE("Regex Replace works correctly in ReplaceText", "[applyReplacements][R
   replace_text.setSearchRegex("a\\w+e");
   replace_text.setReplacementValue("orange");
 
-  CHECK(replace_text.applyReplacements("") == "");

Review Comment:
   Maybe it would be better to create a NOLINT block for this issue with `// NOLINTBEGIN(readability-container-size-empty)`



##########
libminifi/include/controllers/UpdatePolicyControllerService.h:
##########
@@ -50,18 +53,32 @@ class UpdatePolicyControllerService : public core::controller::ControllerService
   MINIFIAPI static constexpr const char* Description = "UpdatePolicyControllerService allows a flow specific policy on allowing or disallowing updates. "
       "Since the flow dictates the purpose of a device it will also be used to dictate updates to specific components.";
 
-  MINIFIAPI static const core::Property AllowAllProperties;
-  MINIFIAPI static const core::Property PersistUpdates;
-  MINIFIAPI static const core::Property AllowedProperties;
-  MINIFIAPI static const core::Property DisallowedProperties;
-  static auto properties() {
-    return std::array{
+  MINIFIAPI static constexpr auto AllowAllProperties = core::PropertyDefinitionBuilder<>::createProperty("Allow All Properties")
+      .withDescription("Allows all properties, which are also not disallowed, to be updated")
+      .withPropertyType(core::StandardPropertyTypes::BOOLEAN_TYPE)
+      .withDefaultValue("false")
+      .build();
+  MINIFIAPI static constexpr auto PersistUpdates = core::PropertyDefinitionBuilder<>::createProperty("Persist Updates")
+      .withDescription("Property that dictates whether updates should persist after a restart")
+      .isRequired(false)
+      .withPropertyType(core::StandardPropertyTypes::BOOLEAN_TYPE)
+      .withDefaultValue("false")
+      .build();
+  MINIFIAPI static constexpr auto AllowedProperties = core::PropertyDefinitionBuilder<>::createProperty("Allowed Properties")
+      .withDescription("Properties for which we will allow updates")
+      .isRequired(false)
+      .build();
+  MINIFIAPI static constexpr auto DisallowedProperties = core::PropertyDefinitionBuilder<>::createProperty("Disallowed Properties")
+      .withDescription("Properties for which we will not allow updates")
+      .isRequired(false)
+      .build();
+  MINIFIAPI static constexpr auto Properties = std::array<core::PropertyReference, 4>{
       AllowAllProperties,
       PersistUpdates,
       AllowedProperties,
       DisallowedProperties
-    };
-  }
+  };
+

Review Comment:
   Nitpick: Added unnecessary newline



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1268190860


##########
extensions/standard-processors/tests/unit/ManifestTests.cpp:
##########
@@ -74,15 +74,13 @@ TEST_CASE("Test Valid Regex", "[validRegex]") {
   const auto &prop_descriptors = proc_0.children[0];
   REQUIRE_FALSE(prop_descriptors.children.empty());
   const auto &prop_0 = prop_descriptors.children[0];
-  REQUIRE(prop_0.children.size() >= 3);
+  REQUIRE(prop_0.children.size() >= 6);

Review Comment:
   what changed here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1257966290


##########
PROCESSORS.md:
##########
@@ -469,12 +471,12 @@ In the list below, the names of required properties appear in bold. Any other pr
 | Resolve Metadata in Attributes     | true                                                                                                                                                                                                                        |                                     | If true, any metadata that is resolved ( such as IDs or keyword metadata ) will be placed into attributes, otherwise it will be replaced in the XML or text output                                                                                                                                                                                                    |
 | Event Header Delimiter             |                                                                                                                                                                                                                             |                                     | If set, the chosen delimiter will be used in the Event output header. Otherwise, a colon followed by spaces will be used.                                                                                                                                                                                                                                             |
 | Event Header                       | LOG_NAME=Log Name, SOURCE = Source, TIME_CREATED = Date,EVENT_RECORDID=Record ID,EVENTID = Event ID,TASK_CATEGORY = Task Category,LEVEL = Level,KEYWORDS = Keywords,USER = User,COMPUTER = Computer, EVENT_TYPE = EventType |                                     | Comma seperated list of key/value pairs with the following keys LOG_NAME, SOURCE, TIME_CREATED,EVENT_RECORDID,EVENTID,TASK_CATEGORY,LEVEL,KEYWORDS,USER,COMPUTER, and EVENT_TYPE. Eliminating fields will remove them from the header.                                                                                                                                |
-| **Output Format**                  | Both                                                                                                                                                                                                                        | Both<br/>JSON<br/>Plaintext<br/>XML | Set the output format type. In case 'Both' is selected the processor generates two flow files for every event captured in format XML and Plaintext                                                                                                                                                                                                                    |
-| **JSON Format**                    | Simple                                                                                                                                                                                                                      | Simple<br/>Raw<br/>Flattened        | Set the json format type. Only applicable if Output Format is set to 'JSON'                                                                                                                                                                                                                                                                                           |
+| **Output Format**                  | Both                                                                                                                                                                                                                        | XML<br/>Both<br/>Plaintext<br/>JSON | Set the output format type. In case 'Both' is selected the processor generates two flow files for every event captured in format XML and Plaintext                                                                                                                                                                                                                    |
+| **JSON Format**                    | Simple                                                                                                                                                                                                                      | Raw<br/>Simple<br/>Flattened        | Set the json format type. Only applicable if Output Format is set to 'JSON'                                                                                                                                                                                                                                                                                           |
 | Batch Commit Size                  | 1000                                                                                                                                                                                                                        |                                     | Maximum number of Events to consume and create to Flow Files from before committing.                                                                                                                                                                                                                                                                                  |
 | State Directory                    | CWELState                                                                                                                                                                                                                   |                                     | DEPRECATED. Only use it for state migration from the state file, supplying the legacy state directory.                                                                                                                                                                                                                                                                |
 | **Process Old Events**             | false                                                                                                                                                                                                                       |                                     | This property defines if old events (which are created before first time server is started) should be processed.                                                                                                                                                                                                                                                      |
-| Cache SID Lookups                  | true                                                                                                                                                                                                                        | true<br>false                       | Determines whether SID to name lookups are cached in memory                                                                                                                                                                                                                                                                                                           |
+| Cache SID Lookups                  | true                                                                                                                                                                                                                        |                                     | Determines whether SID to name lookups are cached in memory                                                                                                                                                                                                                                                                                                           |

Review Comment:
   https://issues.apache.org/jira/browse/MINIFICPP-2158



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259593659


##########
libminifi/include/core/PropertyDefinition.h:
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <array>
+#include <optional>
+#include <string_view>
+#include <utility>
+
+#include "core/PropertyType.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::core {
+
+template<size_t NumAllowedValues = 0, size_t NumAllowedTypes = 0, size_t NumDependentProperties = 0, size_t NumExclusiveOfProperties = 0>
+struct PropertyDefinition {
+  std::string_view name;
+  std::string_view display_name;
+  std::string_view description;
+  bool is_required = false;
+  std::string_view valid_regex;
+  std::array<std::string_view, NumAllowedValues> allowed_values;
+  std::array<std::string_view, NumAllowedTypes> allowed_types;
+  std::array<std::string_view, NumDependentProperties> dependent_properties;
+  std::array<std::pair<std::string_view, std::string_view>, NumExclusiveOfProperties> exclusive_of_properties;
+  std::optional<std::string_view> default_value;
+  gsl::not_null<const PropertyType*> type = gsl::make_not_null(&StandardPropertyTypes::VALID_TYPE);
+  bool supports_expression_language = false;
+};
+
+struct PropertyReference {
+  std::string_view name;
+  std::string_view display_name;
+  std::string_view description;
+  bool is_required = false;
+  std::string_view valid_regex;
+  gsl::span<const std::string_view> allowed_values;
+  gsl::span<const std::string_view> allowed_types;
+  gsl::span<const std::string_view> dependent_properties;
+  gsl::span<const std::pair<std::string_view, std::string_view>> exclusive_of_properties;
+  std::optional<std::string_view> default_value;
+  const PropertyType* type;

Review Comment:
   Can this ever be null? `PropertyDefinition` doesn't allow null.



##########
libminifi/include/core/Core.h:
##########
@@ -95,6 +98,62 @@ static inline std::string getClassName() {
 #endif
 }
 
+constexpr std::string_view removeStructOrClassPrefix(std::string_view input) {
+  constexpr std::string_view STRUCT = "struct ";  // should be static constexpr, but that is only allowed inside a constexpr function with std >= c++23
+  constexpr std::string_view CLASS = "class ";
+
+  for (auto prefix : { STRUCT, CLASS }) {
+    if (input.find(prefix) == 0) {
+      return input.substr(prefix.size());
+    }
+  }
+  return input;
+}
+
+// based on https://bitwizeshift.github.io/posts/2021/03/09/getting-an-unmangled-type-name-at-compile-time/
+template<typename T>
+constexpr auto typeNameArray() {
+#if defined(__clang__)
+  constexpr auto prefix   = std::string_view{"[T = "};
+  constexpr auto suffix   = std::string_view{"]"};
+  constexpr auto function = std::string_view{__PRETTY_FUNCTION__};
+#elif defined(__GNUC__)
+  constexpr auto prefix   = std::string_view{"with T = "};
+  constexpr auto suffix   = std::string_view{"]"};
+  constexpr auto function = std::string_view{__PRETTY_FUNCTION__};
+#elif defined(_MSC_VER)
+  constexpr auto prefix   = std::string_view{"typeNameArray<"};
+  constexpr auto suffix   = std::string_view{">(void)"};
+  constexpr auto function = std::string_view{__FUNCSIG__};
+#else
+# error Unsupported compiler
+#endif
+
+  static_assert(function.find(prefix) != std::string_view::npos && function.rfind(suffix) != std::string_view::npos);
+
+  constexpr auto start = function.find(prefix) + prefix.size();
+  constexpr auto end = function.rfind(suffix);
+  static_assert(start < end);
+
+#if defined(_MSC_VER)
+  constexpr auto name = removeStructOrClassPrefix(function.substr(start, end - start));
+#else
+  constexpr auto name = function.substr(start, end - start);
+#endif
+
+  return utils::string_view_to_array<name.length()>(name);
+}
+
+template<typename T>
+struct TypeNameHolder {
+  static constexpr auto value = typeNameArray<T>();
+};
+
+template<typename T>
+constexpr std::string_view className() {
+  return utils::array_to_string_view(TypeNameHolder<T>::value);
+}

Review Comment:
   The taken code is a non-trivial amount, and the changes to it are fairly minimal, so I think we should add this to the LICENSE and NOTICE files. https://github.com/bitwizeshift/bitwizeshift.github.io/blob/source/LICENSE



##########
libminifi/include/agent/agent_docs.h:
##########
@@ -61,9 +63,12 @@ struct Components {
 };
 
 namespace detail {
-template<typename Container>
-auto toVector(const Container& container) {
-  return std::vector<typename Container::value_type>(container.begin(), container.end());
+inline auto toVector(std::span<const core::PropertyReference> properties) {
+  return std::vector<core::Property>(properties.begin(), properties.end());
+}
+
+inline auto toVector(std::span<const core::RelationshipDefinition> relationships) {
+  return std::vector<core::Relationship>(relationships.begin(), relationships.end());

Review Comment:
   I was expecting something like this
   ```c++
   .class_properties_ = utils::span_to<std::vector>(Class::Properties)
   ```
   
   If it's not that simple, then don't bother changing it.



##########
libminifi/include/core/Core.h:
##########
@@ -95,6 +98,62 @@ static inline std::string getClassName() {
 #endif
 }
 
+constexpr std::string_view removeStructOrClassPrefix(std::string_view input) {
+  constexpr std::string_view STRUCT = "struct ";  // should be static constexpr, but that is only allowed inside a constexpr function with std >= c++23
+  constexpr std::string_view CLASS = "class ";

Review Comment:
   I'm neutral, pick what you think is best.
   
   What about this, using literals inline?
   
   ```cpp
   constexpr std::string_view removeStructOrClassPrefix(std::string_view input) {
     using namespace std::literals;
     for (auto prefix : { "struct "sv, "class "sv }) {
       if (input.find(prefix) == 0) {
         return input.substr(prefix.size());
       }
     }
     return input;
   }
   ```



##########
libminifi/include/core/ProcessContext.h:
##########
@@ -90,6 +91,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return processor_node_;
   }
 
+  // TODO(fgerlits): remove if possible

Review Comment:
   Is it possible?



##########
libminifi/include/utils/ArrayUtils.h:
##########
@@ -37,4 +39,35 @@ constexpr auto array_cat(const std::array<Type, sizes>&... arrays) {
   return result;
 }
 
+template<std::size_t size>
+constexpr auto string_view_to_array(std::string_view input) {
+  std::array<char, size> result;
+  std::copy_n(input.begin(), size, result.begin());
+  return result;
+}
+
+template<std::size_t size>
+constexpr std::string_view array_to_string_view(const std::array<char, size>& input) {
+  return {input.data(), input.size()};
+}
+
+template<typename Key, typename Value, size_t Size>
+constexpr std::array<Key, Size> getKeys(const std::array<std::pair<Key, Value>, Size>& mapping) {
+  std::array<Key, Size> result;
+  for (size_t idx = 0; idx < Size; ++idx) {
+    result.at(idx) = mapping.at(idx).first;
+  }
+  return result;

Review Comment:
   That's a `std::transform`



##########
libminifi/include/utils/ArrayUtils.h:
##########
@@ -37,4 +39,35 @@ constexpr auto array_cat(const std::array<Type, sizes>&... arrays) {
   return result;
 }
 
+template<std::size_t size>
+constexpr auto string_view_to_array(std::string_view input) {
+  std::array<char, size> result;
+  std::copy_n(input.begin(), size, result.begin());
+  return result;
+}
+
+template<std::size_t size>
+constexpr std::string_view array_to_string_view(const std::array<char, size>& input) {
+  return {input.data(), input.size()};
+}
+
+template<typename Key, typename Value, size_t Size>
+constexpr std::array<Key, Size> getKeys(const std::array<std::pair<Key, Value>, Size>& mapping) {
+  std::array<Key, Size> result;
+  for (size_t idx = 0; idx < Size; ++idx) {
+    result.at(idx) = mapping.at(idx).first;
+  }
+  return result;
+}
+
+template<typename Container, typename ComparableToKeyType>
+constexpr auto at(const Container& mapping, ComparableToKeyType key) {
+  for (const auto& kv : mapping) {
+    if (kv.first == key) {
+      return kv.second;
+    }
+  }

Review Comment:
   That's a `std::find_if`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259936865


##########
libminifi/include/core/PropertyDefinition.h:
##########
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <array>
+#include <optional>
+#include <string_view>
+#include <utility>
+
+#include "core/PropertyType.h"
+#include "utils/gsl.h"
+
+namespace org::apache::nifi::minifi::core {
+
+template<size_t NumAllowedValues = 0, size_t NumAllowedTypes = 0, size_t NumDependentProperties = 0, size_t NumExclusiveOfProperties = 0>
+struct PropertyDefinition {
+  std::string_view name;
+  std::string_view display_name;
+  std::string_view description;
+  bool is_required = false;
+  std::string_view valid_regex;
+  std::array<std::string_view, NumAllowedValues> allowed_values;
+  std::array<std::string_view, NumAllowedTypes> allowed_types;
+  std::array<std::string_view, NumDependentProperties> dependent_properties;
+  std::array<std::pair<std::string_view, std::string_view>, NumExclusiveOfProperties> exclusive_of_properties;
+  std::optional<std::string_view> default_value;
+  gsl::not_null<const PropertyType*> type = gsl::make_not_null(&StandardPropertyTypes::VALID_TYPE);
+  bool supports_expression_language = false;
+};
+
+struct PropertyReference {
+  std::string_view name;
+  std::string_view display_name;
+  std::string_view description;
+  bool is_required = false;
+  std::string_view valid_regex;
+  gsl::span<const std::string_view> allowed_values;
+  gsl::span<const std::string_view> allowed_types;
+  gsl::span<const std::string_view> dependent_properties;
+  gsl::span<const std::pair<std::string_view, std::string_view>> exclusive_of_properties;
+  std::optional<std::string_view> default_value;
+  const PropertyType* type;

Review Comment:
   Good point; I've changed it to `gsl::not_null<const PropertyType*>` in 7e27458d3c2e7ed7ee089bcdde61b93c9d6afc68.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259934099


##########
libminifi/include/core/OutputAttributeDefinition.h:
##########
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+#pragma once
+
+#include <array>
+#include <string_view>
+
+#include "utils/gsl.h"
+#include "RelationshipDefinition.h"
+
+namespace org::apache::nifi::minifi::core {
+
+template<size_t NumRelationships = 1>
+struct OutputAttributeDefinition {
+  constexpr OutputAttributeDefinition(std::string_view name, std::array<RelationshipDefinition, NumRelationships> relationships, std::string_view description)
+      : name(name),
+        relationships(relationships),
+        description(description) {
+  }
+
+  std::string_view name;
+  std::array<RelationshipDefinition, NumRelationships> relationships;
+  std::string_view description;
+};
+
+struct OutputAttributeReference {
+  constexpr OutputAttributeReference() = default;  // TODO(fgerlits): is this needed?

Review Comment:
   Unfortunately it is, because VS 2019 cannot construct an empty `OutputAttributeReference` array if there is no default constructor.
   
   I have replaced the TODO with a comment explaining this in 3c761849296b1fed6b74d08fc731a4e9044d54d0.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259952982


##########
libminifi/include/core/Core.h:
##########
@@ -95,6 +98,62 @@ static inline std::string getClassName() {
 #endif
 }
 
+constexpr std::string_view removeStructOrClassPrefix(std::string_view input) {
+  constexpr std::string_view STRUCT = "struct ";  // should be static constexpr, but that is only allowed inside a constexpr function with std >= c++23
+  constexpr std::string_view CLASS = "class ";
+
+  for (auto prefix : { STRUCT, CLASS }) {
+    if (input.find(prefix) == 0) {
+      return input.substr(prefix.size());
+    }
+  }
+  return input;
+}
+
+// based on https://bitwizeshift.github.io/posts/2021/03/09/getting-an-unmangled-type-name-at-compile-time/
+template<typename T>
+constexpr auto typeNameArray() {
+#if defined(__clang__)
+  constexpr auto prefix   = std::string_view{"[T = "};
+  constexpr auto suffix   = std::string_view{"]"};
+  constexpr auto function = std::string_view{__PRETTY_FUNCTION__};
+#elif defined(__GNUC__)
+  constexpr auto prefix   = std::string_view{"with T = "};
+  constexpr auto suffix   = std::string_view{"]"};
+  constexpr auto function = std::string_view{__PRETTY_FUNCTION__};
+#elif defined(_MSC_VER)
+  constexpr auto prefix   = std::string_view{"typeNameArray<"};
+  constexpr auto suffix   = std::string_view{">(void)"};
+  constexpr auto function = std::string_view{__FUNCSIG__};
+#else
+# error Unsupported compiler
+#endif
+
+  static_assert(function.find(prefix) != std::string_view::npos && function.rfind(suffix) != std::string_view::npos);
+
+  constexpr auto start = function.find(prefix) + prefix.size();
+  constexpr auto end = function.rfind(suffix);
+  static_assert(start < end);
+
+#if defined(_MSC_VER)
+  constexpr auto name = removeStructOrClassPrefix(function.substr(start, end - start));
+#else
+  constexpr auto name = function.substr(start, end - start);
+#endif
+
+  return utils::string_view_to_array<name.length()>(name);
+}
+
+template<typename T>
+struct TypeNameHolder {
+  static constexpr auto value = typeNameArray<T>();
+};
+
+template<typename T>
+constexpr std::string_view className() {
+  return utils::array_to_string_view(TypeNameHolder<T>::value);
+}

Review Comment:
   added to the LICENSE and NOTICE files in f93be28fc17ccafdf82f857fd6e2087453fbb3d4



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259938577


##########
extensions/azure/processors/ListAzureDataLakeStorage.h:
##########
@@ -25,36 +25,53 @@
 #include <memory>
 
 #include "AzureDataLakeStorageProcessorBase.h"
+#include "core/PropertyDefinitionBuilder.h"
+#include "core/PropertyType.h"
 #include "utils/ArrayUtils.h"
 
 class ListAzureDataLakeStorageTestsFixture;
 
 namespace org::apache::nifi::minifi::azure::processors {
 
+namespace azure {
+SMART_ENUM(EntityTracking,
+  (NONE, "none"),
+  (TIMESTAMPS, "timestamps")
+)
+}  // namespace azure

Review Comment:
   fixed in 6e0efa05f7da6bbc5f6a079d0d16b1d9100570f6.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265626062


##########
libminifi/include/core/ProcessContext.h:
##########
@@ -102,18 +103,44 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return value;
   }
 
+  template<typename T = std::string>
+  std::enable_if_t<std::is_default_constructible<T>::value, std::optional<T>>
+  getProperty(const PropertyReference& property) const {

Review Comment:
   done in 1f8b8afb8ec970de037384b8af85081fcd370bd1



##########
libminifi/include/core/ProcessContext.h:
##########
@@ -102,18 +103,44 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return value;
   }
 
+  template<typename T = std::string>
+  std::enable_if_t<std::is_default_constructible<T>::value, std::optional<T>>
+  getProperty(const PropertyReference& property) const {
+    T value;
+    try {
+      if (!getProperty(property.name, value)) return std::nullopt;
+    } catch (const utils::internal::ValueException&) {
+      return std::nullopt;
+    }
+    return value;
+  }
+
+  template<typename T>
+  requires(!std::is_convertible_v<T&, const FlowFile&> && !std::is_convertible_v<T&, const std::shared_ptr<FlowFile>&>)
+  bool getProperty(std::string_view name, T &value) const {

Review Comment:
   done in 1f8b8afb8ec970de037384b8af85081fcd370bd1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265631311


##########
libminifi/src/core/Property.cpp:
##########
@@ -90,4 +94,41 @@ std::vector<std::pair<std::string, std::string>> Property::getExclusiveOfPropert
   return exclusive_of_properties_;
 }
 
+namespace {
+std::vector<PropertyValue> createPropertyValues(gsl::span<const std::string_view> values, const core::PropertyParser& property_parser) {
+  return ranges::views::transform(values, [&property_parser](const auto& value) {
+    return property_parser.parse(value);
+  }) | ranges::to<std::vector>;
+}
+
+inline std::vector<std::string> createStrings(gsl::span<const std::string_view> string_views) {
+  return ranges::views::transform(string_views, [](const auto& string_view) { return std::string{string_view}; })
+      | ranges::to<std::vector>;
+}
+
+inline std::vector<std::pair<std::string, std::string>> createStrings(gsl::span<const std::pair<std::string_view, std::string_view>> pairs_of_string_views) {
+  return ranges::views::transform(pairs_of_string_views, [](const auto& pair_of_string_views) { return std::pair<std::string, std::string>(pair_of_string_views); })
+      | ranges::to<std::vector>;
+}
+}  // namespace
+
+Property::Property(const PropertyReference& compile_time_property)
+    : name_(compile_time_property.name),
+      description_(compile_time_property.description),
+      is_required_(compile_time_property.is_required),
+      valid_regex_(compile_time_property.valid_regex),

Review Comment:
   removed in 9ab589d0021d742d2963d910875cd9b6611b6bc2



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259938577


##########
extensions/azure/processors/ListAzureDataLakeStorage.h:
##########
@@ -25,36 +25,53 @@
 #include <memory>
 
 #include "AzureDataLakeStorageProcessorBase.h"
+#include "core/PropertyDefinitionBuilder.h"
+#include "core/PropertyType.h"
 #include "utils/ArrayUtils.h"
 
 class ListAzureDataLakeStorageTestsFixture;
 
 namespace org::apache::nifi::minifi::azure::processors {
 
+namespace azure {
+SMART_ENUM(EntityTracking,
+  (NONE, "none"),
+  (TIMESTAMPS, "timestamps")
+)
+}  // namespace azure

Review Comment:
   fixed in 6e0efa05f7da6bbc5f6a079d0d16b1d9100570f6



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259777005


##########
libminifi/include/agent/agent_docs.h:
##########
@@ -61,9 +63,12 @@ struct Components {
 };
 
 namespace detail {
-template<typename Container>
-auto toVector(const Container& container) {
-  return std::vector<typename Container::value_type>(container.begin(), container.end());
+inline auto toVector(std::span<const core::PropertyReference> properties) {
+  return std::vector<core::Property>(properties.begin(), properties.end());
+}
+
+inline auto toVector(std::span<const core::RelationshipDefinition> relationships) {
+  return std::vector<core::Relationship>(relationships.begin(), relationships.end());

Review Comment:
   The simpler form doesn't work, because the template argument detection can't figure out that we want to convert `PropertyReference`s to `Property`s.  We have to tell this to the compiler explicitly as 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259928495


##########
extensions/standard-processors/tests/unit/ReplaceTextTests.cpp:
##########
@@ -93,7 +93,7 @@ TEST_CASE("Regex Replace works correctly in ReplaceText", "[applyReplacements][R
   replace_text.setSearchRegex("a\\w+e");
   replace_text.setReplacementValue("orange");
 
-  CHECK(replace_text.applyReplacements("") == "");

Review Comment:
   I'm not a fan of this, as the block would have to span multiple test cases, and apply to just one line in each test case.  I think individual `NOLINT`s are OK.  I would also be fine with disabling this check completely.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259925419


##########
extensions/azure/processors/PutAzureDataLakeStorage.h:
##########
@@ -35,18 +37,29 @@ class AzureDataLakeStorageTestsFixture;
 
 namespace org::apache::nifi::minifi::azure::processors {
 
+namespace azure {
+SMART_ENUM(FileExistsResolutionStrategy,
+  (FAIL_FLOW, "fail"),
+  (REPLACE_FILE, "replace"),
+  (IGNORE_REQUEST, "ignore")
+)
+}  // namespace azure

Review Comment:
   fixed in 6e0efa05f7da6bbc5f6a079d0d16b1d9100570f6



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1261346402


##########
extensions/standard-processors/tests/unit/ReplaceTextTests.cpp:
##########
@@ -93,7 +93,7 @@ TEST_CASE("Regex Replace works correctly in ReplaceText", "[applyReplacements][R
   replace_text.setSearchRegex("a\\w+e");
   replace_text.setReplacementValue("orange");
 
-  CHECK(replace_text.applyReplacements("") == "");

Review Comment:
   I'm okay with keeping it as it is



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265626062


##########
libminifi/include/core/ProcessContext.h:
##########
@@ -102,18 +103,44 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return value;
   }
 
+  template<typename T = std::string>
+  std::enable_if_t<std::is_default_constructible<T>::value, std::optional<T>>
+  getProperty(const PropertyReference& property) const {

Review Comment:
   done in 1f8b8afb8ec970de037384b8af85081fcd370bd1
   
   I have also un-templatized another pair of `getProperty()` overloads which were only ever used with `T = std::string`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1262322978


##########
libminifi/include/core/ProcessContext.h:
##########
@@ -90,6 +91,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return processor_node_;
   }
 
+  // TODO(fgerlits): remove if possible

Review Comment:
   Having TODOs is fine, but either they need fixing before merge, or the description needs to be a bit more detailed: why consider removing these, when, why it's not yet possible, etc.
   It may be someone else who eventually gets around to doing the TODO. I'm also fine with not having the TODO, if it doesn't add too much value, just wanted to explain my thought process a bit better than 2 days ago. Not sure if that's the case here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1258454601


##########
extensions/azure/processors/ListAzureBlobStorage.h:
##########
@@ -27,31 +27,42 @@
 #include <vector>
 
 #include "core/Property.h"
+#include "core/PropertyDefinition.h"
 #include "AzureBlobStorageProcessorBase.h"
 #include "core/logging/LoggerConfiguration.h"
 
 namespace org::apache::nifi::minifi::azure::processors {
 
+namespace azure {
+SMART_ENUM(EntityTracking,
+  (NONE, "none"),
+  (TIMESTAMPS, "timestamps")
+)
+}  // namespace azure

Review Comment:
   good catch, thanks!  fixed in a4b930d473c04d82cdc0e139191ce4b7b5fad892



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259676336


##########
libminifi/include/core/Core.h:
##########
@@ -95,6 +98,62 @@ static inline std::string getClassName() {
 #endif
 }
 
+constexpr std::string_view removeStructOrClassPrefix(std::string_view input) {
+  constexpr std::string_view STRUCT = "struct ";  // should be static constexpr, but that is only allowed inside a constexpr function with std >= c++23
+  constexpr std::string_view CLASS = "class ";

Review Comment:
   No, because if `STRUCT` and `CLASS` are only `constinit`, not `constexpr`, then the result of the function is not `constexpr`.
   
   This would work:
   ```c++
   namespace detail {
     inline constexpr std::string_view STRUCT = "struct ";
     inline constexpr std::string_view CLASS = "class ";
   }
   
   constexpr std::string_view removeStructOrClassPrefix(std::string_view input) {
     for (auto prefix : { detail::STRUCT, detail::CLASS }) {
       if (input.find(prefix) == 0) {
         return input.substr(prefix.size());
       }
     }
     return input;
   }
   ```
   Would that be better?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1247998877


##########
extensions/aws/processors/S3Processor.h:
##########
@@ -83,23 +90,67 @@ struct CommonProperties {
 
 class S3Processor : public core::Processor {
  public:
-  static const std::set<std::string> REGIONS;
-
-  EXTENSIONAPI static const core::Property Bucket;
-  EXTENSIONAPI static const core::Property AccessKey;
-  EXTENSIONAPI static const core::Property SecretKey;
-  EXTENSIONAPI static const core::Property CredentialsFile;
-  EXTENSIONAPI static const core::Property AWSCredentialsProviderService;
-  EXTENSIONAPI static const core::Property Region;
-  EXTENSIONAPI static const core::Property CommunicationsTimeout;
-  EXTENSIONAPI static const core::Property EndpointOverrideURL;
-  EXTENSIONAPI static const core::Property ProxyHost;
-  EXTENSIONAPI static const core::Property ProxyPort;
-  EXTENSIONAPI static const core::Property ProxyUsername;
-  EXTENSIONAPI static const core::Property ProxyPassword;
-  EXTENSIONAPI static const core::Property UseDefaultCredentials;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto Bucket = core::PropertyDefinitionBuilder<>::createProperty("Bucket")
+      .withDescription("The S3 bucket")
+      .isRequired(true)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto AccessKey = core::PropertyDefinitionBuilder<>::createProperty("Access Key")
+      .withDescription("AWS account access key")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto SecretKey = core::PropertyDefinitionBuilder<>::createProperty("Secret Key")
+      .withDescription("AWS account secret key")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto CredentialsFile = core::PropertyDefinitionBuilder<>::createProperty("Credentials File")
+      .withDescription("Path to a file containing AWS access key and secret key in properties file format. Properties used: accessKey and secretKey")
+      .build();
+  EXTENSIONAPI static constexpr auto AWSCredentialsProviderService = core::PropertyDefinitionBuilder<>::createProperty("AWS Credentials Provider service")
+      .withDescription("The name of the AWS Credentials Provider controller service that is used to obtain AWS credentials.")
+      .build();
+  EXTENSIONAPI static constexpr auto Region = core::PropertyDefinitionBuilder<region::REGIONS.size()>::createProperty("Region")
+      .isRequired(true)
+      .withDefaultValue(region::US_WEST_2)
+      .withAllowedValues(region::REGIONS)
+      .withDescription("AWS Region")
+      .build();
+  EXTENSIONAPI static constexpr auto CommunicationsTimeout = core::PropertyDefinitionBuilder<>::createProperty("Communications Timeout")
+      .isRequired(true)
+      .withPropertyType(core::StandardPropertyTypes::TIME_PERIOD_TYPE)
+      .withDefaultValue("30 sec")
+      .withDescription("Sets the timeout of the communication between the AWS server and the client")
+      .build();
+  EXTENSIONAPI static constexpr auto EndpointOverrideURL = core::PropertyDefinitionBuilder<>::createProperty("Endpoint Override URL")
+      .withDescription("Endpoint URL to use instead of the AWS default including scheme, host, "
+          "port, and path. The AWS libraries select an endpoint URL based on the AWS "
+          "region, but this property overrides the selected endpoint URL, allowing use "
+          "with other S3-compatible endpoints.")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ProxyHost = core::PropertyDefinitionBuilder<>::createProperty("Proxy Host")
+      .withDescription("Proxy host name or IP")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ProxyPort = core::PropertyDefinitionBuilder<>::createProperty("Proxy Port")
+      .withDescription("The port number of the proxy host")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ProxyUsername =   core::PropertyDefinitionBuilder<>::createProperty("Proxy Username")

Review Comment:
   fixed in bde74b220d3a6755b91e5104648b0a2b0f57fafc (also the other `=  ` issues below)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1268204386


##########
extensions/standard-processors/tests/unit/ManifestTests.cpp:
##########
@@ -74,15 +74,13 @@ TEST_CASE("Test Valid Regex", "[validRegex]") {
   const auto &prop_descriptors = proc_0.children[0];
   REQUIRE_FALSE(prop_descriptors.children.empty());
   const auto &prop_0 = prop_descriptors.children[0];
-  REQUIRE(prop_0.children.size() >= 3);
+  REQUIRE(prop_0.children.size() >= 6);

Review Comment:
   We were dereferencing `children[5]` and `children[6]`, but only checked that `children` has at least 3 elements.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265169957


##########
libminifi/src/core/Property.cpp:
##########
@@ -90,4 +94,41 @@ std::vector<std::pair<std::string, std::string>> Property::getExclusiveOfPropert
   return exclusive_of_properties_;
 }
 
+namespace {
+std::vector<PropertyValue> createPropertyValues(gsl::span<const std::string_view> values, const core::PropertyParser& property_parser) {
+  return ranges::views::transform(values, [&property_parser](const auto& value) {
+    return property_parser.parse(value);
+  }) | ranges::to<std::vector>;
+}
+
+inline std::vector<std::string> createStrings(gsl::span<const std::string_view> string_views) {
+  return ranges::views::transform(string_views, [](const auto& string_view) { return std::string{string_view}; })
+      | ranges::to<std::vector>;
+}
+
+inline std::vector<std::pair<std::string, std::string>> createStrings(gsl::span<const std::pair<std::string_view, std::string_view>> pairs_of_string_views) {
+  return ranges::views::transform(pairs_of_string_views, [](const auto& pair_of_string_views) { return std::pair<std::string, std::string>(pair_of_string_views); })
+      | ranges::to<std::vector>;
+}
+}  // namespace
+
+Property::Property(const PropertyReference& compile_time_property)
+    : name_(compile_time_property.name),
+      description_(compile_time_property.description),
+      is_required_(compile_time_property.is_required),
+      valid_regex_(compile_time_property.valid_regex),

Review Comment:
   I would remove it, and on the off chance that this breaks anyone's workflow, we can always revert it.



##########
libminifi/test/unit/ArrayUtilsTests.cpp:
##########
@@ -0,0 +1,56 @@
+/**
+ * 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 "../Catch.h"
+#include "utils/ArrayUtils.h"
+
+namespace utils = org::apache::nifi::minifi::utils;
+
+TEST_CASE("array_cat() works correctly and is constexpr") {
+  static constexpr auto empty = std::array<int, 0>{};
+  static constexpr auto one_to_three = std::array{1, 2, 3};
+  static constexpr auto four_to_five = utils::array_cat(empty, std::array{4, 5});
+  static constexpr auto all = utils::array_cat(one_to_three, empty, four_to_five, empty);
+  static_assert(all == std::array{1, 2, 3, 4, 5});
+}
+
+TEST_CASE("string_view_to_array() works correctly and is constexpr") {
+  static constexpr std::string_view hello = "Hello world!";
+  static constexpr auto hello_array = utils::string_view_to_array<hello.size()>(hello);
+  static_assert(std::string_view{hello_array.data(), hello_array.size()} == "Hello world!");
+
+  static constexpr auto hello_again = utils::array_to_string_view(hello_array);
+  static_assert(hello_again == "Hello world!");
+}
+
+TEST_CASE("getKeys() works correctly and is constexpr") {
+  static constexpr std::array<std::pair<std::string_view, int>, 3> mapping{{ {"one", 1}, {"two", 2}, {"three", 3} }};
+  static constexpr auto keys = utils::getKeys(mapping);
+  static_assert(keys == std::array<std::string_view, 3>{"one", "two", "three"});
+}
+
+TEST_CASE("at() works correctly and is constexpr") {
+  static constexpr std::array<std::pair<int, std::string_view>, 3> mapping{{ {1, "one"}, {2, "two"}, {3, "three"} }};
+  static constexpr auto two = utils::at(mapping, 2);
+  static_assert(two == "two");
+
+  int one = 1;
+  CHECK(utils::at(mapping, one) == "one");  // non-constexpr argument is OK, but the result is not constexpr
+
+  // static constexpr auto four = utils::at(mapping, 4);  // does not compile

Review Comment:
   Could we check that this doesn't compile using SFINAE or concepts? I used something similar in GeneralUtilsTest to test constexpr `intdiv_ceil`. That one was in 2020, so it hacks around `std::enable_if`, but I think `std::void_t` would also work now, or a single-use concept definition.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1237222027


##########
extensions/gcp/processors/GCSProcessor.h:
##########
@@ -34,19 +38,33 @@ class GCSProcessor : public core::Processor {
  public:
   GCSProcessor(std::string name, const minifi::utils::Identifier& uuid, std::shared_ptr<core::logging::Logger> logger)
       : core::Processor(std::move(name), uuid),
-        logger_(std::move(logger)) {
+     #include "PropertyDefinitionBuilder.h"

Review Comment:
   Yes, thank you -- removed in bee1b6d8bb985c7cc817b9249470506d094cc0cb.
   
   I was wondering how this compiled, but of course the `#pragma once` caused this to be an empty include.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265116915


##########
libminifi/test/unit/CoreTests.cpp:
##########
@@ -0,0 +1,49 @@
+/**
+ * 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 "../Catch.h"
+#include "core/Core.h"
+
+namespace org::apache::nifi::minifi::test {
+
+struct DummyStruct {};
+class DummyClass {};
+template<typename T> struct DummyStructTemplate {};
+template<typename T> class DummyClassTemplate {};
+
+TEST_CASE("getClassName() works correctly") {
+  CHECK(core::getClassName<DummyStruct>() == "org::apache::nifi::minifi::test::DummyStruct");
+  CHECK(core::getClassName<DummyClass>() == "org::apache::nifi::minifi::test::DummyClass");
+  CHECK(core::getClassName<DummyStructTemplate<int>>() == "org::apache::nifi::minifi::test::DummyStructTemplate<int>");
+  CHECK(core::getClassName<DummyClassTemplate<int>>() == "org::apache::nifi::minifi::test::DummyClassTemplate<int>");
+}
+
+TEST_CASE("className() works correctly and is constexpr") {
+  static constexpr auto struct_name = core::className<DummyStruct>();
+  static_assert(struct_name == "org::apache::nifi::minifi::test::DummyStruct");

Review Comment:
   I would prefer to that in a separate pull request, as it would affect many files not already changed in this PR.  I have created a Jira for this: https://issues.apache.org/jira/browse/MINIFICPP-2162.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1262335130


##########
libminifi/include/core/ProcessContext.h:
##########
@@ -90,6 +91,7 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return processor_node_;
   }
 
+  // TODO(fgerlits): remove if possible

Review Comment:
   I added these TODOs quite early on, when I still thought I could get rid of the `Property` and `Relationship` classes completely.  We might still do that eventually, but I don't think it will be soon, and it will require changes in many places, not just these three.  So removing the TODOs is probably the best option.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] szaszm commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "szaszm (via GitHub)" <gi...@apache.org>.
szaszm commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1263795580


##########
libminifi/include/core/ProcessContext.h:
##########
@@ -102,18 +103,44 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return value;
   }
 
+  template<typename T = std::string>
+  std::enable_if_t<std::is_default_constructible<T>::value, std::optional<T>>
+  getProperty(const PropertyReference& property) const {
+    T value;
+    try {
+      if (!getProperty(property.name, value)) return std::nullopt;
+    } catch (const utils::internal::ValueException&) {
+      return std::nullopt;
+    }
+    return value;
+  }
+
+  template<typename T>
+  requires(!std::is_convertible_v<T&, const FlowFile&> && !std::is_convertible_v<T&, const std::shared_ptr<FlowFile>&>)
+  bool getProperty(std::string_view name, T &value) const {

Review Comment:
   ```suggestion
     template<typename T>
     concept NotAFlowFile = !std::convertible_to<T&, const FlowFile&> && !std::convertible_to<T&, const std::shared_ptr<FlowFile>&>;
   
     bool getProperty(std::string_view name, NotAFlowFile auto& value) const {
   ```



##########
libminifi/include/core/ProcessContext.h:
##########
@@ -102,18 +103,44 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return value;
   }
 
+  template<typename T = std::string>
+  std::enable_if_t<std::is_default_constructible<T>::value, std::optional<T>>
+  getProperty(const PropertyReference& property) const {

Review Comment:
   Let's convert this to concept constraints, since we're touching it, and the new one below is also written like this.
   ```suggestion
     template<std::default_initializable T = std::string>
     std::optional<T> getProperty(const PropertyReference& property) const {
   ```



##########
libminifi/include/core/ProcessContext.h:
##########
@@ -102,18 +103,44 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return value;
   }
 
+  template<typename T = std::string>
+  std::enable_if_t<std::is_default_constructible<T>::value, std::optional<T>>
+  getProperty(const PropertyReference& property) const {
+    T value;
+    try {
+      if (!getProperty(property.name, value)) return std::nullopt;
+    } catch (const utils::internal::ValueException&) {
+      return std::nullopt;
+    }
+    return value;
+  }
+
+  template<typename T>
+  requires(!std::is_convertible_v<T&, const FlowFile&> && !std::is_convertible_v<T&, const std::shared_ptr<FlowFile>&>)
+  bool getProperty(std::string_view name, T &value) const {
+    return getPropertyImp<typename std::common_type<T>::type>(std::string{name}, value);
+  }
+
   template<typename T>
-  std::enable_if_t<!std::is_convertible_v<T&, const FlowFile&> && !std::is_convertible_v<T&, const std::shared_ptr<FlowFile>&>,
-      bool> getProperty(const std::string &name, T &value) const {
-    return getPropertyImp<typename std::common_type<T>::type>(name, value);
+  requires(!std::is_convertible_v<T&, const FlowFile&> && !std::is_convertible_v<T&, const std::shared_ptr<FlowFile>&>)
+  bool getProperty(const PropertyReference& property, T &value) const {

Review Comment:
   ```suggestion
     bool getProperty(const PropertyReference& property, NotAFlowFile auto& value) const {
   ```



##########
libminifi/src/core/Property.cpp:
##########
@@ -90,4 +94,41 @@ std::vector<std::pair<std::string, std::string>> Property::getExclusiveOfPropert
   return exclusive_of_properties_;
 }
 
+namespace {
+std::vector<PropertyValue> createPropertyValues(gsl::span<const std::string_view> values, const core::PropertyParser& property_parser) {
+  return ranges::views::transform(values, [&property_parser](const auto& value) {
+    return property_parser.parse(value);
+  }) | ranges::to<std::vector>;
+}
+
+inline std::vector<std::string> createStrings(gsl::span<const std::string_view> string_views) {
+  return ranges::views::transform(string_views, [](const auto& string_view) { return std::string{string_view}; })
+      | ranges::to<std::vector>;
+}
+
+inline std::vector<std::pair<std::string, std::string>> createStrings(gsl::span<const std::pair<std::string_view, std::string_view>> pairs_of_string_views) {
+  return ranges::views::transform(pairs_of_string_views, [](const auto& pair_of_string_views) { return std::pair<std::string, std::string>(pair_of_string_views); })
+      | ranges::to<std::vector>;
+}
+}  // namespace
+
+Property::Property(const PropertyReference& compile_time_property)
+    : name_(compile_time_property.name),
+      description_(compile_time_property.description),
+      is_required_(compile_time_property.is_required),
+      valid_regex_(compile_time_property.valid_regex),

Review Comment:
   Is this `valid_regex_` ever used? The builder doesn't seem to set it. If it's always empty, we might as well drop it.



##########
libminifi/include/core/PropertyType.h:
##########
@@ -90,12 +47,19 @@ class PropertyValidator {
   [[nodiscard]] virtual ValidationResult validate(const std::string &subject, const std::shared_ptr<minifi::state::response::Value> &input) const = 0;
 
   [[nodiscard]] virtual ValidationResult validate(const std::string &subject, const std::string &input) const = 0;
+};
+
+class PropertyType : public PropertyParser, public PropertyValidator {

Review Comment:
   I think composition would be better design here, than multiple inheritance.
   
   Most property types have names changed to end with "_TYPE", except `UnsignedIntPropertyType`. This particular case may be an oversight.
   
   But the pattern ultimately overrides the `getName` method of `PropertyValidator`, so it should retuirn a validator name (like the old version), but a method called `getName` in `PropertyType` has no business returning a validator name. I'm also not sure how the type name gets converted back to a validator name when used with a validator-aware C2 server.



##########
libminifi/test/unit/CoreTests.cpp:
##########
@@ -0,0 +1,49 @@
+/**
+ * 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 "../Catch.h"
+#include "core/Core.h"
+
+namespace org::apache::nifi::minifi::test {
+
+struct DummyStruct {};
+class DummyClass {};
+template<typename T> struct DummyStructTemplate {};
+template<typename T> class DummyClassTemplate {};
+
+TEST_CASE("getClassName() works correctly") {
+  CHECK(core::getClassName<DummyStruct>() == "org::apache::nifi::minifi::test::DummyStruct");
+  CHECK(core::getClassName<DummyClass>() == "org::apache::nifi::minifi::test::DummyClass");
+  CHECK(core::getClassName<DummyStructTemplate<int>>() == "org::apache::nifi::minifi::test::DummyStructTemplate<int>");
+  CHECK(core::getClassName<DummyClassTemplate<int>>() == "org::apache::nifi::minifi::test::DummyClassTemplate<int>");
+}
+
+TEST_CASE("className() works correctly and is constexpr") {
+  static constexpr auto struct_name = core::className<DummyStruct>();
+  static_assert(struct_name == "org::apache::nifi::minifi::test::DummyStruct");

Review Comment:
   Can we remove `getClassName` and replace its usages with `className`? It seems like it's doing the same thing, except that `className` can also work at compile-time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265642911


##########
libminifi/test/unit/ArrayUtilsTests.cpp:
##########
@@ -0,0 +1,56 @@
+/**
+ * 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 "../Catch.h"
+#include "utils/ArrayUtils.h"
+
+namespace utils = org::apache::nifi::minifi::utils;
+
+TEST_CASE("array_cat() works correctly and is constexpr") {
+  static constexpr auto empty = std::array<int, 0>{};
+  static constexpr auto one_to_three = std::array{1, 2, 3};
+  static constexpr auto four_to_five = utils::array_cat(empty, std::array{4, 5});
+  static constexpr auto all = utils::array_cat(one_to_three, empty, four_to_five, empty);
+  static_assert(all == std::array{1, 2, 3, 4, 5});
+}
+
+TEST_CASE("string_view_to_array() works correctly and is constexpr") {
+  static constexpr std::string_view hello = "Hello world!";
+  static constexpr auto hello_array = utils::string_view_to_array<hello.size()>(hello);
+  static_assert(std::string_view{hello_array.data(), hello_array.size()} == "Hello world!");
+
+  static constexpr auto hello_again = utils::array_to_string_view(hello_array);
+  static_assert(hello_again == "Hello world!");
+}
+
+TEST_CASE("getKeys() works correctly and is constexpr") {
+  static constexpr std::array<std::pair<std::string_view, int>, 3> mapping{{ {"one", 1}, {"two", 2}, {"three", 3} }};
+  static constexpr auto keys = utils::getKeys(mapping);
+  static_assert(keys == std::array<std::string_view, 3>{"one", "two", "three"});
+}
+
+TEST_CASE("at() works correctly and is constexpr") {
+  static constexpr std::array<std::pair<int, std::string_view>, 3> mapping{{ {1, "one"}, {2, "two"}, {3, "three"} }};
+  static constexpr auto two = utils::at(mapping, 2);
+  static_assert(two == "two");
+
+  int one = 1;
+  CHECK(utils::at(mapping, one) == "one");  // non-constexpr argument is OK, but the result is not constexpr
+
+  // static constexpr auto four = utils::at(mapping, 4);  // does not compile

Review Comment:
   done in 529796c366c26e1909b541a071875c6561aaaee7



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1265627207


##########
libminifi/include/core/ProcessContext.h:
##########
@@ -102,18 +103,44 @@ class ProcessContext : public controller::ControllerServiceLookup, public core::
     return value;
   }
 
+  template<typename T = std::string>
+  std::enable_if_t<std::is_default_constructible<T>::value, std::optional<T>>
+  getProperty(const PropertyReference& property) const {
+    T value;
+    try {
+      if (!getProperty(property.name, value)) return std::nullopt;
+    } catch (const utils::internal::ValueException&) {
+      return std::nullopt;
+    }
+    return value;
+  }
+
+  template<typename T>
+  requires(!std::is_convertible_v<T&, const FlowFile&> && !std::is_convertible_v<T&, const std::shared_ptr<FlowFile>&>)
+  bool getProperty(std::string_view name, T &value) const {
+    return getPropertyImp<typename std::common_type<T>::type>(std::string{name}, value);
+  }
+
   template<typename T>
-  std::enable_if_t<!std::is_convertible_v<T&, const FlowFile&> && !std::is_convertible_v<T&, const std::shared_ptr<FlowFile>&>,
-      bool> getProperty(const std::string &name, T &value) const {
-    return getPropertyImp<typename std::common_type<T>::type>(name, value);
+  requires(!std::is_convertible_v<T&, const FlowFile&> && !std::is_convertible_v<T&, const std::shared_ptr<FlowFile>&>)
+  bool getProperty(const PropertyReference& property, T &value) const {

Review Comment:
   done in 1f8b8afb8ec970de037384b8af85081fcd370bd1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] lordgamez closed pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "lordgamez (via GitHub)" <gi...@apache.org>.
lordgamez closed pull request #1589: MINIFICPP-1825 Create Properties at compile time
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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


[GitHub] [nifi-minifi-cpp] fgerlits commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

Posted by "fgerlits (via GitHub)" <gi...@apache.org>.
fgerlits commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1259929670


##########
libminifi/test/unit/ArrayUtilsTests.cpp:
##########


Review Comment:
   fixed in 0352bb48727b839c9a4288193ea72348a70dc57e



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

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