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/02/06 16:53:19 UTC

[GitHub] [nifi-minifi-cpp] hunyadi-dev opened a new pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

hunyadi-dev opened a new pull request #995:
URL: https://github.com/apache/nifi-minifi-cpp/pull/995


   This is probably a lot to review. Most of the integration test definitions are in `steps.py` and `MiNiFi_integration_test_driver.py`. The other changes are decoupling functionalities of already existing python classes.


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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()

Review comment:
       I moved it before the disconnect loop and it does not seem to matter, where it is, but its presence is required in the loop. My reasoning was that it is probably needed to sync up the network after the disconnects happen, so that there is no exception related to attempting remove on non-empty container.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/DockerVerify.sh
##########
@@ -71,4 +73,20 @@ export PATH
 PYTHONPATH="${PYTHONPATH}:${docker_dir}/test/integration"
 export PYTHONPATH
 
-exec pytest -s -v "${docker_dir}"/test/integration
+cd "${docker_dir}/test/integration"
+exec 
+  behave -f pretty --logging-level INFO --no-capture "features/file_system_operations.feature" -n "Get and put operations run in a simple flow" &&

Review comment:
       Updated.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/minifi/core/Connectable.py
##########
@@ -54,13 +66,15 @@ def __rshift__(self, other):
             connected.out_proc = copy(connected.out_proc)
 
         if isinstance(other, tuple):
-            if isinstance(other[0], tuple):
-                for rel_tuple in other:
-                    rel = {rel_tuple[0]: rel_tuple[1]}
-                    connected.out_proc.connect(rel)
-            else:
-                rel = {other[0]: other[1]}
-                connected.out_proc.connect(rel)
+            # if isinstance(other[0], tuple):
+            #     for rel_tuple in other:
+            #         rel = {rel_tuple[0]: rel_tuple[1]}
+            #         connected.out_proc.connect(rel)
+            # else:
+            #     rel = {other[0]: other[1]}
+            #     connected.out_proc.connect(rel)

Review comment:
       Removed `__rshift__` and `__invert__` altogether.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/DockerVerify.sh
##########
@@ -58,7 +58,9 @@ if ! command swig -version &> /dev/null; then
 fi
 
 pip install --upgrade \
+            behave \
             pytest \

Review comment:
       Probably not, will remove.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:

Review comment:
       Updated to send kill signal once. I should do the same thing irl, when an application fails to respond, it is unnecessary to repeatedly send kill signals... :rage3:  It feels good though... :grin: 




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass

Review comment:
       How would you go about it?




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/minifi/core/Processor.py
##########
@@ -37,6 +39,16 @@ def __init__(self,
         }
         self.schedule.update(schedule)
 
+    def set_property(self, key, value):
+        # if value == "false":
+        #     self.properties[key] = False
+        # if value == "true":
+        #     self.properties[key] = True

Review comment:
       Removed.

##########
File path: docker/test/integration/minifi/core/DockerTestDirectoryBindings.py
##########
@@ -0,0 +1,104 @@
+import logging
+import os
+import shutil
+
+class DockerTestDirectoryBindings:
+    def __init__(self):
+        self.data_directories = {}
+
+    def __del__(self):
+        self.delete_data_directories()
+
+    def create_new_data_directories(self, test_id):
+        self.data_directories[test_id] = {
+            "input_dir": "/tmp/.nifi-test-input." + test_id,
+            "output_dir": "/tmp/.nifi-test-output." + test_id,
+            "resources_dir": "/tmp/.nifi-test-resources." + test_id
+        }
+
+        [self.create_directory(directory) for directory in self.data_directories[test_id].values()]
+
+        # Add resources
+        test_dir = os.environ['PYTHONPATH'].split(':')[-1] # Based on DockerVerify.sh
+        shutil.copytree(test_dir + "/resources/kafka_broker/conf/certs", self.data_directories[test_id]["resources_dir"] + "/certs")
+
+    def get_data_directories(self, test_id):
+        return self.data_directories[test_id]
+
+    def docker_path_to_local_path(self, test_id, docker_path):
+        # Docker paths are currently hard-coded
+        if docker_path == "/tmp/input":
+            return self.data_directories[test_id]["input_dir"]
+        if docker_path == "/tmp/output":
+            return self.data_directories[test_id]["output_dir"]
+        if docker_path == "/tmp/resources":
+            return self.data_directories[test_id]["resources_dir"]
+        # Might be worth reworking these
+        if docker_path == "/tmp/output/success":
+            self.create_directory(self.data_directories[test_id]["output_dir"] + "/success")
+            return self.data_directories[test_id]["output_dir"] + "/success"
+        if docker_path == "/tmp/output/failure":
+            self.create_directory(self.data_directories[test_id]["output_dir"] + "/failure")
+            return self.data_directories[test_id]["output_dir"] + "/failure"
+        raise Exception("Docker directory \"%s\" has no preset bindings." % docker_path)
+
+    def get_directory_bindings(self, test_id):
+        """
+        Performs a standard container flow deployment with the addition
+        of volumes supporting test input/output directories.
+        """
+        vols = {}
+        vols[self.data_directories[test_id]["input_dir"]] = {"bind": "/tmp/input", "mode": "rw"}
+        vols[self.data_directories[test_id]["output_dir"]] = {"bind": "/tmp/output", "mode": "rw"}
+        vols[self.data_directories[test_id]["resources_dir"]] = {"bind": "/tmp/resources", "mode": "rw"}
+        return vols
+
+    @staticmethod
+    def create_directory(dir):
+        logging.info("Creating tmp dir: %s", dir)
+        os.makedirs(dir)
+        os.chmod(dir, 0o777)
+
+    @staticmethod
+    def delete_directory(dir):

