Fix race condition on snapshots

This commit is contained in:
alesapin 2021-03-26 13:20:07 +03:00
parent 4c28c25a75
commit ba5c151037

View File

@ -4,6 +4,7 @@
#include <IO/ReadHelpers.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Coordination/NuKeeperSnapshotManager.h>
#include <future>
namespace DB
{
@ -227,7 +228,28 @@ void NuKeeperStateMachine::save_logical_snp_obj(
nuraft::ptr<nuraft::buffer> snp_buf = s.serialize();
cloned_meta = nuraft::snapshot::deserialize(*snp_buf);
auto result_path = snapshot_manager.serializeSnapshotBufferToDisk(*cloned_buffer, s.get_last_log_idx());
/// Sometimes NuRaft can call save and create snapshots from different threads
/// at onces. To avoid race conditions we serialize snapshots through snapshots_queue
/// TODO: make something better
CreateSnapshotTask snapshot_task;
std::shared_ptr<std::promise<void>> waiter = std::make_shared<std::promise<void>>();
auto future = waiter->get_future();
snapshot_task.snapshot = nullptr;
snapshot_task.create_snapshot = [this, waiter, cloned_buffer, log_idx = s.get_last_log_idx()] (NuKeeperStorageSnapshotPtr &&)
{
try
{
auto result_path = snapshot_manager.serializeSnapshotBufferToDisk(*cloned_buffer, log_idx);
LOG_DEBUG(log, "Saved snapshot {} to path {}", log_idx, result_path);
}
catch (...)
{
tryLogCurrentException(log);
}
waiter->set_value();
};
snapshots_queue.push(std::move(snapshot_task));
future.wait();
{
std::lock_guard lock(snapshots_lock);
@ -235,7 +257,6 @@ void NuKeeperStateMachine::save_logical_snp_obj(
latest_snapshot_meta = cloned_meta;
}
LOG_DEBUG(log, "Created snapshot {} with path {}", s.get_last_log_idx(), result_path);
obj_id++;
}