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/26 15:15:37 UTC

[GitHub] [nifi-minifi-cpp] lordgamez opened a new pull request #1143: MINIFICPP-1522 Log invalid attribute in case of YAML parse failure

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


   Previously when parsing a single property value failed due to the failure of a custom validator there was only a generic error message:
   ```
   [2021-03-02 16:35:37.377] [org::apache::nifi::minifi::core::YamlConfiguration] [error] Invalid yaml configuration file
   [2021-03-02 16:35:37.377] [main] [error] Failed to load configuration due to exception: General Operation: Cannot convert invalid value
   ```
   The message did not indicate which property had an invalid value, which made it hard for the user to find the source of the failure. With this change the property name is logged in case of validator failure to pinpoint the source of the problem:
   ```
   [2021-07-26 16:53:31.190] [org::apache::nifi::minifi::core::YamlConfiguration] [error] Invalid value was set for property 'Max Poll Time'
   [2021-07-26 16:53:31.190] [org::apache::nifi::minifi::core::YamlConfiguration] [error] Invalid yaml configuration file
   [2021-07-26 16:53:31.190] [main] [error] Failed to load configuration due to exception: General Operation: Cannot convert invalid value
   ```
   
   https://issues.apache.org/jira/browse/MINIFICPP-1522
   
   --------------------------------------------------------------------
   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] adamdebreceni commented on a change in pull request #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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



##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -764,8 +764,15 @@ void YamlConfiguration::parseSingleProperty(const std::string& propertyName, con
   core::Property myProp(propertyName, "", "");
   processor->getProperty(propertyName, myProp);
   const PropertyValue coercedValue = getValidatedProcessorPropertyForDefaultTypeInfo(myProp, propertyValueNode);

Review comment:
       what about this `getValidatedProcessorPropertyForDefaultTypeInfo` function? it seems to log errors on exception, in which case does it not log, when it should?




-- 
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 #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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


   


-- 
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 #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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



##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -764,8 +764,15 @@ void YamlConfiguration::parseSingleProperty(const std::string& propertyName, con
   core::Property myProp(propertyName, "", "");
   processor->getProperty(propertyName, myProp);
   const PropertyValue coercedValue = getValidatedProcessorPropertyForDefaultTypeInfo(myProp, propertyValueNode);

Review comment:
       That function does the cast through `yaml-cpp` lib's `YAML::Node` class and it does not run our custom validators for those properties, those are done through the `PropertyValue` class.




-- 
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 change in pull request #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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



##########
File path: libminifi/src/core/yaml/YamlConfiguration.cpp
##########
@@ -764,8 +764,15 @@ void YamlConfiguration::parseSingleProperty(const std::string& propertyName, con
   core::Property myProp(propertyName, "", "");
   processor->getProperty(propertyName, myProp);
   const PropertyValue coercedValue = getValidatedProcessorPropertyForDefaultTypeInfo(myProp, propertyValueNode);

Review comment:
       oh now I see, so even though `PropertyValue` contains a `validator_`, assignment in itself does not execute it, it only happens in `setValue` (which is called from `processor->setProperty`)




-- 
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 change in pull request #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
+    REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
+    REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+
+    std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+    rootFlowConfig->getConnections(connectionMap);
+    REQUIRE(2 == connectionMap.size());
+    // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
+    for (auto it : connectionMap) {
+      REQUIRE(it.second);
+      REQUIRE(!it.second->getUUIDStr().empty());
+      REQUIRE(it.second->getDestination());
+      REQUIRE(it.second->getSource());
+      REQUIRE(60000 == it.second->getFlowExpirationDuration());
+    }
   }