Review comment:
       Renamed.




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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/minifi/core/DockerTestCluster.py
##########
@@ -198,38 +110,20 @@ def is_s3_bucket_empty(self):
         ls_result = subprocess.check_output(["docker", "exec", "s3-server", "ls", s3_mock_dir + "/test_bucket/"]).decode(sys.stdout.encoding)
         return not ls_result
 
-    def rm_out_child(self, dir):
-        logging.info('Removing %s from output folder', os.path.join(self.tmp_test_output_dir, dir))
-        shutil.rmtree(os.path.join(self.tmp_test_output_dir, dir))
-
     def wait_for_container_logs(self, container_name, log, timeout, count=1):
         logging.info('Waiting for logs `%s` in container `%s`', log, container_name)
         container = self.containers[container_name]
         check_count = 0
         while check_count <= timeout:
-            if container.logs().decode("utf-8").count(log) == count:
+            if count <= container.logs().decode("utf-8").count(log):
                 return True
             else:
                 check_count += 1
                 time.sleep(1)
         return False
-    
-    def put_file_contents(self, contents, file_abs_path):
-        logging.info('Writing %d bytes of content to file: %s', len(contents), file_abs_path)
-        with open(file_abs_path, 'wb') as test_input_file:
-            test_input_file.write(contents)
 
+    def segfault_happened(self):
+        return self.segfault
 
     def __exit__(self, exc_type, exc_val, exc_tb):

Review comment:
       It seems that if we interrupt the tests now docker resources do not get cleaned up automatically, that needs to be checked.

##########
File path: docker/DockerVerify.sh
##########
@@ -58,7 +58,9 @@ if ! command swig -version &> /dev/null; then
 fi
 
 pip install --upgrade \
+            behave \
             pytest \

Review comment:
       Do we still need pytest?

##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable

Review comment:
       Currently it seems to me that this is the only place where we remove the containers, so it does not seem to be a backup.

##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass
+
+        del self.docker_directory_bindings
+
+    def docker_path_to_local_path(self, docker_path):
+        return self.docker_directory_bindings.docker_path_to_local_path(self.test_id, docker_path)
+
+    def get_test_id(self):
+        return self.test_id
+
+    def acquire_cluster(self, name):
+        return self.clusters.setdefault(name, DockerTestCluster())
+
+    def set_up_cluster_network(self):
+        self.docker_network = SingleNodeDockerCluster.create_docker_network()
+        for cluster in self.clusters.values():
+            cluster.set_network(self.docker_network)
+
+    def start(self):
+        logging.info("MiNiFi_integration_test start")
+        self.set_up_cluster_network()
+        for cluster in self.clusters.values():
+            cluster.set_directory_bindings(self.docker_directory_bindings.get_directory_bindings(self.test_id))
+            cluster.deploy_flow()
+        for cluster_name, cluster in self.clusters.items():
+            startup_success = True
+            logging.info("Engine: %s", cluster.get_engine())
+            if cluster.get_engine() == "minifi-cpp":
+                logging.info("Engine is minifi-cpp")
+                startup_success = cluster.wait_for_app_logs("Starting Flow Controller", 120)
+            elif cluster.get_engine() == "nifi":
+                startup_success = cluster.wait_for_app_logs("Starting Flow Controller...", 120)
+            elif cluster.get_engine() == "kafka-broker":
+                startup_success = cluster.wait_for_app_logs("Startup complete.", 120)

