You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/07/15 10:53:51 UTC

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request #1132: MINIFICPP-1603 Support Funnel element in flow definition

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


   Added support for `Funnels` in flow configuration to merge multiple connections and forward the flowfiles to the connected processors. Funnels are implemented as an event driven processor transferring every incoming flowfile to the connected processors. A funnel definition only requires and id and looks like the following in a config yaml file:
   
   ```
   Funnels:
   - id: 01a2f910-7050-41c1-8528-942764e7591d
   Connections:
   - id: 97c6bdfb-3909-499f-9ae5-011cbe8cadaf
     name: 01a2f910-7050-41c1-8528-942764e7591d//LogAttribute
     source id: 01a2f910-7050-41c1-8528-942764e7591d
     source relationship names: []
     destination id: 695658ba-5b6e-4c7d-9c95-5a980b622c1f
   ```
   
   It does not require to have a `source relationship names` attribute or if it is present it can be empty.
   
   Support for funnels also added in the docker integration tests, with added support of multiple start nodes in a flow.
   
   https://issues.apache.org/jira/browse/MINIFICPP-1603
   
   ----------------------------------------------------------------------------------------------
   
   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check 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] lordgamez commented on a change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,45 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;
+  }
+  auto processor = parent_->findProcessorById(srcUUID);
+  if (!processor) {
+    return;
+  }
+  if (std::dynamic_pointer_cast<minifi::core::Funnel>(processor)) {

Review comment:
       Changed in a76b8c6b77f48f19dbc805a0a26a9f55e015d40e




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: docker/test/integration/steps/steps.py
##########
@@ -248,6 +262,27 @@ def step_impl(context, file_name, content, path):
     context.test.add_test_data(path, content, file_name)
 
 
+@given("a Funnel with the name \"{funnel_name}\" is set up in the flow")
+def step_impl(context, funnel_name):
+    funnel = Funnel()
+    funnel.set_name(funnel_name)
+    context.test.add_node(funnel)
+
+
+@given("in the flow the Funnel with the name \"{source_name}\" is connected to the {destination_name}")

Review comment:
       "in the flow" is implicit from the context, I don't think we need it explicitly written
   ```suggestion
   @given("a Funnel with the name \"{funnel_name}\" is set up")
   def step_impl(context, funnel_name):
       funnel = Funnel()
       funnel.set_name(funnel_name)
       context.test.add_node(funnel)
   
   
   @given("the Funnel with the name \"{source_name}\" is connected to the {destination_name}")
   ```

##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -747,3 +747,93 @@ TEST_CASE("Test Regex Property 2", "[YamlConfigurationRegexProperty2]") {
 }
 
 #endif  // YAML_CONFIGURATION_USE_REGEX
