Pārlūkot izejas kodu

HDDS-1887. Enable all the blockade test-cases. Contibuted by NandaKumar(#1206).

(cherry picked from commit eaf350c414e3fdb8f8654cc695b54b471c094ca7)
Nanda kumar 5 gadi atpakaļ
vecāks
revīzija
daf869ca79

+ 11 - 1
hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/ozone/cluster.py

@@ -230,6 +230,16 @@ class OzoneCluster(object):
             raise ContainerNotFoundError(container_id)
         return Container(container_id, self)
 
+    def is_container_replica_exist(self, container_id, datanode):
+        container_parent_path = "%s/hdds/%s/current/containerDir0" % \
+                                (self.datanode_dir, self.scm_uuid)
+        command = "find %s -type f -name '%s.container'" % (container_parent_path, container_id)
+        exit_code, output = util.run_docker_command(command, datanode)
+        container_path = output.strip()
+        if not container_path:
+            return False
+        return True
+
     def get_containers_on_datanode(self, datanode):
         """
         Returns all the container on given datanode.
@@ -284,7 +294,7 @@ class OzoneCluster(object):
                                     (self.datanode_dir, self.scm_uuid)
             command = "find %s -type f -name '%s.container'" % (container_parent_path, container_id)
             exit_code, output = util.run_docker_command(command, datanode)
-            if exit_code == 0:
+            if output.strip():
                 result.append(datanode)
         return result
 

+ 3 - 0
hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/ozone/container.py

@@ -26,6 +26,9 @@ class Container:
         self.container_id = container_id
         self.cluster = cluster
 
+    def is_on(self, datanode):
+        return self.cluster.is_container_replica_exist(self.container_id, datanode)
+
     def get_datanode_states(self):
         dns = self.cluster.get_container_datanodes(self.container_id)
         states = []

+ 0 - 4
hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/test_blockade_client_failure.py

@@ -19,7 +19,6 @@ import re
 import time
 import logging
 import ozone.util
-import pytest
 
 from ozone.cluster import OzoneCluster
 
@@ -36,8 +35,6 @@ def teardown_function():
     cluster.stop()
 
 
-@pytest.mark.skip(reason="The test-case fails intermittently."
-                         "See HDDS-1817 for more info.")
 def test_client_failure_isolate_two_datanodes():
     """
     In this test, all DNs are isolated from each other.
@@ -79,7 +76,6 @@ def test_client_failure_isolate_two_datanodes():
     assert file_checksum == key_checksum
 
 
-@pytest.mark.skip(reason="HDDS-1817")
 def test_client_failure_isolate_one_datanode():
     """
     In this test, one of the DNs is isolated from all other nodes.

+ 15 - 7
hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/test_blockade_datanode_isolation.py

@@ -16,9 +16,9 @@
 # limitations under the License.
 
 import logging
-import pytest
 
 from ozone.cluster import OzoneCluster
+from ozone.exceptions import ContainerNotFoundError
 
 logger = logging.getLogger(__name__)
 
@@ -33,7 +33,6 @@ def teardown_function():
     cluster.stop()
 
 
-@pytest.mark.skip(reason="HDDS-1850")
 def test_isolate_single_datanode():
     """
     In this test case we will create a network partition in such a way that
@@ -68,15 +67,25 @@ def test_isolate_single_datanode():
 
     oz_client.run_freon(1, 1, 1, 10240)
 
-    logger.info("Waiting for container to be QUASI_CLOSED")
-    containers = cluster.get_containers_on_datanode(dns[2])
+    containers = cluster.get_containers_on_datanode(dns[0])
+
+    # The same set of containers should also be in datanode[2]
+
+    for container in containers:
+        assert container.is_on(dns[2])
+
+    logger.info("Waiting for container to be CLOSED")
     for container in containers:
-        container.wait_until_replica_is_quasi_closed(dns[2])
+        container.wait_until_one_replica_is_closed()
 
     for container in containers:
         assert container.get_state(dns[0]) == 'CLOSED'
         assert container.get_state(dns[1]) == 'CLOSED'
-        assert container.get_state(dns[2]) == 'QUASI_CLOSED'
+        try:
+            assert container.get_state(dns[2]) == 'CLOSING' or \
+                   container.get_state(dns[2]) == 'QUASI_CLOSED'
+        except ContainerNotFoundError:
+            assert True
 
     # Since the replica in datanode[2] doesn't have the latest BCSID,
     # ReplicationManager will delete it and copy a closed replica.
@@ -98,7 +107,6 @@ def test_isolate_single_datanode():
     assert exit_code == 0, "freon run failed with output=[%s]" % output
 
 
-@pytest.mark.skip(reason="RATIS-615")
 def test_datanode_isolation_all():
     """
     In this test case we will create a network partition in such a way that

+ 4 - 6
hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/test_blockade_flaky.py

@@ -36,15 +36,13 @@ def teardown_function():
     cluster.stop()
 
 
-@pytest.mark.parametrize("flaky_node", ["datanode", "scm", "om"])
+@pytest.mark.parametrize("flaky_node", ["datanode"])
 def test_flaky(flaky_node):
     """
     In these tests, we make the network of the nodes as flaky using blockade.
-    There are 4 tests :
-    1) one of the DNs selected randomly and network of the DN is made flaky.
-    2) scm network is made flaky.
-    3) om network is made flaky.
-    4) Network of all the nodes are made flaky.
+    One of the DNs selected randomly and network of the DN is made flaky.
+
+    Once HA is in place, we can go ahead and make OM and SCM network flaky.
 
     """
     flaky_container_name = {

+ 0 - 2
hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/test_blockade_mixed_failure.py

@@ -16,7 +16,6 @@
 # limitations under the License.
 
 import logging
-import pytest
 
 from ozone.cluster import OzoneCluster
 
@@ -73,7 +72,6 @@ def test_one_dn_isolate_scm_other_dn():
     assert exit_code == 0, "freon run failed with output=[%s]" % output
 
 
-@pytest.mark.skip(reason="HDDS-1850")
 def test_one_dn_isolate_other_dn():
     """
     In this test, one of the DNs (first DN) cannot communicate

+ 0 - 2
hadoop-ozone/fault-injection-test/network-tests/src/test/blockade/test_blockade_mixed_failure_two_nodes.py

@@ -16,7 +16,6 @@
 # limitations under the License.
 
 import logging
-import pytest
 
 from ozone.cluster import OzoneCluster
 
@@ -33,7 +32,6 @@ def teardown_function():
     cluster.stop()
 
 
-@pytest.mark.skip(reason="HDDS-1850")
 def test_two_dns_isolate_scm_same_partition():
     """
     In this test, there are three DNs,