Review comment:
       :+1: 

##########
File path: docker/test/integration/minifi/core/Connectable.py
##########
@@ -54,13 +66,15 @@ def __rshift__(self, other):
             connected.out_proc = copy(connected.out_proc)
 
         if isinstance(other, tuple):
-            if isinstance(other[0], tuple):
-                for rel_tuple in other:
-                    rel = {rel_tuple[0]: rel_tuple[1]}
-                    connected.out_proc.connect(rel)
-            else:
-                rel = {other[0]: other[1]}
-                connected.out_proc.connect(rel)
+            # if isinstance(other[0], tuple):
+            #     for rel_tuple in other:
+            #         rel = {rel_tuple[0]: rel_tuple[1]}
+            #         connected.out_proc.connect(rel)
+            # else:
+            #     rel = {other[0]: other[1]}
+            #     connected.out_proc.connect(rel)

Review comment:
       Please remove commented code

##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass
+
+        del self.docker_directory_bindings
+
+    def docker_path_to_local_path(self, docker_path):
+        return self.docker_directory_bindings.docker_path_to_local_path(self.test_id, docker_path)
+
+    def get_test_id(self):
+        return self.test_id
+
+    def acquire_cluster(self, name):
+        return self.clusters.setdefault(name, DockerTestCluster())
+
+    def set_up_cluster_network(self):
+        self.docker_network = SingleNodeDockerCluster.create_docker_network()
+        for cluster in self.clusters.values():
+            cluster.set_network(self.docker_network)
+
+    def start(self):
+        logging.info("MiNiFi_integration_test start")
+        self.set_up_cluster_network()
+        for cluster in self.clusters.values():
+            cluster.set_directory_bindings(self.docker_directory_bindings.get_directory_bindings(self.test_id))
+            cluster.deploy_flow()
+        for cluster_name, cluster in self.clusters.items():
+            startup_success = True
+            logging.info("Engine: %s", cluster.get_engine())
+            if cluster.get_engine() == "minifi-cpp":
+                logging.info("Engine is minifi-cpp")
+                startup_success = cluster.wait_for_app_logs("Starting Flow Controller", 120)
+            elif cluster.get_engine() == "nifi":
+                startup_success = cluster.wait_for_app_logs("Starting Flow Controller...", 120)
+            elif cluster.get_engine() == "kafka-broker":
+                startup_success = cluster.wait_for_app_logs("Startup complete.", 120)
+            if not startup_success:
+                cluster.log_nifi_output()
+            assert startup_success
+
+    def add_node(self, processor):
+        if processor.get_name() in (elem.get_name() for elem in self.connectable_nodes):
+            raise Exception("Trying to register processor with an already registered name: \"%s\"" % processor.get_name())
+        self.connectable_nodes.append(processor)
+
+    def get_or_create_node_by_name(self, node_name):
+        node = self.get_node_by_name(node_name) 
+        if node == None:

Review comment:
       Should use `is None` instead

##########
File path: docker/test/integration/minifi/core/SSL_cert_utils.py
##########
@@ -0,0 +1,54 @@
+import time
+import logging
+
+from M2Crypto import X509, EVP, RSA, ASN1
+
+def gen_cert():
+    """
+    Generate TLS certificate request for testing
+    """
+
+    req, key = gen_req()
+    pub_key = req.get_pubkey()
+    subject = req.get_subject()
+    cert = X509.X509()
+    # noinspection PyTypeChecker
+    cert.set_serial_number(1)
+    cert.set_version(2)
+    cert.set_subject(subject)
+    t = int(time.time())
+    now = ASN1.ASN1_UTCTIME()
+    now.set_time(t)
+    now_plus_year = ASN1.ASN1_UTCTIME()
+    now_plus_year.set_time(t + 60 * 60 * 24 * 365)
+    cert.set_not_before(now)
+    cert.set_not_after(now_plus_year)
+    issuer = X509.X509_Name()
+    issuer.C = 'US'
+    issuer.CN = 'minifi-listen'
+    cert.set_issuer(issuer)
+    cert.set_pubkey(pub_key)
+    cert.sign(key, 'sha256')
+
+    return cert, key
+
+def rsa_gen_key_callback():
+        pass

Review comment:
       Bad indentation

