Files
scylladb/test/pylib/repair.py
Piotr Smaron 0fcae72530 test: bootstrap tombstone gc repair cluster sequentially
Avoid concurrent topology changes in the tombstone GC repair setup, where debug-mode nodes running hinted handoff and materialized view startup work can time out while applying Raft entries before the test starts.

Keep the sequential path opt-in so unrelated repair tests still exercise concurrent bootstrap behavior.

Closes scylladb/scylladb#29829
2026-05-13 13:58:44 +03:00

120 lines
5.1 KiB
Python

#
# Copyright (C) 2024-present ScyllaDB
#
# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.1
#
from test.pylib.internal_types import ServerInfo
from test.pylib.util import wait_for_cql_and_get_hosts, Host
from test.cluster.util import create_new_test_keyspace
from test.pylib.rest_client import read_barrier
from cassandra.cluster import Session as CassandraSession
import asyncio
import time
import logging
import json
async def load_tablet_repair_time(cql, hosts, table_id):
all_rows = []
repair_time_map = {}
for host in hosts:
logging.debug(f'Query hosts={host}');
rows = await cql.run_async(f"SELECT last_token, repair_time from system.tablets where table_id = {table_id}", host=host)
all_rows += rows
for row in all_rows:
logging.debug(f"Got system.tablets={row}")
for row in all_rows:
key = str(row[0])
repair_time_map[key] = row[1]
return repair_time_map
async def load_tablet_sstables_repaired_at(manager, cql, server, host, table_id):
# And when ensuring that local tablet metadata on the queried node reflects the finalized tablet tasks
await read_barrier(manager.api, server.ip_addr)
all_rows = []
ret = {}
logging.debug(f'Query hosts={host}');
all_rows = await cql.run_async(f"SELECT last_token, sstables_repaired_at from system.tablets where table_id = {table_id}", host=host)
for row in all_rows:
logging.debug(f"Got system.tablets={row}")
for row in all_rows:
key = str(row[0])
ret[key] = row[1]
return ret
async def load_tablet_repair_task_infos(cql, host, table_id):
repair_task_infos = {}
rows = await cql.run_async(f"SELECT last_token, repair_task_info from system.tablets where table_id = {table_id}", host=host)
for row in rows:
if row.repair_task_info is not None:
key = str(row.last_token)
repair_task_infos[key] = row.repair_task_info
return repair_task_infos
async def create_table_insert_data_for_repair(manager, rf=3, tablets=8, fast_stats_refresh=True, nr_keys=256,
disable_flush_cache_time=False, cmdline=None,
sequential_server_add=False) -> tuple[list[ServerInfo], CassandraSession, list[Host], str, str]:
assert rf <= 3, "A keyspace with RF > 3 will be RF-rack-invalid if there are fewer racks than the RF"
if fast_stats_refresh:
config = {'tablet_load_stats_refresh_interval_in_seconds': 1}
else:
config = {}
if disable_flush_cache_time:
config.update({'repair_hints_batchlog_flush_cache_time_in_ms': 0})
property_files = [{"dc": "dc1", "rack": f"r{i % rf}"} for i in range(3)]
if sequential_server_add:
servers = []
for property_file in property_files:
servers.append(await manager.server_add(config=config, cmdline=cmdline, property_file=property_file))
else:
servers = await manager.servers_add(len(property_files), config=config, cmdline=cmdline, property_file=property_files)
cql = manager.get_cql()
ks = await create_new_test_keyspace(cql, "WITH replication = {{'class': 'NetworkTopologyStrategy', "
"'replication_factor': {}}} AND tablets = {{'initial': {}}};".format(rf, tablets))
await cql.run_async(f"CREATE TABLE {ks}.test (pk int PRIMARY KEY, c int) WITH tombstone_gc = {{'mode':'repair'}};")
keys = range(nr_keys)
await asyncio.gather(*[cql.run_async(f"INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});") for k in keys])
hosts = await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60)
logging.info(f'Got hosts={hosts}');
table_id = await manager.get_table_id(ks, "test")
return (servers, cql, hosts, ks, table_id)
async def get_tablet_task_id(cql, host, table_id, token):
rows = await cql.run_async(f"SELECT last_token, repair_task_info from system.tablets where table_id = {table_id}", host=host)
for row in rows:
if row.last_token == token:
if row.repair_task_info == None:
return None
else:
return str(row.repair_task_info.tablet_task_id)
return None
async def create_table_insert_data_for_repair_multiple_rows(manager, rf = 3 , tablets = 8, cmdline = None):
assert rf <= 3, "A keyspace with RF > 3 will be RF-rack-invalid if there are fewer racks than the RF"
config = {}
servers = await manager.servers_add(3, config=config, cmdline=cmdline,
property_file=[{"dc": "dc1", "rack": f"r{i % rf}"} for i in range(rf)])
cql = manager.get_cql()
ks = await create_new_test_keyspace(cql, "WITH replication = {{'class': 'NetworkTopologyStrategy', "
"'replication_factor': {}}} AND tablets = {{'initial': {}}};".format(rf, tablets))
create_table_cql = f"CREATE TABLE IF NOT EXISTS {ks}.test ( pk int, ck int, data int, PRIMARY KEY (pk, ck)) WITH tombstone_gc = {{'mode':'repair'}};"
await cql.run_async(create_table_cql)
hosts = await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60)
table_id = await manager.get_table_id(ks, "test")
return (servers, cql, hosts, ks, table_id)