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/11/24 10:18:02 UTC

[GitHub] [nifi-minifi-cpp] fgerlits commented on a change in pull request #1216: MINIFICPP-1290 Create test coverage for OPC processors

fgerlits commented on a change in pull request #1216:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1216#discussion_r755272082



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -141,7 +141,7 @@ def check_for_multiple_files_generated(self, file_count, timeout_seconds, expect
     def check_for_at_least_one_file_with_content_generated(self, content, timeout_seconds):
         output_validator = SingleOrMultiFileOutputValidator(decode_escaped_str(content))
         output_validator.set_output_dir(self.file_system_observer.get_output_dir())
-        self.check_output(timeout_seconds, output_validator, 1)
+        self.check_output(timeout_seconds, output_validator, timeout_seconds)

Review comment:
       The 3rd argument is `max_files`, ie. number of files expected -- "1" seems like the correct value here.

##########
File path: docker/test/integration/features/opcua.feature
##########
@@ -0,0 +1,153 @@
+Feature: Putting and fetching data to OPC UA server
+  In order to send and fetch data from an OPC UA server
+  As a user of MiNiFi
+  I need to have PutOPCProcessor and FetchOPCProcessor
+
+  Background:
+    Given the content of "/tmp/output" is monitored
+
+  Scenario Outline: Create and fetch data from an OPC UA node
+    Given a GetFile processor with the "Input Directory" property set to "/tmp/input" in the "create-opc-ua-node" flow
+    And a file with the content "<Value>" is present in "/tmp/input"
+    And a PutOPCProcessor processor in the "create-opc-ua-node" flow
+    And a FetchOPCProcessor processor in the "fetch-opc-ua-node" flow
+    And a PutFile processor with the "Directory" property set to "/tmp/output" in the "fetch-opc-ua-node" flow
+    And these processor properties are set:
+      | processor name    | property name               | property value                  |
+      | PutOPCProcessor   | Parent node ID              | 85                              |
+      | PutOPCProcessor   | Parent node ID type         | Int                             |
+      | PutOPCProcessor   | Target node ID              | 9999                            |
+      | PutOPCProcessor   | Target node ID type         | Int                             |
+      | PutOPCProcessor   | Target node namespace index | 1                               |
+      | PutOPCProcessor   | Value type                  | <Value Type>                    |
+      | PutOPCProcessor   | OPC server endpoint         | opc.tcp://opcua-server:4840/    |
+      | PutOPCProcessor   | Target node browse name     | testnodename                    |
+      | FetchOPCProcessor | Node ID                     | 9999                            |
+      | FetchOPCProcessor | Node ID type                | Int                             |
+      | FetchOPCProcessor | Namespace index             | 1                               |
+      | FetchOPCProcessor | OPC server endpoint         | opc.tcp://opcua-server:4840/    |
+      | FetchOPCProcessor | Max depth                   | 1                               |
+
+    And the "success" relationship of the GetFile processor is connected to the PutOPCProcessor
+    And the "success" relationship of the FetchOPCProcessor processor is connected to the PutFile
+
+    And an OPC UA server is set up
+
+    When all instances start up
+    Then at least one flowfile with the content "<Value>" is placed in the monitored directory in less than 60 seconds
+
+  Examples: Topic names and formats to test
+    | Value Type   | Value   |
+    | String       | Test    |
+    | UInt32       | 42      |
+    | Double       | 123.321 |
+    | Boolean      | False   |
+
+  Scenario Outline: Update and fetch data from an OPC UA node

Review comment:
       What does "update" refer to in this test description?  The only difference I can see with the previous test is that the node ID is configurable.

##########
File path: docker/test/integration/minifi/core/MinifiContainer.py
##########
@@ -8,8 +8,10 @@ class MinifiContainer(FlowContainer):
     MINIFI_VERSION = os.environ['MINIFI_VERSION']
     MINIFI_ROOT = '/opt/minifi/nifi-minifi-cpp-' + MINIFI_VERSION
 
-    def __init__(self, config_dir, name, vols, network, image_store):
-        super().__init__(config_dir, name, 'minifi-cpp', vols, network, image_store)
+    def __init__(self, config_dir, name, vols, network, image_store, command=None):
+        if not command:
+            command = ["/bin/sh", "-c", "cp /tmp/minifi_config/config.yml " + MinifiContainer.MINIFI_ROOT + "/conf && /opt/minifi/minifi-current/bin/minifi.sh run"]

Review comment:
       I think this would be clearer:
   
   ```suggestion
       DEFAULT_COMMAND = ["/bin/sh", "-c", "cp /tmp/minifi_config/config.yml " + MinifiContainer.MINIFI_ROOT + "/conf && /opt/minifi/minifi-current/bin/minifi.sh run"]
   
       def __init__(self, config_dir, name, vols, network, image_store, command=DEFAULT_COMMAND):
   ```

##########
File path: extensions/opc/src/opcbase.cpp
##########
@@ -84,45 +83,41 @@ namespace processors {
 
     auto certificatePathRes = context->getProperty(CertificatePath.getName(), certpath_);
     auto keyPathRes = context->getProperty(KeyPath.getName(), keypath_);
-    auto trustedPathRes = context->getProperty(TrustedPath.getName(), trustpath_);
-    if (certificatePathRes != keyPathRes || keyPathRes != trustedPathRes) {
-      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "All or none of Certificate path, Key path and Trusted server certificate path should be provided!");
+    context->getProperty(TrustedPath.getName(), trustpath_);
+    if (certificatePathRes != keyPathRes) {
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "All or none of Certificate path and Key path should be provided!");
     }
 
-    if (!password_.empty() && (certpath_.empty() || keypath_.empty() || trustpath_.empty() || applicationURI_.empty())) {
-      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Certificate path, Key path, Trusted server certificate path and Application URI must be provided in case Password is provided!");
+    if (certpath_.empty()) {
+      return;
+    }
+    if (applicationURI_.empty()) {
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Application URI must be provided if Certificate path is provided!");
     }
 
-    if (!certpath_.empty()) {
-      if (applicationURI_.empty()) {
-        throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Application URI must be provided if Certificate path is provided!");
-      }
+    std::ifstream input_cert(certpath_, std::ios::binary);
+    if (input_cert.good()) {
+      certBuffer_ = std::vector<char>(std::istreambuf_iterator<char>(input_cert), {});
+    }
+    std::ifstream input_key(keypath_, std::ios::binary);
+    if (input_key.good()) {
+      keyBuffer_ = std::vector<char>(std::istreambuf_iterator<char>(input_key), {});
+    }
 
-      std::ifstream input_cert(certpath_, std::ios::binary);
-      if (input_cert.good()) {
-        certBuffer_ = std::vector<char>(std::istreambuf_iterator<char>(input_cert), {});
-      }
-      std::ifstream input_key(keypath_, std::ios::binary);
-      if (input_key.good()) {
-        keyBuffer_ = std::vector<char>(std::istreambuf_iterator<char>(input_key), {});
-      }
+    if (certBuffer_.empty()) {
+      auto error_msg = utils::StringUtils::join_pack("Failed to load cert from path: ", certpath_);
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, error_msg);
+    }
+    if (keyBuffer_.empty()) {
+      auto error_msg = utils::StringUtils::join_pack("Failed to load key from path: ", keypath_);
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, error_msg);
+    }
 
-      trustBuffers_.emplace_back();
+    if (!trustpath_.empty()) {
       std::ifstream input_trust(trustpath_, std::ios::binary);
       if (input_trust.good()) {
-        trustBuffers_[0] = std::vector<char>(std::istreambuf_iterator<char>(input_trust), {});
-      }
-
-      if (certBuffer_.empty()) {
-        auto error_msg = utils::StringUtils::join_pack("Failed to load cert from path: ", certpath_);
-        throw Exception(PROCESS_SCHEDULE_EXCEPTION, error_msg);
-      }
-      if (keyBuffer_.empty()) {
-        auto error_msg = utils::StringUtils::join_pack("Failed to load key from path: ", keypath_);
-        throw Exception(PROCESS_SCHEDULE_EXCEPTION, error_msg);
-      }
-
-      if (trustBuffers_[0].empty()) {
+        trustBuffers_.push_back(std::vector<char>(std::istreambuf_iterator<char>(input_trust), {}));
+      } else {

Review comment:
       The logic has changed here: before, we would throw if `trustpath_` is empty or the file at `trustpath_` doesn't exist or the file is empty; now we only throw if `trustpath_` is non-empty and the file at `trustpath_` doesn't exist.  If this is intentional, a comment would be useful to explain why we do this.

##########
File path: docker/test/integration/minifi/core/FileSystemObserver.py
##########
@@ -31,21 +31,26 @@ def restart_observer_if_needed(self):
         self.observer.schedule(self.event_handler, self.test_output_dir, recursive=True)
         self.observer.start()
 
-    def wait_for_output(self, timeout_seconds, max_files):
+    def wait_for_output(self, timeout_seconds, max_files, output_validator):

Review comment:
       Previously, this function had a single responsibility: to wait until output is available.  Now, waiting and validation are combined.  To me, this makes the code less clear.

##########
File path: docker/test/integration/steps/steps.py
##########
@@ -63,19 +63,31 @@ def step_impl(context, processor_type, property, property_value, processor_name)
                           format(processor_type=processor_type, property=property, property_value=property_value, minifi_container_name="minifi-cpp-flow", processor_name=processor_name))
 
 
-@given("a {processor_type} processor in the \"{minifi_container_name}\" flow")
-@given("a {processor_type} processor in a \"{minifi_container_name}\" flow")
-@given("a {processor_type} processor set up in a \"{minifi_container_name}\" flow")
-def step_impl(context, processor_type, minifi_container_name):
+@given("a {processor_type} processor with the name \"{processor_name}\" in the \"{minifi_container_name}\" flow")
+def step_impl(context, processor_type, processor_name, minifi_container_name):
     container = context.test.acquire_container(minifi_container_name)
     processor = locate("minifi.processors." + processor_type + "." + processor_type)()
-    processor.set_name(processor_type)
+    processor.set_name(processor_name)
     context.test.add_node(processor)
     # Assume that the first node declared is primary unless specified otherwise
     if not container.get_start_nodes():
         container.add_start_node(processor)
 
 
+@given("a {processor_type} processor with the name \"{processor_name}\"")
+def step_impl(context, processor_type, processor_name):
+    context.execute_steps("given a {processor_type} processor with the name \"{processor_name}\" in the \"{minifi_container_name}\" flow".
+                          format(processor_type=processor_type, minifi_container_name="minifi-cpp-flow", processor_name=processor_name))

Review comment:
       very minor, but it triggers my OCD that the order of `processor_name` and `minifi_container_name` is reversed in this line; can you switch them, please?  (also in line 88)

##########
File path: thirdparty/open62541/open62541.patch
##########
@@ -11,11 +11,20 @@ index d426e1da..5f1a4044 100644
  find_package(PythonInterp REQUIRED)
  find_package(Git)
  include(AssignSourceGroup)
-@@ -416,17 +416,17 @@ if(NOT UA_COMPILE_AS_CXX AND (CMAKE_COMPILER_IS_GNUCC OR "x${CMAKE_C_COMPILER_ID
+@@ -526,7 +526,7 @@ if(NOT UA_FORCE_CPP AND (CMAKE_COMPILER_IS_GNUCC OR "x${CMAKE_C_COMPILER_ID}" ST
+     check_add_cc_flag("-Wall")      # Warnings
+     check_add_cc_flag("-Wextra")    # More warnings
+     check_add_cc_flag("-Wpedantic") # Standard compliance
+-    check_add_cc_flag("-Werror")    # All warnings are errors
++    # check_add_cc_flag("-Werror")    # All warnings are errors
+ 
+     check_add_cc_flag("-Wno-static-in-inline") # Clang doesn't like the use of static inline methods inside static inline methods

Review comment:
       I think it's the use of static functions inside non-static inline functions that Clang doesn't like

##########
File path: extensions/opc/src/opcbase.cpp
##########
@@ -84,45 +83,41 @@ namespace processors {
 
     auto certificatePathRes = context->getProperty(CertificatePath.getName(), certpath_);
     auto keyPathRes = context->getProperty(KeyPath.getName(), keypath_);
-    auto trustedPathRes = context->getProperty(TrustedPath.getName(), trustpath_);
-    if (certificatePathRes != keyPathRes || keyPathRes != trustedPathRes) {
-      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "All or none of Certificate path, Key path and Trusted server certificate path should be provided!");
+    context->getProperty(TrustedPath.getName(), trustpath_);
+    if (certificatePathRes != keyPathRes) {
+      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "All or none of Certificate path and Key path should be provided!");
     }
 
-    if (!password_.empty() && (certpath_.empty() || keypath_.empty() || trustpath_.empty() || applicationURI_.empty())) {
-      throw Exception(PROCESS_SCHEDULE_EXCEPTION, "Certificate path, Key path, Trusted server certificate path and Application URI must be provided in case Password is provided!");

Review comment:
       is this check no longer required?




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