##########
File path: docker/test/integration/minifi/core/SingleNodeDockerCluster.py
##########
@@ -32,57 +38,96 @@ def __init__(self):
         # Get docker client
         self.client = docker.from_env()
 
-    def deploy_flow(self,
-                    flow,
-                    name=None,
-                    vols=None,
-                    engine='minifi-cpp'):
+    def __del__(self):
+        """
+        Clean up ephemeral cluster resources
+        """
+
+        # Clean up containers
+        for container in self.containers.values():
+            logging.info('Cleaning up container: %s', container.name)

Review comment:
       We don't cleanup anything here

##########
File path: docker/DockerVerify.sh
##########
@@ -71,4 +73,20 @@ export PATH
 PYTHONPATH="${PYTHONPATH}:${docker_dir}/test/integration"
 export PYTHONPATH
 
-exec pytest -s -v "${docker_dir}"/test/integration
+cd "${docker_dir}/test/integration"
+exec 
+  behave -f pretty --logging-level INFO --no-capture "features/file_system_operations.feature" -n "Get and put operations run in a simple flow" &&

Review comment:
       The common parameters should be extracted to have less copy paste and be able to change logging level in a single place.

##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:

Review comment:
       I think it may be better to send a single forced removed for the container and check periodically only if the container still exists. I'm not sure periodically sending a remove would make a difference.

##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()

Review comment:
       Do we need this additional kill call? The force removal sends a SIGKILL as well.

##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass

Review comment:
       If the previously mentioned periodic check for the container existence is implemented this can be merge to it as well.

##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()

Review comment:
       Shouldn't we start with a docker_network reload before the loop? Just to be sure the container list is properly updated before checking the length of it in the loop.

##########
File path: docker/test/integration/environment.py
##########
@@ -0,0 +1,36 @@
+from behave import fixture, use_fixture
+import sys
+sys.path.append('../minifi')
+import logging
+
+from MiNiFi_integration_test_driver import MiNiFi_integration_test
+from minifi import *
+
+def raise_exception(exception):
+    raise exception
+
+@fixture
+def test_driver_fixture(context):
+    logging.info("Integration test setup")
+    context.test = MiNiFi_integration_test(context)
+    yield context.test
+    logging.info("Integration test teardown...")
+    del context.test
+
+def before_scenario(context, scenario):
+    use_fixture(test_driver_fixture, context)
+
+def after_scenario(context, scenario):
+	pass
+
+# @fixture
+# def setup_minifi_instance(context):
+#     print("Setup", end="\n\n")
+#     context.flow = None
+#     context.test = MiNiFi_integration_test(context)
+
+def before_all(context):
+    context.config.setup_logging()
+
+# def before_scenario(context, scenario):
+# 	use_fixture(setup_minifi_instance, context)

Review comment:
       Please remove commented code

##########
File path: docker/test/integration/minifi/core/DockerTestDirectoryBindings.py
##########
@@ -0,0 +1,104 @@
+import logging
+import os
+import shutil
+
+class DockerTestDirectoryBindings:
+    def __init__(self):
+        self.data_directories = {}
+
+    def __del__(self):
+        self.delete_data_directories()
+
+    def create_new_data_directories(self, test_id):
+        self.data_directories[test_id] = {
+            "input_dir": "/tmp/.nifi-test-input." + test_id,
+            "output_dir": "/tmp/.nifi-test-output." + test_id,
+            "resources_dir": "/tmp/.nifi-test-resources." + test_id
+        }
+
+        [self.create_directory(directory) for directory in self.data_directories[test_id].values()]
+
+        # Add resources
+        test_dir = os.environ['PYTHONPATH'].split(':')[-1] # Based on DockerVerify.sh
+        shutil.copytree(test_dir + "/resources/kafka_broker/conf/certs", self.data_directories[test_id]["resources_dir"] + "/certs")
+
+    def get_data_directories(self, test_id):
+        return self.data_directories[test_id]
+
+    def docker_path_to_local_path(self, test_id, docker_path):
+        # Docker paths are currently hard-coded
+        if docker_path == "/tmp/input":
+            return self.data_directories[test_id]["input_dir"]
+        if docker_path == "/tmp/output":
+            return self.data_directories[test_id]["output_dir"]
+        if docker_path == "/tmp/resources":
+            return self.data_directories[test_id]["resources_dir"]
+        # Might be worth reworking these
+        if docker_path == "/tmp/output/success":
+            self.create_directory(self.data_directories[test_id]["output_dir"] + "/success")
+            return self.data_directories[test_id]["output_dir"] + "/success"
+        if docker_path == "/tmp/output/failure":
+            self.create_directory(self.data_directories[test_id]["output_dir"] + "/failure")
+            return self.data_directories[test_id]["output_dir"] + "/failure"
+        raise Exception("Docker directory \"%s\" has no preset bindings." % docker_path)
+
+    def get_directory_bindings(self, test_id):
+        """
+        Performs a standard container flow deployment with the addition
+        of volumes supporting test input/output directories.
+        """
+        vols = {}
+        vols[self.data_directories[test_id]["input_dir"]] = {"bind": "/tmp/input", "mode": "rw"}
+        vols[self.data_directories[test_id]["output_dir"]] = {"bind": "/tmp/output", "mode": "rw"}
+        vols[self.data_directories[test_id]["resources_dir"]] = {"bind": "/tmp/resources", "mode": "rw"}
+        return vols
+
+    @staticmethod
+    def create_directory(dir):
+        logging.info("Creating tmp dir: %s", dir)
+        os.makedirs(dir)
+        os.chmod(dir, 0o777)
+
+    @staticmethod
+    def delete_directory(dir):

