mirror of
https://github.com/scylladb/scylladb.git
synced 2026-04-22 09:30:45 +00:00
Merge 'test: cluster: Deflake test_write_cl_any_to_dead_node_generates_hints' from Dawid Mędrek
Before these changes, we would send mutations to the node and
immediately query the metrics to see how many hints had been written.
However, that could lead to random failures of the test: even if the
mutations have finished executing, hints are stored asynchronously, so
we don't have a guarantee they have already been processed.
To prevent such failures, we rewrite the check: we will perform multiple
checks against the metrics until we have confirmed that the hints have
indeed been written or we hit the timeout.
We're generous with the timeout: we give the test 60 seconds. That
should be enough time to avoid flakiness even on super slow machines,
and if the test does fail, we will know something is really wrong.
As a bonus, we improve the test in general too. We explicitly express
the preconditions we rely on, as well as bump the log level. If the
test fails in the future, it might be very difficult do debug it
without this additional information.
Fixes SCYLLADB-1133
Backport: The test is present on all supported branches. To avoid
running into more failures, we should backport these changes
to them.
Closes scylladb/scylladb#29191
* github.com:scylladb/scylladb:
test: cluster: Increase log level in test_write_cl_any_to_dead_node_generates_hints
test: cluster: Await all mutations concurrently in test_write_cl_any_to_dead_node_generates_hints
test: cluster: Specify min_tablet_count in test_write_cl_any_to_dead_node_generates_hints
test: cluster: Use new_test_table in test_write_cl_any_to_dead_node_generates_hints
test: cluster: Introduce auxiliary function keyspace_has_tablets
test: cluster: Deflake test_write_cl_any_to_dead_node_generates_hints
(cherry picked from commit ffd58ca1f0)
Closes scylladb/scylladb#29246
This commit is contained in:
committed by
Patryk Jędrzejczak
parent
3836757486
commit
1689736223
@@ -17,10 +17,10 @@ from test.pylib.manager_client import ManagerClient
|
||||
from test.pylib.rest_client import ScyllaMetricsClient, TCPRESTClient, inject_error
|
||||
from test.pylib.tablets import get_tablet_replicas
|
||||
from test.pylib.scylla_cluster import ReplaceConfig
|
||||
from test.pylib.util import wait_for
|
||||
from test.pylib.util import gather_safely, wait_for
|
||||
|
||||
from test.cluster.conftest import skip_mode
|
||||
from test.cluster.util import get_topology_coordinator, find_server_by_host_id, new_test_keyspace
|
||||
from test.cluster.util import get_topology_coordinator, find_server_by_host_id, keyspace_has_tablets, new_test_keyspace, new_test_table
|
||||
|
||||
|
||||
logger = logging.getLogger(__name__)
|
||||
@@ -52,28 +52,42 @@ async def await_sync_point(client: TCPRESTClient, server_ip: IPAddress, sync_poi
|
||||
@pytest.mark.asyncio
|
||||
async def test_write_cl_any_to_dead_node_generates_hints(manager: ManagerClient):
|
||||
node_count = 2
|
||||
servers = await manager.servers_add(node_count)
|
||||
cmdline = ["--logger-log-level", "hints_manager=trace"]
|
||||
servers = await manager.servers_add(node_count, cmdline=cmdline)
|
||||
|
||||
async def wait_for_hints_written(min_hint_count: int, timeout: int):
|
||||
async def aux():
|
||||
hints_written = await get_hint_metrics(manager.metrics, servers[0].ip_addr, "written")
|
||||
if hints_written >= min_hint_count:
|
||||
return True
|
||||
return None
|
||||
assert await wait_for(aux, time.time() + timeout)
|
||||
|
||||
cql = manager.get_cql()
|
||||
async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}") as ks:
|
||||
table = f"{ks}.t"
|
||||
await cql.run_async(f"CREATE TABLE {table} (pk int primary key, v int)")
|
||||
uses_tablets = await keyspace_has_tablets(manager, ks)
|
||||
# If the keyspace uses tablets, let's explicitly require the table to use multiple tablets.
|
||||
# Otherwise, it could happen that all mutations would target servers[0] only, which would
|
||||
# ultimately lead to a test failure here. We rely on the assumption that mutations will be
|
||||
# distributed more or less uniformly!
|
||||
extra_opts = "WITH tablets = {'min_tablet_count': 16}" if uses_tablets else ""
|
||||
async with new_test_table(manager, ks, "pk int PRIMARY KEY, v int", extra_opts) as table:
|
||||
await manager.server_stop_gracefully(servers[1].server_id)
|
||||
|
||||
await manager.server_stop_gracefully(servers[1].server_id)
|
||||
hints_before = await get_hint_metrics(manager.metrics, servers[0].ip_addr, "written")
|
||||
|
||||
hints_before = await get_hint_metrics(manager.metrics, servers[0].ip_addr, "written")
|
||||
stmt = cql.prepare(f"INSERT INTO {table} (pk, v) VALUES (?, ?)")
|
||||
stmt.consistency_level = ConsistencyLevel.ANY
|
||||
|
||||
# Some of the inserts will be targeted to the dead node.
|
||||
# The coordinator doesn't have live targets to send the write to, but it should write a hint.
|
||||
for i in range(100):
|
||||
await cql.run_async(SimpleStatement(f"INSERT INTO {table} (pk, v) VALUES ({i}, {i+1})", consistency_level=ConsistencyLevel.ANY))
|
||||
# Some of the inserts will be targeted to the dead node.
|
||||
# The coordinator doesn't have live targets to send the write to, but it should write a hint.
|
||||
await gather_safely(*[cql.run_async(stmt, (i, i + 1)) for i in range(100)])
|
||||
|
||||
# Verify hints are written
|
||||
hints_after = await get_hint_metrics(manager.metrics, servers[0].ip_addr, "written")
|
||||
assert hints_after > hints_before
|
||||
# Verify hints are written
|
||||
await wait_for_hints_written(hints_before + 1, timeout=60)
|
||||
|
||||
# For dropping the keyspace
|
||||
await manager.server_start(servers[1].server_id)
|
||||
# For dropping the keyspace
|
||||
await manager.server_start(servers[1].server_id)
|
||||
|
||||
@pytest.mark.asyncio
|
||||
async def test_limited_concurrency_of_writes(manager: ManagerClient):
|
||||
|
||||
@@ -614,6 +614,17 @@ async def new_materialized_view(manager: ManagerClient, table, select, pk, where
|
||||
await manager.get_cql().run_async(f"DROP MATERIALIZED VIEW {mv}")
|
||||
|
||||
|
||||
async def keyspace_has_tablets(manager: ManagerClient, keyspace: str) -> bool:
|
||||
"""
|
||||
Checks whether the given keyspace uses tablets.
|
||||
Adapted from its counterpart in the cqlpy test: cqlpy/util.py::keyspace_has_tablets.
|
||||
"""
|
||||
cql = manager.get_cql()
|
||||
rows_iter = await cql.run_async(f"SELECT * FROM system_schema.scylla_keyspaces WHERE keyspace_name='{keyspace}'")
|
||||
rows = list(rows_iter)
|
||||
return len(rows) > 0 and getattr(rows[0], "initial_tablets", None) is not None
|
||||
|
||||
|
||||
async def get_raft_log_size(cql, host) -> int:
|
||||
query = "select count(\"index\") from system.raft"
|
||||
return (await cql.run_async(query, host=host))[0][0]
|
||||
|
||||
Reference in New Issue
Block a user