better test

This commit is contained in:
Antonio Andelic 2024-02-29 12:46:12 +01:00
parent 9e3355acef
commit 75837152cb
2 changed files with 43 additions and 15 deletions

View File

@ -1790,7 +1790,6 @@ void testLogAndStateMachine(
keeper_context);
changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items);
std::optional<SnapshotFileInfo> latest_snapshot;
for (size_t i = 1; i < total_logs + 1; ++i)
{
std::shared_ptr<ZooKeeperCreateRequest> request = std::make_shared<ZooKeeperCreateRequest>();
@ -1819,7 +1818,7 @@ void testLogAndStateMachine(
bool pop_result = snapshots_queue.pop(snapshot_task);
EXPECT_TRUE(pop_result);
latest_snapshot = snapshot_task.create_snapshot(std::move(snapshot_task.snapshot));
snapshot_task.create_snapshot(std::move(snapshot_task.snapshot));
}
if (snapshot_created && changelog.size() > settings->reserved_log_items)
@ -1862,19 +1861,6 @@ void testLogAndStateMachine(
auto path = "/hello_" + std::to_string(i);
EXPECT_EQ(source_storage.container.getValue(path).getData(), restored_storage.container.getValue(path).getData());
}
if (latest_snapshot.has_value())
{
const auto & [path, disk] = *latest_snapshot;
EXPECT_TRUE(disk->exists(path));
DB::WriteBufferFromFile plain_buf(
fs::path("./snapshots") / path, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY);
plain_buf.truncate(0);
SnapshotsQueue snapshots_queue2{1};
keeper_context = get_keeper_context();
auto invalid_snapshot_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue2, keeper_context, nullptr);
ASSERT_DEATH(invalid_snapshot_machine->init(), "Aborting because of failure to load from latest snapshot with");
}
}
TEST_P(CoordinationTest, TestStateMachineAndLogStore)

View File

@ -161,3 +161,45 @@ def test_ephemeral_after_restart(started_cluster):
node_zk2.close()
except:
pass
def test_invalid_snapshot(started_cluster):
keeper_utils.wait_until_connected(started_cluster, node)
node_zk = None
try:
node_zk = get_connection_zk("node")
node_zk.create("/test_invalid_snapshot", b"somevalue")
keeper_utils.send_4lw_cmd(started_cluster, node, "csnp")
node.stop_clickhouse()
snapshots = (
node.exec_in_container(["ls", "/var/lib/clickhouse/coordination/snapshots"])
.strip()
.split("\n")
)
def snapshot_sort_key(snapshot_name):
snapshot_prefix_size = len("snapshot_")
last_log_idx = snapshot_name.split(".")[0][snapshot_prefix_size:]
return int(last_log_idx)
snapshots.sort(key=snapshot_sort_key)
last_snapshot = snapshots[-1]
node.exec_in_container(
[
"truncate",
"-s",
"0",
f"/var/lib/clickhouse/coordination/snapshots/{last_snapshot}",
]
)
node.start_clickhouse(expected_to_fail=True)
assert node.contains_in_log(
"Aborting because of failure to load from latest snapshot with index"
)
finally:
try:
if node_zk is not None:
node_zk.stop()
node_zk.close()
except:
pass