Review comment:
       It may be better to name it something like `delete_tmp_directory` as it can only be used for directories in `/tmp/`

##########
File path: docker/test/integration/minifi/core/Processor.py
##########
@@ -37,6 +39,16 @@ def __init__(self,
         }
         self.schedule.update(schedule)
 
+    def set_property(self, key, value):
+        # if value == "false":
+        #     self.properties[key] = False
+        # if value == "true":
+        #     self.properties[key] = True

Review comment:
       Please remove commented code

##########
File path: docker/test/integration/steps/steps.py
##########
@@ -0,0 +1,243 @@
+from MiNiFi_integration_test_driver import MiNiFi_integration_test

Review comment:
       Could these steps possible be split up into multiple files corresponding to each test suite?




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/minifi/core/SingleNodeDockerCluster.py
##########
@@ -32,57 +38,96 @@ def __init__(self):
         # Get docker client
         self.client = docker.from_env()
 
-    def deploy_flow(self,
-                    flow,
-                    name=None,
-                    vols=None,
-                    engine='minifi-cpp'):
+    def __del__(self):
+        """
+        Clean up ephemeral cluster resources
+        """
+
+        # Clean up containers
+        for container in self.containers.values():
+            logging.info('Cleaning up container: %s', container.name)

Review comment:
       Removed. Also I see that it is ugly as hell to do part of the deletion outside the object that is expected to manage docker container lifetimes. However, if I add deletion here, I would still be able to access the container via the python API, but if if I tried calling delete on it, it would crash. I wonder if there is a way to sync up the list of available containers.




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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass

Review comment:
       I would extract the removal of the container into a separate function and return the result if it was successful or not and check that.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/environment.py
##########
@@ -0,0 +1,36 @@
+from behave import fixture, use_fixture
+import sys
+sys.path.append('../minifi')
+import logging
+
+from MiNiFi_integration_test_driver import MiNiFi_integration_test
+from minifi import *
+
+def raise_exception(exception):
+    raise exception
+
+@fixture
+def test_driver_fixture(context):
+    logging.info("Integration test setup")
+    context.test = MiNiFi_integration_test(context)
+    yield context.test
+    logging.info("Integration test teardown...")
+    del context.test
+
+def before_scenario(context, scenario):
+    use_fixture(test_driver_fixture, context)
+
+def after_scenario(context, scenario):
+	pass
+
+# @fixture
+# def setup_minifi_instance(context):
+#     print("Setup", end="\n\n")
+#     context.flow = None
+#     context.test = MiNiFi_integration_test(context)
+
+def before_all(context):
+    context.config.setup_logging()
+
+# def before_scenario(context, scenario):
+# 	use_fixture(setup_minifi_instance, context)

Review comment:
       Removed.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/minifi/core/SSL_cert_utils.py