-}
 
   SECTION("missing required field in YAML throws exception") {
-  static const std::string CONFIG_YAML_NO_RPG_PORT_ID = ""
-  "MiNiFi Config Version: 1\n"
-  "Flow Controller:\n"
-  "  name: MiNiFi Flow\n"
-  "Processors: []\n"
-  "Connections: []\n"
-  "Remote Processing Groups:\n"
-  "    - name: NiFi Flow\n"
-  "      comment:\n"
-  "      url: https://localhost:8090/nifi\n"
-  "      timeout: 30 secs\n"
-  "      yield period: 10 sec\n"
-  "      Input Ports:\n"
-  "          - name: tailed log\n"
-  "            comments:\n"
-  "            max concurrent tasks: 1\n"
-  "            use compression: false\n"
-  "\n";
-
-  std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
-  REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
-}
+    static const std::string CONFIG_YAML_NO_RPG_PORT_ID =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+  name: MiNiFi Flow
+Processors: []
+Connections: []
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
+  }
+
+  SECTION("Validated YAML property failure throws exception and logs invalid attribute name") {
+    static const std::string CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE =
+      R"(
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+  - name: RetryFlowFile
+    class: org.apache.nifi.processors.standard.RetryFlowFile
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    auto-terminated relationships list:
+    Properties:
+        Retry Attribute: ""
+Connections: []
+Remote Processing Groups: []
+Provenance Reporting:
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), utils::internal::InvalidValueException);
+    REQUIRE(LogTestController::getInstance().contains("Invalid value was set for property 'Retry Attribute'"));

Review comment:
       but don't we have access to the raw string value at the point of log (without needing to go through a property)?
   ```
   const std::string rawValueString = propertyValueNode.as<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] lordgamez commented on a change in pull request #1143: MINIFICPP-1522 Log invalid attribute in case of YAML parse failure

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());

Review comment:
       Updated in 37f146fc203f51adc62044c8c9e2fc11f358c63b

##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());

Review comment:
       Updated in 37f146fc203f51adc62044c8c9e2fc11f358c63b




-- 
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 #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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


   


-- 
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 pull request #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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


   > The change itself looks good to me, but I think the commit message is misleading: this change is about processor property values, not attributes. :)
   
   That's true, renamed it.


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

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 #1143: MINIFICPP-1522 Log invalid attribute in case of YAML parse failure

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
+    REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
+    REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+
+    std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+    rootFlowConfig->getConnections(connectionMap);
+    REQUIRE(2 == connectionMap.size());
+    // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
+    for (auto it : connectionMap) {
+      REQUIRE(it.second);
+      REQUIRE(!it.second->getUUIDStr().empty());
+      REQUIRE(it.second->getDestination());
+      REQUIRE(it.second->getSource());
+      REQUIRE(60000 == it.second->getFlowExpirationDuration());
+    }
   }
-}
 
   SECTION("missing required field in YAML throws exception") {
-  static const std::string CONFIG_YAML_NO_RPG_PORT_ID = ""
-  "MiNiFi Config Version: 1\n"
-  "Flow Controller:\n"
-  "  name: MiNiFi Flow\n"
-  "Processors: []\n"
-  "Connections: []\n"
-  "Remote Processing Groups:\n"
-  "    - name: NiFi Flow\n"
-  "      comment:\n"
-  "      url: https://localhost:8090/nifi\n"
-  "      timeout: 30 secs\n"
-  "      yield period: 10 sec\n"
-  "      Input Ports:\n"
-  "          - name: tailed log\n"
-  "            comments:\n"
-  "            max concurrent tasks: 1\n"
-  "            use compression: false\n"
-  "\n";
-
-  std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
-  REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
-}
+    static const std::string CONFIG_YAML_NO_RPG_PORT_ID =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+  name: MiNiFi Flow
+Processors: []
+Connections: []
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
+  }
+
+  SECTION("Validated YAML property failure throws exception and logs invalid attribute name") {
+    static const std::string CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE =
+      R"(
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+  - name: RetryFlowFile
+    class: org.apache.nifi.processors.standard.RetryFlowFile
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    auto-terminated relationships list:
+    Properties:
+        Retry Attribute: ""
+Connections: []
+Remote Processing Groups: []
+Provenance Reporting:
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), utils::internal::InvalidValueException);
+    REQUIRE(LogTestController::getInstance().contains("Invalid value was set for property 'Retry Attribute'"));

Review comment:
       It would require to cast the `PropertyValue` type to a `string` which also runs the custom validator and it would also throw an exception in this case.




-- 
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 change in pull request #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
+    REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
+    REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+
+    std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+    rootFlowConfig->getConnections(connectionMap);
+    REQUIRE(2 == connectionMap.size());
+    // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
+    for (auto it : connectionMap) {
+      REQUIRE(it.second);
+      REQUIRE(!it.second->getUUIDStr().empty());
+      REQUIRE(it.second->getDestination());
+      REQUIRE(it.second->getSource());
+      REQUIRE(60000 == it.second->getFlowExpirationDuration());
+    }
   }
-}
 
   SECTION("missing required field in YAML throws exception") {
-  static const std::string CONFIG_YAML_NO_RPG_PORT_ID = ""
-  "MiNiFi Config Version: 1\n"
-  "Flow Controller:\n"
-  "  name: MiNiFi Flow\n"
-  "Processors: []\n"
-  "Connections: []\n"
-  "Remote Processing Groups:\n"
-  "    - name: NiFi Flow\n"
-  "      comment:\n"
-  "      url: https://localhost:8090/nifi\n"
-  "      timeout: 30 secs\n"
-  "      yield period: 10 sec\n"
-  "      Input Ports:\n"
-  "          - name: tailed log\n"
-  "            comments:\n"
-  "            max concurrent tasks: 1\n"
-  "            use compression: false\n"
-  "\n";
-
-  std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
-  REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
-}
+    static const std::string CONFIG_YAML_NO_RPG_PORT_ID =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+  name: MiNiFi Flow
+Processors: []
+Connections: []
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
+  }
+
+  SECTION("Validated YAML property failure throws exception and logs invalid attribute name") {
+    static const std::string CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE =
+      R"(
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+  - name: RetryFlowFile
+    class: org.apache.nifi.processors.standard.RetryFlowFile
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    auto-terminated relationships list:
+    Properties:
+        Retry Attribute: ""
+Connections: []
+Remote Processing Groups: []
+Provenance Reporting:
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), utils::internal::InvalidValueException);
+    REQUIRE(LogTestController::getInstance().contains("Invalid value was set for property 'Retry Attribute'"));

Review comment:
       but don't we have access to the raw string value at the point of log?
   ```
   const std::string rawValueString = propertyValueNode.as<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] adamdebreceni commented on a change in pull request #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
