mirror of
https://github.com/scylladb/scylladb.git
synced 2026-04-20 08:30:35 +00:00
test: add test_failed_tablet_rebuild_is_retried_on_alter
Test if alter keyspace statement with the current rf values will fix the state of replicas.
This commit is contained in:
@@ -3973,6 +3973,11 @@ future<std::optional<group0_guard>> topology_coordinator::maybe_start_tablet_res
|
||||
future<bool> topology_coordinator::maybe_retry_failed_rf_change_tablet_rebuilds(group0_guard guard) {
|
||||
rtlogger.debug("Retrying failed rebuilds");
|
||||
|
||||
if (utils::get_local_injector().enter("maybe_retry_failed_rf_change_tablet_rebuilds_skip")) {
|
||||
rtlogger.debug("Skipping retrying failed rebuilds due to error injection");
|
||||
co_return false;
|
||||
}
|
||||
|
||||
auto tmptr = get_token_metadata_ptr();
|
||||
utils::chunked_vector<canonical_mutation> updates;
|
||||
for (auto& ks_name : _db.get_tablets_keyspaces()) {
|
||||
|
||||
@@ -708,6 +708,49 @@ async def test_failed_tablet_rebuild_is_retried(request: pytest.FixtureRequest,
|
||||
|
||||
await alter_keyspace("'dc1': ['rack1a', 'rack1b', 'rack1c']")
|
||||
|
||||
@pytest.mark.asyncio
|
||||
@pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode')
|
||||
async def test_failed_tablet_rebuild_is_retried_on_alter(manager: ManagerClient) -> None:
|
||||
async def alter_keyspace(new_rf):
|
||||
await cql.run_async(f"alter keyspace ks1 with replication = {{'class': 'NetworkTopologyStrategy', {new_rf}}};")
|
||||
|
||||
fail_injection = "rebuild_repair_stage_fail"
|
||||
skip_fix_injection = "maybe_retry_failed_rf_change_tablet_rebuilds_skip"
|
||||
config = {
|
||||
"tablets_mode_for_new_keyspaces": "enabled",
|
||||
"error_injections_at_startup": [skip_fix_injection, fail_injection],
|
||||
}
|
||||
cmdline = [
|
||||
'--logger-log-level', 'load_balancer=debug',
|
||||
'--smp=2',
|
||||
]
|
||||
|
||||
servers = [await manager.server_add(config=config, cmdline=cmdline, property_file={'dc': 'dc1', 'rack': 'rack1a'}),
|
||||
await manager.server_add(config=config, cmdline=cmdline, property_file={'dc': 'dc1', 'rack': 'rack1b'}),
|
||||
await manager.server_add(config=config, cmdline=cmdline, property_file={'dc': 'dc1', 'rack': 'rack1c'})]
|
||||
|
||||
cql = manager.get_cql()
|
||||
|
||||
await cql.run_async(f"create keyspace ks1 with replication = {{'class': 'NetworkTopologyStrategy', 'dc1': ['rack1a']}} and tablets = {{'initial': 4}};")
|
||||
await cql.run_async("create table ks1.t (pk int primary key);")
|
||||
await asyncio.gather(*[cql.run_async(f"INSERT INTO ks1.t (pk) VALUES ({pk});") for pk in range(16)])
|
||||
|
||||
await alter_keyspace("'dc1': ['rack1a', 'rack1b']")
|
||||
|
||||
tablet_replicas = await get_all_tablet_replicas(manager, servers[0], "ks1", "t")
|
||||
assert len(tablet_replicas) == 4
|
||||
for r in tablet_replicas:
|
||||
assert len(r.replicas) == 1
|
||||
|
||||
[await manager.api.disable_injection(s.ip_addr, fail_injection) for s in servers]
|
||||
|
||||
await alter_keyspace("'dc1': ['rack1a', 'rack1b']")
|
||||
|
||||
tablet_replicas = await get_all_tablet_replicas(manager, servers[0], "ks1", "t")
|
||||
assert len(tablet_replicas) == 4
|
||||
for r in tablet_replicas:
|
||||
assert len(r.replicas) == 2
|
||||
|
||||
# Reproducer for https://github.com/scylladb/scylladb/issues/18110
|
||||
# Check that an existing cached read, will be cleaned up when the tablet it reads
|
||||
# from is migrated away.
|
||||
|
||||
Reference in New Issue
Block a user