Merge '[Backport 2025.3] test: cluster: deflake consistency checks after decommission' from Scylladb[bot]

In the Raft-based topology, a decommissioning node is removed from group
0 after the decommission request is considered finished (and the token
ring is updated). Therefore, `check_token_ring_and_group0_consistency`
called just after decommission might fail when the decommissioned node
is still in group 0 (as a non-voter). We deflake all tests that call
`check_token_ring_and_group0_consistency` after decommission in this PR.

Fixes #25809

This PR improves CI stability and changes only tests, so it should be
backported to all supported branches.

- (cherry picked from commit e41fc841cd)

- (cherry picked from commit bb9fb7848a)

Parent PR: #25927

Closes scylladb/scylladb#25963

* https://github.com/scylladb/scylladb:
  test: cluster: deflake consistency checks after decommission
  test: cluster: util: handle group 0 changes after token ring changes in wait_for_token_ring_and_group0_consistency
This commit is contained in:
Patryk Jędrzejczak
2025-09-11 13:01:53 +02:00
6 changed files with 25 additions and 16 deletions

View File

@@ -12,7 +12,7 @@ from cassandra import ConsistencyLevel # type: ignore
from cassandra.query import SimpleStatement # type: ignore
from test.pylib.manager_client import ManagerClient
from test.pylib.util import wait_for_cql_and_get_hosts
from test.cluster.util import check_token_ring_and_group0_consistency, new_test_keyspace
from test.cluster.util import new_test_keyspace, wait_for_token_ring_and_group0_consistency
from test.pylib.util import wait_for
logger = logging.getLogger(__name__)
@@ -106,7 +106,7 @@ async def test_change_replication_factor_1_to_0_and_decommission(request: pytest
# decommission dc1
await manager.decommission_node(srvs[1].server_id)
await check_token_ring_and_group0_consistency(manager)
await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30)
# ensure this no-op alter still works
async with asyncio.timeout(30):

View File

@@ -7,7 +7,7 @@
from test.pylib.manager_client import ManagerClient
from test.pylib.rest_client import inject_error_one_shot
from test.cluster.conftest import skip_mode
from test.cluster.util import check_token_ring_and_group0_consistency, new_test_keyspace
from test.cluster.util import new_test_keyspace, wait_for_token_ring_and_group0_consistency
import pytest
import asyncio
@@ -67,7 +67,7 @@ async def test_data_resurrection_after_cleanup(manager: ManagerClient):
logger.info(f"Decommissioning node {servers[1]}")
await manager.decommission_node(servers[1].server_id)
await check_token_ring_and_group0_consistency(manager)
await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30)
time.sleep(1)
await check(range(128))

View File

@@ -4,10 +4,11 @@
# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0
#
import logging
import time
import pytest
from test.pylib.manager_client import ManagerClient
from test.cluster.util import check_token_ring_and_group0_consistency
from test.cluster.util import wait_for_token_ring_and_group0_consistency
logger = logging.getLogger(__name__)
@@ -27,7 +28,7 @@ async def test_decommissioned_node_cant_rejoin(request, manager: ManagerClient):
# to communicate with other nodes to discover a leader.
logger.info(f"Decommissioning node {servers[1]}")
await manager.decommission_node(servers[1].server_id)
await check_token_ring_and_group0_consistency(manager)
await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30)
logger.info(f"Attempting to start the node {servers[1]} after it was decommissioned")
await manager.server_start(servers[1].server_id,
expected_error='This node was decommissioned and will not rejoin the ring')

View File

@@ -7,7 +7,7 @@ from test.pylib.scylla_cluster import ReplaceConfig
from test.pylib.manager_client import ManagerClient
from test.pylib.util import wait_for_cql_and_get_hosts
from test.cluster.util import check_token_ring_and_group0_consistency, reconnect_driver, \
check_node_log_for_failed_mutations, start_writes
check_node_log_for_failed_mutations, start_writes, wait_for_token_ring_and_group0_consistency
from cassandra.cluster import ConsistencyLevel
@@ -45,7 +45,7 @@ async def test_topology_ops(request, manager: ManagerClient, tablets_enabled: bo
logger.info(f"Decommissioning node {servers[0]}")
await manager.decommission_node(servers[0].server_id)
await check_token_ring_and_group0_consistency(manager)
await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30)
servers = servers[1:]
logger.info(f"Restarting node {servers[0]} when other nodes have bootstrapped")

View File

@@ -6,7 +6,8 @@
from test.pylib.scylla_cluster import ReplaceConfig
from test.pylib.manager_client import ManagerClient
from test.pylib.util import wait_for_cql_and_get_hosts
from test.cluster.util import check_token_ring_and_group0_consistency, reconnect_driver
from test.cluster.util import check_token_ring_and_group0_consistency, reconnect_driver, \
wait_for_token_ring_and_group0_consistency
from test.cluster.test_topology_ops import check_node_log_for_failed_mutations, start_writes
from cassandra.cluster import ConsistencyLevel
@@ -51,7 +52,7 @@ async def test_topology_ops_encrypted(request, manager: ManagerClient, tablets_e
logger.info(f"Decommissioning node {servers[0]}")
await manager.decommission_node(servers[0].server_id)
await check_token_ring_and_group0_consistency(manager)
await wait_for_token_ring_and_group0_consistency(manager, time.time() + 30)
servers = servers[1:]
logger.info(f"Restarting node {servers[0]} when other nodes have bootstrapped")

View File

@@ -116,15 +116,22 @@ 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/decommission - the normal tokens of the new node propagate through gossip.
"""
Weaker version of the above check.
In the Raft-based topology, a decommissioning node is removed from group 0 after the decommission request is
considered finished (and the token ring is updated).
Moreover, in the gossip-based topology, the token ring is not immediately updated after
bootstrap/replace/decommission - the normal tokens 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():
async def token_ring_and_group0_match():
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)
diff = token_ring_ids ^ group0_ids
if diff:
@@ -132,7 +139,7 @@ async def wait_for_token_ring_and_group0_consistency(manager: ManagerClient, dea
f" according to {srv}, symmetric difference: {diff}")
return None
return True
await wait_for(token_ring_matches, deadline, period=.5)
await wait_for(token_ring_and_group0_match, deadline, period=.5)
async def wait_for_upgrade_state(state: str, cql: Session, host: Host, deadline: float) -> None: