2021-01-21 11:07:55 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Coordination/TestKeeperStorage.h>
|
|
|
|
#include <libnuraft/nuraft.hxx>
|
|
|
|
#include <common/logger_useful.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
class NuKeeperStateMachine : public nuraft::state_machine
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
NuKeeperStateMachine();
|
|
|
|
|
|
|
|
nuraft::ptr<nuraft::buffer> pre_commit(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override { return nullptr; }
|
|
|
|
|
|
|
|
nuraft::ptr<nuraft::buffer> commit(const size_t log_idx, nuraft::buffer & data) override;
|
|
|
|
|
|
|
|
void rollback(const size_t /*log_idx*/, nuraft::buffer & /*data*/) override {}
|
|
|
|
|
|
|
|
size_t last_commit_index() override { return last_committed_idx; }
|
|
|
|
|
|
|
|
bool apply_snapshot(nuraft::snapshot & s) override;
|
|
|
|
|
|
|
|
nuraft::ptr<nuraft::snapshot> last_snapshot() override;
|
|
|
|
|
|
|
|
void create_snapshot(
|
|
|
|
nuraft::snapshot & s,
|
|
|
|
nuraft::async_result<bool>::handler_type & when_done) override;
|
|
|
|
|
|
|
|
void save_logical_snp_obj(
|
|
|
|
nuraft::snapshot & s,
|
|
|
|
size_t & obj_id,
|
|
|
|
nuraft::buffer & data,
|
|
|
|
bool is_first_obj,
|
|
|
|
bool is_last_obj) override;
|
|
|
|
|
|
|
|
int read_logical_snp_obj(
|
|
|
|
nuraft::snapshot & s,
|
|
|
|
void* & user_snp_ctx,
|
|
|
|
ulong obj_id,
|
|
|
|
nuraft::ptr<nuraft::buffer> & data_out,
|
|
|
|
bool & is_last_obj) override;
|
|
|
|
|
2021-01-21 15:09:48 +00:00
|
|
|
TestKeeperStorage & getStorage()
|
2021-01-21 11:07:55 +00:00
|
|
|
{
|
|
|
|
return storage;
|
|
|
|
}
|
|
|
|
|
2021-01-27 13:37:58 +00:00
|
|
|
TestKeeperStorage::ResponsesForSessions processReadRequest(const TestKeeperStorage::RequestForSession & requests);
|
|
|
|
|
2021-01-21 11:07:55 +00:00
|
|
|
private:
|
2021-01-21 13:53:10 +00:00
|
|
|
struct StorageSnapshot
|
|
|
|
{
|
2021-01-21 15:09:48 +00:00
|
|
|
StorageSnapshot(const nuraft::ptr<nuraft::snapshot> & s, const TestKeeperStorage & storage_)
|
2021-01-21 13:53:10 +00:00
|
|
|
: snapshot(s)
|
|
|
|
, storage(storage_)
|
|
|
|
{}
|
|
|
|
|
|
|
|
nuraft::ptr<nuraft::snapshot> snapshot;
|
2021-01-21 15:09:48 +00:00
|
|
|
TestKeeperStorage storage;
|
2021-01-21 13:53:10 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
using StorageSnapshotPtr = std::shared_ptr<StorageSnapshot>;
|
|
|
|
|
|
|
|
StorageSnapshotPtr createSnapshotInternal(nuraft::snapshot & s);
|
|
|
|
|
2021-01-26 08:17:19 +00:00
|
|
|
static StorageSnapshotPtr readSnapshot(nuraft::snapshot & s, nuraft::buffer & in);
|
2021-01-21 13:53:10 +00:00
|
|
|
|
2021-01-26 08:17:19 +00:00
|
|
|
static void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr<nuraft::buffer> & out);
|
2021-01-21 13:53:10 +00:00
|
|
|
|
2021-01-21 15:09:48 +00:00
|
|
|
TestKeeperStorage storage;
|
2021-01-21 13:53:10 +00:00
|
|
|
/// Mutex for snapshots
|
2021-01-21 11:07:55 +00:00
|
|
|
std::mutex snapshots_lock;
|
|
|
|
|
2021-01-21 13:53:10 +00:00
|
|
|
/// Lock for storage
|
|
|
|
std::mutex storage_lock;
|
|
|
|
|
2021-01-21 11:07:55 +00:00
|
|
|
/// Fake snapshot storage
|
2021-01-21 13:53:10 +00:00
|
|
|
std::map<uint64_t, StorageSnapshotPtr> snapshots;
|
2021-01-21 11:07:55 +00:00
|
|
|
|
|
|
|
/// Last committed Raft log number.
|
|
|
|
std::atomic<size_t> last_committed_idx;
|
|
|
|
Poco::Logger * log;
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|