+    REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
+    REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+
+    std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+    rootFlowConfig->getConnections(connectionMap);
+    REQUIRE(2 == connectionMap.size());
+    // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
+    for (auto it : connectionMap) {
+      REQUIRE(it.second);
+      REQUIRE(!it.second->getUUIDStr().empty());
+      REQUIRE(it.second->getDestination());
+      REQUIRE(it.second->getSource());
+      REQUIRE(60000 == it.second->getFlowExpirationDuration());
+    }
   }
-}
 
   SECTION("missing required field in YAML throws exception") {
-  static const std::string CONFIG_YAML_NO_RPG_PORT_ID = ""
-  "MiNiFi Config Version: 1\n"
-  "Flow Controller:\n"
-  "  name: MiNiFi Flow\n"
-  "Processors: []\n"
-  "Connections: []\n"
-  "Remote Processing Groups:\n"
-  "    - name: NiFi Flow\n"
-  "      comment:\n"
-  "      url: https://localhost:8090/nifi\n"
-  "      timeout: 30 secs\n"
-  "      yield period: 10 sec\n"
-  "      Input Ports:\n"
-  "          - name: tailed log\n"
-  "            comments:\n"
-  "            max concurrent tasks: 1\n"
-  "            use compression: false\n"
-  "\n";
-
-  std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
-  REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
-}
+    static const std::string CONFIG_YAML_NO_RPG_PORT_ID =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+  name: MiNiFi Flow
+Processors: []
+Connections: []
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
+  }
+
+  SECTION("Validated YAML property failure throws exception and logs invalid attribute name") {
+    static const std::string CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE =
+      R"(
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+  - name: RetryFlowFile
+    class: org.apache.nifi.processors.standard.RetryFlowFile
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    auto-terminated relationships list:
+    Properties:
+        Retry Attribute: ""
+Connections: []
+Remote Processing Groups: []
+Provenance Reporting:
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), utils::internal::InvalidValueException);
+    REQUIRE(LogTestController::getInstance().contains("Invalid value was set for property 'Retry Attribute'"));

Review comment:
       uh tough one, should we maybe log the processor we encountered the error then? or can the user infer which processor the error originated from? (maybe we log something like `creating processor XY`?) 




-- 
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 #1143: MINIFICPP-1522 Log invalid attribute in case of YAML parse failure

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
+    REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
+    REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+
+    std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+    rootFlowConfig->getConnections(connectionMap);
+    REQUIRE(2 == connectionMap.size());
+    // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
+    for (auto it : connectionMap) {
+      REQUIRE(it.second);
+      REQUIRE(!it.second->getUUIDStr().empty());
+      REQUIRE(it.second->getDestination());
+      REQUIRE(it.second->getSource());
+      REQUIRE(60000 == it.second->getFlowExpirationDuration());
+    }
   }
