|
@@ -16,132 +16,123 @@
|
|
# limitations under the License.
|
|
# limitations under the License.
|
|
|
|
|
|
import os
|
|
import os
|
|
-import time
|
|
|
|
-import re
|
|
|
|
import logging
|
|
import logging
|
|
-from blockadeUtils.blockade import Blockade
|
|
|
|
-from clusterUtils.cluster_utils import ClusterUtils
|
|
|
|
|
|
+import util
|
|
|
|
+from ozone.cluster import Cluster
|
|
|
|
|
|
logger = logging.getLogger(__name__)
|
|
logger = logging.getLogger(__name__)
|
|
-parent_dir = os.path.dirname(os.path.dirname(os.path.realpath(__file__)))
|
|
|
|
-FILE = os.path.join(parent_dir, "compose", "ozoneblockade",
|
|
|
|
- "docker-compose.yaml")
|
|
|
|
-os.environ["DOCKER_COMPOSE_FILE"] = FILE
|
|
|
|
-SCALE = 3
|
|
|
|
-INCREASED_SCALE = 5
|
|
|
|
-CONTAINER_LIST = []
|
|
|
|
-OM = []
|
|
|
|
-SCM = []
|
|
|
|
-DATANODES = []
|
|
|
|
|
|
|
|
|
|
|
|
-def setup():
|
|
|
|
- global CONTAINER_LIST, OM, SCM, DATANODES
|
|
|
|
- Blockade.blockade_destroy()
|
|
|
|
- CONTAINER_LIST = ClusterUtils.cluster_setup(FILE, SCALE)
|
|
|
|
- exit_code, output = Blockade.blockade_status()
|
|
|
|
- assert exit_code == 0, "blockade status command failed with output=[%s]" % \
|
|
|
|
- output
|
|
|
|
- OM, SCM, _, DATANODES = \
|
|
|
|
- ClusterUtils.find_om_scm_client_datanodes(CONTAINER_LIST)
|
|
|
|
|
|
+def setup_function(function):
|
|
|
|
+ global cluster
|
|
|
|
+ cluster = Cluster.create()
|
|
|
|
+ cluster.start()
|
|
|
|
|
|
- exit_code, output = ClusterUtils.run_freon(FILE, 1, 1, 1, 10240, "RATIS",
|
|
|
|
- "THREE")
|
|
|
|
- assert exit_code == 0, "freon run failed with output=[%s]" % output
|
|
|
|
|
|
|
|
|
|
+def teardown_function(function):
|
|
|
|
+ cluster.stop()
|
|
|
|
|
|
-def teardown():
|
|
|
|
- logger.info("Inside teardown")
|
|
|
|
- Blockade.blockade_destroy()
|
|
|
|
|
|
|
|
|
|
+def test_isolate_single_datanode():
|
|
|
|
+ """
|
|
|
|
+ In this test case we will create a network partition in such a way that
|
|
|
|
+ one of the datanode will not be able to communicate with other datanodes
|
|
|
|
+ but it will be able to communicate with SCM.
|
|
|
|
|
|
-def teardown_module():
|
|
|
|
- ClusterUtils.cluster_destroy(FILE)
|
|
|
|
|
|
+ Once the network partition happens, SCM detects it and closes the pipeline,
|
|
|
|
+ which in-turn closes the containers.
|
|
|
|
|
|
|
|
+ The container on the first two datanode will get CLOSED as they have quorum.
|
|
|
|
+ The container replica on the third node will be QUASI_CLOSED as it is not
|
|
|
|
+ able to connect with the other datanodes and it doesn't have latest BCSID.
|
|
|
|
+
|
|
|
|
+ Once we restore the network, the stale replica on the third datanode will be
|
|
|
|
+ deleted and a latest replica will be copied from any one of the other
|
|
|
|
+ datanodes.
|
|
|
|
|
|
-def test_isolatedatanode_singlenode(run_second_phase):
|
|
|
|
- """
|
|
|
|
- In this test, one of the datanodes (first datanode) cannot communicate
|
|
|
|
- with other two datanodes.
|
|
|
|
- All datanodes can communicate with SCM.
|
|
|
|
- Expectation :
|
|
|
|
- The container replica state in first datanode should be quasi-closed.
|
|
|
|
- The container replica state in other datanodes should be closed.
|
|
|
|
"""
|
|
"""
|
|
- first_set = [OM[0], SCM[0], DATANODES[0]]
|
|
|
|
- second_set = [OM[0], SCM[0], DATANODES[1], DATANODES[2]]
|
|
|
|
- Blockade.blockade_create_partition(first_set, second_set)
|
|
|
|
- Blockade.blockade_status()
|
|
|
|
- ClusterUtils.run_freon(FILE, 1, 1, 1, 10240, "RATIS", "THREE")
|
|
|
|
- logger.info("Waiting for %s seconds before checking container status",
|
|
|
|
- os.environ["CONTAINER_STATUS_SLEEP"])
|
|
|
|
- time.sleep(int(os.environ["CONTAINER_STATUS_SLEEP"]))
|
|
|
|
- all_datanodes_container_status = \
|
|
|
|
- ClusterUtils.findall_container_status(FILE, SCALE)
|
|
|
|
- first_datanode_status = all_datanodes_container_status[0]
|
|
|
|
- closed_container_datanodes = [x for x in all_datanodes_container_status
|
|
|
|
- if x == 'CLOSED']
|
|
|
|
- assert first_datanode_status == 'QUASI_CLOSED'
|
|
|
|
- assert len(closed_container_datanodes) == 2, \
|
|
|
|
- "The container should have two closed replicas."
|
|
|
|
-
|
|
|
|
- if str(run_second_phase).lower() == "true":
|
|
|
|
- ClusterUtils.cluster_setup(FILE, INCREASED_SCALE, False)
|
|
|
|
- Blockade.blockade_status()
|
|
|
|
- logger.info("Waiting for %s seconds before checking container status",
|
|
|
|
- os.environ["CONTAINER_STATUS_SLEEP"])
|
|
|
|
- time.sleep(int(os.environ["CONTAINER_STATUS_SLEEP"]))
|
|
|
|
- all_datanodes_container_status = \
|
|
|
|
- ClusterUtils.findall_container_status(FILE, INCREASED_SCALE)
|
|
|
|
- closed_container_datanodes = [x for x in all_datanodes_container_status
|
|
|
|
- if x == 'CLOSED']
|
|
|
|
- assert len(closed_container_datanodes) >= 3, \
|
|
|
|
- "The container should have at least three closed replicas."
|
|
|
|
- Blockade.blockade_join()
|
|
|
|
- Blockade.blockade_status()
|
|
|
|
- _, output = \
|
|
|
|
- ClusterUtils.run_freon(FILE, 1, 1, 1, 10240, "RATIS", "THREE")
|
|
|
|
- assert re.search("Status: Success", output) is not None
|
|
|
|
-
|
|
|
|
-
|
|
|
|
-def test_datanode_isolation_all(run_second_phase):
|
|
|
|
|
|
+ cluster.run_freon(1, 1, 1, 10240)
|
|
|
|
+ first_set = [cluster.om, cluster.scm,
|
|
|
|
+ cluster.datanodes[0], cluster.datanodes[1]]
|
|
|
|
+ second_set = [cluster.om, cluster.scm, cluster.datanodes[2]]
|
|
|
|
+ logger.info("Partitioning the network")
|
|
|
|
+ cluster.partition_network(first_set, second_set)
|
|
|
|
+ cluster.run_freon(1, 1, 1, 10240)
|
|
|
|
+ logger.info("Waiting for container to be QUASI_CLOSED")
|
|
|
|
+
|
|
|
|
+ util.wait_until(lambda: cluster.get_container_states(cluster.datanodes[2])
|
|
|
|
+ .popitem()[1] == 'QUASI_CLOSED',
|
|
|
|
+ int(os.environ["CONTAINER_STATUS_SLEEP"]), 10)
|
|
|
|
+ container_states_dn_0 = cluster.get_container_states(cluster.datanodes[0])
|
|
|
|
+ container_states_dn_1 = cluster.get_container_states(cluster.datanodes[1])
|
|
|
|
+ container_states_dn_2 = cluster.get_container_states(cluster.datanodes[2])
|
|
|
|
+ assert len(container_states_dn_0) != 0
|
|
|
|
+ assert len(container_states_dn_1) != 0
|
|
|
|
+ assert len(container_states_dn_2) != 0
|
|
|
|
+ for key in container_states_dn_0:
|
|
|
|
+ assert container_states_dn_0.get(key) == 'CLOSED'
|
|
|
|
+ for key in container_states_dn_1:
|
|
|
|
+ assert container_states_dn_1.get(key) == 'CLOSED'
|
|
|
|
+ for key in container_states_dn_2:
|
|
|
|
+ assert container_states_dn_2.get(key) == 'QUASI_CLOSED'
|
|
|
|
+
|
|
|
|
+ # Since the replica in datanode[2] doesn't have the latest BCSID,
|
|
|
|
+ # ReplicationManager will delete it and copy a closed replica.
|
|
|
|
+ # We will now restore the network and datanode[2] should get a
|
|
|
|
+ # closed replica of the container
|
|
|
|
+ logger.info("Restoring the network")
|
|
|
|
+ cluster.restore_network()
|
|
|
|
+
|
|
|
|
+ logger.info("Waiting for the replica to be CLOSED")
|
|
|
|
+ util.wait_until(
|
|
|
|
+ lambda: cluster.container_state_predicate(cluster.datanodes[2], 'CLOSED'),
|
|
|
|
+ int(os.environ["CONTAINER_STATUS_SLEEP"]), 10)
|
|
|
|
+ container_states_dn_2 = cluster.get_container_states(cluster.datanodes[2])
|
|
|
|
+ assert len(container_states_dn_2) != 0
|
|
|
|
+ for key in container_states_dn_2:
|
|
|
|
+ assert container_states_dn_2.get(key) == 'CLOSED'
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+def test_datanode_isolation_all():
|
|
"""
|
|
"""
|
|
- In this test, none of the datanodes can communicate with other two
|
|
|
|
- datanodes.
|
|
|
|
- All datanodes can communicate with SCM.
|
|
|
|
- Expectation : The container should eventually have at least two closed
|
|
|
|
- replicas.
|
|
|
|
|
|
+ In this test case we will create a network partition in such a way that
|
|
|
|
+ all datanodes cannot communicate with each other.
|
|
|
|
+ All datanodes will be able to communicate with SCM.
|
|
|
|
+
|
|
|
|
+ Once the network partition happens, SCM detects it and closes the pipeline,
|
|
|
|
+ which in-turn tries to close the containers.
|
|
|
|
+ At least one of the replica should be in closed state
|
|
|
|
+
|
|
|
|
+ Once we restore the network, there will be three closed replicas.
|
|
|
|
+
|
|
"""
|
|
"""
|
|
- first_set = [OM[0], SCM[0], DATANODES[0]]
|
|
|
|
- second_set = [OM[0], SCM[0], DATANODES[1]]
|
|
|
|
- third_set = [OM[0], SCM[0], DATANODES[2]]
|
|
|
|
- Blockade.blockade_create_partition(first_set, second_set, third_set)
|
|
|
|
- Blockade.blockade_status()
|
|
|
|
- ClusterUtils.run_freon(FILE, 1, 1, 1, 10240, "RATIS", "THREE")
|
|
|
|
- logger.info("Waiting for %s seconds before checking container status",
|
|
|
|
- os.environ["CONTAINER_STATUS_SLEEP"])
|
|
|
|
- time.sleep(int(os.environ["CONTAINER_STATUS_SLEEP"]))
|
|
|
|
- all_datanodes_container_status = \
|
|
|
|
- ClusterUtils.findall_container_status(FILE, SCALE)
|
|
|
|
- closed_container_datanodes = [x for x in all_datanodes_container_status
|
|
|
|
- if x == 'CLOSED']
|
|
|
|
- assert len(closed_container_datanodes) >= 2, \
|
|
|
|
- "The container should have at least two closed replicas."
|
|
|
|
-
|
|
|
|
- if str(run_second_phase).lower() == "true":
|
|
|
|
- ClusterUtils.cluster_setup(FILE, INCREASED_SCALE, False)
|
|
|
|
- Blockade.blockade_status()
|
|
|
|
- logger.info("Waiting for %s seconds before checking container status",
|
|
|
|
- os.environ["CONTAINER_STATUS_SLEEP"])
|
|
|
|
- time.sleep(int(os.environ["CONTAINER_STATUS_SLEEP"]))
|
|
|
|
- all_datanodes_container_status = \
|
|
|
|
- ClusterUtils.findall_container_status(FILE, INCREASED_SCALE)
|
|
|
|
- closed_container_datanodes = [x for x in all_datanodes_container_status
|
|
|
|
- if x == 'CLOSED']
|
|
|
|
- assert len(closed_container_datanodes) >= 3, \
|
|
|
|
- "The container should have at least three closed replicas."
|
|
|
|
- Blockade.blockade_join()
|
|
|
|
- Blockade.blockade_status()
|
|
|
|
- _, output = \
|
|
|
|
- ClusterUtils.run_freon(FILE, 1, 1, 1, 10240, "RATIS", "THREE")
|
|
|
|
- assert re.search("Status: Success", output) is not None
|
|
|
|
|
|
+ cluster.run_freon(1, 1, 1, 10240)
|
|
|
|
+
|
|
|
|
+ assert len(cluster.get_container_states(cluster.datanodes[0])) != 0
|
|
|
|
+ assert len(cluster.get_container_states(cluster.datanodes[1])) != 0
|
|
|
|
+ assert len(cluster.get_container_states(cluster.datanodes[2])) != 0
|
|
|
|
+
|
|
|
|
+ logger.info("Partitioning the network")
|
|
|
|
+ first_set = [cluster.om, cluster.scm, cluster.datanodes[0]]
|
|
|
|
+ second_set = [cluster.om, cluster.scm, cluster.datanodes[1]]
|
|
|
|
+ third_set = [cluster.om, cluster.scm, cluster.datanodes[2]]
|
|
|
|
+ cluster.partition_network(first_set, second_set, third_set)
|
|
|
|
+
|
|
|
|
+ logger.info("Waiting for the replica to be CLOSED")
|
|
|
|
+ util.wait_until(
|
|
|
|
+ lambda: cluster.container_state_predicate_one_closed(cluster.datanodes),
|
|
|
|
+ int(os.environ["CONTAINER_STATUS_SLEEP"]), 10)
|
|
|
|
+
|
|
|
|
+ # At least one of the replica should be in closed state
|
|
|
|
+ assert cluster.container_state_predicate_one_closed(cluster.datanodes)
|
|
|
|
+
|
|
|
|
+ # After restoring the network all the replicas should be in
|
|
|
|
+ # CLOSED state
|
|
|
|
+ logger.info("Restoring the network")
|
|
|
|
+ cluster.restore_network()
|
|
|
|
+
|
|
|
|
+ logger.info("Waiting for the container to be replicated")
|
|
|
|
+ util.wait_until(
|
|
|
|
+ lambda: cluster.container_state_predicate_all_closed(cluster.datanodes),
|
|
|
|
+ int(os.environ["CONTAINER_STATUS_SLEEP"]), 10)
|
|
|
|
+ assert cluster.container_state_predicate_all_closed(cluster.datanodes)
|