mirror of
https://github.com/scylladb/scylladb.git
synced 2026-05-22 15:52:13 +00:00
Add a regression test that reproduces the race between tablet split and truncation. The test: 1. Creates a single-tablet table and inserts data. 2. Triggers truncation and pauses it (via database_truncate_wait) after compaction is disabled but before discard_sstables() runs. 3. Triggers tablet split and pauses it (via tablet_split_monitor_wait) at the start of process_tablet_split_candidate(). 4. Releases split so set_split_mode() creates new compaction groups. 5. Waits for the set_split_mode log confirming the groups exist. 6. Releases truncation so discard_sstables() encounters the new groups. 7. Verifies truncation completes and split finishes. Adds a tablet_split_monitor_wait error injection point in process_tablet_split_candidate() to allow pausing the split monitor before it enters the split loop.
424 lines
19 KiB
Python
424 lines
19 KiB
Python
#
|
|
# Copyright (C) 2024-present ScyllaDB
|
|
#
|
|
# SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.1
|
|
#
|
|
from cassandra.query import SimpleStatement, ConsistencyLevel
|
|
from cassandra.protocol import InvalidRequest
|
|
from cassandra.cluster import TruncateError
|
|
from cassandra.policies import FallthroughRetryPolicy
|
|
from test.pylib.manager_client import ManagerClient
|
|
from test.cluster.util import get_topology_coordinator, new_test_keyspace
|
|
from test.pylib.tablets import get_all_tablet_replicas, get_tablet_count
|
|
from test.pylib.util import wait_for_cql_and_get_hosts, wait_for
|
|
import time
|
|
import pytest
|
|
import logging
|
|
import asyncio
|
|
|
|
logger = logging.getLogger(__name__)
|
|
|
|
@pytest.mark.asyncio
|
|
@pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode')
|
|
async def test_truncate_while_migration(manager: ManagerClient):
|
|
|
|
logger.info('Bootstrapping cluster')
|
|
cfg = { 'tablets_mode_for_new_keyspaces': 'enabled',
|
|
'error_injections_at_startup': ['migration_streaming_wait']
|
|
}
|
|
|
|
servers = []
|
|
servers.append(await manager.server_add(config=cfg))
|
|
|
|
cql = manager.get_cql()
|
|
|
|
# Create a keyspace with tablets and initial_tablets == 2, then insert data
|
|
async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 2}") as ks:
|
|
await cql.run_async(f'CREATE TABLE {ks}.test (pk int PRIMARY KEY, c int);')
|
|
|
|
keys = range(1024)
|
|
await asyncio.gather(*[cql.run_async(f'INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});') for k in keys])
|
|
|
|
# Add a node to the cluster. This will cause the tablet load balancer to migrate one tablet to the new node
|
|
servers.append(await manager.server_add(config=cfg))
|
|
|
|
# Wait for tablet streaming to start
|
|
pending_node = servers[1]
|
|
pending_log = await manager.server_open_log(pending_node.server_id)
|
|
|
|
await pending_log.wait_for('migration_streaming_wait: start')
|
|
await manager.api.message_injection(pending_node.ip_addr, 'migration_streaming_wait')
|
|
|
|
# Do a TRUNCATE TABLE while the tablet is being streamed
|
|
await cql.run_async(f'TRUNCATE TABLE {ks}.test')
|
|
|
|
# Wait for streaming to complete
|
|
await pending_log.wait_for('raft_topology - Streaming for tablet migration of.*successful')
|
|
|
|
# Check if we have any data
|
|
row = await cql.run_async(SimpleStatement(f'SELECT COUNT(*) FROM {ks}.test', consistency_level=ConsistencyLevel.ALL))
|
|
assert row[0].count == 0
|
|
|
|
|
|
async def get_raft_leader_and_log(manager: ManagerClient, servers):
|
|
raft_leader_host_id = await get_topology_coordinator(manager)
|
|
for s in servers:
|
|
if raft_leader_host_id == await manager.get_host_id(s.server_id):
|
|
raft_leader = s
|
|
break
|
|
raft_leader_log = await manager.server_open_log(raft_leader.server_id)
|
|
return (raft_leader, raft_leader_log)
|
|
|
|
|
|
@pytest.mark.asyncio
|
|
@pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode')
|
|
async def test_truncate_with_concurrent_drop(manager: ManagerClient):
|
|
|
|
logger.info('Bootstrapping cluster')
|
|
cfg = { 'tablets_mode_for_new_keyspaces': 'enabled',
|
|
'error_injections_at_startup': ['truncate_table_wait']
|
|
}
|
|
|
|
servers = []
|
|
servers.append(await manager.server_add(config=cfg))
|
|
servers.append(await manager.server_add(config=cfg))
|
|
servers.append(await manager.server_add(config=cfg))
|
|
|
|
cql = manager.get_cql()
|
|
hosts = await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60)
|
|
|
|
# Create a keyspace with tablets and initial_tablets == 2, then insert data
|
|
async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 2}") as ks:
|
|
await cql.run_async(f'CREATE TABLE {ks}.test (pk int PRIMARY KEY, c int);')
|
|
|
|
keys = range(1024)
|
|
await asyncio.gather(*[cql.run_async(f'INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});') for k in keys])
|
|
|
|
(raft_leader, raft_leader_log) = await get_raft_leader_and_log(manager, servers)
|
|
|
|
if raft_leader == servers[0]:
|
|
trunc_host = hosts[1]
|
|
drop_host = hosts[2]
|
|
elif raft_leader == servers[1]:
|
|
trunc_host = hosts[0]
|
|
drop_host = hosts[2]
|
|
elif raft_leader == servers[2]:
|
|
trunc_host = hosts[0]
|
|
drop_host = hosts[1]
|
|
else:
|
|
assert False, 'Unable to determine raft leader'
|
|
|
|
# Start a TRUNCATE in the background
|
|
trunc_future = cql.run_async(f'TRUNCATE TABLE {ks}.test', host=trunc_host)
|
|
# Wait for the topology coordinator to reach a point wher it is about to start sending the truncate RPCs
|
|
await raft_leader_log.wait_for('truncate_table_wait: waiting for message')
|
|
# Execute DROP TABLE
|
|
await cql.run_async(f'DROP TABLE {ks}.test', host=drop_host)
|
|
# Release TRUNCATE table in topology coordinator
|
|
await manager.api.message_injection(raft_leader.ip_addr, 'truncate_table_wait')
|
|
# Check we received an error
|
|
with pytest.raises(InvalidRequest, match='unconfigured table test'):
|
|
await trunc_future
|
|
|
|
|
|
@pytest.mark.asyncio
|
|
@pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode')
|
|
async def test_truncate_while_node_restart(manager: ManagerClient):
|
|
|
|
logger.info('Bootstrapping cluster')
|
|
cfg = { 'tablets_mode_for_new_keyspaces': 'enabled' }
|
|
|
|
servers = []
|
|
servers.append(await manager.server_add(config=cfg))
|
|
servers.append(await manager.server_add(config=cfg))
|
|
servers.append(await manager.server_add(config=cfg))
|
|
|
|
cql = manager.get_cql()
|
|
hosts = await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60)
|
|
|
|
# Create a keyspace with tablets and initial_tablets == 2, then insert data
|
|
async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 2}") as ks:
|
|
await cql.run_async(f'CREATE TABLE {ks}.test (pk int PRIMARY KEY, c int);')
|
|
|
|
keys = range(1024)
|
|
await asyncio.gather(*[cql.run_async(f'INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});') for k in keys])
|
|
|
|
(raft_leader, raft_leader_log) = await get_raft_leader_and_log(manager, servers)
|
|
|
|
# Decide which node to restart; select a node with a replica but not the raft leader
|
|
tablet_replicas = await get_all_tablet_replicas(manager, raft_leader, ks, 'test')
|
|
replica_hosts = [tr.replicas[0][0] for tr in tablet_replicas]
|
|
for s in servers:
|
|
if s != raft_leader:
|
|
host_id = await manager.get_host_id(s.server_id)
|
|
if host_id in replica_hosts:
|
|
restart_node = s
|
|
break
|
|
|
|
# Shutdown the node containing a replica
|
|
await manager.server_stop_gracefully(restart_node.server_id)
|
|
# Start truncating in the background
|
|
trunc_future = cql.run_async(f'TRUNCATE TABLE {ks}.test', host=hosts[0])
|
|
# Restart the node
|
|
await manager.server_start(restart_node.server_id)
|
|
# Wait for truncate to complete
|
|
await trunc_future
|
|
|
|
# Check if truncate was successful
|
|
row = await cql.run_async(SimpleStatement(f'SELECT COUNT(*) FROM {ks}.test', consistency_level=ConsistencyLevel.ALL))
|
|
assert row[0].count == 0
|
|
|
|
|
|
@pytest.mark.asyncio
|
|
@pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode')
|
|
async def test_truncate_with_coordinator_crash(manager: ManagerClient):
|
|
|
|
logger.info('Bootstrapping cluster')
|
|
cfg = { 'tablets_mode_for_new_keyspaces': 'enabled' }
|
|
|
|
servers = []
|
|
servers.append(await manager.server_add(config=cfg))
|
|
servers.append(await manager.server_add(config=cfg))
|
|
|
|
cql = manager.get_cql()
|
|
hosts = await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60)
|
|
|
|
# Create a keyspace with tablets and initial_tablets == 2, then insert data
|
|
async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 2}") as ks:
|
|
await cql.run_async(f'CREATE TABLE {ks}.test (pk int PRIMARY KEY, c int);')
|
|
|
|
keys = range(1024)
|
|
await asyncio.gather(*[cql.run_async(f'INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});') for k in keys])
|
|
|
|
(raft_leader, raft_leader_log) = await get_raft_leader_and_log(manager, servers)
|
|
|
|
if raft_leader == servers[0]:
|
|
trunc_host = hosts[1]
|
|
else:
|
|
trunc_host = hosts[0]
|
|
|
|
# Enable injection to crash the raft leader after truncate cleared the session ID
|
|
await manager.api.enable_injection(raft_leader.ip_addr, 'truncate_crash_after_session_clear', one_shot=False)
|
|
|
|
# Start a TRUNCATE in the background
|
|
trunc_future = cql.run_async(f'TRUNCATE TABLE {ks}.test', host=trunc_host)
|
|
# Wait for the topology coordinator to crash
|
|
await raft_leader_log.wait_for('truncate_crash_after_session_clear hit, killing the node')
|
|
await manager.server_stop(raft_leader.server_id)
|
|
# Restart the crashed node
|
|
await manager.server_start(raft_leader.server_id)
|
|
# Wait for truncate to complete
|
|
await trunc_future
|
|
|
|
# Check if we have any data
|
|
row = await cql.run_async(SimpleStatement(f'SELECT COUNT(*) FROM {ks}.test', consistency_level=ConsistencyLevel.ALL))
|
|
assert row[0].count == 0
|
|
|
|
|
|
@pytest.mark.asyncio
|
|
@pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode')
|
|
async def test_truncate_while_truncate_already_waiting(manager: ManagerClient):
|
|
|
|
logger.info('Bootstrapping cluster')
|
|
cfg = { 'tablets_mode_for_new_keyspaces': 'enabled',
|
|
'error_injections_at_startup': ['migration_streaming_wait']
|
|
}
|
|
|
|
servers = []
|
|
servers.append(await manager.server_add(config=cfg))
|
|
|
|
cql = manager.get_cql()
|
|
|
|
# Create a keyspace with tablets and initial_tablets == 2, then insert data
|
|
async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 2}") as ks:
|
|
await cql.run_async(f'CREATE TABLE {ks}.test (pk int PRIMARY KEY, c int);')
|
|
|
|
keys = range(1024)
|
|
await asyncio.gather(*[cql.run_async(f'INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});') for k in keys])
|
|
|
|
# Add a node to the cluster. This will cause the load balancer to migrate one tablet to the new node
|
|
servers.append(await manager.server_add(config=cfg))
|
|
|
|
hosts = await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60)
|
|
s1_log = await manager.server_open_log(servers[1].server_id)
|
|
|
|
# Wait for tablet streaming to start
|
|
await s1_log.wait_for('migration_streaming_wait: start')
|
|
|
|
# Run a truncate which will quickly time out, but the truncate fiber remains alive
|
|
# Do not attempt to retry automatically (hense the FallthroughRetryPolicy)
|
|
with pytest.raises((TruncateError), match=f'Timeout during TRUNCATE TABLE of {ks}.test'):
|
|
await cql.run_async(SimpleStatement(f'TRUNCATE TABLE {ks}.test USING TIMEOUT 100ms', retry_policy=FallthroughRetryPolicy()))
|
|
|
|
# Run another truncate on the same table while the timedout one is still waiting
|
|
truncate_future = cql.run_async(f'TRUNCATE TABLE {ks}.test', host=hosts[1])
|
|
|
|
# Release streaming
|
|
await manager.api.message_injection(servers[1].ip_addr, 'migration_streaming_wait')
|
|
|
|
# Wait for the joined truncate to complete
|
|
await truncate_future
|
|
|
|
# Check if we have any data
|
|
row = await cql.run_async(SimpleStatement(f'SELECT COUNT(*) FROM {ks}.test', consistency_level=ConsistencyLevel.ALL))
|
|
assert row[0].count == 0
|
|
|
|
# Reproduces https://github.com/scylladb/scylladb/issues/23771.
|
|
@pytest.mark.asyncio
|
|
@pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode')
|
|
async def test_replay_position_check_during_truncate(manager):
|
|
logger.info("Bootstrapping cluster")
|
|
cfg = { 'auto_snapshot': True }
|
|
cmdline = ['--smp=1']
|
|
servers = await manager.servers_add(1, cmdline=cmdline, config=cfg)
|
|
server = servers[0]
|
|
|
|
cql = manager.get_cql()
|
|
async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 1}") as ks:
|
|
await cql.run_async(f"CREATE TABLE {ks}.test (pk int PRIMARY KEY, c int);")
|
|
|
|
keys = range(10)
|
|
await asyncio.gather(*[cql.run_async(f"INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});") for k in keys])
|
|
|
|
#await manager.api.flush_keyspace(server.ip_addr, ks)
|
|
|
|
await manager.api.enable_injection(server.ip_addr, "database_truncate_wait", True)
|
|
|
|
s1_log = await manager.server_open_log(server.server_id)
|
|
s1_mark = await s1_log.mark()
|
|
|
|
truncate_task = cql.run_async(f"TRUNCATE {ks}.test")
|
|
|
|
await s1_log.wait_for(f"database_truncate_wait: waiting", from_mark=s1_mark)
|
|
|
|
keys = range(10)
|
|
await asyncio.gather(*[cql.run_async(f"INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});") for k in keys])
|
|
await manager.api.flush_keyspace(server.ip_addr, ks)
|
|
|
|
await manager.api.message_injection(server.ip_addr, "database_truncate_wait")
|
|
await s1_log.wait_for(f"database_truncate_wait: message received", from_mark=s1_mark)
|
|
await truncate_task
|
|
|
|
@pytest.mark.asyncio
|
|
@pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode')
|
|
async def test_parallel_truncate(manager: ManagerClient):
|
|
|
|
logger.info('Bootstrapping cluster')
|
|
cfg = { 'tablets_mode_for_new_keyspaces': 'enabled',
|
|
'error_injections_at_startup': ['migration_streaming_wait']
|
|
}
|
|
|
|
servers = []
|
|
servers.append(await manager.server_add(config=cfg))
|
|
|
|
cql = manager.get_cql()
|
|
|
|
# Create a keyspace with tablets and initial_tablets == 2, then insert data
|
|
async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1} AND tablets = {'initial': 2}") as ks:
|
|
await cql.run_async(f'CREATE TABLE {ks}.test (pk int PRIMARY KEY, c int);')
|
|
await cql.run_async(f'CREATE TABLE {ks}.test1 (pk int PRIMARY KEY, c int);')
|
|
|
|
keys = range(1024)
|
|
await asyncio.gather(*[cql.run_async(f'INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});') for k in keys])
|
|
await asyncio.gather(*[cql.run_async(f'INSERT INTO {ks}.test1 (pk, c) VALUES ({k}, {k});') for k in keys])
|
|
|
|
# Add a node to the cluster. This will cause the load balancer to migrate one tablet to the new node
|
|
servers.append(await manager.server_add(config=cfg))
|
|
|
|
hosts = await wait_for_cql_and_get_hosts(cql, servers, time.time() + 60)
|
|
s1_log = await manager.server_open_log(servers[1].server_id)
|
|
|
|
# Wait for tablet streaming to start
|
|
await s1_log.wait_for('migration_streaming_wait: start')
|
|
|
|
tf1 = cql.run_async(SimpleStatement(f'TRUNCATE TABLE {ks}.test', retry_policy=FallthroughRetryPolicy()))
|
|
tf2 = cql.run_async(SimpleStatement(f'TRUNCATE TABLE {ks}.test1', retry_policy=FallthroughRetryPolicy()))
|
|
|
|
# Release streaming
|
|
await manager.api.message_injection(servers[1].ip_addr, 'migration_streaming_wait')
|
|
|
|
# Wait for the joined truncate to complete
|
|
await tf1
|
|
await tf2
|
|
|
|
# Check if we have any data
|
|
row = await cql.run_async(SimpleStatement(f'SELECT COUNT(*) FROM {ks}.test', consistency_level=ConsistencyLevel.ALL))
|
|
assert row[0].count == 0
|
|
row = await cql.run_async(SimpleStatement(f'SELECT COUNT(*) FROM {ks}.test1', consistency_level=ConsistencyLevel.ALL))
|
|
assert row[0].count == 0
|
|
|
|
@pytest.mark.asyncio
|
|
@pytest.mark.skip_mode(mode='release', reason='error injections are not supported in release mode')
|
|
async def test_split_emitted_during_truncate(manager: ManagerClient):
|
|
"""Tests that truncation handles new compaction groups introduced by tablet
|
|
split after compaction was already disabled on existing groups.
|
|
|
|
The scenario:
|
|
1. Truncation disables compaction on all existing compaction groups, then pauses.
|
|
2. While paused, tablet split creates new (split-ready) compaction groups.
|
|
3. Truncation resumes and runs discard_sstables(), which encounters groups
|
|
that don't have compaction disabled.
|
|
|
|
Without the fix, discard_sstables() would fire on_internal_error because the
|
|
new groups don't have compaction disabled. With the fix, it tolerates them as
|
|
long as they contain no sstables older than the truncation time.
|
|
"""
|
|
logger.info("Bootstrapping cluster")
|
|
cfg = { 'tablets_mode_for_new_keyspaces': 'enabled',
|
|
'tablet_load_stats_refresh_interval_in_seconds': 1,
|
|
}
|
|
cmdline = [
|
|
'--logger-log-level', 'table=debug',
|
|
'--logger-log-level', 'load_balancer=debug',
|
|
'--logger-log-level', 'debug_error_injection=debug',
|
|
]
|
|
servers = await manager.servers_add(1, cmdline=cmdline, config=cfg)
|
|
server = servers[0]
|
|
|
|
cql = manager.get_cql()
|
|
async with new_test_keyspace(manager, "WITH replication = {'class': 'NetworkTopologyStrategy', 'replication_factor': 1}") as ks:
|
|
await cql.run_async(f"CREATE TABLE {ks}.test (pk int PRIMARY KEY, c int) WITH tablets = {{'min_tablet_count': 1}};")
|
|
|
|
keys = range(10)
|
|
await asyncio.gather(*[cql.run_async(f"INSERT INTO {ks}.test (pk, c) VALUES ({k}, {k});") for k in keys])
|
|
|
|
# Flush so sstables exist on disk, exercising the discard logic more thoroughly
|
|
await manager.api.flush_keyspace(server.ip_addr, ks)
|
|
|
|
await manager.api.enable_injection(server.ip_addr, "database_truncate_wait", True)
|
|
await manager.api.enable_injection(server.ip_addr, "tablet_split_monitor_wait", True)
|
|
|
|
log = await manager.server_open_log(server.server_id)
|
|
mark = await log.mark()
|
|
|
|
expected_tablet_count = 2
|
|
# Force split on the test table
|
|
await cql.run_async(f"ALTER TABLE {ks}.test WITH tablets = {{'min_tablet_count': {expected_tablet_count}}}")
|
|
await log.wait_for("tablet_split_monitor_wait: waiting", from_mark=mark)
|
|
|
|
truncate_task = cql.run_async(f"TRUNCATE {ks}.test")
|
|
|
|
# Wait for truncation to disable compaction on existing groups and pause
|
|
await log.wait_for("database_truncate_wait: waiting", from_mark=mark)
|
|
|
|
# Release split monitor so it calls set_split_mode(), creating new compaction groups
|
|
await manager.api.message_injection(server.ip_addr, "tablet_split_monitor_wait")
|
|
|
|
# Wait for set_split_mode to create the new groups before releasing truncation
|
|
await log.wait_for('storage_group::set_split_mode', from_mark=mark)
|
|
|
|
# Now release truncation - discard_sstables() will see the new groups
|
|
await manager.api.message_injection(server.ip_addr, "database_truncate_wait")
|
|
|
|
await truncate_task
|
|
|
|
# Verify truncation actually removed the data
|
|
row = await cql.run_async(SimpleStatement(f'SELECT COUNT(*) FROM {ks}.test', consistency_level=ConsistencyLevel.ALL))
|
|
assert row[0].count == 0
|
|
|
|
async def finished_splitting():
|
|
tablet_count = await get_tablet_count(manager, server, ks, 'test')
|
|
return tablet_count >= expected_tablet_count or None
|
|
# Give enough time for split to happen in debug mode
|
|
await wait_for(finished_splitting, time.time() + 120)
|