-}
 
   SECTION("missing required field in YAML throws exception") {
-  static const std::string CONFIG_YAML_NO_RPG_PORT_ID = ""
-  "MiNiFi Config Version: 1\n"
-  "Flow Controller:\n"
-  "  name: MiNiFi Flow\n"
-  "Processors: []\n"
-  "Connections: []\n"
-  "Remote Processing Groups:\n"
-  "    - name: NiFi Flow\n"
-  "      comment:\n"
-  "      url: https://localhost:8090/nifi\n"
-  "      timeout: 30 secs\n"
-  "      yield period: 10 sec\n"
-  "      Input Ports:\n"
-  "          - name: tailed log\n"
-  "            comments:\n"
-  "            max concurrent tasks: 1\n"
-  "            use compression: false\n"
-  "\n";
-
-  std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
-  REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
-}
+    static const std::string CONFIG_YAML_NO_RPG_PORT_ID =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+  name: MiNiFi Flow
+Processors: []
+Connections: []
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
+  }
+
+  SECTION("Validated YAML property failure throws exception and logs invalid attribute name") {
+    static const std::string CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE =
+      R"(
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+  - name: RetryFlowFile
+    class: org.apache.nifi.processors.standard.RetryFlowFile
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    auto-terminated relationships list:
+    Properties:
+        Retry Attribute: ""
+Connections: []
+Remote Processing Groups: []
+Provenance Reporting:
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), utils::internal::InvalidValueException);
+    REQUIRE(LogTestController::getInstance().contains("Invalid value was set for property 'Retry Attribute'"));

Review comment:
       Ouch, this sounds like tech debt. 




-- 
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 #1143: MINIFICPP-1522 Log invalid attribute in case of YAML parse failure

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());

Review comment:
       Duplicate condition

##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());

Review comment:
       Consider using standard literal suffixes. I also recognize that this is old code just reformatted, so I'm 100% fine with not touching these parts.

##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
+    REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
+    REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+
+    std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+    rootFlowConfig->getConnections(connectionMap);
+    REQUIRE(2 == connectionMap.size());
+    // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
+    for (auto it : connectionMap) {
+      REQUIRE(it.second);
+      REQUIRE(!it.second->getUUIDStr().empty());
+      REQUIRE(it.second->getDestination());
+      REQUIRE(it.second->getSource());
+      REQUIRE(60000 == it.second->getFlowExpirationDuration());
+    }
   }
-}
 
   SECTION("missing required field in YAML throws exception") {
-  static const std::string CONFIG_YAML_NO_RPG_PORT_ID = ""
-  "MiNiFi Config Version: 1\n"
-  "Flow Controller:\n"
-  "  name: MiNiFi Flow\n"
-  "Processors: []\n"
-  "Connections: []\n"
-  "Remote Processing Groups:\n"
-  "    - name: NiFi Flow\n"
-  "      comment:\n"
-  "      url: https://localhost:8090/nifi\n"
-  "      timeout: 30 secs\n"
-  "      yield period: 10 sec\n"
-  "      Input Ports:\n"
-  "          - name: tailed log\n"
-  "            comments:\n"
-  "            max concurrent tasks: 1\n"
-  "            use compression: false\n"
-  "\n";
-
-  std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
-  REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
-}
+    static const std::string CONFIG_YAML_NO_RPG_PORT_ID =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+  name: MiNiFi Flow
+Processors: []
+Connections: []
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
+  }
+
+  SECTION("Validated YAML property failure throws exception and logs invalid attribute name") {
+    static const std::string CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE =
+      R"(
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+  - name: RetryFlowFile
+    class: org.apache.nifi.processors.standard.RetryFlowFile
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    auto-terminated relationships list:
+    Properties:
+        Retry Attribute: ""
+Connections: []
+Remote Processing Groups: []
+Provenance Reporting:
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), utils::internal::InvalidValueException);
+    REQUIRE(LogTestController::getInstance().contains("Invalid value was set for property 'Retry Attribute'"));

Review comment:
       Can we log the value itself? 