##########
@@ -0,0 +1,54 @@
+import time
+import logging
+
+from M2Crypto import X509, EVP, RSA, ASN1
+
+def gen_cert():
+    """
+    Generate TLS certificate request for testing
+    """
+
+    req, key = gen_req()
+    pub_key = req.get_pubkey()
+    subject = req.get_subject()
+    cert = X509.X509()
+    # noinspection PyTypeChecker
+    cert.set_serial_number(1)
+    cert.set_version(2)
+    cert.set_subject(subject)
+    t = int(time.time())
+    now = ASN1.ASN1_UTCTIME()
+    now.set_time(t)
+    now_plus_year = ASN1.ASN1_UTCTIME()
+    now_plus_year.set_time(t + 60 * 60 * 24 * 365)
+    cert.set_not_before(now)
+    cert.set_not_after(now_plus_year)
+    issuer = X509.X509_Name()
+    issuer.C = 'US'
+    issuer.CN = 'minifi-listen'
+    cert.set_issuer(issuer)
+    cert.set_pubkey(pub_key)
+    cert.sign(key, 'sha256')
+
+    return cert, key
+
+def rsa_gen_key_callback():
+        pass

Review comment:
       Updated.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable

Review comment:
       Updated comment.




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

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



[GitHub] [nifi-minifi-cpp] arpadboda closed pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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


   


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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass
+
+        del self.docker_directory_bindings
+
+    def docker_path_to_local_path(self, docker_path):
+        return self.docker_directory_bindings.docker_path_to_local_path(self.test_id, docker_path)
+
+    def get_test_id(self):
+        return self.test_id
+
+    def acquire_cluster(self, name):
+        return self.clusters.setdefault(name, DockerTestCluster())
+
+    def set_up_cluster_network(self):
+        self.docker_network = SingleNodeDockerCluster.create_docker_network()
+        for cluster in self.clusters.values():
+            cluster.set_network(self.docker_network)
+
+    def start(self):
+        logging.info("MiNiFi_integration_test start")
+        self.set_up_cluster_network()
+        for cluster in self.clusters.values():
+            cluster.set_directory_bindings(self.docker_directory_bindings.get_directory_bindings(self.test_id))
+            cluster.deploy_flow()
+        for cluster_name, cluster in self.clusters.items():
+            startup_success = True
+            logging.info("Engine: %s", cluster.get_engine())
+            if cluster.get_engine() == "minifi-cpp":
+                logging.info("Engine is minifi-cpp")
+                startup_success = cluster.wait_for_app_logs("Starting Flow Controller", 120)
+            elif cluster.get_engine() == "nifi":
+                startup_success = cluster.wait_for_app_logs("Starting Flow Controller...", 120)
+            elif cluster.get_engine() == "kafka-broker":
+                startup_success = cluster.wait_for_app_logs("Startup complete.", 120)
+            if not startup_success:
+                cluster.log_nifi_output()
+            assert startup_success
+
+    def add_node(self, processor):
+        if processor.get_name() in (elem.get_name() for elem in self.connectable_nodes):
+            raise Exception("Trying to register processor with an already registered name: \"%s\"" % processor.get_name())
+        self.connectable_nodes.append(processor)
+
+    def get_or_create_node_by_name(self, node_name):
+        node = self.get_node_by_name(node_name) 
+        if node == None:

Review comment:
       Updated.




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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()

Review comment:
       Yes, probably. I was just wondering if we need to sync up additionally before checking the containers list size, but if not that's okay.




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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/steps/steps.py
##########
@@ -0,0 +1,243 @@
+from MiNiFi_integration_test_driver import MiNiFi_integration_test

Review comment:
       I see, thanks for the info.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()

Review comment:
       Removed the extra kill.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass

Review comment:
       All right, but the problem is that you do not know if the removal succeeded unless you query for the same container again. The extracted function would look like this:
   ```python
   def delete_docker_container_by_id(container_id):
     try:
       container = docker_client.containers.get(container_id)
       container.remove(v=True, force=True)
     except docker.errors.NotFound:
       logging.error("Attempt to clean up a contaner that has already been removed before.")
       return True
     try:
       docker_client.containers.get(container_id)
       logging.error("Failure when trying to clean up containers %s.", container_id)
       return False
     except docker.errors.NotFound:
       logging.info("Docker container removal successful for container %d", container_id)
       return True
   
   ```
   This is just as bad is it not?




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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/minifi/core/DockerTestCluster.py
##########
@@ -198,38 +110,20 @@ def is_s3_bucket_empty(self):
         ls_result = subprocess.check_output(["docker", "exec", "s3-server", "ls", s3_mock_dir + "/test_bucket/"]).decode(sys.stdout.encoding)
         return not ls_result
 
-    def rm_out_child(self, dir):
-        logging.info('Removing %s from output folder', os.path.join(self.tmp_test_output_dir, dir))
-        shutil.rmtree(os.path.join(self.tmp_test_output_dir, dir))
-
     def wait_for_container_logs(self, container_name, log, timeout, count=1):
         logging.info('Waiting for logs `%s` in container `%s`', log, container_name)
         container = self.containers[container_name]
         check_count = 0
         while check_count <= timeout:
