From a0ba3b3350dfe87429df20d78fea1e8883623b28 Mon Sep 17 00:00:00 2001 From: Nadav Har'El Date: Sun, 12 Feb 2023 12:13:25 +0200 Subject: [PATCH 01/28] Merge 'test.py: improve test failure handling' from Kamil Braun Improve logging by printing the cluster at the end of each test. Stop performing operations like attempting queries or dropping keyspaces on dirty clusters. Dirty clusters might be completely dead and these operations would only cause more "errors" to happen after a failed test, making it harder to find the real cause of failure. Mark cluster as dirty when a test that uses it fails - after a failed test, we shouldn't assume that the cluster is in a usable state, so we shouldn't reuse it for another test. Rely on the `is_dirty` flag in `PythonTest`s and `CQLApprovalTest`s, similarly to what `TopologyTest`s do. Closes #12652 * github.com:scylladb/scylladb: test.py: rely on ScyllaCluster.is_dirty flag for recycling clusters test/topology: don't drop random_tables keyspace after a failed test test/pylib: mark cluster as dirty after a failed test test: pylib, topology: don't perform operations after test on a dirty cluster test/pylib: print cluster at the end of test (cherry picked from commit 2653865b34d86ef338fa6f55fb29fe81675924bc) --- test.py | 11 +++++----- test/pylib/manager_client.py | 7 +++--- test/pylib/scylla_cluster.py | 31 ++++++++++++++++++--------- test/topology/conftest.py | 41 ++++++++++++++++++++++++++++++++---- 4 files changed, 67 insertions(+), 23 deletions(-) diff --git a/test.py b/test.py index 633aa3c755..0ff4ce0d45 100755 --- a/test.py +++ b/test.py @@ -696,7 +696,7 @@ class CQLApprovalTest(Test): logger.info("Server log:\n%s", self.server_log) # TODO: consider dirty_on_exception=True - async with self.suite.clusters.instance(False, logger) as cluster: + async with (cm := self.suite.clusters.instance(False, logger)) as cluster: try: cluster.before_test(self.uname) logger.info("Leasing Scylla cluster %s for test %s", cluster, self.uname) @@ -706,7 +706,8 @@ class CQLApprovalTest(Test): self.is_before_test_ok = True cluster.take_log_savepoint() self.is_executed_ok = await run_test(self, options, env=self.env) - cluster.after_test(self.uname) + cluster.after_test(self.uname, self.is_executed_ok) + cm.dirty = cluster.is_dirty self.is_after_test_ok = True if self.is_executed_ok is False: @@ -860,7 +861,7 @@ class PythonTest(Test): self.is_before_test_ok = True cluster.take_log_savepoint() status = await run_test(self, options) - cluster.after_test(self.uname) + cluster.after_test(self.uname, status) self.is_after_test_ok = True self.success = status except Exception as e: @@ -874,9 +875,7 @@ class PythonTest(Test): print("Test {} post-check failed: {}".format(self.name, str(e))) print("Server log of the first server:\n{}".format(self.server_log)) logger.info(f"Discarding cluster after failed test %s...", self.name) - await self.suite.clusters.put(cluster, is_dirty=True) - else: - await self.suite.clusters.put(cluster, is_dirty=False) + await self.suite.clusters.put(cluster, is_dirty=cluster.is_dirty) logger.info("Test %s %s", self.uname, "succeeded" if self.success else "failed ") return self diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index eabd958285..2834647294 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -86,10 +86,11 @@ class ManagerClient(): # await self._wait_for_cluster() await self.driver_connect() # Connect driver to new cluster - async def after_test(self, test_case_name: str) -> None: + async def after_test(self, test_case_name: str, success: bool) -> None: """Tell harness this test finished""" - logger.debug("after_test for %s", test_case_name) - await self.client.get(f"/cluster/after-test") + logger.debug("after_test for %s (success: %s)", test_case_name, success) + cluster_str = await self.client.get_text(f"/cluster/after-test/{success}") + logger.info("Cluster after test %s: %s", test_case_name, cluster_str) async def is_manager_up(self) -> bool: """Check if Manager server is up""" diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index b63166d7f3..1d64e4a8d5 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -730,18 +730,26 @@ class ScyllaCluster: if self.start_exception: # Mark as dirty so further test cases don't try to reuse this cluster. self.is_dirty = True - raise self.start_exception + raise Exception(f'Exception when starting cluster {self}:\n{self.start_exception}') for server in self.running.values(): server.write_log_marker(f"------ Starting test {name} ------\n") - def after_test(self, name) -> None: - """Check that the cluster is still alive and the test + def after_test(self, name: str, success: bool) -> None: + """Mark the cluster as dirty after a failed test. + If the cluster is not dirty, check that it's still alive and the test hasn't left any garbage.""" assert self.start_exception is None - if self._get_keyspace_count() != self.keyspace_count: - raise RuntimeError("Test post-condition failed, " - "the test must drop all keyspaces it creates.") + if not success: + self.logger.debug(f"Test failed using cluster {self.name}, marking the cluster as dirty") + self.is_dirty = True + if self.is_dirty: + self.logger.info(f"The cluster {self.name} is dirty, not checking" + f" keyspace count post-condition") + else: + if self._get_keyspace_count() != self.keyspace_count: + raise RuntimeError(f"Test post-condition on cluster {self.name} failed, " + f"the test must drop all keyspaces it creates.") for server in itertools.chain(self.running.values(), self.stopped.values()): server.write_log_marker(f"------ Ending test {name} ------\n") @@ -930,7 +938,7 @@ class ScyllaClusterManager: add_get('/cluster/host-ip/{server_id}', self._cluster_server_ip_addr) add_get('/cluster/host-id/{server_id}', self._cluster_host_id) add_get('/cluster/before-test/{test_case_name}', self._before_test_req) - add_get('/cluster/after-test', self._after_test) + add_get('/cluster/after-test/{success}', self._after_test) add_get('/cluster/mark-dirty', self._mark_dirty) add_get('/cluster/server/{server_id}/stop', self._cluster_server_stop) add_get('/cluster/server/{server_id}/stop_gracefully', self._cluster_server_stop_gracefully) @@ -982,13 +990,16 @@ class ScyllaClusterManager: async def _after_test(self, _request) -> aiohttp.web.Response: assert self.cluster is not None assert self.current_test_case_full_name - self.logger.info("Finished test %s, cluster: %s", self.current_test_case_full_name, self.cluster) + success = _request.match_info["success"] == "True" + self.logger.info("Test %s %s, cluster: %s", self.current_test_case_full_name, + "SUCCEEDED" if success else "FAILED", self.cluster) try: - self.cluster.after_test(self.current_test_case_full_name) + self.cluster.after_test(self.current_test_case_full_name, success) finally: self.current_test_case_full_name = '' self.is_after_test_ok = True - return aiohttp.web.Response(text="True") + cluster_str = str(self.cluster) + return aiohttp.web.Response(text=cluster_str) async def _mark_dirty(self, _request) -> aiohttp.web.Response: """Mark current cluster dirty""" diff --git a/test/topology/conftest.py b/test/topology/conftest.py index 82a7627e96..da83ee17ce 100644 --- a/test/topology/conftest.py +++ b/test/topology/conftest.py @@ -37,6 +37,26 @@ def pytest_addoption(parser): parser.addoption('--ssl', action='store_true', help='Connect to CQL via an encrypted TLSv1.2 connection') + +# This is a constant used in `pytest_runtest_makereport` below to store a flag +# indicating test failure in a stash which can then be accessed from fixtures. +FAILED_KEY = pytest.StashKey[bool]() + + +@pytest.hookimpl(tryfirst=True, hookwrapper=True) +def pytest_runtest_makereport(item, call): + """This is a post-test hook execucted by the pytest library. + Use it to access the test result and store a flag indicating failure + so we can later retrieve it in our fixtures like `manager`. + + `item.stash` is the same stash as `request.node.stash` (in the `request` + fixture provided by pytest). + """ + outcome = yield + report = outcome.get_result() + item.stash[FAILED_KEY] = report.when == "call" and report.failed + + # Change default pytest-asyncio event_loop fixture scope to session to # allow async fixtures with scope larger than function. (e.g. manager fixture) # See https://github.com/pytest-dev/pytest-asyncio/issues/68 @@ -159,7 +179,10 @@ async def manager(request, manager_internal): test_case_name = request.node.name await manager_internal.before_test(test_case_name) yield manager_internal - await manager_internal.after_test(test_case_name) + # `request.node.stash` contains a flag stored in `pytest_runtest_makereport` + # that indicates test failure. + failed = request.node.stash[FAILED_KEY] + await manager_internal.after_test(test_case_name, not failed) # "cql" fixture: set up client object for communicating with the CQL API. # Since connection is managed by manager just return that object @@ -192,9 +215,19 @@ def fails_without_consistent_cluster_management(request, check_pre_consistent_cl # "random_tables" fixture: Creates and returns a temporary RandomTables object -# used in tests to make schema changes. Tables are dropped after finished. +# used in tests to make schema changes. Tables are dropped after test finishes +# unless the cluster is dirty or the test has failed. @pytest.fixture(scope="function") -def random_tables(request, manager): +async def random_tables(request, manager): tables = RandomTables(request.node.name, manager, unique_name()) yield tables - tables.drop_all() + + # Don't drop tables at the end if we failed or the cluster is dirty - it may be impossible + # (e.g. the cluster is completely dead) and it doesn't matter (we won't reuse the cluster + # anyway). + # The cluster will be marked as dirty if the test failed, but that happens + # at the end of `manager` fixture which we depend on (so these steps will be + # executed after us) - so at this point, we need to check for failure ourselves too. + failed = request.node.stash[FAILED_KEY] + if not failed and not await manager.is_dirty(): + tables.drop_all() From 3aa73e8b5a43d4d1f02739e362aaa97be1dae765 Mon Sep 17 00:00:00 2001 From: Kamil Braun Date: Thu, 26 Jan 2023 12:07:12 +0100 Subject: [PATCH 02/28] Merge 'pytest: start after ungraceful stop' from Alecco If a server is stopped suddenly (i.e. not graceful), schema tables might be in inconsistent state. Add a test case and enable Scylla configuration option (force_schema_commit_log) to handle this. Fixes #12218 Closes #12630 * github.com:scylladb/scylladb: pytest: test start after ungraceful stop test.py: enable force_schema_commit_log (cherry picked from commit 5eadea301ecc0cffbd4a6b6bd89acc04e90c59ad) --- test/pylib/scylla_cluster.py | 2 ++ test/topology/test_topology.py | 11 +++++++++++ 2 files changed, 13 insertions(+) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 1d64e4a8d5..529cd9e1ea 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -99,6 +99,8 @@ def make_scylla_conf(workdir: pathlib.Path, host_addr: str, seed_addrs: List[str 'permissions_update_interval_in_ms': 100, 'permissions_validity_in_ms': 100, + + 'force_schema_commit_log': True, } # Seastar options can not be passed through scylla.yaml, use command line diff --git a/test/topology/test_topology.py b/test/topology/test_topology.py index e6b711f836..78739588b9 100644 --- a/test/topology/test_topology.py +++ b/test/topology/test_topology.py @@ -372,3 +372,14 @@ async def test_remove_node_with_concurrent_ddl(manager, random_tables): stopped = True await ddl_task logger.debug("ddl fiber done, finished") + + +@pytest.mark.asyncio +async def test_start_after_sudden_stop(manager: ManagerClient, random_tables) -> None: + """Tests a server can rejoin the cluster after being stopped suddenly""" + servers = await manager.running_servers() + table = await random_tables.add_table(ncolumns=5) + await manager.server_stop(servers[0].server_id) + await table.add_column() + await manager.server_start(servers[0].server_id) + await random_tables.verify_schema() From 05c3f7ecef065956f7e7ec629aea134a7378ba04 Mon Sep 17 00:00:00 2001 From: Kamil Braun Date: Mon, 23 Jan 2023 16:56:11 +0100 Subject: [PATCH 03/28] test: topology: verify that group 0 and token ring are consistent After topology changes like removing a node, verify that the set of group 0 members and token ring members is the same. Modify `get_token_ring_host_ids` to only return NORMAL members. The previous version which used the `/storage_service/host_id` endpoint might have returned non-NORMAL members as well. Fixes: #12153 Closes #12619 (cherry picked from commit fa9cf81af28af58ecf23d6a2d42d7b1c4ce727c7) --- test/topology/test_topology.py | 68 +++++++++++++++++++++++++++------- 1 file changed, 54 insertions(+), 14 deletions(-) diff --git a/test/topology/test_topology.py b/test/topology/test_topology.py index 78739588b9..889ee4415f 100644 --- a/test/topology/test_topology.py +++ b/test/topology/test_topology.py @@ -25,9 +25,16 @@ logger = logging.getLogger(__name__) async def get_token_ring_host_ids(manager: ManagerClient, srv: ServerInfo) -> set[str]: - """Get the host IDs of token ring members known by `srv`.""" + """Get the host IDs of normal token owners known by `srv`.""" + token_endpoint_map = await manager.api.client.get_json("/storage_service/tokens_endpoint", srv.ip_addr) + normal_endpoints = {e["value"] for e in token_endpoint_map} + logger.info(f"Normal endpoints' IPs by {srv}: {normal_endpoints}") host_id_map = await manager.api.client.get_json('/storage_service/host_id', srv.ip_addr) - return {e['value'] for e in host_id_map} + all_host_ids = {e["value"] for e in host_id_map} + logger.info(f"All host IDs by {srv}: {all_host_ids}") + normal_host_ids = {e["value"] for e in host_id_map if e["key"] in normal_endpoints} + logger.info(f"Normal endpoints' host IDs by {srv}: {normal_host_ids}") + return normal_host_ids async def get_current_group0_config(manager: ManagerClient, srv: ServerInfo) -> set[tuple[str, bool]]: @@ -43,7 +50,41 @@ async def get_current_group0_config(manager: ManagerClient, srv: ServerInfo) -> config = await manager.cql.run_async( f"select server_id, can_vote from system.raft_state where group_id = {group0_id} and disposition = 'CURRENT'", host=host) - return {(str(m.server_id), bool(m.can_vote)) for m in config} + result = {(str(m.server_id), bool(m.can_vote)) for m in config} + logger.info(f"Group 0 members by {srv}: {result}") + return result + + +async def check_token_ring_and_group0_consistency(manager: ManagerClient) -> None: + """Ensure that the normal token owners and group 0 members match + according to each currently running server. + """ + servers = await manager.running_servers() + for srv in servers: + group0_members = await get_current_group0_config(manager, srv) + group0_ids = {m[0] for m in group0_members} + token_ring_ids = await get_token_ring_host_ids(manager, srv) + assert token_ring_ids == group0_ids + + +async def wait_for_token_ring_and_group0_consistency(manager: ManagerClient, deadline: float) -> None: + """Weaker version of the above check; the token ring is not immediately updated + after bootstrap/replace - the normal tokens of the new node propagate through gossip. + Take this into account and wait for the equality condition to hold, with a timeout. + """ + servers = await manager.running_servers() + for srv in servers: + group0_members = await get_current_group0_config(manager, srv) + group0_ids = {m[0] for m in group0_members} + async def token_ring_matches(): + token_ring_ids = await get_token_ring_host_ids(manager, srv) + diff = token_ring_ids ^ group0_ids + if diff: + logger.warning(f"Group 0 members and token ring members don't yet match" \ + f" according to {srv}, symmetric difference: {diff}") + return None + return True + await wait_for(token_ring_matches, deadline, period=.5) @pytest.mark.asyncio @@ -51,6 +92,7 @@ async def test_add_server_add_column(manager, random_tables): """Add a node and then add a column to a table and verify""" table = await random_tables.add_table(ncolumns=5) await manager.server_add() + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) await table.add_column() await random_tables.verify_schema() @@ -84,9 +126,9 @@ async def test_remove_node_add_column(manager, random_tables): await manager.server_add() await manager.server_stop_gracefully(servers[1].server_id) # stop [1] await manager.remove_node(servers[0].server_id, servers[1].server_id) # Remove [1] + await check_token_ring_and_group0_consistency(manager) await table.add_column() await random_tables.verify_schema() - # TODO: check that group 0 no longer contains the removed node (#12153) @pytest.mark.asyncio @@ -117,9 +159,9 @@ async def test_decommission_node_add_column(manager, random_tables): await manager.api.enable_injection( bootstrapped_server.ip_addr, 'storage_service_decommission_prepare_handler_sleep', one_shot=True) await manager.decommission_node(decommission_target.server_id) + await check_token_ring_and_group0_consistency(manager) await table.add_column() await random_tables.verify_schema() - # TODO: check that group 0 no longer contains the decommissioned node (#12153) @pytest.mark.asyncio async def test_replace_different_ip(manager: ManagerClient, random_tables) -> None: @@ -127,7 +169,7 @@ async def test_replace_different_ip(manager: ManagerClient, random_tables) -> No await manager.server_stop(servers[0].server_id) replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = False, use_host_id = False) await manager.server_add(replace_cfg) - # TODO: check that group 0 no longer contains the replaced node (#12153) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) @pytest.mark.asyncio async def test_replace_different_ip_using_host_id(manager: ManagerClient, random_tables) -> None: @@ -135,7 +177,7 @@ async def test_replace_different_ip_using_host_id(manager: ManagerClient, random await manager.server_stop(servers[0].server_id) replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = False, use_host_id = True) await manager.server_add(replace_cfg) - # TODO: check that group 0 no longer contains the replaced node (#12153) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) @pytest.mark.asyncio async def test_replace_reuse_ip(manager: ManagerClient, random_tables) -> None: @@ -143,7 +185,7 @@ async def test_replace_reuse_ip(manager: ManagerClient, random_tables) -> None: await manager.server_stop(servers[0].server_id) replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = True, use_host_id = False) await manager.server_add(replace_cfg) - # TODO: check that group 0 no longer contains the replaced node (#12153) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) @pytest.mark.asyncio async def test_replace_reuse_ip_using_host_id(manager: ManagerClient, random_tables) -> None: @@ -151,7 +193,7 @@ async def test_replace_reuse_ip_using_host_id(manager: ManagerClient, random_tab await manager.server_stop(servers[0].server_id) replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = True, use_host_id = True) await manager.server_add(replace_cfg) - # TODO: check that group 0 no longer contains the replaced node (#12153) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) # Checks basic functionality on the cluster with different values of the --smp parameter on the nodes. @@ -190,6 +232,8 @@ async def test_nodes_with_different_smp(manager: ManagerClient, random_tables: R logger.info(f'Adding --smp=5 server') await manager.server_add(cmdline=['--smp', '5']) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + logger.info(f'Creating new tables') await random_tables.add_tables(ntables=4, ncolumns=5) await random_tables.verify_schema() @@ -301,11 +345,7 @@ async def test_remove_garbage_group0_members(manager: ManagerClient, random_tabl logging.info(f'removenode {servers[2]} using {servers[3]}') await manager.remove_node(servers[3].server_id, servers[2].server_id) - group0_members = await get_current_group0_config(manager, servers[3]) - logging.info(f'group 0 members: {group0_members}') - group0_ids = {m[0] for m in group0_members} - - assert token_ring_ids == group0_ids + await check_token_ring_and_group0_consistency(manager) @pytest.mark.asyncio From 91aa2cd8d778da650bac5ecc152833451c5f3286 Mon Sep 17 00:00:00 2001 From: Kamil Braun Date: Tue, 31 Jan 2023 17:03:32 +0100 Subject: [PATCH 04/28] test: topology: wait for token ring/group 0 consistency after decommission There was a check for immediate consistency after a decommission operation has finished in one of the tests, but it turns out that also after decommission it might take some time for token ring to be updated on other nodes. Replace the check with a wait. Also do the wait in another test that performs a sequence of decommissions. We won't attempt to start another decommission until every node learns that the previously decommissioned node has left. Closes #12686 (cherry picked from commit 40142a51d0dbb242db714f54750e232f2f4ff280) --- test/topology/test_topology.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/test/topology/test_topology.py b/test/topology/test_topology.py index 889ee4415f..ff61420241 100644 --- a/test/topology/test_topology.py +++ b/test/topology/test_topology.py @@ -68,9 +68,9 @@ async def check_token_ring_and_group0_consistency(manager: ManagerClient) -> Non async def wait_for_token_ring_and_group0_consistency(manager: ManagerClient, deadline: float) -> None: - """Weaker version of the above check; the token ring is not immediately updated - after bootstrap/replace - the normal tokens of the new node propagate through gossip. - Take this into account and wait for the equality condition to hold, with a timeout. + """Weaker version of the above check; the token ring is not immediately updated after + bootstrap/replace/decommission - the normal tokens of the new node propagate through gossip. + Take this into account and wait for the equality condition to hold, with a timeout. """ servers = await manager.running_servers() for srv in servers: @@ -159,7 +159,7 @@ async def test_decommission_node_add_column(manager, random_tables): await manager.api.enable_injection( bootstrapped_server.ip_addr, 'storage_service_decommission_prepare_handler_sleep', one_shot=True) await manager.decommission_node(decommission_target.server_id) - await check_token_ring_and_group0_consistency(manager) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) await table.add_column() await random_tables.verify_schema() @@ -226,6 +226,7 @@ async def test_nodes_with_different_smp(manager: ManagerClient, random_tables: R servers = await manager.running_servers() for s in servers[:-1]: await manager.decommission_node(s.server_id) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) logger.info(f'Adding --smp=4 server') await manager.server_add(cmdline=['--smp', '4']) From 6e2c547388b5a049b453147d3b2b9dad91befa8d Mon Sep 17 00:00:00 2001 From: Alejo Sanchez Date: Mon, 30 Jan 2023 11:43:04 +0100 Subject: [PATCH 05/28] test/pylib: one-shot error injection helper Existing helper with async context manager only worked for non one-shot error injections. Fix it and add another helper for one-shot without a context manager. Fix tests using the previous helper. Signed-off-by: Alejo Sanchez (cherry picked from commit 9ceb6aba8143c97e144d8834e6db8bc6bbf1d798) --- test/pylib/rest_client.py | 19 +++++- test/topology/test_topology.py | 18 +++--- .../test_raft_upgrade.py | 59 +++++++++---------- 3 files changed, 54 insertions(+), 42 deletions(-) diff --git a/test/pylib/rest_client.py b/test/pylib/rest_client.py index c2ea2a5133..2a03e09cbf 100644 --- a/test/pylib/rest_client.py +++ b/test/pylib/rest_client.py @@ -217,12 +217,13 @@ class ScyllaRESTAPIClient(): @asynccontextmanager -async def inject_error(api: ScyllaRESTAPIClient, node_ip: IPAddress, injection: str, - one_shot: bool): +async def inject_error(api: ScyllaRESTAPIClient, node_ip: IPAddress, injection: str): """Attempts to inject an error. Works only in specific build modes: debug,dev,sanitize. It will trigger a test to be skipped if attempting to enable an injection has no effect. + This is a context manager for enabling and disabling when done, therefore it can't be + used for one shot. """ - await api.enable_injection(node_ip, injection, one_shot) + await api.enable_injection(node_ip, injection, False) enabled = await api.get_enabled_injections(node_ip) logging.info(f"Error injections enabled on {node_ip}: {enabled}") if not enabled: @@ -232,3 +233,15 @@ async def inject_error(api: ScyllaRESTAPIClient, node_ip: IPAddress, injection: finally: logger.info(f"Disabling error injection {injection}") await api.disable_injection(node_ip, injection) + + +async def inject_error_one_shot(api: ScyllaRESTAPIClient, node_ip: IPAddress, injection: str): + """Attempts to inject an error. Works only in specific build modes: debug,dev,sanitize. + It will trigger a test to be skipped if attempting to enable an injection has no effect. + This is a one-shot injection enable. + """ + await api.enable_injection(node_ip, injection, True) + enabled = await api.get_enabled_injections(node_ip) + logging.info(f"Error injections enabled on {node_ip}: {enabled}") + if not enabled: + pytest.skip("Error injection not enabled in Scylla - try compiling in dev/debug/sanitize mode") diff --git a/test/topology/test_topology.py b/test/topology/test_topology.py index ff61420241..257788002f 100644 --- a/test/topology/test_topology.py +++ b/test/topology/test_topology.py @@ -19,7 +19,7 @@ from test.pylib.scylla_cluster import ReplaceConfig from test.pylib.manager_client import ManagerClient from cassandra.cluster import Session from test.pylib.random_tables import RandomTables -from test.pylib.rest_client import inject_error +from test.pylib.rest_client import inject_error_one_shot logger = logging.getLogger(__name__) @@ -256,14 +256,14 @@ async def test_remove_garbage_group0_members(manager: ManagerClient, random_tabl logging.info(f'removenode {servers[0]} using {servers[1]}') # removenode will fail after removing the server from the token ring, # but before removing it from group 0 - async with inject_error(manager.api, servers[1].ip_addr, - 'removenode_fail_before_remove_from_group0', one_shot=True): - try: - await manager.remove_node(servers[1].server_id, servers[0].server_id) - except Exception: - # Note: the exception returned here is only '500 internal server error', - # need to look in test.py log for the actual message coming from Scylla. - logging.info(f'expected exception during injection') + await inject_error_one_shot(manager.api, servers[1].ip_addr, + 'removenode_fail_before_remove_from_group0') + try: + await manager.remove_node(servers[1].server_id, servers[0].server_id) + except Exception: + # Note: the exception returned here is only '500 internal server error', + # need to look in test.py log for the actual message coming from Scylla. + logging.info(f'expected exception during injection') # Query the storage_service/host_id endpoint to calculate a list of known token ring members' Host IDs # (internally, this endpoint uses token_metadata) diff --git a/test/topology_raft_disabled/test_raft_upgrade.py b/test/topology_raft_disabled/test_raft_upgrade.py index 1f384da4b1..1d0d35ee9b 100644 --- a/test/topology_raft_disabled/test_raft_upgrade.py +++ b/test/topology_raft_disabled/test_raft_upgrade.py @@ -16,7 +16,7 @@ from cassandra.pool import Host # type: ignore # pylint: from test.pylib.manager_client import ManagerClient, IPAddress, ServerInfo from test.pylib.random_tables import RandomTables -from test.pylib.rest_client import ScyllaRESTAPIClient, inject_error +from test.pylib.rest_client import ScyllaRESTAPIClient, inject_error_one_shot from test.pylib.util import wait_for, wait_for_cql_and_get_hosts @@ -163,42 +163,41 @@ async def test_recover_stuck_raft_upgrade(manager: ManagerClient, random_tables: # TODO error injection should probably be done through ScyllaClusterManager (we may need to mark the cluster as dirty). # In this test the cluster is dirty anyway due to a restart so it's safe. - async with inject_error(manager.api, srv1.ip_addr, 'group0_upgrade_before_synchronize', - one_shot=True): - logging.info(f"Enabling Raft on {others} and restarting") - await asyncio.gather(*(enable_raft_and_restart(manager, srv) for srv in others)) - cql = await reconnect_driver(manager) + await inject_error_one_shot(manager.api, srv1.ip_addr, 'group0_upgrade_before_synchronize') + logging.info(f"Enabling Raft on {others} and restarting") + await asyncio.gather(*(enable_raft_and_restart(manager, srv) for srv in others)) + cql = await reconnect_driver(manager) - logging.info(f"Cluster restarted, waiting until driver reconnects to {others}") - hosts = await wait_for_cql_and_get_hosts(cql, others, time.time() + 60) - logging.info(f"Driver reconnected, hosts: {hosts}") + logging.info(f"Cluster restarted, waiting until driver reconnects to {others}") + hosts = await wait_for_cql_and_get_hosts(cql, others, time.time() + 60) + logging.info(f"Driver reconnected, hosts: {hosts}") - logging.info(f"Waiting until {hosts} enter 'synchronize' state") - await asyncio.gather(*(wait_for_upgrade_state('synchronize', cql, h, time.time() + 60) for h in hosts)) - logging.info(f"{hosts} entered synchronize") + logging.info(f"Waiting until {hosts} enter 'synchronize' state") + await asyncio.gather(*(wait_for_upgrade_state('synchronize', cql, h, time.time() + 60) for h in hosts)) + logging.info(f"{hosts} entered synchronize") - # TODO ensure that srv1 failed upgrade - look at logs? - # '[shard 0] raft_group0_upgrade - Raft upgrade failed: std::runtime_error (error injection before group 0 upgrade enters synchronize).' + # TODO ensure that srv1 failed upgrade - look at logs? + # '[shard 0] raft_group0_upgrade - Raft upgrade failed: std::runtime_error (error injection before group 0 upgrade enters synchronize).' - logging.info(f"Setting recovery state on {hosts}") - for host in hosts: - await cql.run_async( - "update system.scylla_local set value = 'recovery' where key = 'group0_upgrade_state'", - host=host) + logging.info(f"Setting recovery state on {hosts}") + for host in hosts: + await cql.run_async( + "update system.scylla_local set value = 'recovery' where key = 'group0_upgrade_state'", + host=host) - logging.info(f"Restarting {others}") - await asyncio.gather(*(restart(manager, srv) for srv in others)) - cql = await reconnect_driver(manager) + logging.info(f"Restarting {others}") + await asyncio.gather(*(restart(manager, srv) for srv in others)) + cql = await reconnect_driver(manager) - logging.info(f"{others} restarted, waiting until driver reconnects to them") - hosts = await wait_for_cql_and_get_hosts(cql, others, time.time() + 60) + logging.info(f"{others} restarted, waiting until driver reconnects to them") + hosts = await wait_for_cql_and_get_hosts(cql, others, time.time() + 60) - logging.info(f"Checking if {hosts} are in recovery state") - for host in hosts: - rs = await cql.run_async( - "select value from system.scylla_local where key = 'group0_upgrade_state'", - host=host) - assert rs[0].value == 'recovery' + logging.info(f"Checking if {hosts} are in recovery state") + for host in hosts: + rs = await cql.run_async( + "select value from system.scylla_local where key = 'group0_upgrade_state'", + host=host) + assert rs[0].value == 'recovery' logging.info("Creating a table while in recovery state") table = await random_tables.add_table(ncolumns=5) From f90fe6f3123737c4b788aa5ffa2859887e71e0c3 Mon Sep 17 00:00:00 2001 From: Asias He Date: Fri, 3 Feb 2023 21:11:33 +0800 Subject: [PATCH 06/28] test: Increase START_TIMEOUT It is observed that CI machine is slow to run the test. Increase the timeout of adding servers. (cherry picked from commit fc604844227a9c1f2aa5f973d232be4ae340e1f5) --- test/pylib/manager_client.py | 5 +++-- test/pylib/rest_client.py | 4 ++-- test/pylib/scylla_cluster.py | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index 2834647294..8ff2498feb 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -15,7 +15,7 @@ import logging from test.pylib.rest_client import UnixRESTClient, ScyllaRESTAPIClient from test.pylib.util import wait_for from test.pylib.internal_types import ServerNum, IPAddress, HostID, ServerInfo -from test.pylib.scylla_cluster import ReplaceConfig +from test.pylib.scylla_cluster import ReplaceConfig, ScyllaServer from cassandra.cluster import Session as CassandraSession # type: ignore # pylint: disable=no-name-in-module from cassandra.cluster import Cluster as CassandraCluster # type: ignore # pylint: disable=no-name-in-module import aiohttp @@ -157,7 +157,8 @@ class ManagerClient(): data['replace_cfg'] = replace_cfg._asdict() if cmdline: data['cmdline'] = cmdline - server_info = await self.client.put_json("/cluster/addserver", data, response_type="json") + server_info = await self.client.put_json("/cluster/addserver", data, response_type="json", + timeout=ScyllaServer.START_TIMEOUT) except Exception as exc: raise Exception("Failed to add server") from exc try: diff --git a/test/pylib/rest_client.py b/test/pylib/rest_client.py index 2a03e09cbf..f7634caad0 100644 --- a/test/pylib/rest_client.py +++ b/test/pylib/rest_client.py @@ -100,9 +100,9 @@ class RESTClient(metaclass=ABCMeta): async def put_json(self, resource_uri: str, data: Mapping, host: Optional[str] = None, port: Optional[int] = None, params: Optional[dict[str, str]] = None, - response_type: Optional[str] = None) -> Any: + response_type: Optional[str] = None, timeout: Optional[float] = None) -> Any: ret = await self._fetch("PUT", resource_uri, response_type = response_type, host = host, - port = port, params = params, json = data) + port = port, params = params, json = data, timeout = timeout) return ret async def delete(self, resource_uri: str, host: Optional[str] = None, diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 529cd9e1ea..0a673cbb45 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -187,7 +187,7 @@ class ScyllaServer: """Starts and handles a single Scylla server, managing logs, checking if responsive, and cleanup when finished.""" # pylint: disable=too-many-instance-attributes - START_TIMEOUT = 300 # seconds + START_TIMEOUT = 1000 # seconds start_time: float sleep_interval: float log_file: BufferedWriter From ea80fe20ad057964c5036239220f9c1755a361a1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Botond=20D=C3=A9nes?= Date: Mon, 13 Feb 2023 09:56:06 +0200 Subject: [PATCH 07/28] Merge 'test/pylib: use larger timeout for decommission/removenode' from Kamil Braun Recently we enabled RBNO by default in all topology operations. This made the operations a bit slower (repair-based topology ops are a bit slower than classic streaming - they do more work), and in debug mode with large number of concurrent tests running, they might timeout. The timeout for bootstrap was already increased before, do the same for decommission/removenode. The previously used timeout was 300 seconds (this is the default used by aiohttp library when it makes HTTP requests), now use the TOPOLOGY_TIMEOUT constant from ScyllaServer which is 1000 seconds. Closes #12765 * github.com:scylladb/scylladb: test/pylib: use larger timeout for decommission/removenode test/pylib: scylla_cluster: rename START_TIMEOUT to TOPOLOGY_TIMEOUT (cherry picked from commit e55f475db1fdce91bb5877689b72615d63395b39) --- test/pylib/manager_client.py | 8 +++++--- test/pylib/rest_client.py | 13 +++++++------ test/pylib/scylla_cluster.py | 15 +++++++++------ 3 files changed, 21 insertions(+), 15 deletions(-) diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index 8ff2498feb..47cb3cefdc 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -158,7 +158,7 @@ class ManagerClient(): if cmdline: data['cmdline'] = cmdline server_info = await self.client.put_json("/cluster/addserver", data, response_type="json", - timeout=ScyllaServer.START_TIMEOUT) + timeout=ScyllaServer.TOPOLOGY_TIMEOUT) except Exception as exc: raise Exception("Failed to add server") from exc try: @@ -176,13 +176,15 @@ class ManagerClient(): """Invoke remove node Scylla REST API for a specified server""" logger.debug("ManagerClient remove node %s on initiator %s", server_id, initiator_id) data = {"server_id": server_id, "ignore_dead": ignore_dead} - await self.client.put_json(f"/cluster/remove-node/{initiator_id}", data) + await self.client.put_json(f"/cluster/remove-node/{initiator_id}", data, + timeout=ScyllaServer.TOPOLOGY_TIMEOUT) self._driver_update() async def decommission_node(self, server_id: ServerNum) -> None: """Tell a node to decommission with Scylla REST API""" logger.debug("ManagerClient decommission %s", server_id) - await self.client.get_text(f"/cluster/decommission-node/{server_id}") + await self.client.get_text(f"/cluster/decommission-node/{server_id}", + timeout=ScyllaServer.TOPOLOGY_TIMEOUT) self._driver_update() async def server_get_config(self, server_id: ServerNum) -> dict[str, object]: diff --git a/test/pylib/rest_client.py b/test/pylib/rest_client.py index f7634caad0..d190dbf172 100644 --- a/test/pylib/rest_client.py +++ b/test/pylib/rest_client.py @@ -94,9 +94,9 @@ class RESTClient(metaclass=ABCMeta): async def post(self, resource_uri: str, host: Optional[str] = None, port: Optional[int] = None, params: Optional[Mapping[str, str]] = None, - json: Mapping = None) -> None: + json: Mapping = None, timeout: Optional[float] = None) -> None: await self._fetch("POST", resource_uri, host = host, port = port, params = params, - json = json) + json = json, timeout = timeout) async def put_json(self, resource_uri: str, data: Mapping, host: Optional[str] = None, port: Optional[int] = None, params: Optional[dict[str, str]] = None, @@ -161,19 +161,20 @@ class ScyllaRESTAPIClient(): return result async def remove_node(self, initiator_ip: IPAddress, host_id: HostID, - ignore_dead: list[IPAddress]) -> None: + ignore_dead: list[IPAddress], timeout: float) -> None: """Initiate remove node of host_id in initiator initiator_ip""" logger.info("remove_node for %s on %s", host_id, initiator_ip) await self.client.post("/storage_service/remove_node", params = {"host_id": host_id, "ignore_nodes": ",".join(ignore_dead)}, - host = initiator_ip) + host = initiator_ip, timeout = timeout) logger.debug("remove_node for %s finished", host_id) - async def decommission_node(self, host_ip: str) -> None: + async def decommission_node(self, host_ip: str, timeout: float) -> None: """Initiate decommission node of host_ip""" logger.debug("decommission_node %s", host_ip) - await self.client.post("/storage_service/decommission", host = host_ip) + await self.client.post("/storage_service/decommission", host = host_ip, + timeout = timeout) logger.debug("decommission_node %s finished", host_ip) async def get_gossip_generation_number(self, node_ip: str, target_ip: str) -> int: diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 0a673cbb45..ab75854b68 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -187,7 +187,9 @@ class ScyllaServer: """Starts and handles a single Scylla server, managing logs, checking if responsive, and cleanup when finished.""" # pylint: disable=too-many-instance-attributes - START_TIMEOUT = 1000 # seconds + + # in seconds, used for topology operations such as bootstrap or decommission + TOPOLOGY_TIMEOUT = 1000 start_time: float sleep_interval: float log_file: BufferedWriter @@ -319,7 +321,7 @@ class ScyllaServer: # initializing. When the role is ready, queries begin to # work, so rely on this "side effect". profile = ExecutionProfile(load_balancing_policy=WhiteListRoundRobinPolicy([self.ip_addr]), - request_timeout=self.START_TIMEOUT) + request_timeout=self.TOPOLOGY_TIMEOUT) connected = False try: # In a cluster setup, it's possible that the CQL @@ -379,7 +381,7 @@ class ScyllaServer: sleep_interval = 0.1 cql_up_state = CqlUpState.NOT_CONNECTED - while time.time() < self.start_time + self.START_TIMEOUT: + while time.time() < self.start_time + self.TOPOLOGY_TIMEOUT: if self.cmd.returncode: with self.log_filename.open('r') as log_file: self.logger.error("failed to start server at host %s in %s", @@ -424,7 +426,7 @@ class ScyllaServer: previous state propagation was missed.""" auth = PlainTextAuthProvider(username='cassandra', password='cassandra') profile = ExecutionProfile(load_balancing_policy=WhiteListRoundRobinPolicy(self.seeds), - request_timeout=self.START_TIMEOUT) + request_timeout=self.TOPOLOGY_TIMEOUT) with Cluster(execution_profiles={EXEC_PROFILE_DEFAULT: profile}, contact_points=self.seeds, auth_provider=auth, @@ -1073,7 +1075,8 @@ class ScyllaClusterManager: # initate remove try: - await self.cluster.api.remove_node(initiator.ip_addr, to_remove.host_id, ignore_dead) + await self.cluster.api.remove_node(initiator.ip_addr, to_remove.host_id, ignore_dead, + timeout=ScyllaServer.TOPOLOGY_TIMEOUT) except RuntimeError as exc: self.logger.error("_cluster_remove_node failed initiator %s server %s ignore_dead %s, check log at %s", initiator, to_remove, ignore_dead, initiator.log_filename) @@ -1092,7 +1095,7 @@ class ScyllaClusterManager: self.logger.warning("_cluster_decommission_node %s is only running node left", server_id) server = self.cluster.running[server_id] try: - await self.cluster.api.decommission_node(server.ip_addr) + await self.cluster.api.decommission_node(server.ip_addr, timeout=ScyllaServer.TOPOLOGY_TIMEOUT) except RuntimeError as exc: self.logger.error("_cluster_decommission_node %s, check log at %s", server, server.log_filename) From 7b60cddae7423ddf1328e3fc81c4167b60ae3e5e Mon Sep 17 00:00:00 2001 From: Nadav Har'El Date: Sun, 12 Feb 2023 17:53:48 +0200 Subject: [PATCH 08/28] Merge 'test/pylib: split and refactor topology tests' from Alecco Move long running topology tests out of `test_topology.py` and into their own files, so they can be run in parallel. While there, merge simple schema tests. Closes #12804 * github.com:scylladb/scylladb: test/topology: rename topology test file test/topology: lint and type for topology tests test/topology: move topology ip tests to own file test/topology: move topology test remove garbaje... test/topology: move topology rejoin test to own file test/topology: merge topology schema tests and... test/topology: isolate topology smp params test test/topology: move topology helpers to common file (cherry picked from commit a24600a662b9a481986c63305d5523d2578690e2) --- test/topology/test_topology.py | 426 ------------------ test/topology/test_topology_ip.py | 50 ++ test/topology/test_topology_rejoin.py | 21 + test/topology/test_topology_remove_decom.py | 133 ++++++ .../test_topology_remove_garbage_group0.py | 126 ++++++ test/topology/test_topology_schema.py | 34 ++ test/topology/test_topology_smp.py | 60 +++ test/topology/util.py | 79 ++++ 8 files changed, 503 insertions(+), 426 deletions(-) delete mode 100644 test/topology/test_topology.py create mode 100644 test/topology/test_topology_ip.py create mode 100644 test/topology/test_topology_rejoin.py create mode 100644 test/topology/test_topology_remove_decom.py create mode 100644 test/topology/test_topology_remove_garbage_group0.py create mode 100644 test/topology/test_topology_schema.py create mode 100644 test/topology/test_topology_smp.py create mode 100644 test/topology/util.py diff --git a/test/topology/test_topology.py b/test/topology/test_topology.py deleted file mode 100644 index 257788002f..0000000000 --- a/test/topology/test_topology.py +++ /dev/null @@ -1,426 +0,0 @@ -# -# Copyright (C) 2022-present ScyllaDB -# -# SPDX-License-Identifier: AGPL-3.0-or-later -# -""" -Test consistency of schema changes with topology changes. -""" -import pytest -import logging -import asyncio -import random -import time - -from test.pylib.util import wait_for, wait_for_cql_and_get_hosts - -from test.pylib.internal_types import ServerInfo -from test.pylib.scylla_cluster import ReplaceConfig -from test.pylib.manager_client import ManagerClient -from cassandra.cluster import Session -from test.pylib.random_tables import RandomTables -from test.pylib.rest_client import inject_error_one_shot - -logger = logging.getLogger(__name__) - - -async def get_token_ring_host_ids(manager: ManagerClient, srv: ServerInfo) -> set[str]: - """Get the host IDs of normal token owners known by `srv`.""" - token_endpoint_map = await manager.api.client.get_json("/storage_service/tokens_endpoint", srv.ip_addr) - normal_endpoints = {e["value"] for e in token_endpoint_map} - logger.info(f"Normal endpoints' IPs by {srv}: {normal_endpoints}") - host_id_map = await manager.api.client.get_json('/storage_service/host_id', srv.ip_addr) - all_host_ids = {e["value"] for e in host_id_map} - logger.info(f"All host IDs by {srv}: {all_host_ids}") - normal_host_ids = {e["value"] for e in host_id_map if e["key"] in normal_endpoints} - logger.info(f"Normal endpoints' host IDs by {srv}: {normal_host_ids}") - return normal_host_ids - - -async def get_current_group0_config(manager: ManagerClient, srv: ServerInfo) -> set[tuple[str, bool]]: - """Get the current Raft group 0 configuration known by `srv`. - The first element of each tuple is the Raft ID of the node (which is equal to the Host ID), - the second element indicates whether the node is a voter. - """ - assert(manager.cql) - host = (await wait_for_cql_and_get_hosts(manager.cql, [srv], time.time() + 60))[0] - group0_id = (await manager.cql.run_async( - "select value from system.scylla_local where key = 'raft_group0_id'", - host=host))[0].value - config = await manager.cql.run_async( - f"select server_id, can_vote from system.raft_state where group_id = {group0_id} and disposition = 'CURRENT'", - host=host) - result = {(str(m.server_id), bool(m.can_vote)) for m in config} - logger.info(f"Group 0 members by {srv}: {result}") - return result - - -async def check_token_ring_and_group0_consistency(manager: ManagerClient) -> None: - """Ensure that the normal token owners and group 0 members match - according to each currently running server. - """ - servers = await manager.running_servers() - for srv in servers: - group0_members = await get_current_group0_config(manager, srv) - group0_ids = {m[0] for m in group0_members} - token_ring_ids = await get_token_ring_host_ids(manager, srv) - assert token_ring_ids == group0_ids - - -async def wait_for_token_ring_and_group0_consistency(manager: ManagerClient, deadline: float) -> None: - """Weaker version of the above check; the token ring is not immediately updated after - bootstrap/replace/decommission - the normal tokens of the new node propagate through gossip. - Take this into account and wait for the equality condition to hold, with a timeout. - """ - servers = await manager.running_servers() - for srv in servers: - group0_members = await get_current_group0_config(manager, srv) - group0_ids = {m[0] for m in group0_members} - async def token_ring_matches(): - token_ring_ids = await get_token_ring_host_ids(manager, srv) - diff = token_ring_ids ^ group0_ids - if diff: - logger.warning(f"Group 0 members and token ring members don't yet match" \ - f" according to {srv}, symmetric difference: {diff}") - return None - return True - await wait_for(token_ring_matches, deadline, period=.5) - - -@pytest.mark.asyncio -async def test_add_server_add_column(manager, random_tables): - """Add a node and then add a column to a table and verify""" - table = await random_tables.add_table(ncolumns=5) - await manager.server_add() - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) - await table.add_column() - await random_tables.verify_schema() - - -@pytest.mark.asyncio -async def test_stop_server_add_column(manager, random_tables): - """Add a node, stop an original node, add a column""" - servers = await manager.running_servers() - table = await random_tables.add_table(ncolumns=5) - await manager.server_add() - await manager.server_stop(servers[1].server_id) - await table.add_column() - await random_tables.verify_schema() - - -@pytest.mark.asyncio -async def test_restart_server_add_column(manager, random_tables): - """Add a node, stop an original node, add a column""" - servers = await manager.running_servers() - table = await random_tables.add_table(ncolumns=5) - ret = await manager.server_restart(servers[1].server_id) - await table.add_column() - await random_tables.verify_schema() - - -@pytest.mark.asyncio -async def test_remove_node_add_column(manager, random_tables): - """Add a node, remove an original node, add a column""" - servers = await manager.running_servers() - table = await random_tables.add_table(ncolumns=5) - await manager.server_add() - await manager.server_stop_gracefully(servers[1].server_id) # stop [1] - await manager.remove_node(servers[0].server_id, servers[1].server_id) # Remove [1] - await check_token_ring_and_group0_consistency(manager) - await table.add_column() - await random_tables.verify_schema() - - -@pytest.mark.asyncio -async def test_decommission_node_add_column(manager, random_tables): - """Add a node, remove an original node, add a column""" - table = await random_tables.add_table(ncolumns=5) - servers = await manager.running_servers() - decommission_target = servers[1] - # The sleep injections significantly increase the probability of reproducing #11780: - # 1. bootstrapped_server finishes bootstrapping and enters NORMAL state - # 2. decommission_target starts storage_service::handle_state_normal(bootstrapped_server), - # enters sleep before calling storage_service::notify_joined - # 3. we start decommission on decommission_target - # 4. decommission_target sends node_ops_verb with decommission_prepare request to bootstrapped_server - # 5. bootstrapped_server receives the RPC and enters sleep - # 6. decommission_target handle_state_normal wakes up, - # calls storage_service::notify_joined which drops some RPC clients - # 7. If #11780 is not fixed, this will fail the node_ops_verb RPC, causing decommission to fail - await manager.api.enable_injection( - decommission_target.ip_addr, 'storage_service_notify_joined_sleep', one_shot=True) - bootstrapped_server = await manager.server_add() - async def no_joining_nodes(): - joining_nodes = await manager.api.get_joining_nodes(decommission_target.ip_addr) - return not joining_nodes - # Wait until decommission_target thinks that bootstrapped_server is NORMAL - # note: when this wait finishes, we're usually in the middle of storage_service::handle_state_normal - await wait_for(no_joining_nodes, time.time() + 30, period=.1) - await manager.api.enable_injection( - bootstrapped_server.ip_addr, 'storage_service_decommission_prepare_handler_sleep', one_shot=True) - await manager.decommission_node(decommission_target.server_id) - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) - await table.add_column() - await random_tables.verify_schema() - -@pytest.mark.asyncio -async def test_replace_different_ip(manager: ManagerClient, random_tables) -> None: - servers = await manager.running_servers() - await manager.server_stop(servers[0].server_id) - replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = False, use_host_id = False) - await manager.server_add(replace_cfg) - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) - -@pytest.mark.asyncio -async def test_replace_different_ip_using_host_id(manager: ManagerClient, random_tables) -> None: - servers = await manager.running_servers() - await manager.server_stop(servers[0].server_id) - replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = False, use_host_id = True) - await manager.server_add(replace_cfg) - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) - -@pytest.mark.asyncio -async def test_replace_reuse_ip(manager: ManagerClient, random_tables) -> None: - servers = await manager.running_servers() - await manager.server_stop(servers[0].server_id) - replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = True, use_host_id = False) - await manager.server_add(replace_cfg) - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) - -@pytest.mark.asyncio -async def test_replace_reuse_ip_using_host_id(manager: ManagerClient, random_tables) -> None: - servers = await manager.running_servers() - await manager.server_stop(servers[0].server_id) - replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = True, use_host_id = True) - await manager.server_add(replace_cfg) - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) - - -# Checks basic functionality on the cluster with different values of the --smp parameter on the nodes. -@pytest.mark.asyncio -async def test_nodes_with_different_smp(manager: ManagerClient, random_tables: RandomTables) -> None: - # In this test it's more convenient to start with a fresh cluster. - # We don't need the default nodes, - # but there is currently no way to express this in the test infrastructure - - # When the node starts it tries to communicate with others - # by sending group0_peer_exchange message to them. - # This message can be handled on arbitrary shard of the target node. - # The method manager.server_add waits for node to start, which can only happen - # if this message has been handled correctly. - # - # Note: messaging_service is initialized with server_socket::load_balancing_algorithm::port - # policy, this means that the shard for message will be chosen as client_port % smp::count. - # The client port in turn is chosen as rand() * smp::count + current_shard - # (posix_socket_impl::find_port_and_connect). - # If this succeeds to occupy a free port in 5 tries and smp::count is the same - # on both nodes, then it's guaranteed that the message will be - # processed on the same shard as the calling code. - # In the general case, we cannot assume that this same shard guarantee holds. - logger.info(f'Adding --smp=3 server') - await manager.server_add(cmdline=['--smp', '3']) - - # Remove the original 3 servers, the problem is easier to reproduce with --smp values - # that we pick, not the (currently) default --smp=2 coming from the suite. - logger.info(f'Decommissioning old servers') - servers = await manager.running_servers() - for s in servers[:-1]: - await manager.decommission_node(s.server_id) - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) - - logger.info(f'Adding --smp=4 server') - await manager.server_add(cmdline=['--smp', '4']) - logger.info(f'Adding --smp=5 server') - await manager.server_add(cmdline=['--smp', '5']) - - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) - - logger.info(f'Creating new tables') - await random_tables.add_tables(ntables=4, ncolumns=5) - await random_tables.verify_schema() - - -@pytest.mark.asyncio -async def test_remove_garbage_group0_members(manager: ManagerClient, random_tables): - """ - Verify that failing to leave group 0 or remove a node from group 0 in removenode/decommission - can be handled by executing removenode (which should clear the 'garbage' group 0 member), - even though the node is no longer a token ring member. - """ - # 4 servers, one dead - await manager.server_add() - servers = await manager.running_servers() - removed_host_id = await manager.get_host_id(servers[0].server_id) - await manager.server_stop_gracefully(servers[0].server_id) - - logging.info(f'removenode {servers[0]} using {servers[1]}') - # removenode will fail after removing the server from the token ring, - # but before removing it from group 0 - await inject_error_one_shot(manager.api, servers[1].ip_addr, - 'removenode_fail_before_remove_from_group0') - try: - await manager.remove_node(servers[1].server_id, servers[0].server_id) - except Exception: - # Note: the exception returned here is only '500 internal server error', - # need to look in test.py log for the actual message coming from Scylla. - logging.info(f'expected exception during injection') - - # Query the storage_service/host_id endpoint to calculate a list of known token ring members' Host IDs - # (internally, this endpoint uses token_metadata) - token_ring_ids = await get_token_ring_host_ids(manager, servers[1]) - logging.info(f'token ring members: {token_ring_ids}') - - group0_members = await get_current_group0_config(manager, servers[1]) - logging.info(f'group 0 members: {group0_members}') - group0_ids = {m[0] for m in group0_members} - - # Token ring members should currently be a subset of group 0 members - assert token_ring_ids <= group0_ids - - garbage_members = group0_ids - token_ring_ids - logging.info(f'garbage members: {garbage_members}') - assert len(garbage_members) == 1 - garbage_member = next(iter(garbage_members)) - - # The garbage member is the one that we failed to remove - assert garbage_member == removed_host_id - - # Verify that at least it's a non-voter. - assert garbage_member in {m[0] for m in group0_members if not m[1]} - - logging.info(f'removenode {servers[0]} using {servers[1]} again') - # Retry removenode. It should skip the token ring removal step and remove the server from group 0. - await manager.remove_node(servers[1].server_id, servers[0].server_id) - - group0_members = await get_current_group0_config(manager, servers[1]) - logging.info(f'group 0 members: {group0_members}') - group0_ids = {m[0] for m in group0_members} - - # Token ring members and group 0 members should now be the same. - assert token_ring_ids == group0_ids - - # Verify that availability is not reduced. - # Stop one of the 3 remaining servers and try to remove it. It should succeed with only 2 servers. - - logging.info(f'stop {servers[1]}') - await manager.server_stop_gracefully(servers[1].server_id) - - logging.info(f'removenode {servers[1]} using {servers[2]}') - await manager.remove_node(servers[2].server_id, servers[1].server_id) - - # Perform a similar scenario with decommission. One of the node fails to decommission fully, - # but it manages to leave the token ring. We observe the leftovers using the same APIs as above - # and remove the leftovers. - # We can do this with only 2 nodes because during decommission we become a non-voter before - # leaving the token ring, thus the remaining single node will become a voting majority - # and will be able to perform removenode alone. - - decommissioned_host_id = await manager.get_host_id(servers[2].server_id) - await manager.api.enable_injection( - servers[2].ip_addr, 'decommission_fail_before_leave_group0', one_shot=True) - logging.info(f'decommission {servers[2]}') - try: - await manager.decommission_node(servers[2].server_id) - except Exception: - logging.info(f'expected exception during injection') - logging.info(f'stop {servers[2]}') - await manager.server_stop_gracefully(servers[2].server_id) - - token_ring_ids = await get_token_ring_host_ids(manager, servers[3]) - logging.info(f'token ring members: {token_ring_ids}') - - group0_members = await get_current_group0_config(manager, servers[3]) - logging.info(f'group 0 members: {group0_members}') - group0_ids = {m[0] for m in group0_members} - - assert token_ring_ids <= group0_ids - - garbage_members = group0_ids - token_ring_ids - logging.info(f'garbage members: {garbage_members}') - assert len(garbage_members) == 1 - garbage_member = next(iter(garbage_members)) - - assert garbage_member == decommissioned_host_id - assert garbage_member in {m[0] for m in group0_members if not m[1]} - - logging.info(f'removenode {servers[2]} using {servers[3]}') - await manager.remove_node(servers[3].server_id, servers[2].server_id) - - await check_token_ring_and_group0_consistency(manager) - - -@pytest.mark.asyncio -@pytest.mark.skip(reason="Wait for @slow attribute, #11713") -async def test_remove_node_with_concurrent_ddl(manager, random_tables): - stopped = False - ddl_failed = False - - async def do_ddl(): - nonlocal ddl_failed - iteration = 0 - while not stopped: - logger.debug(f'ddl, iteration {iteration} started') - try: - # If the node was removed, the driver may retry "create table" on another node, - # but the request might have already been completed. - # The same applies to drop_table. - - await random_tables.add_tables(5, 5, if_not_exists=True) - await random_tables.verify_schema() - while len(random_tables.tables) > 0: - await random_tables.drop_table(random_tables.tables[-1], if_exists=True) - logger.debug(f'ddl, iteration {iteration} finished') - except: - logger.exception(f'ddl, iteration {iteration} failed') - ddl_failed = True - raise - iteration += 1 - - async def do_remove_node(): - for i in range(10): - logger.debug(f'do_remove_node [{i}], iteration started') - if ddl_failed: - logger.debug(f'do_remove_node [{i}], ddl failed, exiting') - break - server_ids = await manager.running_servers() - host_ids = await asyncio.gather(*(manager.get_host_id(s) for s in server_ids)) - initiator_index, target_index = random.sample(range(len(server_ids)), 2) - initiator_ip = server_ids[initiator_index] - target_ip = server_ids[target_index] - target_host_id = host_ids[target_index] - logger.info(f'do_remove_node [{i}], running remove_node, ' - f'initiator server [{initiator_ip}], target ip [{target_ip}], ' - f'target host id [{target_host_id}]') - await manager.wait_for_host_known(initiator_ip, target_host_id) - logger.info(f'do_remove_node [{i}], stopping target server [{target_ip}], host_id [{target_host_id}]') - await manager.server_stop_gracefully(target_ip) - logger.info(f'do_remove_node [{i}], target server [{target_ip}] stopped, ' - f'waiting for it to be down on [{initiator_ip}]') - await manager.wait_for_host_down(initiator_ip, target_ip) - logger.info(f'do_remove_node [{i}], invoking remove_node') - await manager.remove_node(initiator_ip, target_ip, target_host_id) - # TODO: check that group 0 no longer contains the removed node (#12153) - logger.info(f'do_remove_node [{i}], remove_node done') - new_server_ip = await manager.server_add() - logger.info(f'do_remove_node [{i}], server_add [{new_server_ip}] done') - logger.info(f'do_remove_node [{i}], iteration finished') - - ddl_task = asyncio.create_task(do_ddl()) - try: - await do_remove_node() - finally: - logger.debug("do_remove_node finished, waiting for ddl fiber") - stopped = True - await ddl_task - logger.debug("ddl fiber done, finished") - - -@pytest.mark.asyncio -async def test_start_after_sudden_stop(manager: ManagerClient, random_tables) -> None: - """Tests a server can rejoin the cluster after being stopped suddenly""" - servers = await manager.running_servers() - table = await random_tables.add_table(ncolumns=5) - await manager.server_stop(servers[0].server_id) - await table.add_column() - await manager.server_start(servers[0].server_id) - await random_tables.verify_schema() diff --git a/test/topology/test_topology_ip.py b/test/topology/test_topology_ip.py new file mode 100644 index 0000000000..e2b32a8787 --- /dev/null +++ b/test/topology/test_topology_ip.py @@ -0,0 +1,50 @@ +# +# Copyright (C) 2022-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +""" +Test replacing node in different scenarios +""" +import time +from test.pylib.scylla_cluster import ReplaceConfig +from test.pylib.manager_client import ManagerClient +from test.topology.util import wait_for_token_ring_and_group0_consistency +import pytest + + +@pytest.mark.asyncio +async def test_replace_different_ip(manager: ManagerClient) -> None: + """Replace an existing node with new node using a different IP address""" + servers = await manager.running_servers() + await manager.server_stop(servers[0].server_id) + replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = False, use_host_id = False) + await manager.server_add(replace_cfg) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + +@pytest.mark.asyncio +async def test_replace_different_ip_using_host_id(manager: ManagerClient) -> None: + """Replace an existing node with new node reusing the replaced node host id""" + servers = await manager.running_servers() + await manager.server_stop(servers[0].server_id) + replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = False, use_host_id = True) + await manager.server_add(replace_cfg) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + +@pytest.mark.asyncio +async def test_replace_reuse_ip(manager: ManagerClient) -> None: + """Replace an existing node with new node using the same IP address""" + servers = await manager.running_servers() + await manager.server_stop(servers[0].server_id) + replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = True, use_host_id = False) + await manager.server_add(replace_cfg) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + +@pytest.mark.asyncio +async def test_replace_reuse_ip_using_host_id(manager: ManagerClient) -> None: + """Replace an existing node with new node using the same IP address and same host id""" + servers = await manager.running_servers() + await manager.server_stop(servers[0].server_id) + replace_cfg = ReplaceConfig(replaced_id = servers[0].server_id, reuse_ip_addr = True, use_host_id = True) + await manager.server_add(replace_cfg) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) diff --git a/test/topology/test_topology_rejoin.py b/test/topology/test_topology_rejoin.py new file mode 100644 index 0000000000..afc25c3309 --- /dev/null +++ b/test/topology/test_topology_rejoin.py @@ -0,0 +1,21 @@ +# +# Copyright (C) 2022-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +""" +Test rejoin of a server after it was stopped suddenly (crash-like) +""" +from test.pylib.manager_client import ManagerClient +import pytest + + +@pytest.mark.asyncio +async def test_start_after_sudden_stop(manager: ManagerClient, random_tables) -> None: + """Tests a server can rejoin the cluster after being stopped suddenly""" + servers = await manager.running_servers() + table = await random_tables.add_table(ncolumns=5) + await manager.server_stop(servers[0].server_id) + await table.add_column() + await manager.server_start(servers[0].server_id) + await random_tables.verify_schema() diff --git a/test/topology/test_topology_remove_decom.py b/test/topology/test_topology_remove_decom.py new file mode 100644 index 0000000000..302e72e660 --- /dev/null +++ b/test/topology/test_topology_remove_decom.py @@ -0,0 +1,133 @@ +# +# Copyright (C) 2022-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +""" +Test consistency of schema changes with topology changes. +""" +import logging +import asyncio +import random +import time +from test.pylib.util import wait_for +from test.pylib.manager_client import ManagerClient +from test.pylib.random_tables import RandomTables +from test.topology.util import check_token_ring_and_group0_consistency, \ + wait_for_token_ring_and_group0_consistency +import pytest + + +logger = logging.getLogger(__name__) + + +@pytest.mark.asyncio +async def test_remove_node_add_column(manager: ManagerClient, random_tables: RandomTables): + """Add a node, remove an original node, add a column""" + servers = await manager.running_servers() + table = await random_tables.add_table(ncolumns=5) + await manager.server_add() + await manager.server_stop_gracefully(servers[1].server_id) # stop [1] + await manager.remove_node(servers[0].server_id, servers[1].server_id) # Remove [1] + await check_token_ring_and_group0_consistency(manager) + await table.add_column() + await random_tables.verify_schema() + + +@pytest.mark.asyncio +async def test_decommission_node_add_column(manager: ManagerClient, random_tables: RandomTables): + """Add a node, remove an original node, add a column""" + table = await random_tables.add_table(ncolumns=5) + servers = await manager.running_servers() + decommission_target = servers[1] + # The sleep injections significantly increase the probability of reproducing #11780: + # 1. bootstrapped_server finishes bootstrapping and enters NORMAL state + # 2. decommission_target starts storage_service::handle_state_normal(bootstrapped_server), + # enters sleep before calling storage_service::notify_joined + # 3. we start decommission on decommission_target + # 4. decommission_target sends node_ops_verb with decommission_prepare request to bootstrapped_server + # 5. bootstrapped_server receives the RPC and enters sleep + # 6. decommission_target handle_state_normal wakes up, + # calls storage_service::notify_joined which drops some RPC clients + # 7. If #11780 is not fixed, this will fail the node_ops_verb RPC, causing decommission to fail + await manager.api.enable_injection( + decommission_target.ip_addr, 'storage_service_notify_joined_sleep', one_shot=True) + bootstrapped_server = await manager.server_add() + async def no_joining_nodes(): + joining_nodes = await manager.api.get_joining_nodes(decommission_target.ip_addr) + return not joining_nodes + # Wait until decommission_target thinks that bootstrapped_server is NORMAL + # note: when this wait finishes, we're usually in the middle of storage_service::handle_state_normal + await wait_for(no_joining_nodes, time.time() + 30, period=.1) + await manager.api.enable_injection( + bootstrapped_server.ip_addr, 'storage_service_decommission_prepare_handler_sleep', one_shot=True) + await manager.decommission_node(decommission_target.server_id) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + await table.add_column() + await random_tables.verify_schema() + + +@pytest.mark.asyncio +@pytest.mark.skip(reason="Wait for @slow attribute, #11713") +async def test_remove_node_with_concurrent_ddl(manager: ManagerClient, random_tables: RandomTables): + stopped = False + ddl_failed = False + + async def do_ddl(): + nonlocal ddl_failed + iteration = 0 + while not stopped: + logger.debug(f'ddl, iteration {iteration} started') + try: + # If the node was removed, the driver may retry "create table" on another node, + # but the request might have already been completed. + # The same applies to drop_table. + + await random_tables.add_tables(5, 5, if_not_exists=True) + await random_tables.verify_schema() + while len(random_tables.tables) > 0: + await random_tables.drop_table(random_tables.tables[-1], if_exists=True) + logger.debug(f'ddl, iteration {iteration} finished') + except: + logger.exception(f'ddl, iteration {iteration} failed') + ddl_failed = True + raise + iteration += 1 + + async def do_remove_node(): + for i in range(10): + logger.debug(f'do_remove_node [{i}], iteration started') + if ddl_failed: + logger.debug(f'do_remove_node [{i}], ddl failed, exiting') + break + server_ids = await manager.running_servers() + host_ids = await asyncio.gather(*(manager.get_host_id(s) for s in server_ids)) + initiator_index, target_index = random.sample(range(len(server_ids)), 2) + initiator_ip = server_ids[initiator_index] + target_ip = server_ids[target_index] + target_host_id = host_ids[target_index] + logger.info(f'do_remove_node [{i}], running remove_node, ' + f'initiator server [{initiator_ip}], target ip [{target_ip}], ' + f'target host id [{target_host_id}]') + await manager.wait_for_host_known(initiator_ip, target_host_id) + logger.info(f'do_remove_node [{i}], stopping target server [{target_ip}], host_id [{target_host_id}]') + await manager.server_stop_gracefully(target_ip) + logger.info(f'do_remove_node [{i}], target server [{target_ip}] stopped, ' + f'waiting for it to be down on [{initiator_ip}]') + await manager.wait_for_host_down(initiator_ip, target_ip) + logger.info(f'do_remove_node [{i}], invoking remove_node') + await manager.remove_node(initiator_ip, target_ip, target_host_id) + # TODO: check that group 0 no longer contains the removed node (#12153) + logger.info(f'do_remove_node [{i}], remove_node done') + new_server_ip = await manager.server_add() + logger.info(f'do_remove_node [{i}], server_add [{new_server_ip}] done') + logger.info(f'do_remove_node [{i}], iteration finished') + + ddl_task = asyncio.create_task(do_ddl()) + try: + await do_remove_node() + finally: + logger.debug("do_remove_node finished, waiting for ddl fiber") + stopped = True + await ddl_task + logger.debug("ddl fiber done, finished") diff --git a/test/topology/test_topology_remove_garbage_group0.py b/test/topology/test_topology_remove_garbage_group0.py new file mode 100644 index 0000000000..90558b58d3 --- /dev/null +++ b/test/topology/test_topology_remove_garbage_group0.py @@ -0,0 +1,126 @@ +# +# Copyright (C) 2022-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +""" +Test removenode with node with node no longer member +""" +import logging +from test.pylib.manager_client import ManagerClient +from test.pylib.rest_client import inject_error_one_shot +from test.topology.util import get_token_ring_host_ids, get_current_group0_config, \ + check_token_ring_and_group0_consistency +import pytest + + +logger = logging.getLogger(__name__) + + +async def test_remove_garbage_group0_members(manager: ManagerClient): + """ + Verify that failing to leave group 0 or remove a node from group 0 in removenode/decommission + can be handled by executing removenode (which should clear the 'garbage' group 0 member), + even though the node is no longer a token ring member. + """ + # 4 servers, one dead + await manager.server_add() + servers = await manager.running_servers() + removed_host_id = await manager.get_host_id(servers[0].server_id) + await manager.server_stop_gracefully(servers[0].server_id) + + logging.info(f'removenode {servers[0]} using {servers[1]}') + # removenode will fail after removing the server from the token ring, + # but before removing it from group 0 + await inject_error_one_shot(manager.api, servers[1].ip_addr, + 'removenode_fail_before_remove_from_group0') + try: + await manager.remove_node(servers[1].server_id, servers[0].server_id) + except Exception: + # Note: the exception returned here is only '500 internal server error', + # need to look in test.py log for the actual message coming from Scylla. + logging.info(f'expected exception during injection') + + # Query the storage_service/host_id endpoint to calculate a list of known token ring members' Host IDs + # (internally, this endpoint uses token_metadata) + token_ring_ids = await get_token_ring_host_ids(manager, servers[1]) + logging.info(f'token ring members: {token_ring_ids}') + + group0_members = await get_current_group0_config(manager, servers[1]) + logging.info(f'group 0 members: {group0_members}') + group0_ids = {m[0] for m in group0_members} + + # Token ring members should currently be a subset of group 0 members + assert token_ring_ids <= group0_ids + + garbage_members = group0_ids - token_ring_ids + logging.info(f'garbage members: {garbage_members}') + assert len(garbage_members) == 1 + garbage_member = next(iter(garbage_members)) + + # The garbage member is the one that we failed to remove + assert garbage_member == removed_host_id + + # Verify that at least it's a non-voter. + assert garbage_member in {m[0] for m in group0_members if not m[1]} + + logging.info(f'removenode {servers[0]} using {servers[1]} again') + # Retry removenode. It should skip the token ring removal step and remove the server from group 0. + await manager.remove_node(servers[1].server_id, servers[0].server_id) + + group0_members = await get_current_group0_config(manager, servers[1]) + logging.info(f'group 0 members: {group0_members}') + group0_ids = {m[0] for m in group0_members} + + # Token ring members and group 0 members should now be the same. + assert token_ring_ids == group0_ids + + # Verify that availability is not reduced. + # Stop one of the 3 remaining servers and try to remove it. It should succeed with only 2 servers. + + logging.info(f'stop {servers[1]}') + await manager.server_stop_gracefully(servers[1].server_id) + + logging.info(f'removenode {servers[1]} using {servers[2]}') + await manager.remove_node(servers[2].server_id, servers[1].server_id) + + # Perform a similar scenario with decommission. One of the node fails to decommission fully, + # but it manages to leave the token ring. We observe the leftovers using the same APIs as above + # and remove the leftovers. + # We can do this with only 2 nodes because during decommission we become a non-voter before + # leaving the token ring, thus the remaining single node will become a voting majority + # and will be able to perform removenode alone. + + decommissioned_host_id = await manager.get_host_id(servers[2].server_id) + await manager.api.enable_injection( + servers[2].ip_addr, 'decommission_fail_before_leave_group0', one_shot=True) + logging.info(f'decommission {servers[2]}') + try: + await manager.decommission_node(servers[2].server_id) + except Exception: + logging.info(f'expected exception during injection') + logging.info(f'stop {servers[2]}') + await manager.server_stop_gracefully(servers[2].server_id) + + token_ring_ids = await get_token_ring_host_ids(manager, servers[3]) + logging.info(f'token ring members: {token_ring_ids}') + + group0_members = await get_current_group0_config(manager, servers[3]) + logging.info(f'group 0 members: {group0_members}') + group0_ids = {m[0] for m in group0_members} + + assert token_ring_ids <= group0_ids + + garbage_members = group0_ids - token_ring_ids + logging.info(f'garbage members: {garbage_members}') + assert len(garbage_members) == 1 + garbage_member = next(iter(garbage_members)) + + assert garbage_member == decommissioned_host_id + assert garbage_member in {m[0] for m in group0_members if not m[1]} + + logging.info(f'removenode {servers[2]} using {servers[3]}') + await manager.remove_node(servers[3].server_id, servers[2].server_id) + + await check_token_ring_and_group0_consistency(manager) + diff --git a/test/topology/test_topology_schema.py b/test/topology/test_topology_schema.py new file mode 100644 index 0000000000..3d5e30091b --- /dev/null +++ b/test/topology/test_topology_schema.py @@ -0,0 +1,34 @@ +# +# Copyright (C) 2022-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +""" +Test consistency of schema changes with server hard stop. +""" +import time +from test.topology.util import wait_for_token_ring_and_group0_consistency +import pytest + + +@pytest.mark.asyncio +async def test_topology_schema_changes(manager, random_tables): + """Test schema consistency with restart, add, and sudden stop of servers""" + table = await random_tables.add_table(ncolumns=5) + servers = await manager.running_servers() + + # Test add column after server restart + await manager.server_restart(servers[1].server_id) + await table.add_column() + await random_tables.verify_schema() + + # Test add column after adding a server + await manager.server_add() + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + await table.add_column() + await random_tables.verify_schema() + + # Test add column after hard stop of a server (1/3) + await manager.server_stop(servers[1].server_id) + await table.add_column() + await random_tables.verify_schema() diff --git a/test/topology/test_topology_smp.py b/test/topology/test_topology_smp.py new file mode 100644 index 0000000000..f753ec790b --- /dev/null +++ b/test/topology/test_topology_smp.py @@ -0,0 +1,60 @@ +# +# Copyright (C) 2022-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +""" +Test functionality on the cluster with different values of the --smp parameter on the nodes. +""" +import logging +import time +from test.pylib.manager_client import ManagerClient +from test.pylib.random_tables import RandomTables +from test.topology.util import wait_for_token_ring_and_group0_consistency +import pytest + +logger = logging.getLogger(__name__) + + +# Checks basic functionality on the cluster with different values of the --smp parameter on the nodes. +@pytest.mark.asyncio +async def test_nodes_with_different_smp(manager: ManagerClient, random_tables: RandomTables) -> None: + # In this test it's more convenient to start with a fresh cluster. + # We don't need the default nodes, + # but there is currently no way to express this in the test infrastructure + + # When the node starts it tries to communicate with others + # by sending group0_peer_exchange message to them. + # This message can be handled on arbitrary shard of the target node. + # The method manager.server_add waits for node to start, which can only happen + # if this message has been handled correctly. + # + # Note: messaging_service is initialized with server_socket::load_balancing_algorithm::port + # policy, this means that the shard for message will be chosen as client_port % smp::count. + # The client port in turn is chosen as rand() * smp::count + current_shard + # (posix_socket_impl::find_port_and_connect). + # If this succeeds to occupy a free port in 5 tries and smp::count is the same + # on both nodes, then it's guaranteed that the message will be + # processed on the same shard as the calling code. + # In the general case, we cannot assume that this same shard guarantee holds. + logger.info(f'Adding --smp=3 server') + await manager.server_add(cmdline=['--smp', '3']) + + # Remove the original 3 servers, the problem is easier to reproduce with --smp values + # that we pick, not the (currently) default --smp=2 coming from the suite. + logger.info(f'Decommissioning old servers') + servers = await manager.running_servers() + for s in servers[:-1]: + await manager.decommission_node(s.server_id) + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + + logger.info(f'Adding --smp=4 server') + await manager.server_add(cmdline=['--smp', '4']) + logger.info(f'Adding --smp=5 server') + await manager.server_add(cmdline=['--smp', '5']) + + await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) + + logger.info(f'Creating new tables') + await random_tables.add_tables(ntables=4, ncolumns=5) + await random_tables.verify_schema() diff --git a/test/topology/util.py b/test/topology/util.py new file mode 100644 index 0000000000..87ba5dddb4 --- /dev/null +++ b/test/topology/util.py @@ -0,0 +1,79 @@ +# +# Copyright (C) 2022-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +""" +Test consistency of schema changes with topology changes. +""" +import logging +import time +from test.pylib.util import wait_for, wait_for_cql_and_get_hosts +from test.pylib.internal_types import ServerInfo +from test.pylib.manager_client import ManagerClient + + +logger = logging.getLogger(__name__) + + +async def get_token_ring_host_ids(manager: ManagerClient, srv: ServerInfo) -> set[str]: + """Get the host IDs of normal token owners known by `srv`.""" + token_endpoint_map = await manager.api.client.get_json("/storage_service/tokens_endpoint", srv.ip_addr) + normal_endpoints = {e["value"] for e in token_endpoint_map} + logger.info(f"Normal endpoints' IPs by {srv}: {normal_endpoints}") + host_id_map = await manager.api.client.get_json('/storage_service/host_id', srv.ip_addr) + all_host_ids = {e["value"] for e in host_id_map} + logger.info(f"All host IDs by {srv}: {all_host_ids}") + normal_host_ids = {e["value"] for e in host_id_map if e["key"] in normal_endpoints} + logger.info(f"Normal endpoints' host IDs by {srv}: {normal_host_ids}") + return normal_host_ids + + +async def get_current_group0_config(manager: ManagerClient, srv: ServerInfo) -> set[tuple[str, bool]]: + """Get the current Raft group 0 configuration known by `srv`. + The first element of each tuple is the Raft ID of the node (which is equal to the Host ID), + the second element indicates whether the node is a voter. + """ + assert manager.cql + host = (await wait_for_cql_and_get_hosts(manager.cql, [srv], time.time() + 60))[0] + group0_id = (await manager.cql.run_async( + "select value from system.scylla_local where key = 'raft_group0_id'", + host=host))[0].value + config = await manager.cql.run_async( + f"select server_id, can_vote from system.raft_state where group_id = {group0_id} and disposition = 'CURRENT'", + host=host) + result = {(str(m.server_id), bool(m.can_vote)) for m in config} + logger.info(f"Group 0 members by {srv}: {result}") + return result + + +async def check_token_ring_and_group0_consistency(manager: ManagerClient) -> None: + """Ensure that the normal token owners and group 0 members match + according to each currently running server. + """ + servers = await manager.running_servers() + for srv in servers: + group0_members = await get_current_group0_config(manager, srv) + group0_ids = {m[0] for m in group0_members} + token_ring_ids = await get_token_ring_host_ids(manager, srv) + assert token_ring_ids == group0_ids + + +async def wait_for_token_ring_and_group0_consistency(manager: ManagerClient, deadline: float) -> None: + """Weaker version of the above check; the token ring is not immediately updated after + bootstrap/replace/decommission - the normal tokens of the new node propagate through gossip. + Take this into account and wait for the equality condition to hold, with a timeout. + """ + servers = await manager.running_servers() + for srv in servers: + group0_members = await get_current_group0_config(manager, srv) + group0_ids = {m[0] for m in group0_members} + async def token_ring_matches(): + token_ring_ids = await get_token_ring_host_ids(manager, srv) + diff = token_ring_ids ^ group0_ids + if diff: + logger.warning(f"Group 0 members and token ring members don't yet match" \ + f" according to {srv}, symmetric difference: {diff}") + return None + return True + await wait_for(token_ring_matches, deadline, period=.5) From b39cdadff3e8a9f4ab7377f1896fb6fa8cc42e38 Mon Sep 17 00:00:00 2001 From: Alejo Sanchez Date: Tue, 21 Mar 2023 17:06:36 +0100 Subject: [PATCH 09/28] test/pylib: topology: support clusters of initial size 0 To allow tests with custom clusters, allow configuration of initial cluster size of 0. Add a proof-of-concept test to be removed later. Signed-off-by: Alejo Sanchez Closes #13342 (cherry picked from commit e3b462507d4d8a92ce017b6d97ca893490430095) --- test/pylib/manager_client.py | 8 ++++++-- test/pylib/scylla_cluster.py | 5 +++-- test/topology_custom/__init__.py | 0 test/topology_custom/conftest.py | 9 +++++++++ test/topology_custom/pytest.ini | 9 +++++++++ test/topology_custom/suite.yaml | 6 ++++++ test/topology_custom/test_custom.py | 19 +++++++++++++++++++ 7 files changed, 52 insertions(+), 4 deletions(-) create mode 100644 test/topology_custom/__init__.py create mode 100644 test/topology_custom/conftest.py create mode 100644 test/topology_custom/pytest.ini create mode 100644 test/topology_custom/suite.yaml create mode 100644 test/topology_custom/test_custom.py diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index 47cb3cefdc..24c8e601f7 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -81,7 +81,8 @@ class ManagerClient(): logger.info(f"Using cluster: {cluster_str} for test {test_case_name}") except aiohttp.ClientError as exc: raise RuntimeError(f"Failed before test check {exc}") from exc - if self.cql is None: + servers = await self.running_servers() + if self.cql is None and servers: # TODO: if cluster is not up yet due to taking long and HTTP timeout, wait for it # await self._wait_for_cluster() await self.driver_connect() # Connect driver to new cluster @@ -168,7 +169,10 @@ class ManagerClient(): except Exception as exc: raise RuntimeError(f"server_add got invalid server data {server_info}") from exc logger.debug("ManagerClient added %s", s_info) - self._driver_update() + if self.cql: + self._driver_update() + else: + await self.driver_connect() return s_info async def remove_node(self, initiator_id: ServerNum, server_id: ServerNum, diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index ab75854b68..c4f47e90d7 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -570,7 +570,8 @@ class ScyllaCluster: try: for _ in range(self.replicas): await self.add_server() - self.keyspace_count = self._get_keyspace_count() + if self.replicas > 0: + self.keyspace_count = self._get_keyspace_count() except Exception as exc: # If start fails, swallow the error to throw later, # at test time. @@ -751,7 +752,7 @@ class ScyllaCluster: self.logger.info(f"The cluster {self.name} is dirty, not checking" f" keyspace count post-condition") else: - if self._get_keyspace_count() != self.keyspace_count: + if self.running and self._get_keyspace_count() != self.keyspace_count: raise RuntimeError(f"Test post-condition on cluster {self.name} failed, " f"the test must drop all keyspaces it creates.") for server in itertools.chain(self.running.values(), self.stopped.values()): diff --git a/test/topology_custom/__init__.py b/test/topology_custom/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/test/topology_custom/conftest.py b/test/topology_custom/conftest.py new file mode 100644 index 0000000000..74f19717c3 --- /dev/null +++ b/test/topology_custom/conftest.py @@ -0,0 +1,9 @@ +# +# Copyright (C) 2023-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +# This file configures pytest for all tests in this directory, and also +# defines common test fixtures for all of them to use + +from test.topology.conftest import * diff --git a/test/topology_custom/pytest.ini b/test/topology_custom/pytest.ini new file mode 100644 index 0000000000..aa0aef9e89 --- /dev/null +++ b/test/topology_custom/pytest.ini @@ -0,0 +1,9 @@ +[pytest] +asyncio_mode = auto + +log_cli = true +log_format = %(asctime)s.%(msecs)03d %(levelname)s> %(message)s +log_date_format = %H:%M:%S + +markers = + slow: tests that take more than 30 seconds to run diff --git a/test/topology_custom/suite.yaml b/test/topology_custom/suite.yaml new file mode 100644 index 0000000000..194dd0fc2a --- /dev/null +++ b/test/topology_custom/suite.yaml @@ -0,0 +1,6 @@ +type: Topology +pool_size: 4 +cluster_size: 0 +extra_scylla_config_options: + authenticator: AllowAllAuthenticator + authorizer: AllowAllAuthorizer diff --git a/test/topology_custom/test_custom.py b/test/topology_custom/test_custom.py new file mode 100644 index 0000000000..691f4000c3 --- /dev/null +++ b/test/topology_custom/test_custom.py @@ -0,0 +1,19 @@ +# +# Copyright (C) 2023-present ScyllaDB +# +# SPDX-License-Identifier: AGPL-3.0-or-later +# +from test.pylib.manager_client import ManagerClient +from test.pylib.random_tables import RandomTables +from test.pylib.util import unique_name +import pytest + + +@pytest.mark.asyncio +async def test_custom(request, manager: ManagerClient): + servers = [await manager.server_add(), await manager.server_add(), await manager.server_add()] + tables = RandomTables(request.node.name, manager, unique_name()) + table = await tables.add_table(ncolumns=5) + await table.insert_seq() + await table.add_index(2) + await tables.verify_schema(table) From 6474edd6917041651b954623c2790b0f21886d5e Mon Sep 17 00:00:00 2001 From: Alejo Sanchez Date: Thu, 20 Apr 2023 10:19:19 +0200 Subject: [PATCH 10/28] test: topology smp test with custom cluster Instead of decommission of initial cluster, use custom cluster. Signed-off-by: Alejo Sanchez Closes #13589 (cherry picked from commit ce87aedd306dd503467cd36d6a6c8aa641b8786b) --- .../test_topology_smp.py | 20 ++++++------------- 1 file changed, 6 insertions(+), 14 deletions(-) rename test/{topology => topology_custom}/test_topology_smp.py (72%) diff --git a/test/topology/test_topology_smp.py b/test/topology_custom/test_topology_smp.py similarity index 72% rename from test/topology/test_topology_smp.py rename to test/topology_custom/test_topology_smp.py index f753ec790b..28d31a3cf4 100644 --- a/test/topology/test_topology_smp.py +++ b/test/topology_custom/test_topology_smp.py @@ -10,18 +10,18 @@ import logging import time from test.pylib.manager_client import ManagerClient from test.pylib.random_tables import RandomTables +from test.pylib.util import unique_name from test.topology.util import wait_for_token_ring_and_group0_consistency import pytest +from pytest import FixtureRequest logger = logging.getLogger(__name__) # Checks basic functionality on the cluster with different values of the --smp parameter on the nodes. @pytest.mark.asyncio -async def test_nodes_with_different_smp(manager: ManagerClient, random_tables: RandomTables) -> None: +async def test_nodes_with_different_smp(request: FixtureRequest, manager: ManagerClient) -> None: # In this test it's more convenient to start with a fresh cluster. - # We don't need the default nodes, - # but there is currently no way to express this in the test infrastructure # When the node starts it tries to communicate with others # by sending group0_peer_exchange message to them. @@ -39,15 +39,6 @@ async def test_nodes_with_different_smp(manager: ManagerClient, random_tables: R # In the general case, we cannot assume that this same shard guarantee holds. logger.info(f'Adding --smp=3 server') await manager.server_add(cmdline=['--smp', '3']) - - # Remove the original 3 servers, the problem is easier to reproduce with --smp values - # that we pick, not the (currently) default --smp=2 coming from the suite. - logger.info(f'Decommissioning old servers') - servers = await manager.running_servers() - for s in servers[:-1]: - await manager.decommission_node(s.server_id) - await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) - logger.info(f'Adding --smp=4 server') await manager.server_add(cmdline=['--smp', '4']) logger.info(f'Adding --smp=5 server') @@ -56,5 +47,6 @@ async def test_nodes_with_different_smp(manager: ManagerClient, random_tables: R await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30) logger.info(f'Creating new tables') - await random_tables.add_tables(ntables=4, ncolumns=5) - await random_tables.verify_schema() + tables = RandomTables(request.node.name, manager, unique_name(), 3) + await tables.add_tables(ntables=4, ncolumns=5) + await tables.verify_schema() From d2caaef18889fe04d4f65736c4ad3b85262c71be Mon Sep 17 00:00:00 2001 From: Konstantin Osipov Date: Fri, 3 Mar 2023 21:14:29 +0300 Subject: [PATCH 11/28] test: improve logging in ScyllaCluster Print IP addresses and cluster identifiers in more log messages, it helps debugging. (cherry picked from commit 7309a1bd6bb7e1b6e55217e622b352bef8e6809e) --- test/pylib/scylla_cluster.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index c4f47e90d7..96de276efd 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -583,7 +583,7 @@ class ScyllaCluster: async def uninstall(self) -> None: """Stop running servers and uninstall all servers""" self.is_dirty = True - self.logger.info("Uninstalling cluster") + self.logger.info("Uninstalling cluster %s", self) await self.stop() await asyncio.gather(*(srv.uninstall() for srv in self.stopped.values())) await asyncio.gather(*(self.host_registry.release_host(Host(ip)) @@ -593,6 +593,7 @@ class ScyllaCluster: """Release all IPs leased from the host registry by this cluster. Call this function only if the cluster is stopped and will not be started again.""" assert not self.running + self.logger.info("Cluster %s releases ips %s", self, self.leased_ips) while self.leased_ips: ip = self.leased_ips.pop() await self.host_registry.release_host(Host(ip)) @@ -785,7 +786,6 @@ class ScyllaCluster: async def server_start(self, server_id: ServerNum) -> ActionReturn: """Start a stopped server""" - self.logger.info("Cluster %s starting server %s", self, server_id) if server_id in self.running: return ScyllaCluster.ActionReturn(success=True, msg=f"{self.running[server_id]} already running") @@ -793,6 +793,8 @@ class ScyllaCluster: return ScyllaCluster.ActionReturn(success=False, msg=f"Server {server_id} unknown") self.is_dirty = True server = self.stopped.pop(server_id) + self.logger.info("Cluster %s starting server %s ip %s", self, + server_id, server.ip_addr) server.seeds = self._seeds() # Put the server in `running` before starting it. # Starting may fail and if we didn't add it now it might leak. From 930c4e65d636c68e02db0246a7570a48ac3b9fca Mon Sep 17 00:00:00 2001 From: Petr Gusev Date: Wed, 22 Mar 2023 13:30:08 +0400 Subject: [PATCH 12/28] scylla_cluster.py: fix ScyllaServer.start, reset cmd if start failed The ScyllaServer expects cmd to be None if the Scylla process is not running. Otherwise, if start failed and the test called update_config, the latter will try to send a signal to a non-existent process via cmd. (cherry picked from commit 21b505e67c98239eaa01c52c74c1cc274c310cbd) --- test/pylib/scylla_cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 96de276efd..69cdef093b 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -383,6 +383,7 @@ class ScyllaServer: while time.time() < self.start_time + self.TOPOLOGY_TIMEOUT: if self.cmd.returncode: + self.cmd = None with self.log_filename.open('r') as log_file: self.logger.error("failed to start server at host %s in %s", self.ip_addr, self.workdir.name) From 43525aec83757133af35dcc9084cffe976ea76ed Mon Sep 17 00:00:00 2001 From: Petr Gusev Date: Wed, 22 Mar 2023 15:10:44 +0400 Subject: [PATCH 13/28] scylla_cluster.py: ScyllaServer.start, refactor error reporting Extract the function that encapsulates all the error reporting logic. We are going to use it in several other places to implement expected_error feature. (cherry picked from commit a4411e9ec4f06beb9fa7d4a6959c45f0a93453d0) --- test/pylib/scylla_cluster.py | 46 ++++++++++++++++-------------------- 1 file changed, 20 insertions(+), 26 deletions(-) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 69cdef093b..a631c1aa27 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -381,24 +381,28 @@ class ScyllaServer: sleep_interval = 0.1 cql_up_state = CqlUpState.NOT_CONNECTED + def report_error(message: str): + message += f", server_id {self.server_id}, IP {self.ip_addr}, workdir {self.workdir.name}" + message += f", host_id {self.host_id if hasattr(self, 'host_id') else ''}" + message += f", cql [{'connected' if cql_up_state == CqlUpState.CONNECTED else 'not connected'}]" + self.logger.error(message) + self.logger.error("last line of %s:", self.log_filename) + with self.log_filename.open('r') as log_file: + log_file.seek(0, 0) + self.logger.error(log_file.readlines()[-1].rstrip()) + log_handler = logging.getLogger().handlers[0] + if hasattr(log_handler, 'baseFilename'): + logpath = log_handler.baseFilename # type: ignore + else: + logpath = "?" + raise RuntimeError(message + "\nCheck the log files:\n" + f"{logpath}\n" + f"{self.log_filename}") + while time.time() < self.start_time + self.TOPOLOGY_TIMEOUT: if self.cmd.returncode: self.cmd = None - with self.log_filename.open('r') as log_file: - self.logger.error("failed to start server at host %s in %s", - self.ip_addr, self.workdir.name) - self.logger.error("last line of %s:", self.log_filename) - log_file.seek(0, 0) - self.logger.error(log_file.readlines()[-1].rstrip()) - log_handler = logging.getLogger().handlers[0] - if hasattr(log_handler, 'baseFilename'): - logpath = log_handler.baseFilename # type: ignore - else: - logpath = "?" - raise RuntimeError(f"Failed to start server with ID = {self.server_id}, IP = {self.ip_addr}.\n" - "Check the log files:\n" - f"{logpath}\n" - f"{self.log_filename}") + report_error("failed to start the node") if hasattr(self, "host_id") or await self.get_host_id(api): cql_up_state = await self.cql_is_up() @@ -408,17 +412,7 @@ class ScyllaServer: # Sleep and retry await asyncio.sleep(sleep_interval) - err = f"Failed to start server with ID = {self.server_id}, IP = {self.ip_addr}." - if hasattr(self, "host_id"): - err += f" Managed to obtain the server's Host ID ({self.host_id})" - if cql_up_state == CqlUpState.CONNECTED: - err += " and to connect the CQL driver, but failed to execute a query." - else: - err += " but failed to connect the CQL driver." - else: - err += " Failed to obtain the server's Host ID." - err += f"\nCheck server log at {self.log_filename}." - raise RuntimeError(err) + report_error('failed to start the node, timeout reached') async def force_schema_migration(self) -> None: """This is a hack to change schema hash on an existing cluster node From 1959eddf8643192fea85574fbb6a5ccbaada9393 Mon Sep 17 00:00:00 2001 From: Petr Gusev Date: Fri, 17 Mar 2023 16:18:33 +0400 Subject: [PATCH 14/28] scylla_cluster.py: add expected_error to server_start Sometimes it's useful to check that the node has failed to start for a particular reason. If server_start can't find expected_error in the node's log or if the node has started without errors, it throws an exception. (cherry picked from commit c1d0ee2bce135f18351adecdde85f6ab72708adf) --- test/pylib/manager_client.py | 5 +++-- test/pylib/scylla_cluster.py | 22 ++++++++++++++++++---- 2 files changed, 21 insertions(+), 6 deletions(-) diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index 24c8e601f7..b6176edecd 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -138,10 +138,11 @@ class ManagerClient(): logger.debug("ManagerClient stopping gracefully %s", server_id) await self.client.get_text(f"/cluster/server/{server_id}/stop_gracefully") - async def server_start(self, server_id: ServerNum) -> None: + async def server_start(self, server_id: ServerNum, expected_error: Optional[str] = None) -> None: """Start specified server""" logger.debug("ManagerClient starting %s", server_id) - await self.client.get_text(f"/cluster/server/{server_id}/start") + params = {'expected_error': expected_error} if expected_error is not None else None + await self.client.get_text(f"/cluster/server/{server_id}/start", params=params) self._driver_update() async def server_restart(self, server_id: ServerNum) -> None: diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index a631c1aa27..fd2ce8f92c 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -362,7 +362,7 @@ class ScyllaServer: return False # Any other exception may indicate a problem, and is passed to the caller. - async def start(self, api: ScyllaRESTAPIClient) -> None: + async def start(self, api: ScyllaRESTAPIClient, expected_error: Optional[str] = None) -> None: """Start an installed server. May be used for restarts.""" env = os.environ.copy() @@ -385,6 +385,8 @@ class ScyllaServer: message += f", server_id {self.server_id}, IP {self.ip_addr}, workdir {self.workdir.name}" message += f", host_id {self.host_id if hasattr(self, 'host_id') else ''}" message += f", cql [{'connected' if cql_up_state == CqlUpState.CONNECTED else 'not connected'}]" + if expected_error is not None: + message += f", the node log was expected to contain the string [{expected_error}]" self.logger.error(message) self.logger.error("last line of %s:", self.log_filename) with self.log_filename.open('r') as log_file: @@ -402,11 +404,19 @@ class ScyllaServer: while time.time() < self.start_time + self.TOPOLOGY_TIMEOUT: if self.cmd.returncode: self.cmd = None + if expected_error is not None: + with self.log_filename.open('r') as log_file: + for line in log_file.readlines(): + if expected_error in line: + return + report_error("the node startup failed, but the log file doesn't contain the expected error") report_error("failed to start the node") if hasattr(self, "host_id") or await self.get_host_id(api): cql_up_state = await self.cql_is_up() if cql_up_state == CqlUpState.QUERIED: + if expected_error is not None: + report_error("the node started, but was expected to fail with the expected error") return # Sleep and retry @@ -779,7 +789,7 @@ class ScyllaCluster: self.logger.debug("Cluster %s marking server %s as removed", self, server_id) self.removed.add(server_id) - async def server_start(self, server_id: ServerNum) -> ActionReturn: + async def server_start(self, server_id: ServerNum, expected_error: Optional[str] = None) -> ActionReturn: """Start a stopped server""" if server_id in self.running: return ScyllaCluster.ActionReturn(success=True, @@ -794,7 +804,10 @@ class ScyllaCluster: # Put the server in `running` before starting it. # Starting may fail and if we didn't add it now it might leak. self.running[server_id] = server - await server.start(self.api) + await server.start(self.api, expected_error) + if expected_error is not None: + self.running.pop(server_id) + self.stopped[server_id] = server return ScyllaCluster.ActionReturn(success=True, msg=f"{server} started") async def server_restart(self, server_id: ServerNum) -> ActionReturn: @@ -1031,7 +1044,8 @@ class ScyllaClusterManager: """Start a specified server (must be stopped)""" assert self.cluster server_id = ServerNum(int(request.match_info["server_id"])) - ret = await self.cluster.server_start(server_id) + expected_error = request.query.get("expected_error") + ret = await self.cluster.server_start(server_id, expected_error) return aiohttp.web.Response(status=200 if ret[0] else 500, text=ret[1]) async def _cluster_server_restart(self, request) -> aiohttp.web.Response: From 3ab610753ecbc7685b1bdb400868734ef647c94c Mon Sep 17 00:00:00 2001 From: Petr Gusev Date: Fri, 17 Mar 2023 16:25:17 +0400 Subject: [PATCH 15/28] scylla_cluster.py: add config to server_add Sometimes when creating a node it's useful to pass a custom node config. (cherry picked from commit 8e3392c64f624d9d30e3f697672f86dce2e8a649) --- test/pylib/manager_client.py | 4 +++- test/pylib/scylla_cluster.py | 6 +++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index b6176edecd..9be6bc121e 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -151,7 +151,7 @@ class ManagerClient(): await self.client.get_text(f"/cluster/server/{server_id}/restart") self._driver_update() - async def server_add(self, replace_cfg: Optional[ReplaceConfig] = None, cmdline: Optional[List[str]] = None) -> ServerInfo: + async def server_add(self, replace_cfg: Optional[ReplaceConfig] = None, cmdline: Optional[List[str]] = None, config: Optional[dict[str, str]] = None) -> ServerInfo: """Add a new server""" try: data: dict[str, Any] = {} @@ -159,6 +159,8 @@ class ManagerClient(): data['replace_cfg'] = replace_cfg._asdict() if cmdline: data['cmdline'] = cmdline + if config: + data['config'] = config server_info = await self.client.put_json("/cluster/addserver", data, response_type="json", timeout=ScyllaServer.TOPOLOGY_TIMEOUT) except Exception as exc: diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index fd2ce8f92c..ca4fd809ae 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -628,11 +628,11 @@ class ScyllaCluster: def _seeds(self) -> List[str]: return [server.ip_addr for server in self.running.values()] - async def add_server(self, replace_cfg: Optional[ReplaceConfig] = None, cmdline: Optional[List[str]] = None) -> ServerInfo: + async def add_server(self, replace_cfg: Optional[ReplaceConfig] = None, cmdline: Optional[List[str]] = None, config: Optional[dict[str, str]] = None) -> ServerInfo: """Add a new server to the cluster""" self.is_dirty = True - extra_config: dict[str, str] = {} + extra_config: dict[str, str] = config.copy() if config else {} if replace_cfg: replaced_id = replace_cfg.replaced_id assert replaced_id in self.servers, \ @@ -1060,7 +1060,7 @@ class ScyllaClusterManager: assert self.cluster data = await request.json() replace_cfg = ReplaceConfig(**data["replace_cfg"]) if "replace_cfg" in data else None - s_info = await self.cluster.add_server(replace_cfg, data.get('cmdline')) + s_info = await self.cluster.add_server(replace_cfg, data.get('cmdline'), data.get('config')) return aiohttp.web.json_response({"server_id" : s_info.server_id, "ip_addr": s_info.ip_addr, "host_id": s_info.host_id}) From 15ea5bf53f4586738f7cc91dc3ec8c9328f0eff9 Mon Sep 17 00:00:00 2001 From: Petr Gusev Date: Wed, 22 Mar 2023 17:17:47 +0400 Subject: [PATCH 16/28] ServerInfo: drop host_id We are going to allow the ScyllaCluster.add_server function not to start the server if the caller has requested that with a special parameter. The host_id can only be obtained from a running node, so add_server won't be able to return it in this case. I've grepped the tests for host_id and there doesn't seem to be any reference to it in the code. (cherry picked from commit 794d0e4000734e5857c6dc274941755a7b32f1a7) --- test/pylib/internal_types.py | 3 +-- test/pylib/manager_client.py | 5 ++--- test/pylib/scylla_cluster.py | 9 ++++----- 3 files changed, 7 insertions(+), 10 deletions(-) diff --git a/test/pylib/internal_types.py b/test/pylib/internal_types.py index c6673dd999..b3e8cba577 100644 --- a/test/pylib/internal_types.py +++ b/test/pylib/internal_types.py @@ -18,6 +18,5 @@ class ServerInfo(NamedTuple): """Server id (test local) and IP address""" server_id: ServerNum ip_addr: IPAddress - host_id: HostID def __str__(self): - return f"Server({self.server_id}, {self.ip_addr}, {self.host_id})" + return f"Server({self.server_id}, {self.ip_addr})" diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index 9be6bc121e..ca52a669ac 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -120,7 +120,7 @@ class ManagerClient(): except RuntimeError as exc: raise Exception("Failed to get list of running servers") from exc assert isinstance(server_info_list, list), "running_servers got unknown data type" - return [ServerInfo(ServerNum(int(info[0])), IPAddress(info[1]), HostID(info[2])) + return [ServerInfo(ServerNum(int(info[0])), IPAddress(info[1])) for info in server_info_list] async def mark_dirty(self) -> None: @@ -167,8 +167,7 @@ class ManagerClient(): raise Exception("Failed to add server") from exc try: s_info = ServerInfo(ServerNum(int(server_info["server_id"])), - IPAddress(server_info["ip_addr"]), - HostID(server_info["host_id"])) + IPAddress(server_info["ip_addr"])) except Exception as exc: raise RuntimeError(f"server_add got invalid server data {server_info}") from exc logger.debug("ManagerClient added %s", s_info) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index ca4fd809ae..6fea996508 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -683,7 +683,7 @@ class ScyllaCluster: raise self.running[server.server_id] = server self.logger.info("Cluster %s added %s", self, server) - return ServerInfo(server.server_id, server.ip_addr, server.host_id) + return ServerInfo(server.server_id, server.ip_addr) def endpoint(self) -> str: """Get a server id (IP) from running servers""" @@ -715,9 +715,9 @@ class ScyllaCluster: stopped = ", ".join(str(server) for server in self.stopped.values()) return f"ScyllaCluster(name: {self.name}, running: {running}, stopped: {stopped})" - def running_servers(self) -> List[Tuple[ServerNum, IPAddress, HostID]]: + def running_servers(self) -> List[Tuple[ServerNum, IPAddress]]: """Get a list of tuples of server id and IP address of running servers (and not removed)""" - return [(server.server_id, server.ip_addr, server.host_id) for server in self.running.values() + return [(server.server_id, server.ip_addr) for server in self.running.values() if server.server_id not in self.removed] def _get_keyspace_count(self) -> int: @@ -1062,8 +1062,7 @@ class ScyllaClusterManager: replace_cfg = ReplaceConfig(**data["replace_cfg"]) if "replace_cfg" in data else None s_info = await self.cluster.add_server(replace_cfg, data.get('cmdline'), data.get('config')) return aiohttp.web.json_response({"server_id" : s_info.server_id, - "ip_addr": s_info.ip_addr, - "host_id": s_info.host_id}) + "ip_addr": s_info.ip_addr}) async def _cluster_remove_node(self, request: aiohttp.web.Request) -> aiohttp.web.Response: """Run remove node on Scylla REST API for a specified server""" From 7a8c5db55b9ffd874623bda2d4134ea7e04e5865 Mon Sep 17 00:00:00 2001 From: Petr Gusev Date: Fri, 17 Mar 2023 16:32:57 +0400 Subject: [PATCH 17/28] scylla_cluster.py: add start flag to server_add Sometimes when creating a node it's useful to just install it and not start. For example, we may want to try to start it later with expected error. The ScyllaServer.install method has been made exception safe, if an exception occurs, it reverts to the original state. This allows to not duplicate the try/except logic in two of its call sites. (cherry picked from commit e407956e9f0e258326f1373553d98d09d87a2eb3) --- test/pylib/manager_client.py | 4 ++-- test/pylib/scylla_cluster.py | 36 +++++++++++++++++++++++------------- 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index ca52a669ac..d7af493b46 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -151,10 +151,10 @@ class ManagerClient(): await self.client.get_text(f"/cluster/server/{server_id}/restart") self._driver_update() - async def server_add(self, replace_cfg: Optional[ReplaceConfig] = None, cmdline: Optional[List[str]] = None, config: Optional[dict[str, str]] = None) -> ServerInfo: + async def server_add(self, replace_cfg: Optional[ReplaceConfig] = None, cmdline: Optional[List[str]] = None, config: Optional[dict[str, str]] = None, start: bool = True) -> ServerInfo: """Add a new server""" try: - data: dict[str, Any] = {} + data: dict[str, Any] = {'start': start} if replace_cfg: data['replace_cfg'] = replace_cfg._asdict() if cmdline: diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 6fea996508..5481fa1e81 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -228,11 +228,7 @@ class ScyllaServer: async def install_and_start(self, api: ScyllaRESTAPIClient) -> None: """Setup and start this server""" - try: - await self.install() - except: - await self.uninstall() - raise + await self.install() self.logger.info("starting server at host %s in %s...", self.ip_addr, self.workdir.name) @@ -267,11 +263,19 @@ class ScyllaServer: # Cleanup any remains of the previously running server in this path shutil.rmtree(self.workdir, ignore_errors=True) - self.workdir.mkdir(parents=True, exist_ok=True) - self.config_filename.parent.mkdir(parents=True, exist_ok=True) - self._write_config_file() + try: + self.workdir.mkdir(parents=True, exist_ok=True) + self.config_filename.parent.mkdir(parents=True, exist_ok=True) + self._write_config_file() - self.log_file = self.log_filename.open("wb") + self.log_file = self.log_filename.open("wb") + except: + try: + shutil.rmtree(self.workdir) + except FileNotFoundError: + pass + self.log_filename.unlink(missing_ok=True) + raise def get_config(self) -> dict[str, object]: """Return the contents of conf/scylla.yaml as a dict.""" @@ -628,7 +632,7 @@ class ScyllaCluster: def _seeds(self) -> List[str]: return [server.ip_addr for server in self.running.values()] - async def add_server(self, replace_cfg: Optional[ReplaceConfig] = None, cmdline: Optional[List[str]] = None, config: Optional[dict[str, str]] = None) -> ServerInfo: + async def add_server(self, replace_cfg: Optional[ReplaceConfig] = None, cmdline: Optional[List[str]] = None, config: Optional[dict[str, str]] = None, start: bool = True) -> ServerInfo: """Add a new server to the cluster""" self.is_dirty = True @@ -673,7 +677,10 @@ class ScyllaCluster: try: server = self.create_server(params) self.logger.info("Cluster %s adding server...", self) - await server.install_and_start(self.api) + if start: + await server.install_and_start(self.api) + else: + await server.install() except Exception as exc: self.logger.error("Failed to start Scylla server at host %s in %s: %s", ip_addr, server.workdir.name, str(exc)) @@ -681,7 +688,10 @@ class ScyllaCluster: self.leased_ips.remove(ip_addr) await self.host_registry.release_host(Host(ip_addr)) raise - self.running[server.server_id] = server + if start: + self.running[server.server_id] = server + else: + self.stopped[server.server_id] = server self.logger.info("Cluster %s added %s", self, server) return ServerInfo(server.server_id, server.ip_addr) @@ -1060,7 +1070,7 @@ class ScyllaClusterManager: assert self.cluster data = await request.json() replace_cfg = ReplaceConfig(**data["replace_cfg"]) if "replace_cfg" in data else None - s_info = await self.cluster.add_server(replace_cfg, data.get('cmdline'), data.get('config')) + s_info = await self.cluster.add_server(replace_cfg, data.get('cmdline'), data.get('config'), data.get('start', True)) return aiohttp.web.json_response({"server_id" : s_info.server_id, "ip_addr": s_info.ip_addr}) From 59847389d4ecbe4e6143e219c34ebc6e29e68bdb Mon Sep 17 00:00:00 2001 From: Alejo Sanchez Date: Tue, 21 Mar 2023 15:18:33 +0100 Subject: [PATCH 18/28] test/pylib: RandomTables.add_column with value column When adding extra columns in a test, make them value column. Name them with the "v_" prefix and use the value column number counter. Signed-off-by: Alejo Sanchez Closes #13271 (cherry picked from commit 81b40c10de70e1dd02f7f6a4a38ac777bcfea41c) --- test/pylib/random_tables.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/pylib/random_tables.py b/test/pylib/random_tables.py index 7289fcaddf..63e400428f 100644 --- a/test/pylib/random_tables.py +++ b/test/pylib/random_tables.py @@ -173,10 +173,11 @@ class RandomTable(): return await self.manager.cql.run_async(cql_stmt) async def add_column(self, name: str = None, ctype: Type[ValueType] = None, column: Column = None): + """Add a value column to the table""" if column is not None: assert type(column) is Column, "Wrong column type to add_column" else: - name = name if name is not None else f"c_{self.next_clustering_id():02}" + name = name if name is not None else f"v_{self.next_value_id():02}" ctype = ctype if ctype is not None else TextType column = Column(name, ctype=ctype) self.columns.append(column) From 12eec5bb2b3749745b4a90b7bd11884592b5bf7f Mon Sep 17 00:00:00 2001 From: Petr Gusev Date: Fri, 31 Mar 2023 16:21:38 +0400 Subject: [PATCH 19/28] scylla_cluster.py: optimize node logs reading There are two occasions in scylla_cluster where we read the node logs, and in both of them we read the entire file in memory. This is not efficient and may cause an OOM. In the first case we need the last line of the log file, so we seek at the end and move backwards looking for a new line symbol. In the second case we look through the log file to find the expected_error. The readlines() method returns a Python list object, which means it reads the entire file in memory. It's sufficient to just remove it since iterating over the file instance already yields lines lazily one by one. This is a follow-up for #13134. Closes #13399 (cherry picked from commit 09636b20f3c26d3c97094ec4d89db39a18af2299) --- test/pylib/scylla_cluster.py | 30 +++++++++++++++++++++++++----- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 5481fa1e81..1429d1449a 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -385,6 +385,29 @@ class ScyllaServer: sleep_interval = 0.1 cql_up_state = CqlUpState.NOT_CONNECTED + def read_last_line(file_path: pathlib.Path): + block_size = 4 * 1024 + file_size = os.stat(file_path).st_size + pos = file_size + blocks = [] + linesep = os.linesep.encode() + with file_path.open('rb') as f: + linesep_index = -1 + while pos > 0 and linesep_index == -1: + next_pos = max(pos - block_size, 0) + f.seek(next_pos, os.SEEK_SET) + block = f.read(pos - next_pos) + # ignore the last empty line if any + if pos == file_size and block.endswith(linesep): + block = block[:-len(linesep)] + linesep_index = block.rfind(linesep) + blocks.append(block) + pos = next_pos + if linesep_index != -1: + blocks[-1] = block[linesep_index + len(linesep):] + blocks.reverse() + return b''.join(blocks).decode() + def report_error(message: str): message += f", server_id {self.server_id}, IP {self.ip_addr}, workdir {self.workdir.name}" message += f", host_id {self.host_id if hasattr(self, 'host_id') else ''}" @@ -392,10 +415,7 @@ class ScyllaServer: if expected_error is not None: message += f", the node log was expected to contain the string [{expected_error}]" self.logger.error(message) - self.logger.error("last line of %s:", self.log_filename) - with self.log_filename.open('r') as log_file: - log_file.seek(0, 0) - self.logger.error(log_file.readlines()[-1].rstrip()) + self.logger.error("last line of %s:\n%s", self.log_filename, read_last_line(self.log_filename)) log_handler = logging.getLogger().handlers[0] if hasattr(log_handler, 'baseFilename'): logpath = log_handler.baseFilename # type: ignore @@ -410,7 +430,7 @@ class ScyllaServer: self.cmd = None if expected_error is not None: with self.log_filename.open('r') as log_file: - for line in log_file.readlines(): + for line in log_file: if expected_error in line: return report_error("the node startup failed, but the log file doesn't contain the expected error") From 937e890fbab810981d2f226dd26315f3112c8fa1 Mon Sep 17 00:00:00 2001 From: Alejo Sanchez Date: Tue, 3 Jan 2023 15:32:06 +0100 Subject: [PATCH 20/28] test/pylib: configurable replication factor Make replication factor configurable for the RandomTables helper. Signed-off-by: Alejo Sanchez (cherry picked from commit 3508a4e41e7cae7c8849944929186b6a9acdf1f0) --- test/pylib/random_tables.py | 6 ++++-- test/topology/conftest.py | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/test/pylib/random_tables.py b/test/pylib/random_tables.py index 63e400428f..73dabce94e 100644 --- a/test/pylib/random_tables.py +++ b/test/pylib/random_tables.py @@ -245,7 +245,8 @@ class RandomTable(): class RandomTables(): """A list of managed random tables""" - def __init__(self, test_name: str, manager: ManagerClient, keyspace: str): + def __init__(self, test_name: str, manager: ManagerClient, keyspace: str, + replication_factor: int): self.test_name = test_name self.manager = manager self.keyspace = keyspace @@ -253,7 +254,8 @@ class RandomTables(): self.removed_tables: List[RandomTable] = [] assert self.manager.cql is not None self.manager.cql.execute(f"CREATE KEYSPACE {keyspace} WITH REPLICATION = " - "{ 'class' : 'NetworkTopologyStrategy', 'replication_factor' : 1 }") + "{ 'class' : 'NetworkTopologyStrategy', " + f"'replication_factor' : {replication_factor} }}") async def add_tables(self, ntables: int = 1, ncolumns: int = 5, if_not_exists: bool = False) -> None: """Add random tables to the list. diff --git a/test/topology/conftest.py b/test/topology/conftest.py index da83ee17ce..50d42562ec 100644 --- a/test/topology/conftest.py +++ b/test/topology/conftest.py @@ -219,7 +219,7 @@ def fails_without_consistent_cluster_management(request, check_pre_consistent_cl # unless the cluster is dirty or the test has failed. @pytest.fixture(scope="function") async def random_tables(request, manager): - tables = RandomTables(request.node.name, manager, unique_name()) + tables = RandomTables(request.node.name, manager, unique_name(), 1) yield tables # Don't drop tables at the end if we failed or the cluster is dirty - it may be impossible From ad99456a9d1d1b4c69aaf09970b3da72ae226c27 Mon Sep 17 00:00:00 2001 From: Alejo Sanchez Date: Tue, 3 Jan 2023 15:33:48 +0100 Subject: [PATCH 21/28] test/topology: default replication factor 3 For most tests there will be nodes down, increase replication factor to 3 to avoid having problems for partitions belonging to down nodes. Use replication factor 1 for raft upgrade tests. (cherry picked from commit 08d754e13fc5e02fe8852781277db503586c8052) --- test/topology/conftest.py | 4 +++- test/topology_custom/test_custom.py | 2 +- test/topology_raft_disabled/test_raft_upgrade.py | 3 +++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/test/topology/conftest.py b/test/topology/conftest.py index 50d42562ec..7a2478d603 100644 --- a/test/topology/conftest.py +++ b/test/topology/conftest.py @@ -219,7 +219,9 @@ def fails_without_consistent_cluster_management(request, check_pre_consistent_cl # unless the cluster is dirty or the test has failed. @pytest.fixture(scope="function") async def random_tables(request, manager): - tables = RandomTables(request.node.name, manager, unique_name(), 1) + rf_marker = request.node.get_closest_marker("replication_factor") + replication_factor = rf_marker.args[0] if rf_marker is not None else 3 # Default 3 + tables = RandomTables(request.node.name, manager, unique_name(), replication_factor) yield tables # Don't drop tables at the end if we failed or the cluster is dirty - it may be impossible diff --git a/test/topology_custom/test_custom.py b/test/topology_custom/test_custom.py index 691f4000c3..74f4b21284 100644 --- a/test/topology_custom/test_custom.py +++ b/test/topology_custom/test_custom.py @@ -12,7 +12,7 @@ import pytest @pytest.mark.asyncio async def test_custom(request, manager: ManagerClient): servers = [await manager.server_add(), await manager.server_add(), await manager.server_add()] - tables = RandomTables(request.node.name, manager, unique_name()) + tables = RandomTables(request.node.name, manager, unique_name(), 3) table = await tables.add_table(ncolumns=5) await table.insert_seq() await table.add_index(2) diff --git a/test/topology_raft_disabled/test_raft_upgrade.py b/test/topology_raft_disabled/test_raft_upgrade.py index 1d0d35ee9b..5d7085c449 100644 --- a/test/topology_raft_disabled/test_raft_upgrade.py +++ b/test/topology_raft_disabled/test_raft_upgrade.py @@ -110,6 +110,7 @@ def log_run_time(f): @pytest.mark.asyncio @log_run_time +@pytest.mark.replication_factor(1) async def test_raft_upgrade_basic(manager: ManagerClient, random_tables: RandomTables): """ kbr-: the test takes about 7 seconds in dev mode on my laptop. @@ -144,6 +145,7 @@ async def test_raft_upgrade_basic(manager: ManagerClient, random_tables: RandomT @pytest.mark.asyncio @log_run_time +@pytest.mark.replication_factor(1) async def test_recover_stuck_raft_upgrade(manager: ManagerClient, random_tables: RandomTables): """ We enable Raft on every server and the upgrade procedure starts. All servers join group 0. Then one @@ -228,6 +230,7 @@ async def test_recover_stuck_raft_upgrade(manager: ManagerClient, random_tables: @pytest.mark.asyncio @log_run_time +@pytest.mark.replication_factor(1) async def test_recovery_after_majority_loss(manager: ManagerClient, random_tables: RandomTables): """ We successfully upgrade a cluster. Eventually however all servers but one fail - group 0 From ac5dff7de04da010b82833c418ac8e868e937735 Mon Sep 17 00:00:00 2001 From: Alejo Sanchez Date: Sat, 18 Feb 2023 00:37:57 +0100 Subject: [PATCH 22/28] test/pylib: get gossiper alive endpoints Helper to get list of gossiper alive endpoints from REST API. Signed-off-by: Alejo Sanchez (cherry picked from commit 62a945ccd56675360c89d741f8b0e9144379bc64) --- test/pylib/rest_client.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/test/pylib/rest_client.py b/test/pylib/rest_client.py index d190dbf172..e6b5ffec62 100644 --- a/test/pylib/rest_client.py +++ b/test/pylib/rest_client.py @@ -190,6 +190,12 @@ class ScyllaRESTAPIClient(): assert(type(data) == list) return data + async def get_alive_endpoints(self, node_ip: str) -> list: + """Get the list of alive nodes according to `node_ip`.""" + data = await self.client.get_json(f"/gossiper/endpoint/live", host=node_ip) + assert(type(data) == list) + return data + async def enable_injection(self, node_ip: str, injection: str, one_shot: bool) -> None: """Enable error injection named `injection` on `node_ip`. Depending on `one_shot`, the injection will be executed only once or every time the process passes the injection point. From fe4af95745ea7c28fb3e1251d9e60f5db664dc51 Mon Sep 17 00:00:00 2001 From: Tomasz Grabiec Date: Mon, 27 Mar 2023 18:08:53 +0200 Subject: [PATCH 23/28] test: pylib: Add a way to create cql connections with particular coordinators Usage: await manager.driver_connect(server=servers[0]) manager.cql.execute(f"...", execution_profile='whitelist') (cherry picked from commit 041ee3ffddc9b93914b86ffa142f26357abfec49) --- test/pylib/manager_client.py | 5 +++-- test/topology/conftest.py | 9 ++++++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index d7af493b46..bcfb141d1e 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -48,10 +48,11 @@ class ManagerClient(): """Close driver""" self.driver_close() - async def driver_connect(self) -> None: + async def driver_connect(self, server=None) -> None: """Connect to cluster""" if self.con_gen is not None: - servers = [s_info.ip_addr for s_info in await self.running_servers()] + targets = [server] if server else await self.running_servers() + servers = [s_info.ip_addr for s_info in targets] logger.debug("driver connecting to %s", servers) self.ccluster = self.con_gen(servers, self.port, self.use_ssl) self.cql = self.ccluster.connect() diff --git a/test/topology/conftest.py b/test/topology/conftest.py index 7a2478d603..4cdd776e8e 100644 --- a/test/topology/conftest.py +++ b/test/topology/conftest.py @@ -18,6 +18,8 @@ from cassandra.cluster import Session, ResponseFuture # type: from cassandra.cluster import Cluster, ConsistencyLevel # type: ignore # pylint: disable=no-name-in-module from cassandra.cluster import ExecutionProfile, EXEC_PROFILE_DEFAULT # type: ignore # pylint: disable=no-name-in-module from cassandra.policies import RoundRobinPolicy # type: ignore +from cassandra.policies import TokenAwarePolicy # type: ignore +from cassandra.policies import WhiteListRoundRobinPolicy # type: ignore from cassandra.connection import DRIVER_NAME # type: ignore # pylint: disable=no-name-in-module from cassandra.connection import DRIVER_VERSION # type: ignore # pylint: disable=no-name-in-module @@ -125,6 +127,11 @@ def cluster_con(hosts: List[IPAddress], port: int, use_ssl: bool): # See issue #11289. # NOTE: request_timeout is the main cause of timeouts, even if logs say heartbeat request_timeout=200) + whitelist_profile = ExecutionProfile( + load_balancing_policy=TokenAwarePolicy(WhiteListRoundRobinPolicy(hosts)), + consistency_level=ConsistencyLevel.LOCAL_QUORUM, + serial_consistency_level=ConsistencyLevel.LOCAL_SERIAL, + request_timeout=200) if use_ssl: # Scylla does not support any earlier TLS protocol. If you try, # you will get mysterious EOF errors (see issue #6971) :-( @@ -132,7 +139,7 @@ def cluster_con(hosts: List[IPAddress], port: int, use_ssl: bool): else: ssl_context = None - return Cluster(execution_profiles={EXEC_PROFILE_DEFAULT: profile}, + return Cluster(execution_profiles={EXEC_PROFILE_DEFAULT: profile, 'whitelist': whitelist_profile}, contact_points=hosts, port=port, # TODO: make the protocol version an option, to allow testing with From bcf99a37cd38a5cc53fd70f5639a7f9ef250462f Mon Sep 17 00:00:00 2001 From: Alejo Sanchez Date: Tue, 4 Apr 2023 18:09:58 +0200 Subject: [PATCH 24/28] test/pylib: ManagerClient helpers to wait for... server to see other servers after start/restart When starting/restarting a server, provide a way to wait for the server to see at least n other servers. Also leave the implementation methods available for manual use and update previous tests, one to wait for a specific server to be seen, and one to wait for a specific server to not be seen (down). Fixes #13147 Signed-off-by: Alejo Sanchez Closes #13438 (cherry picked from commit 11561a73cbdfc19c547e0ecd393f36b944629d05) Backport note: skipped the test_mutation_schema_change.py fix as the test doesn't exist on this branch. --- test/pylib/manager_client.py | 43 ++++++++++++++++--- .../test_topology_remove_garbage_group0.py | 2 + 2 files changed, 40 insertions(+), 5 deletions(-) diff --git a/test/pylib/manager_client.py b/test/pylib/manager_client.py index bcfb141d1e..9e90b99a63 100644 --- a/test/pylib/manager_client.py +++ b/test/pylib/manager_client.py @@ -48,7 +48,7 @@ class ManagerClient(): """Close driver""" self.driver_close() - async def driver_connect(self, server=None) -> None: + async def driver_connect(self, server: Optional[ServerInfo] = None) -> None: """Connect to cluster""" if self.con_gen is not None: targets = [server] if server else await self.running_servers() @@ -139,17 +139,21 @@ class ManagerClient(): logger.debug("ManagerClient stopping gracefully %s", server_id) await self.client.get_text(f"/cluster/server/{server_id}/stop_gracefully") - async def server_start(self, server_id: ServerNum, expected_error: Optional[str] = None) -> None: - """Start specified server""" + async def server_start(self, server_id: ServerNum, expected_error: Optional[str] = None, + wait_others: int = 0, wait_interval: float = 45) -> None: + """Start specified server and optionally wait for it to learn of other servers""" logger.debug("ManagerClient starting %s", server_id) params = {'expected_error': expected_error} if expected_error is not None else None await self.client.get_text(f"/cluster/server/{server_id}/start", params=params) + await self.server_sees_others(server_id, wait_others, interval = wait_interval) self._driver_update() - async def server_restart(self, server_id: ServerNum) -> None: - """Restart specified server""" + async def server_restart(self, server_id: ServerNum, wait_others: int = 0, + wait_interval: float = 45) -> None: + """Restart specified server and optionally wait for it to learn of other servers""" logger.debug("ManagerClient restarting %s", server_id) await self.client.get_text(f"/cluster/server/{server_id}/restart") + await self.server_sees_others(server_id, wait_others, interval = wait_interval) self._driver_update() async def server_add(self, replace_cfg: Optional[ReplaceConfig] = None, cmdline: Optional[List[str]] = None, config: Optional[dict[str, str]] = None, start: bool = True) -> ServerInfo: @@ -235,3 +239,32 @@ class ManagerClient(): except Exception as exc: raise Exception(f"Failed to get local host id address for server {server_id}") from exc return HostID(host_id) + + async def server_sees_others(self, server_id: ServerNum, count: int, interval: float = 45.): + """Wait till a server sees a minimum given count of other servers""" + if count < 1: + return + server_ip = await self.get_host_ip(server_id) + async def _sees_min_others(): + alive_nodes = await self.api.get_alive_endpoints(server_ip) + if len(alive_nodes) > count: + return True + await wait_for(_sees_min_others, time() + interval, period=.1) + + async def server_sees_other_server(self, server_ip: IPAddress, other_ip: IPAddress, + interval: float = 45.): + """Wait till a server sees another specific server IP as alive""" + async def _sees_another_server(): + alive_nodes = await self.api.get_alive_endpoints(server_ip) + if other_ip in alive_nodes: + return True + await wait_for(_sees_another_server, time() + interval, period=.1) + + async def server_not_sees_other_server(self, server_ip: IPAddress, other_ip: IPAddress, + interval: float = 45.): + """Wait till a server sees another specific server IP as dead""" + async def _not_sees_another_server(): + alive_nodes = await self.api.get_alive_endpoints(server_ip) + if not other_ip in alive_nodes: + return True + await wait_for(_not_sees_another_server, time() + interval, period=.1) diff --git a/test/topology/test_topology_remove_garbage_group0.py b/test/topology/test_topology_remove_garbage_group0.py index 90558b58d3..69975c8535 100644 --- a/test/topology/test_topology_remove_garbage_group0.py +++ b/test/topology/test_topology_remove_garbage_group0.py @@ -81,6 +81,8 @@ async def test_remove_garbage_group0_members(manager: ManagerClient): logging.info(f'stop {servers[1]}') await manager.server_stop_gracefully(servers[1].server_id) + logging.debug(f'waiting for {servers[2]} to see {servers[1]} is down') + await manager.server_not_sees_other_server(servers[2].ip_addr, servers[1].ip_addr) logging.info(f'removenode {servers[1]} using {servers[2]}') await manager.remove_node(servers[2].server_id, servers[1].server_id) From e49a531aaa33fe728890b09ebc60d9e35e635ab2 Mon Sep 17 00:00:00 2001 From: Kamil Braun Date: Fri, 5 May 2023 13:29:15 +0200 Subject: [PATCH 25/28] Merge 'scylla_cluster.py: fix read_last_line' from Gusev Petr This is a follow-up to #13399, the patch addresses the issues mentioned there: * linesep can be split between blocks; * linesep can be part of UTF-8 sequence; * avoid excessively long lines, limit to 256 chars; * the logic of the function made simpler and more maintainable. Closes #13427 * github.com:scylladb/scylladb: pylib_test: add tests for read_last_line pytest: add pylib_test directory scylla_cluster.py: fix read_last_line scylla_cluster.py: move read_last_line to util.py (cherry picked from commit 70f2b09397aa082cd5447ebe83a9c2bf26d111b0) --- test/pylib/scylla_cluster.py | 25 +------------------------ test/pylib/util.py | 18 ++++++++++++++++++ test/pylib_test/__init__.py | 0 test/pylib_test/pytest.ini | 9 +++++++++ test/pylib_test/run | 11 +++++++++++ test/pylib_test/suite.yaml | 1 + test/pylib_test/test_util.py | 28 ++++++++++++++++++++++++++++ 7 files changed, 68 insertions(+), 24 deletions(-) create mode 100644 test/pylib_test/__init__.py create mode 100644 test/pylib_test/pytest.ini create mode 100755 test/pylib_test/run create mode 100644 test/pylib_test/suite.yaml create mode 100644 test/pylib_test/test_util.py diff --git a/test/pylib/scylla_cluster.py b/test/pylib/scylla_cluster.py index 1429d1449a..93f40709db 100644 --- a/test/pylib/scylla_cluster.py +++ b/test/pylib/scylla_cluster.py @@ -25,7 +25,7 @@ from io import BufferedWriter from test.pylib.host_registry import Host, HostRegistry from test.pylib.pool import Pool from test.pylib.rest_client import ScyllaRESTAPIClient, HTTPError -from test.pylib.util import LogPrefixAdapter +from test.pylib.util import LogPrefixAdapter, read_last_line from test.pylib.internal_types import ServerNum, IPAddress, HostID, ServerInfo import aiohttp import aiohttp.web @@ -385,29 +385,6 @@ class ScyllaServer: sleep_interval = 0.1 cql_up_state = CqlUpState.NOT_CONNECTED - def read_last_line(file_path: pathlib.Path): - block_size = 4 * 1024 - file_size = os.stat(file_path).st_size - pos = file_size - blocks = [] - linesep = os.linesep.encode() - with file_path.open('rb') as f: - linesep_index = -1 - while pos > 0 and linesep_index == -1: - next_pos = max(pos - block_size, 0) - f.seek(next_pos, os.SEEK_SET) - block = f.read(pos - next_pos) - # ignore the last empty line if any - if pos == file_size and block.endswith(linesep): - block = block[:-len(linesep)] - linesep_index = block.rfind(linesep) - blocks.append(block) - pos = next_pos - if linesep_index != -1: - blocks[-1] = block[linesep_index + len(linesep):] - blocks.reverse() - return b''.join(blocks).decode() - def report_error(message: str): message += f", server_id {self.server_id}, IP {self.ip_addr}, workdir {self.workdir.name}" message += f", host_id {self.host_id if hasattr(self, 'host_id') else ''}" diff --git a/test/pylib/util.py b/test/pylib/util.py index a426218e8d..e3e462adaa 100644 --- a/test/pylib/util.py +++ b/test/pylib/util.py @@ -6,6 +6,8 @@ import time import asyncio import logging +import pathlib +import os from typing import Callable, Awaitable, Optional, TypeVar, Generic @@ -76,5 +78,21 @@ async def wait_for_cql_and_get_hosts(cql: Session, servers: list[ServerInfo], de return hosts +def read_last_line(file_path: pathlib.Path, max_line_bytes = 512): + file_size = os.stat(file_path).st_size + with file_path.open('rb') as f: + f.seek(max(0, file_size - max_line_bytes), os.SEEK_SET) + line_bytes = f.read() + line_str = line_bytes.decode('utf-8', errors='ignore') + linesep = os.linesep + if line_str.endswith(linesep): + line_str = line_str[:-len(linesep)] + linesep_index = line_str.rfind(linesep) + if linesep_index != -1: + line_str = line_str[linesep_index + len(linesep):] + elif file_size > max_line_bytes: + line_str = '...' + line_str + return line_str + unique_name.last_ms = 0 diff --git a/test/pylib_test/__init__.py b/test/pylib_test/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/test/pylib_test/pytest.ini b/test/pylib_test/pytest.ini new file mode 100644 index 0000000000..1b586bd36f --- /dev/null +++ b/test/pylib_test/pytest.ini @@ -0,0 +1,9 @@ +# Pytest configuration file. If we don't have one in this directory, +# pytest will look for one in our ancestor directories, and may find +# something irrelevant. So we should have one here, even if empty. +[pytest] +asyncio_mode = auto + +log_cli = true +log_format = %(asctime)s.%(msecs)03d %(levelname)s> %(message)s +log_date_format = %H:%M:%S diff --git a/test/pylib_test/run b/test/pylib_test/run new file mode 100755 index 0000000000..0b4ac6762a --- /dev/null +++ b/test/pylib_test/run @@ -0,0 +1,11 @@ +#!/usr/bin/env python3 +# Use the run.py library from ../cql-pytest: +import sys +sys.path.insert(1, sys.path[0] + '/../cql-pytest') +import run + +success = run.run_pytest(sys.path[0], sys.argv[1:]) + +run.summary = 'Pylib tests pass' if success else 'Pylib tests failure' + +exit(0 if success else 1) diff --git a/test/pylib_test/suite.yaml b/test/pylib_test/suite.yaml new file mode 100644 index 0000000000..8919604e66 --- /dev/null +++ b/test/pylib_test/suite.yaml @@ -0,0 +1 @@ +type: Run diff --git a/test/pylib_test/test_util.py b/test/pylib_test/test_util.py new file mode 100644 index 0000000000..8c14186f01 --- /dev/null +++ b/test/pylib_test/test_util.py @@ -0,0 +1,28 @@ +import os +import tempfile +import pathlib +from test.pylib.util import read_last_line + +def test_read_last_line(): + test_cases = [ + (b"This is the first line.\nThis is the second line.\nThis is the third line.", 'This is the third line.'), + (b"This is another file.\nIt has a few lines.\nThe last line is what we're interested in.", 'The last line is what we\'re interested in.'), + (b"This file has only one line.", 'This file has only one line.'), + (b"\n", ""), + (b"\n\n\n", ""), + (b"", ""), + (b"abc\n", 'abc'), + (b"abc", '...bc', 2), + (b"lalala\nbububu", "bububu"), + (b"line1\nline2\nline3\n", "...line3", 6), + (b"line1\nline2\nline3", "line3", 6), + (b"line1\nline2\nline3\n", "line3", 7), + (b"\xbe\xbe\xbe\xbebububu\n", "bububu") + ] + for test_case in test_cases: + with tempfile.NamedTemporaryFile(dir=os.getenv('TMPDIR', '/tmp')) as f: + f.write(test_case[0]) + f.flush() + file_path = pathlib.Path(f.name) + actual = read_last_line(file_path, test_case[2]) if len(test_case) == 3 else read_last_line(file_path) + assert(actual == test_case[1]) From 094bcac399cf66f7d5ed65c4198aa330e6ff834a Mon Sep 17 00:00:00 2001 From: Konstantin Osipov Date: Wed, 3 May 2023 13:51:24 +0300 Subject: [PATCH 26/28] test: issue a read barrier before checking ring consistency Raft replication doesn't guarantee that all replicas see identical Raft state at all times, it only guarantees the same order of events on all replicas. When comparing raft state with gossip state on a node, first issue a read barrier to ensure the node has the latest raft state. To issue a read barrier it is sufficient to alter a non-existing state: in order to validate the DDL the node needs to sync with the leader and fetch its latest group0 state. Fixes #13518 (flaky topology test). Closes #13756 (cherry picked from commit e7c9ca560bf292ccdb2aba03fd04367efff1cee9) --- test/topology/util.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/test/topology/util.py b/test/topology/util.py index 87ba5dddb4..c0401bacb7 100644 --- a/test/topology/util.py +++ b/test/topology/util.py @@ -7,10 +7,12 @@ Test consistency of schema changes with topology changes. """ import logging +import pytest import time -from test.pylib.util import wait_for, wait_for_cql_and_get_hosts +from cassandra.protocol import InvalidRequest, ConfigurationException from test.pylib.internal_types import ServerInfo from test.pylib.manager_client import ManagerClient +from test.pylib.util import wait_for, wait_for_cql_and_get_hosts logger = logging.getLogger(__name__) @@ -36,6 +38,9 @@ async def get_current_group0_config(manager: ManagerClient, srv: ServerInfo) -> """ assert manager.cql host = (await wait_for_cql_and_get_hosts(manager.cql, [srv], time.time() + 60))[0] + # Issue a read barrer on that host. + with pytest.raises(InvalidRequest, match="nosuch"): + _ = await manager.cql.run_async("alter table nosuchkeyspace.nosuchtable with comment=''", host = host) group0_id = (await manager.cql.run_async( "select value from system.scylla_local where key = 'raft_group0_id'", host=host))[0].value From 9c941aba0b8fea78f7c1a91302f86cee699595b4 Mon Sep 17 00:00:00 2001 From: Kamil Braun Date: Fri, 5 May 2023 16:13:44 +0200 Subject: [PATCH 27/28] test: pylib: random_tables: perform read barrier in `verify_schema` `RandomTables.verify_schema` is often called in topology tests after performing a schema change. It compares the schema tables fetched from some node to the expected latest schema stored by the `RandomTables` object. However there's no guarantee that the latest schema change has already propagated to the node which we query. We could have performed the schema change on a different node and the change may not have been applied yet on all nodes. To fix that, pick a specific node and perform a read barrier on it, then use that node to fetch the schema tables. Fixes #13788 Closes #13789 (cherry picked from commit 3f3dcf451bc7dbdd6ceb7f95ac859d056aabc00e) --- test/pylib/random_tables.py | 18 ++++++++++++++---- test/pylib/util.py | 30 ++++++++++++++++++++++++++++-- test/topology/util.py | 7 ++----- 3 files changed, 44 insertions(+), 11 deletions(-) diff --git a/test/pylib/random_tables.py b/test/pylib/random_tables.py index 73dabce94e..9a1da10462 100644 --- a/test/pylib/random_tables.py +++ b/test/pylib/random_tables.py @@ -35,10 +35,12 @@ import itertools import logging import random import uuid +import time from typing import Optional, Type, List, Set, Union, TYPE_CHECKING if TYPE_CHECKING: from cassandra.cluster import Session as CassandraSession # type: ignore from test.pylib.manager_client import ManagerClient +from test.pylib.util import get_available_host, read_barrier logger = logging.getLogger('random_tables') @@ -318,8 +320,17 @@ class RandomTables(): f"WHERE keyspace_name = '{self.keyspace}'" logger.debug(cql_stmt1) - assert self.manager.cql is not None - res1 = {row.table_name for row in await self.manager.cql.run_async(cql_stmt1)} + + cql = self.manager.cql + assert cql + + # Issue a read barrier on some node and then keep using that node to do the queries. + # This ensures that the queries return recent data (at least all data committed + # when `verify_schema` was called). + host = await get_available_host(cql, time.time() + 60) + await read_barrier(cql, host) + + res1 = {row.table_name for row in await cql.run_async(cql_stmt1, host=host)} assert not tables - res1, f"Tables {tables - res1} not present" for table_name in tables: @@ -329,8 +340,7 @@ class RandomTables(): cql_stmt2 = f"SELECT column_name, position, kind, type FROM system_schema.columns " \ f"WHERE keyspace_name = '{self.keyspace}' AND table_name = '{table_name}'" logger.debug(cql_stmt2) - assert self.manager.cql is not None - res2 = {row.column_name: row for row in await self.manager.cql.run_async(cql_stmt2)} + res2 = {row.column_name: row for row in await cql.run_async(cql_stmt2, host=host)} assert res2.keys() == cols.keys(), f"Column names for {table_name} do not match " \ f"expected ({', '.join(cols.keys())}) " \ f"got ({', '.join(res2.keys())})" diff --git a/test/pylib/util.py b/test/pylib/util.py index e3e462adaa..fb787dcb59 100644 --- a/test/pylib/util.py +++ b/test/pylib/util.py @@ -8,11 +8,13 @@ import asyncio import logging import pathlib import os +import pytest from typing import Callable, Awaitable, Optional, TypeVar, Generic -from cassandra.cluster import NoHostAvailable, Session # type: ignore # pylint: disable=no-name-in-module -from cassandra.pool import Host # type: ignore # pylint: disable=no-name-in-module +from cassandra.cluster import NoHostAvailable, Session, Cluster # type: ignore # pylint: disable=no-name-in-module +from cassandra.protocol import InvalidRequest # type: ignore # pylint: disable=no-name-in-module +from cassandra.pool import Host # type: ignore # pylint: disable=no-name-in-module from test.pylib.internal_types import ServerInfo @@ -95,4 +97,28 @@ def read_last_line(file_path: pathlib.Path, max_line_bytes = 512): return line_str +async def get_available_host(cql: Session, deadline: float) -> Host: + hosts = cql.cluster.metadata.all_hosts() + async def find_host(): + for h in hosts: + try: + await cql.run_async( + "select key from system.local where key = 'local'", host=h) + except NoHostAvailable: + logging.debug(f"get_available_host: {h} not available") + continue + return h + return None + return await wait_for(find_host, deadline) + + +async def read_barrier(cql: Session, host: Host): + """To issue a read barrier it is sufficient to attempt altering + a non-existing table: in order to validate the DDL the node needs to sync + with the leader and fetch latest group 0 state. + """ + with pytest.raises(InvalidRequest, match="nosuch"): + await cql.run_async("alter table nosuchkeyspace.nosuchtable with comment=''", host = host) + + unique_name.last_ms = 0 diff --git a/test/topology/util.py b/test/topology/util.py index c0401bacb7..0d34bb0b0b 100644 --- a/test/topology/util.py +++ b/test/topology/util.py @@ -9,10 +9,9 @@ Test consistency of schema changes with topology changes. import logging import pytest import time -from cassandra.protocol import InvalidRequest, ConfigurationException from test.pylib.internal_types import ServerInfo from test.pylib.manager_client import ManagerClient -from test.pylib.util import wait_for, wait_for_cql_and_get_hosts +from test.pylib.util import wait_for, wait_for_cql_and_get_hosts, read_barrier logger = logging.getLogger(__name__) @@ -38,9 +37,7 @@ async def get_current_group0_config(manager: ManagerClient, srv: ServerInfo) -> """ assert manager.cql host = (await wait_for_cql_and_get_hosts(manager.cql, [srv], time.time() + 60))[0] - # Issue a read barrer on that host. - with pytest.raises(InvalidRequest, match="nosuch"): - _ = await manager.cql.run_async("alter table nosuchkeyspace.nosuchtable with comment=''", host = host) + await read_barrier(manager.cql, host) group0_id = (await manager.cql.run_async( "select value from system.scylla_local where key = 'raft_group0_id'", host=host))[0].value From f4115528d6a80ea761ecc47a55aae5363c52c21f Mon Sep 17 00:00:00 2001 From: Kamil Braun Date: Thu, 18 May 2023 11:24:32 +0200 Subject: [PATCH 28/28] test: pylib: fix `read_barrier` implementation The previous implementation didn't actually do a read barrier, because the statement failed on an early prepare/validate step which happened before read barrier was even performed. Change it to a statement which does not fail and doesn't perform any schema change but requires a read barrier. This breaks one test which uses `RandomTables.verify_schema()` when only one node is alive, but `verify_schema` performs a read barrier. Unbreak it by skipping the read barrier in this case (it makes sense in this particular test). Closes #13933 (cherry picked from commit 64dc76db555b924727708de83e1e5c9f542fda32) Backport note: skipped the test_snapshot.py change, as the test doesn't exist on this branch. --- test/pylib/random_tables.py | 11 ++++++----- test/pylib/util.py | 10 +++++----- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/test/pylib/random_tables.py b/test/pylib/random_tables.py index 9a1da10462..ecca26c501 100644 --- a/test/pylib/random_tables.py +++ b/test/pylib/random_tables.py @@ -302,7 +302,7 @@ class RandomTables(): assert self.manager.cql is not None self.manager.cql.execute(f"DROP KEYSPACE {self.keyspace}") - async def verify_schema(self, table: Union[RandomTable, str] = None) -> None: + async def verify_schema(self, table: Union[RandomTable, str] = None, do_read_barrier: bool = True) -> None: """Verify schema of all active managed random tables""" if isinstance(table, RandomTable): tables = {table.name} @@ -324,11 +324,12 @@ class RandomTables(): cql = self.manager.cql assert cql - # Issue a read barrier on some node and then keep using that node to do the queries. - # This ensures that the queries return recent data (at least all data committed - # when `verify_schema` was called). host = await get_available_host(cql, time.time() + 60) - await read_barrier(cql, host) + if do_read_barrier: + # Issue a read barrier on some node and then keep using that node to do the queries. + # This ensures that the queries return recent data (at least all data committed + # when `verify_schema` was called). + await read_barrier(cql, host) res1 = {row.table_name for row in await cql.run_async(cql_stmt1, host=host)} assert not tables - res1, f"Tables {tables - res1} not present" diff --git a/test/pylib/util.py b/test/pylib/util.py index fb787dcb59..227bf69b62 100644 --- a/test/pylib/util.py +++ b/test/pylib/util.py @@ -113,12 +113,12 @@ async def get_available_host(cql: Session, deadline: float) -> Host: async def read_barrier(cql: Session, host: Host): - """To issue a read barrier it is sufficient to attempt altering - a non-existing table: in order to validate the DDL the node needs to sync - with the leader and fetch latest group 0 state. + """To issue a read barrier it is sufficient to attempt dropping a + non-existing table. We need to use `if exists`, otherwise the statement + would fail on prepare/validate step which happens before a read barrier is + performed. """ - with pytest.raises(InvalidRequest, match="nosuch"): - await cql.run_async("alter table nosuchkeyspace.nosuchtable with comment=''", host = host) + await cql.run_async("drop table if exists nosuchkeyspace.nosuchtable", host = host) unique_name.last_ms = 0