ClickHouse/src/Coordination/KeeperSnapshotManager.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

187 lines
6.8 KiB
C++
Raw Normal View History

2021-02-01 14:14:59 +00:00
#pragma once
2022-02-15 12:25:19 +00:00
#include <filesystem>
2022-02-14 11:43:08 +00:00
#include <system_error>
2021-03-29 08:24:56 +00:00
#include <Coordination/KeeperStorage.h>
2021-02-01 14:14:59 +00:00
#include <IO/ReadBuffer.h>
2022-05-19 09:45:38 +00:00
#include <IO/WriteBuffer.h>
#include <libnuraft/nuraft.hxx>
2022-07-23 14:27:44 +00:00
#include <Coordination/KeeperContext.h>
2021-02-01 14:14:59 +00:00
namespace DB
{
2021-03-02 14:30:56 +00:00
using SnapshotMetadata = nuraft::snapshot;
using SnapshotMetadataPtr = std::shared_ptr<SnapshotMetadata>;
2021-10-18 15:27:51 +00:00
using ClusterConfig = nuraft::cluster_config;
using ClusterConfigPtr = nuraft::ptr<ClusterConfig>;
2021-03-02 14:30:56 +00:00
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
V3 = 3, /// compress snapshots with ZSTD codec
2021-12-09 15:09:56 +00:00
V4 = 4, /// add Node size to snapshots
2022-05-16 13:08:10 +00:00
V5 = 5, /// add ZXID and digest to snapshots
2021-03-01 13:33:34 +00:00
};
2022-05-19 09:45:38 +00:00
static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V5;
2021-06-17 16:37:08 +00:00
2022-09-02 08:54:48 +00:00
/// What is stored in binary snapshot
2021-10-18 15:27:51 +00:00
struct SnapshotDeserializationResult
{
2021-10-19 14:10:09 +00:00
/// Storage
2021-10-18 15:27:51 +00:00
KeeperStoragePtr storage;
2021-10-19 14:10:09 +00:00
/// Snapshot metadata (up_to_log_idx and so on)
2021-10-18 15:27:51 +00:00
SnapshotMetadataPtr snapshot_meta;
2021-10-19 14:10:09 +00:00
/// Cluster config
2021-10-18 15:27:51 +00:00
ClusterConfigPtr cluster_config;
};
/// In memory keeper snapshot. Keeper Storage based on a hash map which can be
/// turned into snapshot mode. This operation is fast and KeeperStorageSnapshot
/// class do it in constructor. It also copies iterators from storage hash table
/// up to some log index with lock. In destructor this class turn off snapshot
/// mode for KeeperStorage.
///
/// This representation of snapshot have to be serialized into NuRaft
/// buffer and send over network or saved to file.
2021-03-29 08:24:56 +00:00
struct KeeperStorageSnapshot
2021-02-01 14:14:59 +00:00
{
public:
2021-10-18 15:27:51 +00:00
KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr);
2022-05-19 09:45:38 +00:00
KeeperStorageSnapshot(
KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr);
2021-03-02 14:30:56 +00:00
2021-03-29 08:24:56 +00:00
~KeeperStorageSnapshot();
2021-03-01 13:33:34 +00:00
2022-07-23 14:27:44 +00:00
static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context);
2021-02-01 14:14:59 +00:00
2022-07-23 14:27:44 +00:00
static void deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context);
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;
/// Snapshot metadata
2021-03-02 14:30:56 +00:00
SnapshotMetadataPtr snapshot_meta;
/// Max session id
2021-03-01 13:33:34 +00:00
int64_t session_id;
/// Size of snapshot container in amount of nodes after begin iterator
/// so we have for loop for (i = 0; i < snapshot_container_size; ++i) { doSmth(begin + i); }
2021-03-01 13:33:34 +00:00
size_t snapshot_container_size;
/// Iterator to the start of the storage
2021-03-29 08:24:56 +00:00
KeeperStorage::Container::const_iterator begin;
/// Active sessions and their timeouts
2021-03-01 13:33:34 +00:00
SessionAndTimeout session_and_timeout;
/// Sessions credentials
KeeperStorage::SessionAndAuth session_and_auth;
/// ACLs cache for better performance. Without we cannot deserialize storage.
std::unordered_map<uint64_t, Coordination::ACLs> acl_map;
2021-10-18 15:27:51 +00:00
/// Cluster config from snapshot, can be empty
ClusterConfigPtr cluster_config;
2022-05-13 08:24:44 +00:00
/// Last committed ZXID
int64_t zxid;
2022-05-16 13:08:10 +00:00
/// Current digest of committed nodes
uint64_t nodes_digest;
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<std::string(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
/// Class responsible for snapshots serialization and deserialization. Each snapshot
/// has it's path on disk and log index.
2021-03-29 08:24:56 +00:00
class KeeperSnapshotManager
2021-03-01 13:33:34 +00:00
{
public:
KeeperSnapshotManager(
2022-05-19 09:45:38 +00:00
const std::string & snapshots_path_,
size_t snapshots_to_keep_,
2022-07-23 14:27:44 +00:00
const KeeperContextPtr & keeper_context_,
2022-05-19 09:45:38 +00:00
bool compress_snapshots_zstd_ = true,
const std::string & superdigest_ = "",
2022-07-23 14:27:44 +00:00
size_t storage_tick_time_ = 500);
2021-03-01 13:33:34 +00:00
/// Restore storage from latest available snapshot
2021-10-18 15:27:51 +00:00
SnapshotDeserializationResult restoreFromLatestSnapshot();
2021-03-01 13:33:34 +00:00
/// Compress snapshot and serialize it to buffer
2021-09-27 14:21:10 +00:00
nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const;
/// Serialize already compressed snapshot to disk (return path)
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
2022-02-14 11:43:08 +00:00
/// Serialize snapshot directly to disk
std::pair<std::string, std::error_code> serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot);
2021-10-18 15:27:51 +00:00
SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
2021-03-02 14:30:56 +00:00
/// Deserialize snapshot with log index up_to_log_idx from disk into compressed nuraft buffer.
2021-04-08 14:17:57 +00:00
nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const;
/// Deserialize latest snapshot from disk into compressed nuraft buffer.
nuraft::ptr<nuraft::buffer> deserializeLatestSnapshotBufferFromDisk();
/// Remove snapshot with this log_index
2021-04-08 14:17:57 +00:00
void removeSnapshot(uint64_t log_idx);
/// Total amount of snapshots
2022-05-19 09:45:38 +00:00
size_t totalSnapshots() const { return existing_snapshots.size(); }
/// The most fresh snapshot log index we have
size_t getLatestSnapshotIndex() const
{
if (!existing_snapshots.empty())
return existing_snapshots.rbegin()->first;
return 0;
}
2021-03-01 13:33:34 +00:00
2022-02-14 11:43:08 +00:00
std::string getLatestSnapshotPath() const
{
if (!existing_snapshots.empty())
2022-02-15 12:25:19 +00:00
{
2022-02-17 02:34:22 +00:00
const auto & path = existing_snapshots.at(getLatestSnapshotIndex());
2022-02-15 12:25:19 +00:00
std::error_code ec;
if (std::filesystem::exists(path, ec))
return path;
}
2022-02-14 11:43:08 +00:00
return "";
}
2021-03-01 13:33:34 +00:00
private:
2021-03-01 14:54:08 +00:00
void removeOutdatedSnapshotsIfNeeded();
/// Checks first 4 buffer bytes to became sure that snapshot compressed with
/// ZSTD codec.
2021-09-27 14:21:10 +00:00
static bool isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer);
2021-03-01 13:33:34 +00:00
const std::string snapshots_path;
/// How many snapshots to keep before remove
2021-03-01 14:54:08 +00:00
const size_t snapshots_to_keep;
/// All existing snapshots in our path (log_index -> path)
2021-04-08 14:17:57 +00:00
std::map<uint64_t, std::string> existing_snapshots;
/// Compress snapshots in common ZSTD format instead of custom ClickHouse block LZ4 format
const bool compress_snapshots_zstd;
/// Superdigest for deserialization of storage
2021-05-23 17:54:42 +00:00
const std::string superdigest;
/// Storage sessions timeout check interval (also for deserializatopn)
2021-03-19 08:08:43 +00:00
size_t storage_tick_time;
2022-07-23 14:27:44 +00:00
KeeperContextPtr keeper_context;
2021-02-01 14:14:59 +00:00
};
/// Keeper create snapshots in background thread. KeeperStateMachine just create
/// in-memory snapshot from storage and push task for it serialization into
/// special tasks queue. Background thread check this queue and after snapshot
/// successfully serialized notify state machine.
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
}