ClickHouse/src/Coordination/KeeperSnapshotManager.h

97 lines
2.9 KiB
C++
Raw Normal View History

2021-02-01 14:14:59 +00:00
#pragma once
2021-03-01 13:33:34 +00:00
#include <libnuraft/nuraft.hxx> // Y_IGNORE
2021-03-29 08:24:56 +00:00
#include <Coordination/KeeperStorage.h>
2021-02-01 14:14:59 +00:00
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
namespace DB
{
2021-03-02 14:30:56 +00:00
using SnapshotMetadata = nuraft::snapshot;
using SnapshotMetadataPtr = std::shared_ptr<SnapshotMetadata>;
2021-03-01 13:33:34 +00:00
enum SnapshotVersion : uint8_t
{
V0 = 0,
V1 = 1, /// with ACL map
2021-06-17 16:37:08 +00:00
V2 = 2, /// with 64 bit buffer header
2021-03-01 13:33:34 +00:00
};
2021-06-17 16:37:08 +00:00
static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V2;
2021-03-29 08:24:56 +00:00
struct KeeperStorageSnapshot
2021-02-01 14:14:59 +00:00
{
public:
2021-04-08 14:17:57 +00:00
KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_);
2021-03-02 14:30:56 +00:00
2021-03-29 08:24:56 +00:00
KeeperStorageSnapshot(KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_);
~KeeperStorageSnapshot();
2021-03-01 13:33:34 +00:00
2021-03-29 08:24:56 +00:00
static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out);
2021-02-01 14:14:59 +00:00
2021-03-29 08:24:56 +00:00
static SnapshotMetadataPtr deserialize(KeeperStorage & storage, ReadBuffer & in);
2021-03-01 13:33:34 +00:00
2021-03-29 08:24:56 +00:00
KeeperStorage * storage;
2021-03-01 13:33:34 +00:00
2021-06-17 16:37:08 +00:00
SnapshotVersion version = CURRENT_SNAPSHOT_VERSION;
2021-03-02 14:30:56 +00:00
SnapshotMetadataPtr snapshot_meta;
2021-03-01 13:33:34 +00:00
int64_t session_id;
size_t snapshot_container_size;
2021-03-29 08:24:56 +00:00
KeeperStorage::Container::const_iterator begin;
2021-03-01 13:33:34 +00:00
SessionAndTimeout session_and_timeout;
KeeperStorage::SessionAndAuth session_and_auth;
std::unordered_map<uint64_t, Coordination::ACLs> acl_map;
2021-03-01 13:33:34 +00:00
};
2021-03-29 08:24:56 +00:00
using KeeperStorageSnapshotPtr = std::shared_ptr<KeeperStorageSnapshot>;
using CreateSnapshotCallback = std::function<void(KeeperStorageSnapshotPtr &&)>;
2021-03-05 10:40:24 +00:00
2021-03-19 08:08:43 +00:00
2021-03-29 08:24:56 +00:00
using SnapshotMetaAndStorage = std::pair<SnapshotMetadataPtr, KeeperStoragePtr>;
2021-03-19 08:08:43 +00:00
2021-03-29 08:24:56 +00:00
class KeeperSnapshotManager
2021-03-01 13:33:34 +00:00
{
public:
2021-05-23 17:54:42 +00:00
KeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_, const std::string & superdigest_ = "", size_t storage_tick_time_ = 500);
2021-03-01 13:33:34 +00:00
2021-03-19 08:08:43 +00:00
SnapshotMetaAndStorage restoreFromLatestSnapshot();
2021-03-01 13:33:34 +00:00
2021-03-29 08:24:56 +00:00
static nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot);
2021-04-08 14:17:57 +00:00
std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx);
2021-03-01 13:33:34 +00:00
2021-03-19 08:08:43 +00:00
SnapshotMetaAndStorage deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
2021-03-02 14:30:56 +00:00
2021-04-08 14:17:57 +00:00
nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const;
nuraft::ptr<nuraft::buffer> deserializeLatestSnapshotBufferFromDisk();
2021-04-08 14:17:57 +00:00
void removeSnapshot(uint64_t log_idx);
size_t totalSnapshots() const
{
return existing_snapshots.size();
}
size_t getLatestSnapshotIndex() const
{
if (!existing_snapshots.empty())
return existing_snapshots.rbegin()->first;
return 0;
}
2021-03-01 13:33:34 +00:00
private:
2021-03-01 14:54:08 +00:00
void removeOutdatedSnapshotsIfNeeded();
2021-03-01 13:33:34 +00:00
const std::string snapshots_path;
2021-03-01 14:54:08 +00:00
const size_t snapshots_to_keep;
2021-04-08 14:17:57 +00:00
std::map<uint64_t, std::string> existing_snapshots;
2021-05-23 17:54:42 +00:00
const std::string superdigest;
2021-03-19 08:08:43 +00:00
size_t storage_tick_time;
2021-02-01 14:14:59 +00:00
};
2021-03-05 10:40:24 +00:00
struct CreateSnapshotTask
{
2021-03-29 08:24:56 +00:00
KeeperStorageSnapshotPtr snapshot;
2021-03-05 10:40:24 +00:00
CreateSnapshotCallback create_snapshot;
};
2021-02-01 14:14:59 +00:00
}