-            if container.logs().decode("utf-8").count(log) == count:
+            if count <= container.logs().decode("utf-8").count(log):
                 return True
             else:
                 check_count += 1
                 time.sleep(1)
         return False
-    
-    def put_file_contents(self, contents, file_abs_path):
-        logging.info('Writing %d bytes of content to file: %s', len(contents), file_abs_path)
-        with open(file_abs_path, 'wb') as test_input_file:
-            test_input_file.write(contents)
 
+    def segfault_happened(self):
+        return self.segfault
 
     def __exit__(self, exc_type, exc_val, exc_tb):

Review comment:
       Thanks the cleanup seems to be working now as expected.




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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()

Review comment:
       Yes, probably. I was just wondering if we need to sync up additionally before checking the containers list size, but if not that's okay.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass

Review comment:
       All right, but then the extracted function would look like this:
   ```python
   def delete_docker_container_by_id(container_id)
     try:
       container = docker_client.containers.get(container_id)
       container.remove(v=True, force=True)
     except docker.errors.NotFound:
       logging.error("Attempt to clean up a contaner that has already been removed before.")
       return True
     try:
       docker_client.containers.get(container_id)
       logging.error("Failure when trying to clean up containers %s.", container_id)
       return False
     except docker.errors.NotFound:
       logging.info("Docker container removal successful for container %d", container_id)
       return True
   
   ```
   This looks just as bad does it not?

##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass

Review comment:
       All right, but then the extracted function would look like this:
   ```python
   def delete_docker_container_by_id(container_id):
     try:
       container = docker_client.containers.get(container_id)
       container.remove(v=True, force=True)
     except docker.errors.NotFound:
       logging.error("Attempt to clean up a contaner that has already been removed before.")
       return True
     try:
       docker_client.containers.get(container_id)
       logging.error("Failure when trying to clean up containers %s.", container_id)
       return False
     except docker.errors.NotFound:
       logging.info("Docker container removal successful for container %d", container_id)
       return True
   
   ```
   This looks just as bad does it not?




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass

Review comment:
       Updated




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/steps/steps.py
##########
@@ -0,0 +1,243 @@
+from MiNiFi_integration_test_driver import MiNiFi_integration_test

Review comment:
       Yes, but it makes the directory structure worse, as each feature would need to live in its own subdirectory as described here:
   https://stackoverflow.com/questions/36613050/modular-structure-for-behave-tests
   If we need further structural changes, let's do it in another PR.




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass

Review comment:
       All right, but the problem is that you do not know if the removal succeeded unless you query for the same container again. The extracted function would look like this:
   ```python
   def delete_docker_container_by_id(container_id):
     try:
       container = docker_client.containers.get(container_id)
       container.remove(v=True, force=True)
     except docker.errors.NotFound:
       logging.error("Attempt to clean up a contaner that has already been removed before.")
       return True
     try:
       docker_client.containers.get(container_id)
       logging.error("Failure when trying to clean up containers %s.", container_id)
       return False
     except docker.errors.NotFound:
       logging.info("Docker container removal successful for container %d", container_id)
       return True
   
   ```
   This looks just as bad does it not?




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

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



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/MiNiFi_integration_test_driver.py
##########
@@ -0,0 +1,208 @@
+from subprocess import Popen, PIPE, STDOUT
+
+import docker
+import logging
+import os
+import shutil
+import threading
+import time
+import uuid
+
+from pydoc import locate
+
+from minifi.core.InputPort import InputPort
+
+from minifi.core.DockerTestCluster import DockerTestCluster
+from minifi.core.SingleNodeDockerCluster import SingleNodeDockerCluster
+from minifi.core.DockerTestDirectoryBindings import DockerTestDirectoryBindings
+
+from minifi.validators.EmptyFilesOutPutValidator import EmptyFilesOutPutValidator
+from minifi.validators.NoFileOutPutValidator import NoFileOutPutValidator
+from minifi.validators.SingleFileOutputValidator import SingleFileOutputValidator
+
+class MiNiFi_integration_test():
+    def __init__(self, context):
+        logging.info("MiNiFi_integration_test init")
+        self.test_id = str(uuid.uuid4())
+        self.clusters = {}
+
+        self.connectable_nodes = []
+        # Remote process groups are not connectables
+        self.remote_process_groups = []
+        self.file_system_observer = None
+
+        self.docker_network = None
+
+        self.docker_directory_bindings = DockerTestDirectoryBindings()
+        self.docker_directory_bindings.create_new_data_directories(self.test_id)
+
+    def __del__(self):
+        logging.info("MiNiFi_integration_test cleanup")
+
+        # Clean up network, for some reason only this order of events work for cleanup
+        if self.docker_network is not None:
+            logging.info('Cleaning up network network: %s', self.docker_network.name)
+            while len(self.docker_network.containers) != 0:
+                for container in self.docker_network.containers:
+                    self.docker_network.disconnect(container, force=True)
+                self.docker_network.reload()
+            self.docker_network.remove()
+
+        container_ids = []
+        for cluster in self.clusters.values():
+            for container in cluster.containers.values():
+                container_ids.append(container.id)
+            del cluster
+
+        # Backup for cleaning up containers as the cluster deleter is not reliable
+        docker_client = docker.from_env()
+        for container_id in container_ids:    
+            wait_start_time = time.perf_counter()
+            while (time.perf_counter() - wait_start_time) < 35:
+                # There is no clean way to check for container existence
+                try:
+                    container = docker_client.containers.get(container_id)
+                    logging.error("Failure when trying to clean up containers. Attempting secondary cleanup.")
+                    container.kill()
+                    time.sleep(5)
+                    container.remove(v=True, force=True)
+                    time.sleep(5)
+                except docker.errors.NotFound:
+                    break
+            try:
+                container = docker_client.containers.get(container_id)
+                logging.error("All attempts to clean up docker containers were unsuccessful.")
+            except docker.errors.NotFound:
+                logging.info("Docker container secondary cleanup successful.")
+                pass

