mirror of
https://github.com/scylladb/scylladb.git
synced 2026-04-29 04:37:00 +00:00
Merge '[Backport 2025.3] db: batchlog_manager: update _last_replay only if all batches were re…' from Scylladb[bot]
…played Currently, if flushing hints falls within the repair cache timeout, then the flush_time is set to batchlog_manager::_last_replay. _last_replay is updated on each replay, even if some batches weren't replayed. Due to that, we risk the data resurrection. Update _last_replay only if all batches were replayed. Fixes: https://github.com/scylladb/scylladb/issues/24415. Needs backport to all live versions. - (cherry picked from commit4d0de1126f) - (cherry picked from commite3dcb7e827) Parent PR: #26793 Closes scylladb/scylladb#27092 * github.com:scylladb/scylladb: test: extend test_batchlog_replay_failure_during_repair db: batchlog_manager: update _last_replay only if all batches were replayed
This commit is contained in:
@@ -295,7 +295,8 @@ async def test_drop_mutations_for_dropped_table(manager: ManagerClient) -> None:
|
||||
|
||||
@pytest.mark.asyncio
|
||||
@skip_mode("release", "error injections are not supported in release mode")
|
||||
async def test_batchlog_replay_failure_during_repair(manager: ManagerClient) -> None:
|
||||
@pytest.mark.parametrize("repair_cache", [True, False])
|
||||
async def test_batchlog_replay_failure_during_repair(manager: ManagerClient, repair_cache: bool) -> None:
|
||||
"""
|
||||
We want to verify that repair_time will not be updated if batchlog replay fails.
|
||||
|
||||
@@ -315,7 +316,7 @@ async def test_batchlog_replay_failure_during_repair(manager: ManagerClient) ->
|
||||
cmdline=['--enable-cache', '0',
|
||||
"--hinted-handoff-enabled", "0",
|
||||
"--logger-log-level", "batchlog_manager=trace:repair=debug",
|
||||
"--repair-hints-batchlog-flush-cache-time-in-ms", "0"]
|
||||
"--repair-hints-batchlog-flush-cache-time-in-ms", "1000000" if repair_cache else "0"]
|
||||
config = {"error_injections_at_startup": ["short_batchlog_manager_replay_interval"],
|
||||
"write_request_timeout_in_ms": 2000,
|
||||
'group0_tombstone_gc_refresh_interval_in_ms': 1000,
|
||||
@@ -390,6 +391,9 @@ async def test_batchlog_replay_failure_during_repair(manager: ManagerClient) ->
|
||||
# be triggered from now on (if it's present).
|
||||
await disable_injection("skip_batch_replay")
|
||||
|
||||
await s1_log.wait_for("Batchlog replay on shard 0: done", timeout=60, from_mark=s1_mark)
|
||||
await s1_log.wait_for("Batchlog replay on shard 1: done", timeout=60, from_mark=s1_mark)
|
||||
|
||||
await manager.api.repair(s1.ip_addr, ks, "my_table")
|
||||
|
||||
await manager.api.flush_keyspace(s1.ip_addr, ks)
|
||||
|
||||
Reference in New Issue
Block a user