+
+TEST_CASE("Test YAML Config With Funnel", "[YamlConfiguration]") {
+  TestController test_controller;
+
+  std::shared_ptr<core::Repository> testProvRepo = core::createRepository("provenancerepository", true);
+  std::shared_ptr<core::Repository> testFlowFileRepo = core::createRepository("flowfilerepository", true);
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
+  std::shared_ptr<minifi::io::StreamFactory> streamFactory = minifi::io::StreamFactory::getInstance(configuration);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
+
+  static const std::string CONFIG_YAML_WITH_FUNNEL = ""
+      "MiNiFi Config Version: 3\n"
+      "Flow Controller:\n"
+      "  name: root\n"
+      "  comment: ''\n"
+      "Processors:\n"
+      "- id: 0eac51eb-d76c-4ba6-9f0c-351795b2d243\n"
+      "  name: GenerateFlowFile1\n"

Review comment:
       Please use a raw string literal for this. (6) at https://en.cppreference.com/w/cpp/language/string_literal

##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,37 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {

Review comment:
       Did you consider using a range-based for loop?




-- 
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 #1132: MINIFICPP-1603 Support Funnels element in flow definition

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


   


-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -747,3 +747,93 @@ TEST_CASE("Test Regex Property 2", "[YamlConfigurationRegexProperty2]") {
 }
 
 #endif  // YAML_CONFIGURATION_USE_REGEX
+
+TEST_CASE("Test YAML Config With Funnel", "[YamlConfiguration]") {
+  TestController test_controller;
+
+  std::shared_ptr<core::Repository> testProvRepo = core::createRepository("provenancerepository", true);
+  std::shared_ptr<core::Repository> testFlowFileRepo = core::createRepository("flowfilerepository", true);
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
+  std::shared_ptr<minifi::io::StreamFactory> streamFactory = minifi::io::StreamFactory::getInstance(configuration);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
+
+  static const std::string CONFIG_YAML_WITH_FUNNEL = ""
+      "MiNiFi Config Version: 3\n"
+      "Flow Controller:\n"
+      "  name: root\n"
+      "  comment: ''\n"
+      "Processors:\n"
+      "- id: 0eac51eb-d76c-4ba6-9f0c-351795b2d243\n"
+      "  name: GenerateFlowFile1\n"

Review comment:
       Updated in eb396ae43b5dae3ec5c23f19725f5ac6d380bf78

##########
File path: docker/test/integration/steps/steps.py
##########
@@ -248,6 +262,27 @@ def step_impl(context, file_name, content, path):
     context.test.add_test_data(path, content, file_name)
 
 
+@given("a Funnel with the name \"{funnel_name}\" is set up in the flow")
+def step_impl(context, funnel_name):
+    funnel = Funnel()
+    funnel.set_name(funnel_name)
+    context.test.add_node(funnel)
+
+
+@given("in the flow the Funnel with the name \"{source_name}\" is connected to the {destination_name}")

Review comment:
       Updated in eb396ae43b5dae3ec5c23f19725f5ac6d380bf78

##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,37 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {

Review comment:
       Updated in eb396ae43b5dae3ec5c23f19725f5ac6d380bf78




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: docker/test/integration/minifi/flow_serialization/Minifi_flow_yaml_serializer.py
##########
@@ -92,21 +108,22 @@ def serialize(self, connectable, root=None, visited=None):
                     res['Connections'].append({
                         'name': str(uuid.uuid4()),
                         'source id': str(connectable.uuid),
-                        'source relationship name': conn_name,
                         'destination id': str(proc.uuid),
                         'drop empty': ("true" if proc.drop_empty_flowfiles else "false")
                     })
+                    if (str(connectable.uuid) not in [x['id'] for x in res['Funnels']]):
+                        res['Connections'][-1]['source relationship name'] = conn_name
                     if proc not in visited:
-                        self.serialize(proc, res, visited)
+                        self.serialize_node(proc, res, visited)
             else:
                 res['Connections'].append({
                     'name': str(uuid.uuid4()),
                     'source id': str(connectable.uuid),
-                    'source relationship name': conn_name,
                     'destination id': str(conn_procs.uuid)
                 })
+                if (str(connectable.uuid) not in [x['id'] for x in res['Funnels']]):

Review comment:
       Almost okay but needed to be fixed as it was checking for the opposite, fixed in bb83a0d48c8d39d3ff03082e29fabd0ee370ac5b




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,48 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;

Review comment:
       I think there is no need, because `getSourceUUIDFromYaml` already logs all the error cases.

##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,48 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;
+  }
+  auto processor = parent_->findProcessorById(srcUUID);
+  if (!processor) {
+    logger_->log_error("Could not find processor with id %s", srcUUID.to_string());
+    return;
+  }
+
+  auto& processor_ref = *processor.get();
+  if (typeid(minifi::core::Funnel) == typeid(processor_ref)) {
+    addNewRelationshipToConnection(minifi::core::Funnel::Success.getName(), connection);
+  }

Review comment:
       Added in e64b1c0d1fe231936dcde263939fbd8dcfff4083




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,38 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (const auto& element : node) {
+  // for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {
+    YAML::Node funnel_node = element.as<YAML::Node>();
+
+    std::string id = getOrGenerateId(funnel_node);
+
+    // Default name to be same as ID
+    std::string name = funnel_node["name"].as<std::string>(id);
+
+    const utils::optional<utils::Identifier> uuid = utils::Identifier::parse(id);
+    if (!uuid) {
+      logger_->log_debug("Incorrect connection UUID format.");
+      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
+    }

Review comment:
       Thanks I missed that one, changed in ca0962054629946225b27cb0136bb017c6484c32




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -747,3 +747,93 @@ TEST_CASE("Test Regex Property 2", "[YamlConfigurationRegexProperty2]") {
 }
 
 #endif  // YAML_CONFIGURATION_USE_REGEX
+
+TEST_CASE("Test YAML Config With Funnel", "[YamlConfiguration]") {
+  TestController test_controller;
+
+  std::shared_ptr<core::Repository> testProvRepo = core::createRepository("provenancerepository", true);
+  std::shared_ptr<core::Repository> testFlowFileRepo = core::createRepository("flowfilerepository", true);
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
+  std::shared_ptr<minifi::io::StreamFactory> streamFactory = minifi::io::StreamFactory::getInstance(configuration);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
+
+  static const std::string CONFIG_YAML_WITH_FUNNEL = ""
+      "MiNiFi Config Version: 3\n"
+      "Flow Controller:\n"
+      "  name: root\n"
+      "  comment: ''\n"
+      "Processors:\n"
+      "- id: 0eac51eb-d76c-4ba6-9f0c-351795b2d243\n"
+      "  name: GenerateFlowFile1\n"

Review comment:
       Updated in eb396ae43b5dae3ec5c23f19725f5ac6d380bf78

##########
File path: docker/test/integration/steps/steps.py
##########
@@ -248,6 +262,27 @@ def step_impl(context, file_name, content, path):
     context.test.add_test_data(path, content, file_name)
 
 
+@given("a Funnel with the name \"{funnel_name}\" is set up in the flow")
+def step_impl(context, funnel_name):
+    funnel = Funnel()
+    funnel.set_name(funnel_name)
+    context.test.add_node(funnel)
+
+
+@given("in the flow the Funnel with the name \"{source_name}\" is connected to the {destination_name}")

Review comment:
       Updated in eb396ae43b5dae3ec5c23f19725f5ac6d380bf78

##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,37 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {

Review comment:
       Updated in eb396ae43b5dae3ec5c23f19725f5ac6d380bf78




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: docker/test/integration/steps/steps.py
##########
@@ -248,6 +262,27 @@ def step_impl(context, file_name, content, path):
     context.test.add_test_data(path, content, file_name)
 
 
+@given("a Funnel with the name \"{funnel_name}\" is set up in the flow")
+def step_impl(context, funnel_name):
+    funnel = Funnel()
+    funnel.set_name(funnel_name)
+    context.test.add_node(funnel)
+
+
+@given("in the flow the Funnel with the name \"{source_name}\" is connected to the {destination_name}")

Review comment:
       "in the flow" is implicit from the context, I don't think we need it explicitly written
   ```suggestion
   @given("a Funnel with the name \"{funnel_name}\" is set up")
   def step_impl(context, funnel_name):
       funnel = Funnel()
       funnel.set_name(funnel_name)
       context.test.add_node(funnel)
   
   
   @given("the Funnel with the name \"{source_name}\" is connected to the {destination_name}")
   ```

##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -747,3 +747,93 @@ TEST_CASE("Test Regex Property 2", "[YamlConfigurationRegexProperty2]") {
 }
 
 #endif  // YAML_CONFIGURATION_USE_REGEX
+
+TEST_CASE("Test YAML Config With Funnel", "[YamlConfiguration]") {
+  TestController test_controller;
+
+  std::shared_ptr<core::Repository> testProvRepo = core::createRepository("provenancerepository", true);
+  std::shared_ptr<core::Repository> testFlowFileRepo = core::createRepository("flowfilerepository", true);
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
+  std::shared_ptr<minifi::io::StreamFactory> streamFactory = minifi::io::StreamFactory::getInstance(configuration);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
+
+  static const std::string CONFIG_YAML_WITH_FUNNEL = ""
+      "MiNiFi Config Version: 3\n"
+      "Flow Controller:\n"
+      "  name: root\n"
+      "  comment: ''\n"
+      "Processors:\n"
+      "- id: 0eac51eb-d76c-4ba6-9f0c-351795b2d243\n"
+      "  name: GenerateFlowFile1\n"

Review comment:
       Please use a raw string literal for this. (6) at https://en.cppreference.com/w/cpp/language/string_literal

##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,37 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {

Review comment:
       Did you consider using a range-based for loop?




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,38 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (const auto& element : node) {
+  // for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {
+    YAML::Node funnel_node = element.as<YAML::Node>();
+
+    std::string id = getOrGenerateId(funnel_node);
+
+    // Default name to be same as ID
+    std::string name = funnel_node["name"].as<std::string>(id);
+
+    const utils::optional<utils::Identifier> uuid = utils::Identifier::parse(id);
+    if (!uuid) {
+      logger_->log_debug("Incorrect connection UUID format.");
+      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
+    }

Review comment:
       You changed a different occurence of the same pattern. Line 610 vs 826.




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: docker/test/integration/steps/steps.py
##########
@@ -248,6 +262,27 @@ def step_impl(context, file_name, content, path):
     context.test.add_test_data(path, content, file_name)
 
 
+@given("a Funnel with the name \"{funnel_name}\" is set up")
+def step_impl(context, funnel_name):
+    funnel = Funnel()
+    funnel.set_name(funnel_name)

Review comment:
       True, updated in f6a40ef8ad7c2e10b048100e8279dfb00a95c14d




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,38 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (const auto& element : node) {
+  // for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {

Review comment:
       My bad, I left it there by accident, removed it in a76b8c6b77f48f19dbc805a0a26a9f55e015d40e




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,38 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (const auto& element : node) {
+  // for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {
+    YAML::Node funnel_node = element.as<YAML::Node>();
+
+    std::string id = getOrGenerateId(funnel_node);
+
+    // Default name to be same as ID
+    std::string name = funnel_node["name"].as<std::string>(id);
+
+    const utils::optional<utils::Identifier> uuid = utils::Identifier::parse(id);
+    if (!uuid) {
+      logger_->log_debug("Incorrect connection UUID format.");
+      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
+    }

Review comment:
       You changed a different occurrence of the same pattern. Line 610 vs 826.




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,45 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;
+  }
+  auto processor = parent_->findProcessorById(srcUUID);
+  if (!processor) {
+    return;

Review comment:
       Added additional log in a76b8c6b77f48f19dbc805a0a26a9f55e015d40e, `getSourceUUIDFromYaml` should already log errors a few lines 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] szaszm commented on a change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: docker/test/integration/steps/steps.py
##########
@@ -248,6 +262,27 @@ def step_impl(context, file_name, content, path):
     context.test.add_test_data(path, content, file_name)
 
 
+@given("a Funnel with the name \"{funnel_name}\" is set up in the flow")
+def step_impl(context, funnel_name):
+    funnel = Funnel()
+    funnel.set_name(funnel_name)
+    context.test.add_node(funnel)
+
+
+@given("in the flow the Funnel with the name \"{source_name}\" is connected to the {destination_name}")

Review comment:
       "in the flow" is implicit from the context, I don't think we need it explicitly written
   ```suggestion
   @given("a Funnel with the name \"{funnel_name}\" is set up")
   def step_impl(context, funnel_name):
       funnel = Funnel()
       funnel.set_name(funnel_name)
       context.test.add_node(funnel)
   
   
   @given("the Funnel with the name \"{source_name}\" is connected to the {destination_name}")
   ```

##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -747,3 +747,93 @@ TEST_CASE("Test Regex Property 2", "[YamlConfigurationRegexProperty2]") {
 }
 
 #endif  // YAML_CONFIGURATION_USE_REGEX
+
+TEST_CASE("Test YAML Config With Funnel", "[YamlConfiguration]") {
+  TestController test_controller;
+
+  std::shared_ptr<core::Repository> testProvRepo = core::createRepository("provenancerepository", true);
+  std::shared_ptr<core::Repository> testFlowFileRepo = core::createRepository("flowfilerepository", true);
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
+  std::shared_ptr<minifi::io::StreamFactory> streamFactory = minifi::io::StreamFactory::getInstance(configuration);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
+
+  static const std::string CONFIG_YAML_WITH_FUNNEL = ""
+      "MiNiFi Config Version: 3\n"
+      "Flow Controller:\n"
+      "  name: root\n"
+      "  comment: ''\n"
+      "Processors:\n"
+      "- id: 0eac51eb-d76c-4ba6-9f0c-351795b2d243\n"
+      "  name: GenerateFlowFile1\n"

Review comment:
       Please use a raw string literal for this. (6) at https://en.cppreference.com/w/cpp/language/string_literal

##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,37 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {

Review comment:
       Did you consider using a range-based for loop?




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,38 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (const auto& element : node) {
+  // for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {

Review comment:
       Did you mean to leave the old version here?

##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,45 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;
+  }
+  auto processor = parent_->findProcessorById(srcUUID);
+  if (!processor) {
+    return;
+  }
+  if (std::dynamic_pointer_cast<minifi::core::Funnel>(processor)) {

Review comment:
       If `Funnel` can be `final`, then this could be done more efficiently with `typeid` comparison, because it doesn't have to travel the inheritance graph.

##########
File path: libminifi/include/core/Funnel.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 <string>
+#include <memory>
+
+#include "Processor.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class Funnel : public Processor {

Review comment:
       Could this be `final`? Is this class designed for further subclassing?
   `final` gives optimizers the opportunity to devirtualize calls, because when the static type is `final`, then the dynamic type is proven to be the same.

##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,38 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (const auto& element : node) {
+  // for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {
+    YAML::Node funnel_node = element.as<YAML::Node>();
+
+    std::string id = getOrGenerateId(funnel_node);
+
+    // Default name to be same as ID
+    std::string name = funnel_node["name"].as<std::string>(id);
+
+    const utils::optional<utils::Identifier> uuid = utils::Identifier::parse(id);
+    if (!uuid) {
+      logger_->log_debug("Incorrect connection UUID format.");
+      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
+    }

Review comment:
       You might be interested in `utils::orElse` from OptionalUtils.h (inspired by TartanLlama/optional and the associated proposal) if you prefer longer expressions over more statements. 
   ```suggestion
       const utils::optional<utils::Identifier> uuid = utils::Identifier::parse(id) | utils::orElse([this] {
         logger_->log_debug("Incorrect connection UUID format.");
         throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
       });
   ```
   

##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,45 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;
+  }
+  auto processor = parent_->findProcessorById(srcUUID);
+  if (!processor) {
+    return;

Review comment:
       Maybe it would be worth logging if something goes wrong here or a few lines 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] lordgamez commented on a change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,48 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;
+  }
+  auto processor = parent_->findProcessorById(srcUUID);
+  if (!processor) {
+    logger_->log_error("Could not find processor with id %s", srcUUID.to_string());
+    return;
+  }
+
+  auto& processor_ref = *processor.get();
+  if (typeid(minifi::core::Funnel) == typeid(processor_ref)) {
+    addNewRelationshipToConnection(minifi::core::Funnel::Success.getName(), connection);
+  }

Review comment:
       Removed in bb83a0d48c8d39d3ff03082e29fabd0ee370ac5b as the yml parser runs the source parsing for all processors and those that do not have need any source (and not Funnels) like `GetFile` will get this error message printed unnecessarily.




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,45 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;
+  }
+  auto processor = parent_->findProcessorById(srcUUID);
+  if (!processor) {
+    return;
+  }
+  if (std::dynamic_pointer_cast<minifi::core::Funnel>(processor)) {

Review comment:
       If `Funnel` can be `final`, then this could be done more efficiently with `typeid` comparison, because it doesn't have to traverse the inheritance graph.




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: docker/test/integration/minifi/flow_serialization/Minifi_flow_yaml_serializer.py
##########
@@ -92,21 +108,22 @@ def serialize(self, connectable, root=None, visited=None):
                     res['Connections'].append({
                         'name': str(uuid.uuid4()),
                         'source id': str(connectable.uuid),
-                        'source relationship name': conn_name,
                         'destination id': str(proc.uuid),
                         'drop empty': ("true" if proc.drop_empty_flowfiles else "false")
                     })
+                    if (str(connectable.uuid) not in [x['id'] for x in res['Funnels']]):
+                        res['Connections'][-1]['source relationship name'] = conn_name
                     if proc not in visited:
-                        self.serialize(proc, res, visited)
+                        self.serialize_node(proc, res, visited)
             else:
                 res['Connections'].append({
                     'name': str(uuid.uuid4()),
                     'source id': str(connectable.uuid),
-                    'source relationship name': conn_name,
                     'destination id': str(conn_procs.uuid)
                 })
+                if (str(connectable.uuid) not in [x['id'] for x in res['Funnels']]):

Review comment:
       Yeah, I like this better :) Updated in f6a40ef8ad7c2e10b048100e8279dfb00a95c14d




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,38 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (const auto& element : node) {
+  // for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {
+    YAML::Node funnel_node = element.as<YAML::Node>();
+
+    std::string id = getOrGenerateId(funnel_node);
+
+    // Default name to be same as ID
+    std::string name = funnel_node["name"].as<std::string>(id);
+
+    const utils::optional<utils::Identifier> uuid = utils::Identifier::parse(id);
+    if (!uuid) {
+      logger_->log_debug("Incorrect connection UUID format.");
+      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");
+    }

Review comment:
       Looks good, changed it in a76b8c6b77f48f19dbc805a0a26a9f55e015d40e




-- 
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 #1132: MINIFICPP-1603 Support Funnels element in flow definition

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


   


-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -747,3 +747,93 @@ TEST_CASE("Test Regex Property 2", "[YamlConfigurationRegexProperty2]") {
 }
 
 #endif  // YAML_CONFIGURATION_USE_REGEX
+
+TEST_CASE("Test YAML Config With Funnel", "[YamlConfiguration]") {
+  TestController test_controller;
+
+  std::shared_ptr<core::Repository> testProvRepo = core::createRepository("provenancerepository", true);
+  std::shared_ptr<core::Repository> testFlowFileRepo = core::createRepository("flowfilerepository", true);
+  std::shared_ptr<minifi::Configure> configuration = std::make_shared<minifi::Configure>();
+  std::shared_ptr<minifi::io::StreamFactory> streamFactory = minifi::io::StreamFactory::getInstance(configuration);
+  std::shared_ptr<core::ContentRepository> content_repo = std::make_shared<core::repository::VolatileContentRepository>();
+  core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
+
+  static const std::string CONFIG_YAML_WITH_FUNNEL = ""
+      "MiNiFi Config Version: 3\n"
+      "Flow Controller:\n"
+      "  name: root\n"
+      "  comment: ''\n"
+      "Processors:\n"
+      "- id: 0eac51eb-d76c-4ba6-9f0c-351795b2d243\n"
+      "  name: GenerateFlowFile1\n"

Review comment:
       Updated in eb396ae43b5dae3ec5c23f19725f5ac6d380bf78

##########
File path: docker/test/integration/steps/steps.py
##########
@@ -248,6 +262,27 @@ def step_impl(context, file_name, content, path):
     context.test.add_test_data(path, content, file_name)
 
 
+@given("a Funnel with the name \"{funnel_name}\" is set up in the flow")
+def step_impl(context, funnel_name):
+    funnel = Funnel()
+    funnel.set_name(funnel_name)
+    context.test.add_node(funnel)
+
+
+@given("in the flow the Funnel with the name \"{source_name}\" is connected to the {destination_name}")

Review comment:
       Updated in eb396ae43b5dae3ec5c23f19725f5ac6d380bf78

##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +807,37 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (YAML::const_iterator iter = node.begin(); iter != node.end(); ++iter) {

Review comment:
       Updated in eb396ae43b5dae3ec5c23f19725f5ac6d380bf78




-- 
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 #1132: MINIFICPP-1603 Support Funnels element in flow definition

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


   


-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: docker/test/integration/minifi/flow_serialization/Minifi_flow_yaml_serializer.py
##########
@@ -92,21 +108,22 @@ def serialize(self, connectable, root=None, visited=None):
                     res['Connections'].append({
                         'name': str(uuid.uuid4()),
                         'source id': str(connectable.uuid),
-                        'source relationship name': conn_name,
                         'destination id': str(proc.uuid),
                         'drop empty': ("true" if proc.drop_empty_flowfiles else "false")

Review comment:
       Unfortunately, I don't know why is it not included there, but it seems to me as well that there shouldn't be a difference there. I added it in e64b1c0d1fe231936dcde263939fbd8dcfff4083

##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +806,36 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (const auto& element : node) {
+    YAML::Node funnel_node = element.as<YAML::Node>();
+
+    std::string id = getOrGenerateId(funnel_node);
+
+    // Default name to be same as ID
+    std::string name = funnel_node["name"].as<std::string>(id);
+
+    const utils::optional<utils::Identifier> uuid = utils::Identifier::parse(id) | utils::orElse([this] {
+      logger_->log_debug("Incorrect connection UUID format.");
+      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");

Review comment:
       Updated in e64b1c0d1fe231936dcde263939fbd8dcfff4083




-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: docker/test/integration/minifi/flow_serialization/Minifi_flow_yaml_serializer.py
##########
@@ -92,21 +108,22 @@ def serialize(self, connectable, root=None, visited=None):
                     res['Connections'].append({
                         'name': str(uuid.uuid4()),
                         'source id': str(connectable.uuid),
-                        'source relationship name': conn_name,
                         'destination id': str(proc.uuid),
                         'drop empty': ("true" if proc.drop_empty_flowfiles else "false")

Review comment:
       Not related to this change, but do you know why `drop empty` is not included in the other (single-outgoing-connection) case?  That seems to be the only difference in the two blocks, without this, the duplication could be removed.

##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -805,6 +806,36 @@ void YamlConfiguration::parsePropertiesNodeYaml(const YAML::Node& propertiesNode
   validateComponentProperties(processor, component_name, yaml_section);
 }
 
+void YamlConfiguration::parseFunnelsYaml(const YAML::Node& node, core::ProcessGroup* parent) {
+  if (!parent) {
+    logger_->log_error("parseFunnelsYaml: no parent group was provided");
+    return;
+  }
+  if (!node || !node.IsSequence()) {
+    return;
+  }
+
+  for (const auto& element : node) {
+    YAML::Node funnel_node = element.as<YAML::Node>();
+
+    std::string id = getOrGenerateId(funnel_node);
+
+    // Default name to be same as ID
+    std::string name = funnel_node["name"].as<std::string>(id);
+
+    const utils::optional<utils::Identifier> uuid = utils::Identifier::parse(id) | utils::orElse([this] {
+      logger_->log_debug("Incorrect connection UUID format.");
+      throw Exception(ExceptionType::GENERAL_EXCEPTION, "Incorrect connection UUID format.");

Review comment:
       both of these should be "Incorrect funnel UUID format."

##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,48 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;

Review comment:
       should we log an error here?

##########
File path: libminifi/src/core/yaml/YamlConnectionParser.cpp
##########
@@ -29,24 +29,48 @@ namespace yaml {
 // This is no longer needed in c++17
 constexpr const char* YamlConnectionParser::CONFIG_YAML_CONNECTIONS_KEY;
 
+void YamlConnectionParser::addNewRelationshipToConnection(const std::string& relationship_name, const std::shared_ptr<minifi::Connection>& connection) const {
+  core::Relationship relationship(relationship_name, "");
+  logger_->log_debug("parseConnection: relationship => [%s]", relationship_name);
+  connection->addRelationship(std::move(relationship));
+}
+
+void YamlConnectionParser::addFunnelRelationshipToConnection(const std::shared_ptr<minifi::Connection>& connection) const {
+  utils::Identifier srcUUID;
+  try {
+    srcUUID = getSourceUUIDFromYaml();
+  } catch(const std::exception&) {
+    return;
+  }
+  auto processor = parent_->findProcessorById(srcUUID);
+  if (!processor) {
+    logger_->log_error("Could not find processor with id %s", srcUUID.to_string());
+    return;
+  }
+
+  auto& processor_ref = *processor.get();
+  if (typeid(minifi::core::Funnel) == typeid(processor_ref)) {
+    addNewRelationshipToConnection(minifi::core::Funnel::Success.getName(), connection);
+  }

Review comment:
       it could be useful to add an else branch with an error log here, too




-- 
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 closed pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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


   


-- 
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 change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: libminifi/include/core/Funnel.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 <string>
+#include <memory>
+
+#include "Processor.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace core {
+
+class Funnel : public Processor {

Review comment:
       Thanks for the tip, added `final` in a76b8c6b77f48f19dbc805a0a26a9f55e015d40e




-- 
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] arpadboda commented on a change in pull request #1132: MINIFICPP-1603 Support Funnels element in flow definition

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



##########
File path: docker/test/integration/steps/steps.py
##########
@@ -248,6 +262,27 @@ def step_impl(context, file_name, content, path):
     context.test.add_test_data(path, content, file_name)
 
 
+@given("a Funnel with the name \"{funnel_name}\" is set up")
+def step_impl(context, funnel_name):
+    funnel = Funnel()
+    funnel.set_name(funnel_name)

Review comment:
       You wrote the ctor that takes the name as an optional argument :)

##########
File path: docker/test/integration/minifi/flow_serialization/Minifi_flow_yaml_serializer.py
##########
@@ -92,21 +108,22 @@ def serialize(self, connectable, root=None, visited=None):
                     res['Connections'].append({
                         'name': str(uuid.uuid4()),
                         'source id': str(connectable.uuid),
-                        'source relationship name': conn_name,
                         'destination id': str(proc.uuid),
                         'drop empty': ("true" if proc.drop_empty_flowfiles else "false")
                     })
+                    if (str(connectable.uuid) not in [x['id'] for x in res['Funnels']]):
+                        res['Connections'][-1]['source relationship name'] = conn_name
                     if proc not in visited:
-                        self.serialize(proc, res, visited)
+                        self.serialize_node(proc, res, visited)
             else:
                 res['Connections'].append({
                     'name': str(uuid.uuid4()),
                     'source id': str(connectable.uuid),
-                    'source relationship name': conn_name,
                     'destination id': str(conn_procs.uuid)
                 })
+                if (str(connectable.uuid) not in [x['id'] for x in res['Funnels']]):

Review comment:
       Nitpicking (as it seems to be a "how to write it shorter" challenge 😄  ):
   if (any(str(connectable.uuid) == x['id'] for x in res['Funnels']))

##########
File path: docker/test/integration/minifi/flow_serialization/Minifi_flow_yaml_serializer.py
##########
@@ -92,21 +108,22 @@ def serialize(self, connectable, root=None, visited=None):
                     res['Connections'].append({
                         'name': str(uuid.uuid4()),
                         'source id': str(connectable.uuid),
-                        'source relationship name': conn_name,
                         'destination id': str(proc.uuid),
                         'drop empty': ("true" if proc.drop_empty_flowfiles else "false")
                     })
+                    if (str(connectable.uuid) not in [x['id'] for x in res['Funnels']]):
+                        res['Connections'][-1]['source relationship name'] = conn_name
                     if proc not in visited:
-                        self.serialize(proc, res, visited)
+                        self.serialize_node(proc, res, visited)
             else:
                 res['Connections'].append({
                     'name': str(uuid.uuid4()),
                     'source id': str(connectable.uuid),
-                    'source relationship name': conn_name,
                     'destination id': str(conn_procs.uuid)
                 })
+                if (str(connectable.uuid) not in [x['id'] for x in res['Funnels']]):

Review comment:
       Nitpicking (as it seems to be a "how to write it shorter" challenge 😄  ):
   ```
   if (any(str(connectable.uuid) == x['id'] for x in res['Funnels']))
   ```




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