-- 
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 #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
+    REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
+    REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+
+    std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+    rootFlowConfig->getConnections(connectionMap);
+    REQUIRE(2 == connectionMap.size());
+    // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
+    for (auto it : connectionMap) {
+      REQUIRE(it.second);
+      REQUIRE(!it.second->getUUIDStr().empty());
+      REQUIRE(it.second->getDestination());
+      REQUIRE(it.second->getSource());
+      REQUIRE(60000 == it.second->getFlowExpirationDuration());
+    }
   }
-}
 
   SECTION("missing required field in YAML throws exception") {
-  static const std::string CONFIG_YAML_NO_RPG_PORT_ID = ""
-  "MiNiFi Config Version: 1\n"
-  "Flow Controller:\n"
-  "  name: MiNiFi Flow\n"
-  "Processors: []\n"
-  "Connections: []\n"
-  "Remote Processing Groups:\n"
-  "    - name: NiFi Flow\n"
-  "      comment:\n"
-  "      url: https://localhost:8090/nifi\n"
-  "      timeout: 30 secs\n"
-  "      yield period: 10 sec\n"
-  "      Input Ports:\n"
-  "          - name: tailed log\n"
-  "            comments:\n"
-  "            max concurrent tasks: 1\n"
-  "            use compression: false\n"
-  "\n";
-
-  std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
-  REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
-}
+    static const std::string CONFIG_YAML_NO_RPG_PORT_ID =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+  name: MiNiFi Flow
+Processors: []
+Connections: []
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
+  }
+
+  SECTION("Validated YAML property failure throws exception and logs invalid attribute name") {
+    static const std::string CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE =
+      R"(
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+  - name: RetryFlowFile
+    class: org.apache.nifi.processors.standard.RetryFlowFile
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    auto-terminated relationships list:
+    Properties:
+        Retry Attribute: ""
+Connections: []
+Remote Processing Groups: []
+Provenance Reporting:
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), utils::internal::InvalidValueException);
+    REQUIRE(LogTestController::getInstance().contains("Invalid value was set for property 'Retry Attribute'"));

Review comment:
       You are right, we do have access to that, first I thought that it's the same conversion, but actually this is done through the `YAML::Node` class without running the validator and the other one is done through the `PropertyValue` class. 
   
   The only problem I see with this is that the original problem was a validation failure due to defining a time period with quotation marks which caused a validation error: `Max Poll Time: "6 sec"`
   If we log the value it looks like this: `[error] Invalid value '6 sec' was set for property 'Max Poll Time'` which could be misleading as it does not show what the problem with this value is. I'm not sure if we should log that value due to this.




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

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 #1143: MINIFICPP-1522 Log property name while parsing config in case of property validation failure

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