Review comment:
       I think it is better to encapsulate it, with the retries as well. It seems that the only thing we do with the result is write a log entry about it. That could be merged in the function, and we do not need a return value either. The first log entry could be changed to a warn as it does not show an error here. I changed the message a bit as we do not talk about the cleanup of all the containers but a single container.
   
   ```python
   def delete_docker_container_by_id(container_id):
       try:
           container = docker_client.containers.get(container_id)
           container.remove(v=True, force=True)
       except docker.errors.NotFound:
           logging.warn("Contaner '%s' is already cleaned up before.", container_id)
           return
       while (time.perf_counter() - wait_start_time) < 35:
           try:
               docker_client.containers.get(container_id)
               logging.error("Docker container '%s' still exists after remove. Waiting for docker daemon...", container_id)
               time.sleep(5)
           except docker.errors.NotFound:
               logging.info("Docker container cleanup successful for '%s'.", container_id)
               return
       logging.error("Failed to clean up docker container '%s'.", container_id)
   ```




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

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



[GitHub] [nifi-minifi-cpp] hunyadi-dev commented on a change in pull request #995: MINIFICPP-1445 - Move docker integration tests to python behave

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



##########
File path: docker/test/integration/minifi/core/DockerTestCluster.py
##########
@@ -198,38 +110,20 @@ def is_s3_bucket_empty(self):
         ls_result = subprocess.check_output(["docker", "exec", "s3-server", "ls", s3_mock_dir + "/test_bucket/"]).decode(sys.stdout.encoding)
         return not ls_result
 
-    def rm_out_child(self, dir):
-        logging.info('Removing %s from output folder', os.path.join(self.tmp_test_output_dir, dir))
-        shutil.rmtree(os.path.join(self.tmp_test_output_dir, dir))
-
     def wait_for_container_logs(self, container_name, log, timeout, count=1):
         logging.info('Waiting for logs `%s` in container `%s`', log, container_name)
         container = self.containers[container_name]
         check_count = 0
         while check_count <= timeout:
-            if container.logs().decode("utf-8").count(log) == count:
+            if count <= container.logs().decode("utf-8").count(log):
                 return True
             else:
                 check_count += 1
                 time.sleep(1)
         return False
-    
-    def put_file_contents(self, contents, file_abs_path):
-        logging.info('Writing %d bytes of content to file: %s', len(contents), file_abs_path)
-        with open(file_abs_path, 'wb') as test_input_file:
-            test_input_file.write(contents)
 
+    def segfault_happened(self):
+        return self.segfault
 
     def __exit__(self, exc_type, exc_val, exc_tb):

Review comment:
       As far as I understand cleanup on clusters is expected to happen on test exit regardless of the test state. This `__exit__` is probably useless, will remove it.




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

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