##########
File path: extensions/standard-processors/tests/unit/YamlConfigurationTests.cpp
##########
@@ -37,160 +37,186 @@ TEST_CASE("Test YAML Config Processing", "[YamlConfiguration]") {
   core::YamlConfiguration yamlConfig(testProvRepo, testFlowFileRepo, content_repo, streamFactory, configuration);
 
   SECTION("loading YAML without optional component IDs works") {
-  static const std::string CONFIG_YAML_WITHOUT_IDS = ""
-      "MiNiFi Config Version: 1\n"
-      "Flow Controller:\n"
-      "    name: MiNiFi Flow\n"
-      "    comment:\n"
-      "\n"
-      "Core Properties:\n"
-      "    flow controller graceful shutdown period: 10 sec\n"
-      "    flow service write delay interval: 500 ms\n"
-      "    administrative yield duration: 30 sec\n"
-      "    bored yield duration: 10 millis\n"
-      "\n"
-      "FlowFile Repository:\n"
-      "    partitions: 256\n"
-      "    checkpoint interval: 2 mins\n"
-      "    always sync: false\n"
-      "    Swap:\n"
-      "        threshold: 20000\n"
-      "        in period: 5 sec\n"
-      "        in threads: 1\n"
-      "        out period: 5 sec\n"
-      "        out threads: 4\n"
-      "\n"
-      "Provenance Repository:\n"
-      "    provenance rollover time: 1 min\n"
-      "\n"
-      "Content Repository:\n"
-      "    content claim max appendable size: 10 MB\n"
-      "    content claim max flow files: 100\n"
-      "    always sync: false\n"
-      "\n"
-      "Component Status Repository:\n"
-      "    buffer size: 1440\n"
-      "    snapshot frequency: 1 min\n"
-      "\n"
-      "Security Properties:\n"
-      "    keystore: /tmp/ssl/localhost-ks.jks\n"
-      "    keystore type: JKS\n"
-      "    keystore password: localtest\n"
-      "    key password: localtest\n"
-      "    truststore: /tmp/ssl/localhost-ts.jks\n"
-      "    truststore type: JKS\n"
-      "    truststore password: localtest\n"
-      "    ssl protocol: TLS\n"
-      "    Sensitive Props:\n"
-      "        key:\n"
-      "        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL\n"
-      "        provider: BC\n"
-      "\n"
-      "Processors:\n"
-      "    - name: TailFile\n"
-      "      class: org.apache.nifi.processors.standard.TailFile\n"
-      "      max concurrent tasks: 1\n"
-      "      scheduling strategy: TIMER_DRIVEN\n"
-      "      scheduling period: 1 sec\n"
-      "      penalization period: 30 sec\n"
-      "      yield period: 1 sec\n"
-      "      run duration nanos: 0\n"
-      "      auto-terminated relationships list:\n"
-      "      Properties:\n"
-      "          File to Tail: logs/minifi-app.log\n"
-      "          Rolling Filename Pattern: minifi-app*\n"
-      "          Initial Start Position: Beginning of File\n"
-      "\n"
-      "Connections:\n"
-      "    - name: TailToS2S\n"
-      "      source name: TailFile\n"
-      "      source relationship name: success\n"
-      "      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "      max work queue size: 0\n"
-      "      max work queue data size: 1 MB\n"
-      "      flowfile expiration: 60 sec\n"
-      "      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer\n"
-      "\n"
-      "Remote Processing Groups:\n"
-      "    - name: NiFi Flow\n"
-      "      comment:\n"
-      "      url: https://localhost:8090/nifi\n"
-      "      timeout: 30 secs\n"
-      "      yield period: 10 sec\n"
-      "      Input Ports:\n"
-      "          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61\n"
-      "            name: tailed log\n"
-      "            comments:\n"
-      "            max concurrent tasks: 1\n"
-      "            use compression: false\n"
-      "\n"
-      "Provenance Reporting:\n"
-      "    comment:\n"
-      "    scheduling strategy: TIMER_DRIVEN\n"
-      "    scheduling period: 30 sec\n"
-      "    host: localhost\n"
-      "    port name: provenance\n"
-      "    port: 8090\n"
-      "    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56\n"
-      "    url: https://localhost:8090/\n"
-      "    originating url: http://${hostname(true)}:8081/nifi\n"
-      "    use compression: true\n"
-      "    timeout: 30 secs\n"
-      "    batch size: 1000";
-
-  std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
-  std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+    static const std::string CONFIG_YAML_WITHOUT_IDS =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+    name: MiNiFi Flow
+    comment:
 
-  REQUIRE(rootFlowConfig);
-  REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
-  utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
-  REQUIRE(uuid);
-  REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(
-      core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
-  REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
-  REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
-  REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
-  REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
-  REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+Core Properties:
+    flow controller graceful shutdown period: 10 sec
+    flow service write delay interval: 500 ms
+    administrative yield duration: 30 sec
+    bored yield duration: 10 millis
 
-  std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
-  rootFlowConfig->getConnections(connectionMap);
-  REQUIRE(2 == connectionMap.size());
-  // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
-  for (auto it : connectionMap) {
-    REQUIRE(it.second);
-    REQUIRE(!it.second->getUUIDStr().empty());
-    REQUIRE(it.second->getDestination());
-    REQUIRE(it.second->getSource());
-    REQUIRE(60000 == it.second->getFlowExpirationDuration());
+FlowFile Repository:
+    partitions: 256
+    checkpoint interval: 2 mins
+    always sync: false
+    Swap:
+        threshold: 20000
+        in period: 5 sec
+        in threads: 1
+        out period: 5 sec
+        out threads: 4
+
+Provenance Repository:
+    provenance rollover time: 1 min
+
+Content Repository:
+    content claim max appendable size: 10 MB
+    content claim max flow files: 100
+    always sync: false
+
+Component Status Repository:
+    buffer size: 1440
+    snapshot frequency: 1 min
+
+Security Properties:
+    keystore: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    host: localhost
+    port name: provenance
+    port: 8090
+    port uuid: 2f389b8d-83f2-48d3-b465-048f28a1cb56
+    url: https://localhost:8090/
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000;
+        )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_WITHOUT_IDS);
+    std::unique_ptr<core::ProcessGroup> rootFlowConfig = yamlConfig.getYamlRoot(configYamlStream);
+
+    REQUIRE(rootFlowConfig);
+    REQUIRE(rootFlowConfig->findProcessorByName("TailFile"));
+    utils::Identifier uuid = rootFlowConfig->findProcessorByName("TailFile")->getUUID();
+    REQUIRE(uuid);
+    REQUIRE(!rootFlowConfig->findProcessorByName("TailFile")->getUUIDStr().empty());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(
+        core::SchedulingStrategy::TIMER_DRIVEN == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingStrategy());
+    REQUIRE(1 == rootFlowConfig->findProcessorByName("TailFile")->getMaxConcurrentTasks());
+    REQUIRE(1 * 1000 * 1000 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getSchedulingPeriodNano());
+    REQUIRE(std::chrono::seconds(30) == rootFlowConfig->findProcessorByName("TailFile")->getPenalizationPeriod());
+    REQUIRE(1 * 1000 == rootFlowConfig->findProcessorByName("TailFile")->getYieldPeriodMsec());
+    REQUIRE(0 == rootFlowConfig->findProcessorByName("TailFile")->getRunDurationNano());
+
+    std::map<std::string, std::shared_ptr<minifi::Connection>> connectionMap;
+    rootFlowConfig->getConnections(connectionMap);
+    REQUIRE(2 == connectionMap.size());
+    // This is a map of UUID->Connection, and we don't know UUID, so just going to loop over it
+    for (auto it : connectionMap) {
+      REQUIRE(it.second);
+      REQUIRE(!it.second->getUUIDStr().empty());
+      REQUIRE(it.second->getDestination());
+      REQUIRE(it.second->getSource());
+      REQUIRE(60000 == it.second->getFlowExpirationDuration());
+    }
   }
-}
 
   SECTION("missing required field in YAML throws exception") {
-  static const std::string CONFIG_YAML_NO_RPG_PORT_ID = ""
-  "MiNiFi Config Version: 1\n"
-  "Flow Controller:\n"
-  "  name: MiNiFi Flow\n"
-  "Processors: []\n"
-  "Connections: []\n"
-  "Remote Processing Groups:\n"
-  "    - name: NiFi Flow\n"
-  "      comment:\n"
-  "      url: https://localhost:8090/nifi\n"
-  "      timeout: 30 secs\n"
-  "      yield period: 10 sec\n"
-  "      Input Ports:\n"
-  "          - name: tailed log\n"
-  "            comments:\n"
-  "            max concurrent tasks: 1\n"
-  "            use compression: false\n"
-  "\n";
-
-  std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
-  REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
-}
+    static const std::string CONFIG_YAML_NO_RPG_PORT_ID =
+      R"(
+MiNiFi Config Version: 1
+Flow Controller:
+  name: MiNiFi Flow
+Processors: []
+Connections: []
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_NO_RPG_PORT_ID);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), std::invalid_argument);
+  }
+
+  SECTION("Validated YAML property failure throws exception and logs invalid attribute name") {
+    static const std::string CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE =
+      R"(
+Flow Controller:
+  name: MiNiFi Flow
+Processors:
+  - name: RetryFlowFile
+    class: org.apache.nifi.processors.standard.RetryFlowFile
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 1 sec
+    auto-terminated relationships list:
+    Properties:
+        Retry Attribute: ""
+Connections: []
+Remote Processing Groups: []
+Provenance Reporting:
+      )";
+
+    std::istringstream configYamlStream(CONFIG_YAML_EMPTY_RETRY_ATTRIBUTE);
+    REQUIRE_THROWS_AS(yamlConfig.getYamlRoot(configYamlStream), utils::internal::InvalidValueException);
+    REQUIRE(LogTestController::getInstance().contains("Invalid value was set for property 'Retry Attribute'"));

Review comment:
       I like the idea of logging the processor name that would pinpoint the source as there could be duplicate property names in different processors (or controller services). Updated in a601b3e4cab33790af5c241157cf6d2ec25523ec




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