Merge pull request #56626 from hanfei1991/hanfei/keeperrocks

support RocksDB as alternative keeper storage
This commit is contained in:
Han Fei 2024-07-19 21:56:02 +00:00 committed by GitHub
commit 29f1120f05
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
21 changed files with 2631 additions and 1282 deletions

View File

@ -45,16 +45,17 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv)
keeper_context->setDigestEnabled(true);
keeper_context->setSnapshotDisk(std::make_shared<DiskLocal>("Keeper-snapshots", options["output-dir"].as<std::string>()));
DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false);
/// TODO(hanfei): support rocksdb here
DB::KeeperMemoryStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false);
DB::deserializeKeeperStorageFromSnapshotsDir(storage, options["zookeeper-snapshots-dir"].as<std::string>(), logger);
storage.initializeSystemNodes();
DB::deserializeLogsAndApplyToStorage(storage, options["zookeeper-logs-dir"].as<std::string>(), logger);
DB::SnapshotMetadataPtr snapshot_meta = std::make_shared<DB::SnapshotMetadata>(storage.getZXID(), 1, std::make_shared<nuraft::cluster_config>());
DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta);
DB::KeeperStorageSnapshot<DB::KeeperMemoryStorage> snapshot(&storage, snapshot_meta);
DB::KeeperSnapshotManager manager(1, keeper_context);
DB::KeeperSnapshotManager<DB::KeeperMemoryStorage> manager(1, keeper_context);
auto snp = manager.serializeSnapshotToBuffer(snapshot);
auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID());
std::cout << "Snapshot serialized to path:" << fs::path(file_info->disk->getPath()) / file_info->path << std::endl;

View File

@ -55,6 +55,7 @@ struct Settings;
M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests, set to 0 to disable", 0) \
M(UInt64, raft_limits_reconnect_limit, 50, "If connection to a peer is silent longer than this limit * (multiplied by heartbeat interval), we re-establish the connection.", 0) \
M(Bool, async_replication, false, "Enable async replication. All write and read guarantees are preserved while better performance is achieved. Settings is disabled by default to not break backwards compatibility.", 0) \
M(Bool, experimental_use_rocksdb, false, "Use rocksdb as backend storage", 0) \
M(UInt64, latest_logs_cache_size_threshold, 1 * 1024 * 1024 * 1024, "Maximum total size of in-memory cache of latest log entries.", 0) \
M(UInt64, commit_logs_cache_size_threshold, 500 * 1024 * 1024, "Maximum total size of in-memory cache of log entries needed next for commit.", 0) \
M(UInt64, disk_move_retries_wait_ms, 1000, "How long to wait between retries after a failure which happened while a file was being moved between disks.", 0) \

View File

@ -5,18 +5,27 @@
#include <Coordination/CoordinationSettings.h>
#include <Coordination/Defines.h>
#include <Disks/DiskLocal.h>
#include <Interpreters/Context.h>
#include <IO/S3/Credentials.h>
#include <IO/WriteHelpers.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/JSONConfiguration.h>
#include <Coordination/KeeperConstants.h>
#include <Server/CloudPlacementInfo.h>
#include <Coordination/KeeperFeatureFlags.h>
#include <Disks/DiskLocal.h>
#include <Disks/DiskSelector.h>
#include <IO/S3/Credentials.h>
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/logger_useful.h>
#include <boost/algorithm/string.hpp>
#include "config.h"
#if USE_ROCKSDB
#include <rocksdb/table.h>
#include <rocksdb/convenience.h>
#include <rocksdb/utilities/db_ttl.h>
#endif
namespace DB
{
@ -24,6 +33,8 @@ namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int ROCKSDB_ERROR;
}
@ -41,6 +52,95 @@ KeeperContext::KeeperContext(bool standalone_keeper_, CoordinationSettingsPtr co
system_nodes_with_data[keeper_api_version_path] = toString(static_cast<uint8_t>(KeeperApiVersion::WITH_MULTI_READ));
}
#if USE_ROCKSDB
using RocksDBOptions = std::unordered_map<std::string, std::string>;
static RocksDBOptions getOptionsFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & path)
{
RocksDBOptions options;
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(path, keys);
for (const auto & key : keys)
{
const String key_path = path + "." + key;
options[key] = config.getString(key_path);
}
return options;
}
static rocksdb::Options getRocksDBOptionsFromConfig(const Poco::Util::AbstractConfiguration & config)
{
rocksdb::Status status;
rocksdb::Options base;
base.create_if_missing = true;
base.compression = rocksdb::CompressionType::kZSTD;
base.statistics = rocksdb::CreateDBStatistics();
/// It is too verbose by default, and in fact we don't care about rocksdb logs at all.
base.info_log_level = rocksdb::ERROR_LEVEL;
rocksdb::Options merged = base;
rocksdb::BlockBasedTableOptions table_options;
if (config.has("keeper_server.rocksdb.options"))
{
auto config_options = getOptionsFromConfig(config, "keeper_server.rocksdb.options");
status = rocksdb::GetDBOptionsFromMap(merged, config_options, &merged);
if (!status.ok())
{
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.options' : {}",
status.ToString());
}
}
if (config.has("rocksdb.column_family_options"))
{
auto column_family_options = getOptionsFromConfig(config, "rocksdb.column_family_options");
status = rocksdb::GetColumnFamilyOptionsFromMap(merged, column_family_options, &merged);
if (!status.ok())
{
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.column_family_options' at: {}", status.ToString());
}
}
if (config.has("rocksdb.block_based_table_options"))
{
auto block_based_table_options = getOptionsFromConfig(config, "rocksdb.block_based_table_options");
status = rocksdb::GetBlockBasedTableOptionsFromMap(table_options, block_based_table_options, &table_options);
if (!status.ok())
{
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.block_based_table_options' at: {}", status.ToString());
}
}
merged.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options));
return merged;
}
#endif
KeeperContext::Storage KeeperContext::getRocksDBPathFromConfig(const Poco::Util::AbstractConfiguration & config) const
{
const auto create_local_disk = [](const auto & path)
{
if (fs::exists(path))
fs::remove_all(path);
fs::create_directories(path);
return std::make_shared<DiskLocal>("LocalRocksDBDisk", path);
};
if (config.has("keeper_server.rocksdb_path"))
return create_local_disk(config.getString("keeper_server.rocksdb_path"));
if (config.has("keeper_server.storage_path"))
return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")} / "rocksdb");
if (standalone_keeper)
return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "rocksdb");
else
return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/rocksdb");
}
void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_)
{
dispatcher = dispatcher_;
@ -59,6 +159,14 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config,
initializeFeatureFlags(config);
initializeDisks(config);
#if USE_ROCKSDB
if (config.getBool("keeper_server.coordination_settings.experimental_use_rocksdb", false))
{
rocksdb_options = std::make_shared<rocksdb::Options>(getRocksDBOptionsFromConfig(config));
digest_enabled = false; /// TODO: support digest
}
#endif
}
namespace
@ -94,6 +202,8 @@ void KeeperContext::initializeDisks(const Poco::Util::AbstractConfiguration & co
{
disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance(), diskValidator);
rocksdb_storage = getRocksDBPathFromConfig(config);
log_storage = getLogsPathFromConfig(config);
if (config.has("keeper_server.latest_log_storage_disk"))
@ -262,6 +372,37 @@ void KeeperContext::dumpConfiguration(WriteBufferFromOwnString & buf) const
}
}
void KeeperContext::setRocksDBDisk(DiskPtr disk)
{
rocksdb_storage = std::move(disk);
}
DiskPtr KeeperContext::getTemporaryRocksDBDisk() const
{
DiskPtr rocksdb_disk = getDisk(rocksdb_storage);
if (!rocksdb_disk)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "rocksdb storage is not initialized");
}
auto uuid_str = formatUUID(UUIDHelpers::generateV4());
String path_to_create = "rocks_" + std::string(uuid_str.data(), uuid_str.size());
rocksdb_disk->createDirectory(path_to_create);
return std::make_shared<DiskLocal>("LocalTmpRocksDBDisk", fullPath(rocksdb_disk, path_to_create));
}
void KeeperContext::setRocksDBOptions(std::shared_ptr<rocksdb::Options> rocksdb_options_)
{
if (rocksdb_options_ != nullptr)
rocksdb_options = rocksdb_options_;
else
{
#if USE_ROCKSDB
rocksdb_options = std::make_shared<rocksdb::Options>(getRocksDBOptionsFromConfig(Poco::Util::JSONConfiguration()));
#endif
}
}
KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const
{
const auto create_local_disk = [](const auto & path)

View File

@ -6,6 +6,11 @@
#include <cstdint>
#include <memory>
namespace rocksdb
{
struct Options;
}
namespace DB
{
@ -62,6 +67,12 @@ public:
constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; }
void setRocksDBDisk(DiskPtr disk);
DiskPtr getTemporaryRocksDBDisk() const;
void setRocksDBOptions(std::shared_ptr<rocksdb::Options> rocksdb_options_ = nullptr);
std::shared_ptr<rocksdb::Options> getRocksDBOptions() const { return rocksdb_options; }
UInt64 getKeeperMemorySoftLimit() const { return memory_soft_limit; }
void updateKeeperMemorySoftLimit(const Poco::Util::AbstractConfiguration & config);
@ -90,6 +101,7 @@ private:
void initializeFeatureFlags(const Poco::Util::AbstractConfiguration & config);
void initializeDisks(const Poco::Util::AbstractConfiguration & config);
Storage getRocksDBPathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
Storage getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
Storage getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
Storage getStatePathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
@ -111,12 +123,15 @@ private:
std::shared_ptr<DiskSelector> disk_selector;
Storage rocksdb_storage;
Storage log_storage;
Storage latest_log_storage;
Storage snapshot_storage;
Storage latest_snapshot_storage;
Storage state_file_storage;
std::shared_ptr<rocksdb::Options> rocksdb_options;
std::vector<std::string> old_log_disk_names;
std::vector<std::string> old_snapshot_disk_names;

View File

@ -117,13 +117,13 @@ void KeeperDispatcher::requestThread()
RaftAppendResult prev_result = nullptr;
/// Requests from previous iteration. We store them to be able
/// to send errors to the client.
KeeperStorage::RequestsForSessions prev_batch;
KeeperStorageBase::RequestsForSessions prev_batch;
const auto & shutdown_called = keeper_context->isShutdownCalled();
while (!shutdown_called)
{
KeeperStorage::RequestForSession request;
KeeperStorageBase::RequestForSession request;
auto coordination_settings = configuration_and_settings->coordination_settings;
uint64_t max_wait = coordination_settings->operation_timeout_ms.totalMilliseconds();
@ -153,7 +153,7 @@ void KeeperDispatcher::requestThread()
continue;
}
KeeperStorage::RequestsForSessions current_batch;
KeeperStorageBase::RequestsForSessions current_batch;
size_t current_batch_bytes_size = 0;
bool has_read_request = false;
@ -311,7 +311,7 @@ void KeeperDispatcher::responseThread()
const auto & shutdown_called = keeper_context->isShutdownCalled();
while (!shutdown_called)
{
KeeperStorage::ResponseForSession response_for_session;
KeeperStorageBase::ResponseForSession response_for_session;
uint64_t max_wait = configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds();
@ -402,7 +402,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ
return false;
}
KeeperStorage::RequestForSession request_info;
KeeperStorageBase::RequestForSession request_info;
request_info.request = request;
using namespace std::chrono;
request_info.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count();
@ -448,7 +448,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf
snapshots_queue,
keeper_context,
snapshot_s3,
[this](uint64_t /*log_idx*/, const KeeperStorage::RequestForSession & request_for_session)
[this](uint64_t /*log_idx*/, const KeeperStorageBase::RequestForSession & request_for_session)
{
{
/// check if we have queue of read requests depending on this request to be committed
@ -540,7 +540,7 @@ void KeeperDispatcher::shutdown()
update_configuration_thread.join();
}
KeeperStorage::RequestForSession request_for_session;
KeeperStorageBase::RequestForSession request_for_session;
/// Set session expired for all pending requests
while (requests_queue && requests_queue->tryPop(request_for_session))
@ -551,7 +551,7 @@ void KeeperDispatcher::shutdown()
setResponse(request_for_session.session_id, response);
}
KeeperStorage::RequestsForSessions close_requests;
KeeperStorageBase::RequestsForSessions close_requests;
{
/// Clear all registered sessions
std::lock_guard lock(session_to_response_callback_mutex);
@ -565,7 +565,7 @@ void KeeperDispatcher::shutdown()
auto request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close);
request->xid = Coordination::CLOSE_XID;
using namespace std::chrono;
KeeperStorage::RequestForSession request_info
KeeperStorageBase::RequestForSession request_info
{
.session_id = session,
.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count(),
@ -663,7 +663,7 @@ void KeeperDispatcher::sessionCleanerTask()
auto request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close);
request->xid = Coordination::CLOSE_XID;
using namespace std::chrono;
KeeperStorage::RequestForSession request_info
KeeperStorageBase::RequestForSession request_info
{
.session_id = dead_session,
.time = duration_cast<milliseconds>(system_clock::now().time_since_epoch()).count(),
@ -711,16 +711,16 @@ void KeeperDispatcher::finishSession(int64_t session_id)
}
}
void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error)
void KeeperDispatcher::addErrorResponses(const KeeperStorageBase::RequestsForSessions & requests_for_sessions, Coordination::Error error)
{
for (const auto & request_for_session : requests_for_sessions)
{
KeeperStorage::ResponsesForSessions responses;
KeeperStorageBase::ResponsesForSessions responses;
auto response = request_for_session.request->makeResponse();
response->xid = request_for_session.request->xid;
response->zxid = 0;
response->error = error;
if (!responses_queue.push(DB::KeeperStorage::ResponseForSession{request_for_session.session_id, response}))
if (!responses_queue.push(DB::KeeperStorageBase::ResponseForSession{request_for_session.session_id, response}))
throw Exception(ErrorCodes::SYSTEM_ERROR,
"Could not push error response xid {} zxid {} error message {} to responses queue",
response->xid,
@ -730,7 +730,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession
}
nuraft::ptr<nuraft::buffer> KeeperDispatcher::forceWaitAndProcessResult(
RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions, bool clear_requests_on_success)
RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions, bool clear_requests_on_success)
{
if (!result->has_result())
result->get();
@ -755,7 +755,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms)
{
/// New session id allocation is a special request, because we cannot process it in normal
/// way: get request -> put to raft -> set response for registered callback.
KeeperStorage::RequestForSession request_info;
KeeperStorageBase::RequestForSession request_info;
std::shared_ptr<Coordination::ZooKeeperSessionIDRequest> request = std::make_shared<Coordination::ZooKeeperSessionIDRequest>();
/// Internal session id. It's a temporary number which is unique for each client on this server
/// but can be same on different servers.

View File

@ -26,7 +26,7 @@ using ZooKeeperResponseCallback = std::function<void(const Coordination::ZooKeep
class KeeperDispatcher
{
private:
using RequestsQueue = ConcurrentBoundedQueue<KeeperStorage::RequestForSession>;
using RequestsQueue = ConcurrentBoundedQueue<KeeperStorageBase::RequestForSession>;
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
using ClusterUpdateQueue = ConcurrentBoundedQueue<ClusterUpdateAction>;
@ -95,18 +95,18 @@ private:
/// Add error responses for requests to responses queue.
/// Clears requests.
void addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error);
void addErrorResponses(const KeeperStorageBase::RequestsForSessions & requests_for_sessions, Coordination::Error error);
/// Forcefully wait for result and sets errors if something when wrong.
/// Clears both arguments
nuraft::ptr<nuraft::buffer> forceWaitAndProcessResult(
RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions, bool clear_requests_on_success);
RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions, bool clear_requests_on_success);
public:
std::mutex read_request_queue_mutex;
/// queue of read requests that can be processed after a request with specific session ID and XID is committed
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, KeeperStorage::RequestsForSessions>> read_request_queue;
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, KeeperStorageBase::RequestsForSessions>> read_request_queue;
/// Just allocate some objects, real initialization is done by `intialize method`
KeeperDispatcher();
@ -192,7 +192,7 @@ public:
Keeper4LWInfo getKeeper4LWInfo() const;
const KeeperStateMachine & getStateMachine() const
const IKeeperStateMachine & getStateMachine() const
{
return *server->getKeeperStateMachine();
}

View File

@ -123,7 +123,7 @@ KeeperServer::KeeperServer(
SnapshotsQueue & snapshots_queue_,
KeeperContextPtr keeper_context_,
KeeperSnapshotManagerS3 & snapshot_manager_s3,
KeeperStateMachine::CommitCallback commit_callback)
IKeeperStateMachine::CommitCallback commit_callback)
: server_id(configuration_and_settings_->server_id)
, log(getLogger("KeeperServer"))
, is_recovering(config.getBool("keeper_server.force_recovery", false))
@ -134,13 +134,28 @@ KeeperServer::KeeperServer(
if (keeper_context->getCoordinationSettings()->quorum_reads)
LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower.");
state_machine = nuraft::cs_new<KeeperStateMachine>(
responses_queue_,
snapshots_queue_,
keeper_context,
config.getBool("keeper_server.upload_snapshot_on_exit", false) ? &snapshot_manager_s3 : nullptr,
commit_callback,
checkAndGetSuperdigest(configuration_and_settings_->super_digest));
#if USE_ROCKSDB
const auto & coordination_settings = keeper_context->getCoordinationSettings();
if (coordination_settings->experimental_use_rocksdb)
{
state_machine = nuraft::cs_new<KeeperStateMachine<KeeperRocksStorage>>(
responses_queue_,
snapshots_queue_,
keeper_context,
config.getBool("keeper_server.upload_snapshot_on_exit", false) ? &snapshot_manager_s3 : nullptr,
commit_callback,
checkAndGetSuperdigest(configuration_and_settings_->super_digest));
LOG_WARNING(log, "Use RocksDB as Keeper backend storage.");
}
else
#endif
state_machine = nuraft::cs_new<KeeperStateMachine<KeeperMemoryStorage>>(
responses_queue_,
snapshots_queue_,
keeper_context,
config.getBool("keeper_server.upload_snapshot_on_exit", false) ? &snapshot_manager_s3 : nullptr,
commit_callback,
checkAndGetSuperdigest(configuration_and_settings_->super_digest));
state_manager = nuraft::cs_new<KeeperStateManager>(
server_id,
@ -522,7 +537,7 @@ namespace
{
// Serialize the request for the log entry
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestForSession & request_for_session)
nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorageBase::RequestForSession & request_for_session)
{
DB::WriteBufferFromNuraftBuffer write_buf;
DB::writeIntBinary(request_for_session.session_id, write_buf);
@ -530,7 +545,7 @@ nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestFor
DB::writeIntBinary(request_for_session.time, write_buf);
/// we fill with dummy values to eliminate unnecessary copy later on when we will write correct values
DB::writeIntBinary(static_cast<int64_t>(0), write_buf); /// zxid
DB::writeIntBinary(KeeperStorage::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag
DB::writeIntBinary(KeeperStorageBase::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag
DB::writeIntBinary(static_cast<uint64_t>(0), write_buf); /// digest value
/// if new fields are added, update KeeperStateMachine::ZooKeeperLogSerializationVersion along with parseRequest function and PreAppendLog callback handler
return write_buf.getBuffer();
@ -538,7 +553,7 @@ nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(const KeeperStorage::RequestFor
}
void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & request_for_session)
void KeeperServer::putLocalReadRequest(const KeeperStorageBase::RequestForSession & request_for_session)
{
if (!request_for_session.request->isReadRequest())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot process non-read request locally");
@ -546,7 +561,7 @@ void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession &
state_machine->processReadRequest(request_for_session);
}
RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions)
RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorageBase::RequestsForSessions & requests_for_sessions)
{
std::vector<nuraft::ptr<nuraft::buffer>> entries;
entries.reserve(requests_for_sessions.size());
@ -789,7 +804,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
auto entry_buf = entry->get_buf_ptr();
KeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version;
IKeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version;
auto request_for_session = state_machine->parseRequest(*entry_buf, /*final=*/false, &serialization_version);
request_for_session->zxid = next_zxid;
if (!state_machine->preprocess(*request_for_session))
@ -799,10 +814,10 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
/// older versions of Keeper can send logs that are missing some fields
size_t bytes_missing = 0;
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
bytes_missing += sizeof(request_for_session->time);
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_ZXID_DIGEST)
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_ZXID_DIGEST)
bytes_missing += sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value);
if (bytes_missing != 0)
@ -816,19 +831,19 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
size_t write_buffer_header_size
= sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value);
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
write_buffer_header_size += sizeof(request_for_session->time);
auto * buffer_start = reinterpret_cast<BufferBase::Position>(entry_buf->data_begin() + entry_buf->size() - write_buffer_header_size);
WriteBufferFromPointer write_buf(buffer_start, write_buffer_header_size);
if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME)
writeIntBinary(request_for_session->time, write_buf);
writeIntBinary(request_for_session->zxid, write_buf);
writeIntBinary(request_for_session->digest->version, write_buf);
if (request_for_session->digest->version != KeeperStorage::NO_DIGEST)
if (request_for_session->digest->version != KeeperStorageBase::NO_DIGEST)
writeIntBinary(request_for_session->digest->value, write_buf);
write_buf.finalize();

View File

@ -24,7 +24,7 @@ class KeeperServer
private:
const int server_id;
nuraft::ptr<KeeperStateMachine> state_machine;
nuraft::ptr<IKeeperStateMachine> state_machine;
nuraft::ptr<KeeperStateManager> state_manager;
@ -79,26 +79,26 @@ public:
SnapshotsQueue & snapshots_queue_,
KeeperContextPtr keeper_context_,
KeeperSnapshotManagerS3 & snapshot_manager_s3,
KeeperStateMachine::CommitCallback commit_callback);
IKeeperStateMachine::CommitCallback commit_callback);
/// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings.
void startup(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6 = true);
/// Put local read request and execute in state machine directly and response into
/// responses queue
void putLocalReadRequest(const KeeperStorage::RequestForSession & request);
void putLocalReadRequest(const KeeperStorageBase::RequestForSession & request);
bool isRecovering() const { return is_recovering; }
bool reconfigEnabled() const { return enable_reconfiguration; }
/// Put batch of requests into Raft and get result of put. Responses will be set separately into
/// responses_queue.
RaftAppendResult putRequestBatch(const KeeperStorage::RequestsForSessions & requests);
RaftAppendResult putRequestBatch(const KeeperStorageBase::RequestsForSessions & requests);
/// Return set of the non-active sessions
std::vector<int64_t> getDeadSessions();
nuraft::ptr<KeeperStateMachine> getKeeperStateMachine() const { return state_machine; }
nuraft::ptr<IKeeperStateMachine> getKeeperStateMachine() const { return state_machine; }
void forceRecovery();

View File

@ -66,7 +66,8 @@ namespace
return base;
}
void writeNode(const KeeperStorage::Node & node, SnapshotVersion version, WriteBuffer & out)
template<typename Node>
void writeNode(const Node & node, SnapshotVersion version, WriteBuffer & out)
{
writeBinary(node.getData(), out);
@ -86,7 +87,7 @@ namespace
writeBinary(node.aversion, out);
writeBinary(node.ephemeralOwner(), out);
if (version < SnapshotVersion::V6)
writeBinary(static_cast<int32_t>(node.data_size), out);
writeBinary(static_cast<int32_t>(node.getData().size()), out);
writeBinary(node.numChildren(), out);
writeBinary(node.pzxid, out);
@ -96,7 +97,8 @@ namespace
writeBinary(node.sizeInBytes(), out);
}
void readNode(KeeperStorage::Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map)
template<typename Node>
void readNode(Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map)
{
readVarUInt(node.data_size, in);
if (node.data_size != 0)
@ -195,7 +197,8 @@ namespace
}
}
void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context)
template<typename Storage>
void KeeperStorageSnapshot<Storage>::serialize(const KeeperStorageSnapshot<Storage> & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context)
{
writeBinary(static_cast<uint8_t>(snapshot.version), out);
serializeSnapshotMetadata(snapshot.snapshot_meta, out);
@ -205,11 +208,11 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
writeBinary(snapshot.zxid, out);
if (keeper_context->digestEnabled())
{
writeBinary(static_cast<uint8_t>(KeeperStorage::CURRENT_DIGEST_VERSION), out);
writeBinary(static_cast<uint8_t>(Storage::CURRENT_DIGEST_VERSION), out);
writeBinary(snapshot.nodes_digest, out);
}
else
writeBinary(static_cast<uint8_t>(KeeperStorage::NO_DIGEST), out);
writeBinary(static_cast<uint8_t>(Storage::NO_DIGEST), out);
}
writeBinary(snapshot.session_id, out);
@ -255,7 +258,6 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
/// slightly bigger than required.
if (node.mzxid > snapshot.zxid)
break;
writeBinary(path, out);
writeNode(node, snapshot.version, out);
@ -282,7 +284,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
writeBinary(session_id, out);
writeBinary(timeout, out);
KeeperStorage::AuthIDs ids;
KeeperStorageBase::AuthIDs ids;
if (snapshot.session_and_auth.contains(session_id))
ids = snapshot.session_and_auth.at(session_id);
@ -303,7 +305,8 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
}
}
void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context)
template<typename Storage>
void KeeperStorageSnapshot<Storage>::deserialize(SnapshotDeserializationResult<Storage> & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context)
{
uint8_t version;
readBinary(version, in);
@ -312,7 +315,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported snapshot version {}", version);
deserialization_result.snapshot_meta = deserializeSnapshotMetadata(in);
KeeperStorage & storage = *deserialization_result.storage;
Storage & storage = *deserialization_result.storage;
bool recalculate_digest = keeper_context->digestEnabled();
if (version >= SnapshotVersion::V5)
@ -320,11 +323,11 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
readBinary(storage.zxid, in);
uint8_t digest_version;
readBinary(digest_version, in);
if (digest_version != KeeperStorage::DigestVersion::NO_DIGEST)
if (digest_version != Storage::DigestVersion::NO_DIGEST)
{
uint64_t nodes_digest;
readBinary(nodes_digest, in);
if (digest_version == KeeperStorage::CURRENT_DIGEST_VERSION)
if (digest_version == Storage::CURRENT_DIGEST_VERSION)
{
storage.nodes_digest = nodes_digest;
recalculate_digest = false;
@ -374,8 +377,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
size_t snapshot_container_size;
readBinary(snapshot_container_size, in);
storage.container.reserve(snapshot_container_size);
if constexpr (!use_rocksdb)
storage.container.reserve(snapshot_container_size);
if (recalculate_digest)
storage.nodes_digest = 0;
@ -389,7 +392,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
in.readStrict(path_data.get(), path_size);
std::string_view path{path_data.get(), path_size};
KeeperStorage::Node node{};
typename Storage::Node node{};
readNode(node, in, current_version, storage.acl_map);
using enum Coordination::PathMatchResult;
@ -421,7 +424,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT)
{
LOG_ERROR(getLogger("KeeperSnapshotManager"), "{}. Ignoring it", get_error_msg());
node = KeeperStorage::Node{};
node = typename Storage::Node{};
}
else
throw Exception(
@ -433,8 +436,9 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
}
auto ephemeral_owner = node.ephemeralOwner();
if (!node.isEphemeral() && node.numChildren() > 0)
node.getChildren().reserve(node.numChildren());
if constexpr (!use_rocksdb)
if (!node.isEphemeral() && node.numChildren() > 0)
node.getChildren().reserve(node.numChildren());
if (ephemeral_owner != 0)
storage.ephemerals[node.ephemeralOwner()].insert(std::string{path});
@ -447,36 +451,38 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
LOG_TRACE(getLogger("KeeperSnapshotManager"), "Building structure for children nodes");
for (const auto & itr : storage.container)
if constexpr (!use_rocksdb)
{
if (itr.key != "/")
for (const auto & itr : storage.container)
{
auto parent_path = parentNodePath(itr.key);
storage.container.updateValue(
parent_path, [path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path)); });
}
}
for (const auto & itr : storage.container)
{
if (itr.key != "/")
{
if (itr.value.numChildren() != static_cast<int32_t>(itr.value.getChildren().size()))
if (itr.key != "/")
{
auto parent_path = parentNodePath(itr.key);
storage.container.updateValue(
parent_path, [path = itr.key](typename Storage::Node & value) { value.addChild(getBaseNodeName(path)); });
}
}
for (const auto & itr : storage.container)
{
if (itr.key != "/")
{
if (itr.value.numChildren() != static_cast<int32_t>(itr.value.getChildren().size()))
{
#ifdef NDEBUG
/// TODO (alesapin) remove this, it should be always CORRUPTED_DATA.
LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
/// TODO (alesapin) remove this, it should be always CORRUPTED_DATA.
LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
#else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}",
itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}"
" is different from actual children size {} for node {}",
itr.value.numChildren(), itr.value.getChildren().size(), itr.key);
#endif
}
}
}
}
size_t active_sessions_size;
readBinary(active_sessions_size, in);
@ -493,14 +499,14 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
size_t session_auths_size;
readBinary(session_auths_size, in);
KeeperStorage::AuthIDs ids;
typename Storage::AuthIDs ids;
size_t session_auth_counter = 0;
while (session_auth_counter < session_auths_size)
{
String scheme, id;
readBinary(scheme, in);
readBinary(id, in);
ids.emplace_back(KeeperStorage::AuthID{scheme, id});
ids.emplace_back(typename Storage::AuthID{scheme, id});
session_auth_counter++;
}
@ -523,7 +529,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
}
}
KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_)
template<typename Storage>
KeeperStorageSnapshot<Storage>::KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_)
: storage(storage_)
, snapshot_meta(std::make_shared<SnapshotMetadata>(up_to_log_idx_, 0, std::make_shared<nuraft::cluster_config>()))
, session_id(storage->session_id_counter)
@ -540,8 +547,9 @@ KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t
session_and_auth = storage->session_and_auth;
}
KeeperStorageSnapshot::KeeperStorageSnapshot(
KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_)
template<typename Storage>
KeeperStorageSnapshot<Storage>::KeeperStorageSnapshot(
Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_)
: storage(storage_)
, snapshot_meta(snapshot_meta_)
, session_id(storage->session_id_counter)
@ -558,12 +566,14 @@ KeeperStorageSnapshot::KeeperStorageSnapshot(
session_and_auth = storage->session_and_auth;
}
KeeperStorageSnapshot::~KeeperStorageSnapshot()
template<typename Storage>
KeeperStorageSnapshot<Storage>::~KeeperStorageSnapshot()
{
storage->disableSnapshotMode();
}
KeeperSnapshotManager::KeeperSnapshotManager(
template<typename Storage>
KeeperSnapshotManager<Storage>::KeeperSnapshotManager(
size_t snapshots_to_keep_,
const KeeperContextPtr & keeper_context_,
bool compress_snapshots_zstd_,
@ -651,7 +661,8 @@ KeeperSnapshotManager::KeeperSnapshotManager(
moveSnapshotsIfNeeded();
}
SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx)
template<typename Storage>
SnapshotFileInfoPtr KeeperSnapshotManager<Storage>::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx)
{
ReadBufferFromNuraftBuffer reader(buffer);
@ -680,7 +691,8 @@ SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft:
return snapshot_file_info;
}
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk()
template<typename Storage>
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager<Storage>::deserializeLatestSnapshotBufferFromDisk()
{
while (!existing_snapshots.empty())
{
@ -701,7 +713,8 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBuff
return nullptr;
}
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const
template<typename Storage>
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager<Storage>::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const
{
const auto & [snapshot_path, snapshot_disk, size] = *existing_snapshots.at(up_to_log_idx);
WriteBufferFromNuraftBuffer writer;
@ -710,7 +723,8 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFrom
return writer.getBuffer();
}
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const
template<typename Storage>
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager<Storage>::serializeSnapshotToBuffer(const KeeperStorageSnapshot<Storage> & snapshot) const
{
std::unique_ptr<WriteBufferFromNuraftBuffer> writer = std::make_unique<WriteBufferFromNuraftBuffer>();
auto * buffer_raw_ptr = writer.get();
@ -720,13 +734,13 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::serializeSnapshotToBuffer(con
else
compressed_writer = std::make_unique<CompressedWriteBuffer>(*writer);
KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context);
KeeperStorageSnapshot<Storage>::serialize(snapshot, *compressed_writer, keeper_context);
compressed_writer->finalize();
return buffer_raw_ptr->getBuffer();
}
bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
template<typename Storage>
bool KeeperSnapshotManager<Storage>::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
{
static constexpr unsigned char ZSTD_COMPRESSED_MAGIC[4] = {0x28, 0xB5, 0x2F, 0xFD};
@ -737,7 +751,8 @@ bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
return memcmp(magic_from_buffer, ZSTD_COMPRESSED_MAGIC, 4) == 0;
}
SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const
template<typename Storage>
SnapshotDeserializationResult<Storage> KeeperSnapshotManager<Storage>::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const
{
bool is_zstd_compressed = isZstdCompressed(buffer);
@ -749,14 +764,15 @@ SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuff
else
compressed_reader = std::make_unique<CompressedReadBuffer>(*reader);
SnapshotDeserializationResult result;
result.storage = std::make_unique<KeeperStorage>(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false);
KeeperStorageSnapshot::deserialize(result, *compressed_reader, keeper_context);
SnapshotDeserializationResult<Storage> result;
result.storage = std::make_unique<Storage>(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false);
KeeperStorageSnapshot<Storage>::deserialize(result, *compressed_reader, keeper_context);
result.storage->initializeSystemNodes();
return result;
}
SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot()
template<typename Storage>
SnapshotDeserializationResult<Storage> KeeperSnapshotManager<Storage>::restoreFromLatestSnapshot()
{
if (existing_snapshots.empty())
return {};
@ -767,23 +783,27 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot()
return deserializeSnapshotFromBuffer(buffer);
}
DiskPtr KeeperSnapshotManager::getDisk() const
template<typename Storage>
DiskPtr KeeperSnapshotManager<Storage>::getDisk() const
{
return keeper_context->getSnapshotDisk();
}
DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const
template<typename Storage>
DiskPtr KeeperSnapshotManager<Storage>::getLatestSnapshotDisk() const
{
return keeper_context->getLatestSnapshotDisk();
}
void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded()
template<typename Storage>
void KeeperSnapshotManager<Storage>::removeOutdatedSnapshotsIfNeeded()
{
while (existing_snapshots.size() > snapshots_to_keep)
removeSnapshot(existing_snapshots.begin()->first);
}
void KeeperSnapshotManager::moveSnapshotsIfNeeded()
template<typename Storage>
void KeeperSnapshotManager<Storage>::moveSnapshotsIfNeeded()
{
/// move snapshots to correct disks
@ -813,7 +833,8 @@ void KeeperSnapshotManager::moveSnapshotsIfNeeded()
}
void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx)
template<typename Storage>
void KeeperSnapshotManager<Storage>::removeSnapshot(uint64_t log_idx)
{
auto itr = existing_snapshots.find(log_idx);
if (itr == existing_snapshots.end())
@ -823,7 +844,8 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx)
existing_snapshots.erase(itr);
}
SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot)
template<typename Storage>
SnapshotFileInfoPtr KeeperSnapshotManager<Storage>::serializeSnapshotToDisk(const KeeperStorageSnapshot<Storage> & snapshot)
{
auto up_to_log_idx = snapshot.snapshot_meta->get_last_log_idx();
auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd);
@ -842,7 +864,7 @@ SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperS
else
compressed_writer = std::make_unique<CompressedWriteBuffer>(*writer);
KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context);
KeeperStorageSnapshot<Storage>::serialize(snapshot, *compressed_writer, keeper_context);
compressed_writer->finalize();
compressed_writer->sync();
@ -864,14 +886,16 @@ SnapshotFileInfoPtr KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperS
return snapshot_file_info;
}
size_t KeeperSnapshotManager::getLatestSnapshotIndex() const
template<typename Storage>
size_t KeeperSnapshotManager<Storage>::getLatestSnapshotIndex() const
{
if (!existing_snapshots.empty())
return existing_snapshots.rbegin()->first;
return 0;
}
SnapshotFileInfoPtr KeeperSnapshotManager::getLatestSnapshotInfo() const
template<typename Storage>
SnapshotFileInfoPtr KeeperSnapshotManager<Storage>::getLatestSnapshotInfo() const
{
if (!existing_snapshots.empty())
{
@ -890,4 +914,10 @@ SnapshotFileInfoPtr KeeperSnapshotManager::getLatestSnapshotInfo() const
return nullptr;
}
template struct KeeperStorageSnapshot<KeeperMemoryStorage>;
template class KeeperSnapshotManager<KeeperMemoryStorage>;
#if USE_ROCKSDB
template struct KeeperStorageSnapshot<KeeperRocksStorage>;
template class KeeperSnapshotManager<KeeperRocksStorage>;
#endif
}

View File

@ -34,10 +34,11 @@ enum SnapshotVersion : uint8_t
static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V6;
/// What is stored in binary snapshot
template<typename Storage>
struct SnapshotDeserializationResult
{
/// Storage
KeeperStoragePtr storage;
std::unique_ptr<Storage> storage;
/// Snapshot metadata (up_to_log_idx and so on)
SnapshotMetadataPtr snapshot_meta;
/// Cluster config
@ -52,21 +53,31 @@ struct SnapshotDeserializationResult
///
/// This representation of snapshot have to be serialized into NuRaft
/// buffer and send over network or saved to file.
template<typename Storage>
struct KeeperStorageSnapshot
{
#if USE_ROCKSDB
static constexpr bool use_rocksdb = std::is_same_v<Storage, KeeperRocksStorage>;
#else
static constexpr bool use_rocksdb = false;
#endif
public:
KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr);
KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr);
KeeperStorageSnapshot(
KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr);
Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr);
KeeperStorageSnapshot(const KeeperStorageSnapshot<Storage>&) = delete;
KeeperStorageSnapshot(KeeperStorageSnapshot<Storage>&&) = default;
~KeeperStorageSnapshot();
static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context);
static void serialize(const KeeperStorageSnapshot<Storage> & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context);
static void deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context);
static void deserialize(SnapshotDeserializationResult<Storage> & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context);
KeeperStorage * storage;
Storage * storage;
SnapshotVersion version = CURRENT_SNAPSHOT_VERSION;
/// Snapshot metadata
@ -77,11 +88,11 @@ public:
/// so we have for loop for (i = 0; i < snapshot_container_size; ++i) { doSmth(begin + i); }
size_t snapshot_container_size;
/// Iterator to the start of the storage
KeeperStorage::Container::const_iterator begin;
Storage::Container::const_iterator begin;
/// Active sessions and their timeouts
SessionAndTimeout session_and_timeout;
/// Sessions credentials
KeeperStorage::SessionAndAuth session_and_auth;
Storage::SessionAndAuth session_and_auth;
/// ACLs cache for better performance. Without we cannot deserialize storage.
std::unordered_map<uint64_t, Coordination::ACLs> acl_map;
/// Cluster config from snapshot, can be empty
@ -105,14 +116,16 @@ struct SnapshotFileInfo
};
using SnapshotFileInfoPtr = std::shared_ptr<SnapshotFileInfo>;
using KeeperStorageSnapshotPtr = std::shared_ptr<KeeperStorageSnapshot>;
using CreateSnapshotCallback = std::function<std::shared_ptr<SnapshotFileInfo>(KeeperStorageSnapshotPtr &&, bool)>;
using SnapshotMetaAndStorage = std::pair<SnapshotMetadataPtr, KeeperStoragePtr>;
#if USE_ROCKSDB
using KeeperStorageSnapshotPtr = std::variant<std::shared_ptr<KeeperStorageSnapshot<KeeperMemoryStorage>>, std::shared_ptr<KeeperStorageSnapshot<KeeperRocksStorage>>>;
#else
using KeeperStorageSnapshotPtr = std::variant<std::shared_ptr<KeeperStorageSnapshot<KeeperMemoryStorage>>>;
#endif
using CreateSnapshotCallback = std::function<SnapshotFileInfoPtr(KeeperStorageSnapshotPtr &&, bool)>;
/// Class responsible for snapshots serialization and deserialization. Each snapshot
/// has it's path on disk and log index.
template<typename Storage>
class KeeperSnapshotManager
{
public:
@ -124,18 +137,18 @@ public:
size_t storage_tick_time_ = 500);
/// Restore storage from latest available snapshot
SnapshotDeserializationResult restoreFromLatestSnapshot();
SnapshotDeserializationResult<Storage> restoreFromLatestSnapshot();
/// Compress snapshot and serialize it to buffer
nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const;
nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot<Storage> & snapshot) const;
/// Serialize already compressed snapshot to disk (return path)
SnapshotFileInfoPtr serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx);
/// Serialize snapshot directly to disk
SnapshotFileInfoPtr serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot);
SnapshotFileInfoPtr serializeSnapshotToDisk(const KeeperStorageSnapshot<Storage> & snapshot);
SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
SnapshotDeserializationResult<Storage> deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
/// Deserialize snapshot with log index up_to_log_idx from disk into compressed nuraft buffer.
nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const;

View File

@ -44,7 +44,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
KeeperStateMachine::KeeperStateMachine(
IKeeperStateMachine::IKeeperStateMachine(
ResponsesQueue & responses_queue_,
SnapshotsQueue & snapshots_queue_,
const KeeperContextPtr & keeper_context_,
@ -52,12 +52,6 @@ KeeperStateMachine::KeeperStateMachine(
CommitCallback commit_callback_,
const std::string & superdigest_)
: commit_callback(commit_callback_)
, snapshot_manager(
keeper_context_->getCoordinationSettings()->snapshots_to_keep,
keeper_context_,
keeper_context_->getCoordinationSettings()->compress_snapshots_with_zstd_format,
superdigest_,
keeper_context_->getCoordinationSettings()->dead_session_check_period_ms.totalMilliseconds())
, responses_queue(responses_queue_)
, snapshots_queue(snapshots_queue_)
, min_request_size_to_cache(keeper_context_->getCoordinationSettings()->min_request_size_for_cache)
@ -68,6 +62,32 @@ KeeperStateMachine::KeeperStateMachine(
{
}
template<typename Storage>
KeeperStateMachine<Storage>::KeeperStateMachine(
ResponsesQueue & responses_queue_,
SnapshotsQueue & snapshots_queue_,
// const CoordinationSettingsPtr & coordination_settings_,
const KeeperContextPtr & keeper_context_,
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
IKeeperStateMachine::CommitCallback commit_callback_,
const std::string & superdigest_)
: IKeeperStateMachine(
responses_queue_,
snapshots_queue_,
/// coordination_settings_,
keeper_context_,
snapshot_manager_s3_,
commit_callback_,
superdigest_),
snapshot_manager(
keeper_context_->getCoordinationSettings()->snapshots_to_keep,
keeper_context_,
keeper_context_->getCoordinationSettings()->compress_snapshots_with_zstd_format,
superdigest_,
keeper_context_->getCoordinationSettings()->dead_session_check_period_ms.totalMilliseconds())
{
}
namespace
{
@ -78,7 +98,8 @@ bool isLocalDisk(const IDisk & disk)
}
void KeeperStateMachine::init()
template<typename Storage>
void KeeperStateMachine<Storage>::init()
{
/// Do everything without mutexes, no other threads exist.
LOG_DEBUG(log, "Totally have {} snapshots", snapshot_manager.totalSnapshots());
@ -123,7 +144,7 @@ void KeeperStateMachine::init()
LOG_DEBUG(log, "No existing snapshots, last committed log index {}", last_committed_idx);
if (!storage)
storage = std::make_unique<KeeperStorage>(
storage = std::make_unique<Storage>(
keeper_context->getCoordinationSettings()->dead_session_check_period_ms.totalMilliseconds(), superdigest, keeper_context);
}
@ -131,13 +152,13 @@ namespace
{
void assertDigest(
const KeeperStorage::Digest & expected,
const KeeperStorage::Digest & actual,
const KeeperStorageBase::Digest & expected,
const KeeperStorageBase::Digest & actual,
const Coordination::ZooKeeperRequest & request,
uint64_t log_idx,
bool committing)
{
if (!KeeperStorage::checkDigest(expected, actual))
if (!KeeperStorageBase::checkDigest(expected, actual))
{
LOG_FATAL(
getLogger("KeeperStateMachine"),
@ -170,7 +191,8 @@ struct TSA_SCOPED_LOCKABLE LockGuardWithStats final
}
nuraft::ptr<nuraft::buffer> KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data)
template<typename Storage>
nuraft::ptr<nuraft::buffer> KeeperStateMachine<Storage>::pre_commit(uint64_t log_idx, nuraft::buffer & data)
{
auto result = nuraft::buffer::alloc(sizeof(log_idx));
nuraft::buffer_serializer ss(result);
@ -191,10 +213,10 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::pre_commit(uint64_t log_idx, nur
return result;
}
std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version)
std::shared_ptr<KeeperStorageBase::RequestForSession> IKeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version)
{
ReadBufferFromNuraftBuffer buffer(data);
auto request_for_session = std::make_shared<KeeperStorage::RequestForSession>();
auto request_for_session = std::make_shared<KeeperStorageBase::RequestForSession>();
readIntBinary(request_for_session->session_id, buffer);
int32_t length;
@ -267,7 +289,7 @@ std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseReque
request_for_session->digest.emplace();
readIntBinary(request_for_session->digest->version, buffer);
if (request_for_session->digest->version != KeeperStorage::DigestVersion::NO_DIGEST || !buffer.eof())
if (request_for_session->digest->version != KeeperStorageBase::DigestVersion::NO_DIGEST || !buffer.eof())
readIntBinary(request_for_session->digest->value, buffer);
}
@ -283,7 +305,8 @@ std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseReque
return request_for_session;
}
bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session)
template<typename Storage>
bool KeeperStateMachine<Storage>::preprocess(const KeeperStorageBase::RequestForSession & request_for_session)
{
const auto op_num = request_for_session.request->getOpNum();
if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig)
@ -317,10 +340,11 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req
return true;
}
void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session)
template<typename Storage>
void KeeperStateMachine<Storage>::reconfigure(const KeeperStorageBase::RequestForSession& request_for_session)
{
LockGuardWithStats lock(storage_and_responses_lock);
KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session);
KeeperStorageBase::ResponseForSession response = processReconfiguration(request_for_session);
if (!responses_queue.push(response))
{
ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed);
@ -330,8 +354,9 @@ void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& req
}
}
KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
const KeeperStorage::RequestForSession & request_for_session)
template<typename Storage>
KeeperStorageBase::ResponseForSession KeeperStateMachine<Storage>::processReconfiguration(
const KeeperStorageBase::RequestForSession & request_for_session)
{
ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest);
@ -340,7 +365,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
const int64_t zxid = request_for_session.zxid;
using enum Coordination::Error;
auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorage::ResponseForSession
auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorageBase::ResponseForSession
{
auto res = std::make_shared<Coordination::ZooKeeperReconfigResponse>();
res->xid = request.xid;
@ -397,7 +422,8 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
return { session_id, std::move(response) };
}
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)
template<typename Storage>
nuraft::ptr<nuraft::buffer> KeeperStateMachine<Storage>::commit(const uint64_t log_idx, nuraft::buffer & data)
{
auto request_for_session = parseRequest(data, true);
if (!request_for_session->zxid)
@ -408,7 +434,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
if (!keeper_context->localLogsPreprocessed() && !preprocess(*request_for_session))
return nullptr;
auto try_push = [&](const KeeperStorage::ResponseForSession & response)
auto try_push = [&](const KeeperStorageBase::ResponseForSession & response)
{
if (!responses_queue.push(response))
{
@ -430,7 +456,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
std::shared_ptr<Coordination::ZooKeeperSessionIDResponse> response = std::make_shared<Coordination::ZooKeeperSessionIDResponse>();
response->internal_id = session_id_request.internal_id;
response->server_id = session_id_request.server_id;
KeeperStorage::ResponseForSession response_for_session;
KeeperStorageBase::ResponseForSession response_for_session;
response_for_session.session_id = -1;
response_for_session.response = response;
response_for_session.request = request_for_session->request;
@ -451,7 +477,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
}
LockGuardWithStats lock(storage_and_responses_lock);
KeeperStorage::ResponsesForSessions responses_for_sessions
KeeperStorageBase::ResponsesForSessions responses_for_sessions
= storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid);
for (auto & response_for_session : responses_for_sessions)
@ -482,7 +508,8 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
return nullptr;
}
bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
template<typename Storage>
bool KeeperStateMachine<Storage>::apply_snapshot(nuraft::snapshot & s)
{
LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx());
nuraft::ptr<nuraft::buffer> latest_snapshot_ptr;
@ -509,7 +536,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
{ /// deserialize and apply snapshot to storage
LockGuardWithStats lock(storage_and_responses_lock);
SnapshotDeserializationResult snapshot_deserialization_result;
SnapshotDeserializationResult<Storage> snapshot_deserialization_result;
if (latest_snapshot_ptr)
snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr);
else
@ -530,7 +557,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
}
void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf)
void IKeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf)
{
std::lock_guard lock(cluster_config_lock);
auto tmp = new_conf->serialize();
@ -538,7 +565,7 @@ void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr<nuraf
keeper_context->setLastCommitIndex(log_idx);
}
void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
void IKeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
{
/// Don't rollback anything until the first commit because nothing was preprocessed
if (!keeper_context->localLogsPreprocessed())
@ -554,7 +581,8 @@ void KeeperStateMachine::rollback(uint64_t log_idx, nuraft::buffer & data)
rollbackRequest(*request_for_session, false);
}
void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing)
template<typename Storage>
void KeeperStateMachine<Storage>::rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing)
{
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
return;
@ -563,7 +591,8 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession
storage->rollbackRequest(request_for_session.zxid, allow_missing);
}
void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing)
template<typename Storage>
void KeeperStateMachine<Storage>::rollbackRequestNoLock(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing)
{
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
return;
@ -571,14 +600,15 @@ void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSe
storage->rollbackRequest(request_for_session.zxid, allow_missing);
}
nuraft::ptr<nuraft::snapshot> KeeperStateMachine::last_snapshot()
nuraft::ptr<nuraft::snapshot> IKeeperStateMachine::last_snapshot()
{
/// Just return the latest snapshot.
std::lock_guard lock(snapshots_lock);
return latest_snapshot_meta;
}
void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done)
template<typename Storage>
void KeeperStateMachine<Storage>::create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done)
{
LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx());
@ -587,14 +617,15 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
CreateSnapshotTask snapshot_task;
{ /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking.
LockGuardWithStats lock(storage_and_responses_lock);
snapshot_task.snapshot = std::make_shared<KeeperStorageSnapshot>(storage.get(), snapshot_meta_copy, getClusterConfig());
snapshot_task.snapshot = std::make_shared<KeeperStorageSnapshot<Storage>>(storage.get(), snapshot_meta_copy, getClusterConfig());
}
/// create snapshot task for background execution (in snapshot thread)
snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot, bool execute_only_cleanup)
snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot_, bool execute_only_cleanup)
{
nuraft::ptr<std::exception> exception(nullptr);
bool ret = false;
auto && snapshot = std::get<std::shared_ptr<KeeperStorageSnapshot<Storage>>>(std::move(snapshot_));
if (!execute_only_cleanup)
{
try
@ -683,7 +714,8 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
LOG_WARNING(log, "Cannot push snapshot task into queue");
}
void KeeperStateMachine::save_logical_snp_obj(
template<typename Storage>
void KeeperStateMachine<Storage>::save_logical_snp_obj(
nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool /*is_first_obj*/, bool /*is_last_obj*/)
{
LOG_DEBUG(log, "Saving snapshot {} obj_id {}", s.get_last_log_idx(), obj_id);
@ -748,7 +780,7 @@ static int bufferFromFile(LoggerPtr log, const std::string & path, nuraft::ptr<n
return 0;
}
int KeeperStateMachine::read_logical_snp_obj(
int IKeeperStateMachine::read_logical_snp_obj(
nuraft::snapshot & s, void *& /*user_snp_ctx*/, uint64_t obj_id, nuraft::ptr<nuraft::buffer> & data_out, bool & is_last_obj)
{
LOG_DEBUG(log, "Reading snapshot {} obj_id {}", s.get_last_log_idx(), obj_id);
@ -788,7 +820,8 @@ int KeeperStateMachine::read_logical_snp_obj(
return 1;
}
void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session)
template<typename Storage>
void KeeperStateMachine<Storage>::processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session)
{
/// Pure local request, just process it with storage
LockGuardWithStats lock(storage_and_responses_lock);
@ -804,103 +837,120 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi
}
}
void KeeperStateMachine::shutdownStorage()
template<typename Storage>
void KeeperStateMachine<Storage>::shutdownStorage()
{
LockGuardWithStats lock(storage_and_responses_lock);
storage->finalize();
}
std::vector<int64_t> KeeperStateMachine::getDeadSessions()
template<typename Storage>
std::vector<int64_t> KeeperStateMachine<Storage>::getDeadSessions()
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getDeadSessions();
}
int64_t KeeperStateMachine::getNextZxid() const
template<typename Storage>
int64_t KeeperStateMachine<Storage>::getNextZxid() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getNextZXID();
}
KeeperStorage::Digest KeeperStateMachine::getNodesDigest() const
template<typename Storage>
KeeperStorageBase::Digest KeeperStateMachine<Storage>::getNodesDigest() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getNodesDigest(false);
}
uint64_t KeeperStateMachine::getLastProcessedZxid() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getLastProcessedZxid() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getZXID();
}
uint64_t KeeperStateMachine::getNodesCount() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getNodesCount() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getNodesCount();
}
uint64_t KeeperStateMachine::getTotalWatchesCount() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getTotalWatchesCount() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getTotalWatchesCount();
}
uint64_t KeeperStateMachine::getWatchedPathsCount() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getWatchedPathsCount() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getWatchedPathsCount();
}
uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getSessionsWithWatchesCount() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getSessionsWithWatchesCount();
}
uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getTotalEphemeralNodesCount() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getTotalEphemeralNodesCount();
}
uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getSessionWithEphemeralNodesCount() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getSessionWithEphemeralNodesCount();
}
void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const
template<typename Storage>
void KeeperStateMachine<Storage>::dumpWatches(WriteBufferFromOwnString & buf) const
{
LockGuardWithStats lock(storage_and_responses_lock);
storage->dumpWatches(buf);
}
void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
template<typename Storage>
void KeeperStateMachine<Storage>::dumpWatchesByPath(WriteBufferFromOwnString & buf) const
{
LockGuardWithStats lock(storage_and_responses_lock);
storage->dumpWatchesByPath(buf);
}
void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const
template<typename Storage>
void KeeperStateMachine<Storage>::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const
{
LockGuardWithStats lock(storage_and_responses_lock);
storage->dumpSessionsAndEphemerals(buf);
}
uint64_t KeeperStateMachine::getApproximateDataSize() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getApproximateDataSize() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getApproximateDataSize();
}
uint64_t KeeperStateMachine::getKeyArenaSize() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getKeyArenaSize() const
{
LockGuardWithStats lock(storage_and_responses_lock);
return storage->getArenaDataSize();
}
uint64_t KeeperStateMachine::getLatestSnapshotSize() const
template<typename Storage>
uint64_t KeeperStateMachine<Storage>::getLatestSnapshotSize() const
{
auto snapshot_info = [&]
{
@ -923,7 +973,7 @@ uint64_t KeeperStateMachine::getLatestSnapshotSize() const
return size;
}
ClusterConfigPtr KeeperStateMachine::getClusterConfig() const
ClusterConfigPtr IKeeperStateMachine::getClusterConfig() const
{
std::lock_guard lock(cluster_config_lock);
if (cluster_config)
@ -935,11 +985,18 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const
return nullptr;
}
void KeeperStateMachine::recalculateStorageStats()
template<typename Storage>
void KeeperStateMachine<Storage>::recalculateStorageStats()
{
LockGuardWithStats lock(storage_and_responses_lock);
LOG_INFO(log, "Recalculating storage stats");
storage->recalculateStats();
LOG_INFO(log, "Done recalculating storage stats");
}
template class KeeperStateMachine<KeeperMemoryStorage>;
#if USE_ROCKSDB
template class KeeperStateMachine<KeeperRocksStorage>;
#endif
}

View File

@ -11,26 +11,24 @@
namespace DB
{
using ResponsesQueue = ConcurrentBoundedQueue<KeeperStorage::ResponseForSession>;
using ResponsesQueue = ConcurrentBoundedQueue<KeeperStorageBase::ResponseForSession>;
using SnapshotsQueue = ConcurrentBoundedQueue<CreateSnapshotTask>;
/// ClickHouse Keeper state machine. Wrapper for KeeperStorage.
/// Responsible for entries commit, snapshots creation and so on.
class KeeperStateMachine : public nuraft::state_machine
class IKeeperStateMachine : public nuraft::state_machine
{
public:
using CommitCallback = std::function<void(uint64_t, const KeeperStorage::RequestForSession &)>;
using CommitCallback = std::function<void(uint64_t, const KeeperStorageBase::RequestForSession &)>;
KeeperStateMachine(
IKeeperStateMachine(
ResponsesQueue & responses_queue_,
SnapshotsQueue & snapshots_queue_,
const KeeperContextPtr & keeper_context_,
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
CommitCallback commit_callback_ = {},
const std::string & superdigest_ = "");
CommitCallback commit_callback_,
const std::string & superdigest_);
/// Read state from the latest snapshot
void init();
virtual void init() = 0;
enum ZooKeeperLogSerializationVersion
{
@ -47,89 +45,66 @@ public:
///
/// final - whether it's the final time we will fetch the request so we can safely remove it from cache
/// serialization_version - information about which fields were parsed from the buffer so we can modify the buffer accordingly
std::shared_ptr<KeeperStorage::RequestForSession> parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr);
std::shared_ptr<KeeperStorageBase::RequestForSession> parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr);
bool preprocess(const KeeperStorage::RequestForSession & request_for_session);
virtual bool preprocess(const KeeperStorageBase::RequestForSession & request_for_session) = 0;
nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override;
nuraft::ptr<nuraft::buffer> commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT
/// Save new cluster config to our snapshot (copy of the config stored in StateManager)
void commit_config(const uint64_t log_idx, nuraft::ptr<nuraft::cluster_config> & new_conf) override; /// NOLINT
void rollback(uint64_t log_idx, nuraft::buffer & data) override;
// allow_missing - whether the transaction we want to rollback can be missing from storage
// (can happen in case of exception during preprocessing)
void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing);
void rollbackRequestNoLock(
const KeeperStorage::RequestForSession & request_for_session,
bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS;
virtual void rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) = 0;
uint64_t last_commit_index() override { return keeper_context->lastCommittedIndex(); }
/// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state.
bool apply_snapshot(nuraft::snapshot & s) override;
nuraft::ptr<nuraft::snapshot> last_snapshot() override;
/// Create new snapshot from current state.
void create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done) override;
void create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done) override = 0;
/// Save snapshot which was send by leader to us. After that we will apply it in apply_snapshot.
void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override;
void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override = 0;
/// Better name is `serialize snapshot` -- save existing snapshot (created by create_snapshot) into
/// in-memory buffer data_out.
int read_logical_snp_obj(
nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr<nuraft::buffer> & data_out, bool & is_last_obj) override;
// This should be used only for tests or keeper-data-dumper because it violates
// TSA -- we can't acquire the lock outside of this class or return a storage under lock
// in a reasonable way.
KeeperStorage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS
{
return *storage;
}
void shutdownStorage();
virtual void shutdownStorage() = 0;
ClusterConfigPtr getClusterConfig() const;
/// Process local read request
void processReadRequest(const KeeperStorage::RequestForSession & request_for_session);
virtual void processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) = 0;
std::vector<int64_t> getDeadSessions();
virtual std::vector<int64_t> getDeadSessions() = 0;
int64_t getNextZxid() const;
virtual int64_t getNextZxid() const = 0;
KeeperStorage::Digest getNodesDigest() const;
virtual KeeperStorageBase::Digest getNodesDigest() const = 0;
/// Introspection functions for 4lw commands
uint64_t getLastProcessedZxid() const;
virtual uint64_t getLastProcessedZxid() const = 0;
uint64_t getNodesCount() const;
uint64_t getTotalWatchesCount() const;
uint64_t getWatchedPathsCount() const;
uint64_t getSessionsWithWatchesCount() const;
virtual uint64_t getNodesCount() const = 0;
virtual uint64_t getTotalWatchesCount() const = 0;
virtual uint64_t getWatchedPathsCount() const = 0;
virtual uint64_t getSessionsWithWatchesCount() const = 0;
void dumpWatches(WriteBufferFromOwnString & buf) const;
void dumpWatchesByPath(WriteBufferFromOwnString & buf) const;
void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const;
virtual void dumpWatches(WriteBufferFromOwnString & buf) const = 0;
virtual void dumpWatchesByPath(WriteBufferFromOwnString & buf) const = 0;
virtual void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const = 0;
uint64_t getSessionWithEphemeralNodesCount() const;
uint64_t getTotalEphemeralNodesCount() const;
uint64_t getApproximateDataSize() const;
uint64_t getKeyArenaSize() const;
uint64_t getLatestSnapshotSize() const;
virtual uint64_t getSessionWithEphemeralNodesCount() const = 0;
virtual uint64_t getTotalEphemeralNodesCount() const = 0;
virtual uint64_t getApproximateDataSize() const = 0;
virtual uint64_t getKeyArenaSize() const = 0;
virtual uint64_t getLatestSnapshotSize() const = 0;
void recalculateStorageStats();
virtual void recalculateStorageStats() = 0;
void reconfigure(const KeeperStorage::RequestForSession& request_for_session);
virtual void reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) = 0;
private:
protected:
CommitCallback commit_callback;
/// In our state machine we always have a single snapshot which is stored
/// in memory in compressed (serialized) format.
@ -137,12 +112,9 @@ private:
std::shared_ptr<SnapshotFileInfo> latest_snapshot_info;
nuraft::ptr<nuraft::buffer> latest_snapshot_buf = nullptr;
/// Main state machine logic
KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock);
CoordinationSettingsPtr coordination_settings;
/// Save/Load and Serialize/Deserialize logic for snapshots.
KeeperSnapshotManager snapshot_manager;
/// Put processed responses into this queue
ResponsesQueue & responses_queue;
@ -159,7 +131,7 @@ private:
/// for request.
mutable std::mutex storage_and_responses_lock;
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, std::shared_ptr<KeeperStorage::RequestForSession>>> parsed_request_cache;
std::unordered_map<int64_t, std::unordered_map<Coordination::XID, std::shared_ptr<KeeperStorageBase::RequestForSession>>> parsed_request_cache;
uint64_t min_request_size_to_cache{0};
/// we only need to protect the access to the map itself
/// requests can be modified from anywhere without lock because a single request
@ -181,7 +153,104 @@ private:
KeeperSnapshotManagerS3 * snapshot_manager_s3;
KeeperStorage::ResponseForSession processReconfiguration(const KeeperStorage::RequestForSession & request_for_session)
TSA_REQUIRES(storage_and_responses_lock);
virtual KeeperStorageBase::ResponseForSession processReconfiguration(
const KeeperStorageBase::RequestForSession& request_for_session)
TSA_REQUIRES(storage_and_responses_lock) = 0;
};
/// ClickHouse Keeper state machine. Wrapper for KeeperStorage.
/// Responsible for entries commit, snapshots creation and so on.
template<typename Storage>
class KeeperStateMachine : public IKeeperStateMachine
{
public:
/// using CommitCallback = std::function<void(uint64_t, const KeeperStorage::RequestForSession &)>;
KeeperStateMachine(
ResponsesQueue & responses_queue_,
SnapshotsQueue & snapshots_queue_,
/// const CoordinationSettingsPtr & coordination_settings_,
const KeeperContextPtr & keeper_context_,
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
CommitCallback commit_callback_ = {},
const std::string & superdigest_ = "");
/// Read state from the latest snapshot
void init() override;
bool preprocess(const KeeperStorageBase::RequestForSession & request_for_session) override;
nuraft::ptr<nuraft::buffer> pre_commit(uint64_t log_idx, nuraft::buffer & data) override;
nuraft::ptr<nuraft::buffer> commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT
// allow_missing - whether the transaction we want to rollback can be missing from storage
// (can happen in case of exception during preprocessing)
void rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) override;
void rollbackRequestNoLock(
const KeeperStorageBase::RequestForSession & request_for_session,
bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS;
/// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state.
bool apply_snapshot(nuraft::snapshot & s) override;
/// Create new snapshot from current state.
void create_snapshot(nuraft::snapshot & s, nuraft::async_result<bool>::handler_type & when_done) override;
/// Save snapshot which was send by leader to us. After that we will apply it in apply_snapshot.
void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override;
// This should be used only for tests or keeper-data-dumper because it violates
// TSA -- we can't acquire the lock outside of this class or return a storage under lock
// in a reasonable way.
Storage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS
{
return *storage;
}
void shutdownStorage() override;
/// Process local read request
void processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) override;
std::vector<int64_t> getDeadSessions() override;
int64_t getNextZxid() const override;
KeeperStorageBase::Digest getNodesDigest() const override;
/// Introspection functions for 4lw commands
uint64_t getLastProcessedZxid() const override;
uint64_t getNodesCount() const override;
uint64_t getTotalWatchesCount() const override;
uint64_t getWatchedPathsCount() const override;
uint64_t getSessionsWithWatchesCount() const override;
void dumpWatches(WriteBufferFromOwnString & buf) const override;
void dumpWatchesByPath(WriteBufferFromOwnString & buf) const override;
void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const override;
uint64_t getSessionWithEphemeralNodesCount() const override;
uint64_t getTotalEphemeralNodesCount() const override;
uint64_t getApproximateDataSize() const override;
uint64_t getKeyArenaSize() const override;
uint64_t getLatestSnapshotSize() const override;
void recalculateStorageStats() override;
void reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) override;
private:
/// Main state machine logic
std::unique_ptr<Storage> storage; //TSA_PT_GUARDED_BY(storage_and_responses_lock);
/// Save/Load and Serialize/Deserialize logic for snapshots.
KeeperSnapshotManager<Storage> snapshot_manager;
KeeperStorageBase::ResponseForSession processReconfiguration(const KeeperStorageBase::RequestForSession & request_for_session)
TSA_REQUIRES(storage_and_responses_lock) override;
};
}

File diff suppressed because it is too large Load Diff

View File

@ -8,188 +8,384 @@
#include <absl/container/flat_hash_set.h>
#include "config.h"
#if USE_ROCKSDB
#include <Coordination/RocksDBContainer.h>
#endif
namespace DB
{
class KeeperContext;
using KeeperContextPtr = std::shared_ptr<KeeperContext>;
struct KeeperStorageRequestProcessor;
using KeeperStorageRequestProcessorPtr = std::shared_ptr<KeeperStorageRequestProcessor>;
using ResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr &)>;
using ChildrenSet = absl::flat_hash_set<StringRef, StringRefHash>;
using SessionAndTimeout = std::unordered_map<int64_t, int64_t>;
struct KeeperStorageSnapshot;
/// Keeper state machine almost equal to the ZooKeeper's state machine.
/// Implements all logic of operations, data changes, sessions allocation.
/// In-memory and not thread safe.
class KeeperStorage
/// KeeperRocksNodeInfo is used in RocksDB keeper.
/// It is serialized directly as POD to RocksDB.
struct KeeperRocksNodeInfo
{
public:
/// Node should have as minimal size as possible to reduce memory footprint
/// of stored nodes
/// New fields should be added to the struct only if it's really necessary
struct Node
int64_t czxid{0};
int64_t mzxid{0};
int64_t pzxid{0};
uint64_t acl_id = 0; /// 0 -- no ACL by default
int64_t mtime{0};
int32_t version{0};
int32_t cversion{0};
int32_t aversion{0};
int32_t seq_num = 0;
mutable UInt64 digest = 0; /// we cached digest for this node.
/// as ctime can't be negative because it stores the timestamp when the
/// node was created, we can use the MSB for a bool
struct
{
int64_t czxid{0};
int64_t mzxid{0};
int64_t pzxid{0};
uint64_t acl_id = 0; /// 0 -- no ACL by default
bool is_ephemeral : 1;
int64_t ctime : 63;
} is_ephemeral_and_ctime{false, 0};
int64_t mtime{0};
std::unique_ptr<char[]> data{nullptr};
uint32_t data_size{0};
int32_t version{0};
int32_t cversion{0};
int32_t aversion{0};
mutable uint64_t cached_digest = 0;
Node() = default;
Node & operator=(const Node & other);
Node(const Node & other);
Node & operator=(Node && other) noexcept;
Node(Node && other) noexcept;
bool empty() const;
bool isEphemeral() const
{
return is_ephemeral_and_ctime.is_ephemeral;
}
int64_t ephemeralOwner() const
{
if (isEphemeral())
return ephemeral_or_children_data.ephemeral_owner;
return 0;
}
void setEphemeralOwner(int64_t ephemeral_owner)
{
is_ephemeral_and_ctime.is_ephemeral = ephemeral_owner != 0;
ephemeral_or_children_data.ephemeral_owner = ephemeral_owner;
}
int32_t numChildren() const
{
if (isEphemeral())
return 0;
return ephemeral_or_children_data.children_info.num_children;
}
void setNumChildren(int32_t num_children)
{
ephemeral_or_children_data.children_info.num_children = num_children;
}
void increaseNumChildren()
{
chassert(!isEphemeral());
++ephemeral_or_children_data.children_info.num_children;
}
void decreaseNumChildren()
{
chassert(!isEphemeral());
--ephemeral_or_children_data.children_info.num_children;
}
int32_t seqNum() const
{
if (isEphemeral())
return 0;
return ephemeral_or_children_data.children_info.seq_num;
}
void setSeqNum(int32_t seq_num)
{
ephemeral_or_children_data.children_info.seq_num = seq_num;
}
void increaseSeqNum()
{
chassert(!isEphemeral());
++ephemeral_or_children_data.children_info.seq_num;
}
int64_t ctime() const
{
return is_ephemeral_and_ctime.ctime;
}
void setCtime(uint64_t ctime)
{
is_ephemeral_and_ctime.ctime = ctime;
}
void copyStats(const Coordination::Stat & stat);
void setResponseStat(Coordination::Stat & response_stat) const;
/// Object memory size
uint64_t sizeInBytes() const;
void setData(const String & new_data);
std::string_view getData() const noexcept { return {data.get(), data_size}; }
void addChild(StringRef child_path);
void removeChild(StringRef child_path);
const auto & getChildren() const noexcept { return children; }
auto & getChildren() { return children; }
// Invalidate the calculated digest so it's recalculated again on the next
// getDigest call
void invalidateDigestCache() const;
// get the calculated digest of the node
UInt64 getDigest(std::string_view path) const;
// copy only necessary information for preprocessing and digest calculation
// (e.g. we don't need to copy list of children)
void shallowCopy(const Node & other);
private:
/// as ctime can't be negative because it stores the timestamp when the
/// node was created, we can use the MSB for a bool
/// ephemeral notes cannot have children so a node can set either
/// ephemeral_owner OR seq_num + num_children
union
{
int64_t ephemeral_owner;
struct
{
bool is_ephemeral : 1;
int64_t ctime : 63;
} is_ephemeral_and_ctime{false, 0};
int32_t seq_num;
int32_t num_children;
} children_info;
} ephemeral_or_children_data{0};
/// ephemeral notes cannot have children so a node can set either
/// ephemeral_owner OR seq_num + num_children
union
{
int64_t ephemeral_owner;
struct
{
int32_t seq_num;
int32_t num_children;
} children_info;
} ephemeral_or_children_data{0};
bool isEphemeral() const
{
return is_ephemeral_and_ctime.is_ephemeral;
}
ChildrenSet children{};
};
int64_t ephemeralOwner() const
{
if (isEphemeral())
return ephemeral_or_children_data.ephemeral_owner;
#if !defined(ADDRESS_SANITIZER) && !defined(MEMORY_SANITIZER)
static_assert(
sizeof(ListNode<Node>) <= 144,
"std::list node containing ListNode<Node> is > 160 bytes (sizeof(ListNode<Node>) + 16 bytes for pointers) which will increase "
"memory consumption");
return 0;
}
void setEphemeralOwner(int64_t ephemeral_owner)
{
is_ephemeral_and_ctime.is_ephemeral = ephemeral_owner != 0;
ephemeral_or_children_data.ephemeral_owner = ephemeral_owner;
}
int32_t numChildren() const
{
if (isEphemeral())
return 0;
return ephemeral_or_children_data.children_info.num_children;
}
void setNumChildren(int32_t num_children)
{
ephemeral_or_children_data.children_info.num_children = num_children;
}
/// dummy interface for test
void addChild(StringRef) {}
auto getChildren() const
{
return std::vector<int>(numChildren());
}
void increaseNumChildren()
{
chassert(!isEphemeral());
++ephemeral_or_children_data.children_info.num_children;
}
void decreaseNumChildren()
{
chassert(!isEphemeral());
--ephemeral_or_children_data.children_info.num_children;
}
int32_t seqNum() const
{
if (isEphemeral())
return 0;
return ephemeral_or_children_data.children_info.seq_num;
}
void setSeqNum(int32_t seq_num_)
{
ephemeral_or_children_data.children_info.seq_num = seq_num_;
}
void increaseSeqNum()
{
chassert(!isEphemeral());
++ephemeral_or_children_data.children_info.seq_num;
}
int64_t ctime() const
{
return is_ephemeral_and_ctime.ctime;
}
void setCtime(uint64_t ctime)
{
is_ephemeral_and_ctime.ctime = ctime;
}
void copyStats(const Coordination::Stat & stat);
};
/// KeeperRocksNode is the memory structure used by RocksDB
struct KeeperRocksNode : public KeeperRocksNodeInfo
{
#if USE_ROCKSDB
friend struct RocksDBContainer<KeeperRocksNode>;
#endif
using Meta = KeeperRocksNodeInfo;
uint64_t size_bytes = 0; // only for compatible, should be deprecated
uint64_t sizeInBytes() const { return data_size + sizeof(KeeperRocksNodeInfo); }
void setData(String new_data)
{
data_size = static_cast<uint32_t>(new_data.size());
if (data_size != 0)
{
data = std::unique_ptr<char[]>(new char[new_data.size()]);
memcpy(data.get(), new_data.data(), data_size);
}
}
void shallowCopy(const KeeperRocksNode & other)
{
czxid = other.czxid;
mzxid = other.mzxid;
pzxid = other.pzxid;
acl_id = other.acl_id; /// 0 -- no ACL by default
mtime = other.mtime;
is_ephemeral_and_ctime = other.is_ephemeral_and_ctime;
ephemeral_or_children_data = other.ephemeral_or_children_data;
data_size = other.data_size;
if (data_size != 0)
{
data = std::unique_ptr<char[]>(new char[data_size]);
memcpy(data.get(), other.data.get(), data_size);
}
version = other.version;
cversion = other.cversion;
aversion = other.aversion;
/// cached_digest = other.cached_digest;
}
void invalidateDigestCache() const;
UInt64 getDigest(std::string_view path) const;
String getEncodedString();
void decodeFromString(const String & buffer_str);
void recalculateSize() {}
std::string_view getData() const noexcept { return {data.get(), data_size}; }
void setResponseStat(Coordination::Stat & response_stat) const
{
response_stat.czxid = czxid;
response_stat.mzxid = mzxid;
response_stat.ctime = ctime();
response_stat.mtime = mtime;
response_stat.version = version;
response_stat.cversion = cversion;
response_stat.aversion = aversion;
response_stat.ephemeralOwner = ephemeralOwner();
response_stat.dataLength = static_cast<int32_t>(data_size);
response_stat.numChildren = numChildren();
response_stat.pzxid = pzxid;
}
void reset()
{
serialized = false;
}
bool empty() const
{
return data_size == 0 && mzxid == 0;
}
std::unique_ptr<char[]> data{nullptr};
uint32_t data_size{0};
private:
bool serialized = false;
};
/// KeeperMemNode should have as minimal size as possible to reduce memory footprint
/// of stored nodes
/// New fields should be added to the struct only if it's really necessary
struct KeeperMemNode
{
int64_t czxid{0};
int64_t mzxid{0};
int64_t pzxid{0};
uint64_t acl_id = 0; /// 0 -- no ACL by default
int64_t mtime{0};
std::unique_ptr<char[]> data{nullptr};
uint32_t data_size{0};
int32_t version{0};
int32_t cversion{0};
int32_t aversion{0};
mutable uint64_t cached_digest = 0;
KeeperMemNode() = default;
KeeperMemNode & operator=(const KeeperMemNode & other);
KeeperMemNode(const KeeperMemNode & other);
KeeperMemNode & operator=(KeeperMemNode && other) noexcept;
KeeperMemNode(KeeperMemNode && other) noexcept;
bool empty() const;
bool isEphemeral() const
{
return is_ephemeral_and_ctime.is_ephemeral;
}
int64_t ephemeralOwner() const
{
if (isEphemeral())
return ephemeral_or_children_data.ephemeral_owner;
return 0;
}
void setEphemeralOwner(int64_t ephemeral_owner)
{
is_ephemeral_and_ctime.is_ephemeral = ephemeral_owner != 0;
ephemeral_or_children_data.ephemeral_owner = ephemeral_owner;
}
int32_t numChildren() const
{
if (isEphemeral())
return 0;
return ephemeral_or_children_data.children_info.num_children;
}
void setNumChildren(int32_t num_children)
{
ephemeral_or_children_data.children_info.num_children = num_children;
}
void increaseNumChildren()
{
chassert(!isEphemeral());
++ephemeral_or_children_data.children_info.num_children;
}
void decreaseNumChildren()
{
chassert(!isEphemeral());
--ephemeral_or_children_data.children_info.num_children;
}
int32_t seqNum() const
{
if (isEphemeral())
return 0;
return ephemeral_or_children_data.children_info.seq_num;
}
void setSeqNum(int32_t seq_num)
{
ephemeral_or_children_data.children_info.seq_num = seq_num;
}
void increaseSeqNum()
{
chassert(!isEphemeral());
++ephemeral_or_children_data.children_info.seq_num;
}
int64_t ctime() const
{
return is_ephemeral_and_ctime.ctime;
}
void setCtime(uint64_t ctime)
{
is_ephemeral_and_ctime.ctime = ctime;
}
void copyStats(const Coordination::Stat & stat);
void setResponseStat(Coordination::Stat & response_stat) const;
/// Object memory size
uint64_t sizeInBytes() const;
void setData(const String & new_data);
std::string_view getData() const noexcept { return {data.get(), data_size}; }
void addChild(StringRef child_path);
void removeChild(StringRef child_path);
const auto & getChildren() const noexcept { return children; }
auto & getChildren() { return children; }
// Invalidate the calculated digest so it's recalculated again on the next
// getDigest call
void invalidateDigestCache() const;
// get the calculated digest of the node
UInt64 getDigest(std::string_view path) const;
// copy only necessary information for preprocessing and digest calculation
// (e.g. we don't need to copy list of children)
void shallowCopy(const KeeperMemNode & other);
private:
/// as ctime can't be negative because it stores the timestamp when the
/// node was created, we can use the MSB for a bool
struct
{
bool is_ephemeral : 1;
int64_t ctime : 63;
} is_ephemeral_and_ctime{false, 0};
/// ephemeral notes cannot have children so a node can set either
/// ephemeral_owner OR seq_num + num_children
union
{
int64_t ephemeral_owner;
struct
{
int32_t seq_num;
int32_t num_children;
} children_info;
} ephemeral_or_children_data{0};
ChildrenSet children{};
};
class KeeperStorageBase
{
public:
enum DigestVersion : uint8_t
{
@ -200,7 +396,11 @@ public:
V4 = 4 // 0 is not a valid digest value
};
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V4;
struct Digest
{
DigestVersion version{DigestVersion::NO_DIGEST};
uint64_t value{0};
};
struct ResponseForSession
{
@ -210,16 +410,6 @@ public:
};
using ResponsesForSessions = std::vector<ResponseForSession>;
struct Digest
{
DigestVersion version{DigestVersion::NO_DIGEST};
uint64_t value{0};
};
static bool checkDigest(const Digest & first, const Digest & second);
static String generateDigest(const String & userdata);
struct RequestForSession
{
int64_t session_id;
@ -229,6 +419,7 @@ public:
std::optional<Digest> digest;
int64_t log_idx{0};
};
using RequestsForSessions = std::vector<RequestForSession>;
struct AuthID
{
@ -238,9 +429,6 @@ public:
bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; }
};
using RequestsForSessions = std::vector<RequestForSession>;
using Container = SnapshotableHashTable<Node>;
using Ephemerals = std::unordered_map<int64_t, std::unordered_set<std::string>>;
using SessionAndWatcher = std::unordered_map<int64_t, std::unordered_set<std::string>>;
using SessionIDs = std::unordered_set<int64_t>;
@ -250,6 +438,38 @@ public:
using SessionAndAuth = std::unordered_map<int64_t, AuthIDs>;
using Watches = std::unordered_map<String /* path, relative of root_path */, SessionIDs>;
static bool checkDigest(const Digest & first, const Digest & second);
};
/// Keeper state machine almost equal to the ZooKeeper's state machine.
/// Implements all logic of operations, data changes, sessions allocation.
/// In-memory and not thread safe.
template<typename Container_>
class KeeperStorage : public KeeperStorageBase
{
public:
using Container = Container_;
using Node = Container::Node;
#if !defined(ADDRESS_SANITIZER) && !defined(MEMORY_SANITIZER)
static_assert(
sizeof(ListNode<Node>) <= 144,
"std::list node containing ListNode<Node> is > 160 bytes (sizeof(ListNode<Node>) + 16 bytes for pointers) which will increase "
"memory consumption");
#endif
#if USE_ROCKSDB
static constexpr bool use_rocksdb = std::is_same_v<Container_, RocksDBContainer<KeeperRocksNode>>;
#else
static constexpr bool use_rocksdb = false;
#endif
static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V4;
static String generateDigest(const String & userdata);
int64_t session_id_counter{1};
SessionAndAuth session_and_auth;
@ -393,7 +613,7 @@ public:
std::unordered_map<std::string, std::list<const Delta *>, Hash, Equal> deltas_for_path;
std::list<Delta> deltas;
KeeperStorage & storage;
KeeperStorage<Container> & storage;
};
UncommittedState uncommitted_state{*this};
@ -530,10 +750,16 @@ public:
/// Set of methods for creating snapshots
/// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version.
void enableSnapshotMode(size_t up_to_version) { container.enableSnapshotMode(up_to_version); }
void enableSnapshotMode(size_t up_to_version)
{
container.enableSnapshotMode(up_to_version);
}
/// Turn off snapshot mode.
void disableSnapshotMode() { container.disableSnapshotMode(); }
void disableSnapshotMode()
{
container.disableSnapshotMode();
}
Container::const_iterator getSnapshotIteratorBegin() const { return container.begin(); }
@ -572,6 +798,9 @@ private:
void addDigest(const Node & node, std::string_view path);
};
using KeeperStoragePtr = std::unique_ptr<KeeperStorage>;
using KeeperMemoryStorage = KeeperStorage<SnapshotableHashTable<KeeperMemNode>>;
#if USE_ROCKSDB
using KeeperRocksStorage = KeeperStorage<RocksDBContainer<KeeperRocksNode>>;
#endif
}

View File

@ -0,0 +1,460 @@
#pragma once
#include <base/StringRef.h>
#include <Coordination/CoordinationSettings.h>
#include <Coordination/KeeperContext.h>
#include <Common/SipHash.h>
#include <Disks/DiskLocal.h>
#include <IO/WriteBufferFromString.h>
#include <IO/ReadBufferFromString.h>
#include <rocksdb/convenience.h>
#include <rocksdb/options.h>
#include <rocksdb/status.h>
#include <rocksdb/table.h>
#include <rocksdb/snapshot.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ROCKSDB_ERROR;
extern const int LOGICAL_ERROR;
}
/// The key-value format of rocks db will be
/// - key: Int8 (depth of the path) + String (path)
/// - value: SizeOf(keeperRocksNodeInfo) (meta of the node) + String (data)
template <class Node_>
struct RocksDBContainer
{
using Node = Node_;
private:
/// MockNode is only use in test to mock `getChildren()` and `getData()`
struct MockNode
{
std::vector<int> children;
std::string data;
MockNode(size_t children_num, std::string_view data_)
: children(std::vector<int>(children_num)),
data(data_)
{
}
std::vector<int> getChildren() { return children; }
std::string getData() { return data; }
};
UInt16 getKeyDepth(const std::string & key)
{
UInt16 depth = 0;
for (size_t i = 0; i < key.size(); i++)
{
if (key[i] == '/' && i + 1 != key.size())
depth ++;
}
return depth;
}
std::string getEncodedKey(const std::string & key, bool child_prefix = false)
{
WriteBufferFromOwnString key_buffer;
UInt16 depth = getKeyDepth(key) + (child_prefix ? 1 : 0);
writeIntBinary(depth, key_buffer);
writeString(key, key_buffer);
return key_buffer.str();
}
static std::string_view getDecodedKey(const std::string_view & key)
{
return std::string_view(key.begin() + 2, key.end());
}
struct KVPair
{
StringRef key;
Node value;
};
using ValueUpdater = std::function<void(Node & node)>;
public:
/// This is an iterator wrapping rocksdb iterator and the kv result.
struct const_iterator
{
std::shared_ptr<rocksdb::Iterator> iter;
std::shared_ptr<const KVPair> pair;
const_iterator() = default;
explicit const_iterator(std::shared_ptr<KVPair> pair_) : pair(std::move(pair_)) {}
explicit const_iterator(rocksdb::Iterator * iter_) : iter(iter_)
{
updatePairFromIter();
}
const KVPair & operator * () const
{
return *pair;
}
const KVPair * operator->() const
{
return pair.get();
}
bool operator != (const const_iterator & other) const
{
return !(*this == other);
}
bool operator == (const const_iterator & other) const
{
if (pair == nullptr && other == nullptr)
return true;
if (pair == nullptr || other == nullptr)
return false;
return pair->key.toView() == other->key.toView() && iter == other.iter;
}
bool operator == (std::nullptr_t) const
{
return iter == nullptr;
}
bool operator != (std::nullptr_t) const
{
return iter != nullptr;
}
explicit operator bool() const
{
return iter != nullptr;
}
const_iterator & operator ++()
{
iter->Next();
updatePairFromIter();
return *this;
}
private:
void updatePairFromIter()
{
if (iter && iter->Valid())
{
auto new_pair = std::make_shared<KVPair>();
new_pair->key = StringRef(getDecodedKey(iter->key().ToStringView()));
ReadBufferFromOwnString buffer(iter->value().ToStringView());
typename Node::Meta & meta = new_pair->value;
readPODBinary(meta, buffer);
readVarUInt(new_pair->value.data_size, buffer);
if (new_pair->value.data_size)
{
new_pair->value.data = std::unique_ptr<char[]>(new char[new_pair->value.data_size]);
buffer.readStrict(new_pair->value.data.get(), new_pair->value.data_size);
}
pair = new_pair;
}
else
{
pair = nullptr;
iter = nullptr;
}
}
};
bool initialized = false;
const const_iterator end_ptr;
void initialize(const KeeperContextPtr & context)
{
DiskPtr disk = context->getTemporaryRocksDBDisk();
if (disk == nullptr)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get rocksdb disk");
}
auto options = context->getRocksDBOptions();
if (options == nullptr)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get rocksdb options");
}
rocksdb_dir = disk->getPath();
rocksdb::DB * db;
auto status = rocksdb::DB::Open(*options, rocksdb_dir, &db);
if (!status.ok())
{
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Failed to open rocksdb path at: {}: {}",
rocksdb_dir, status.ToString());
}
rocksdb_ptr = std::unique_ptr<rocksdb::DB>(db);
write_options.disableWAL = true;
initialized = true;
}
~RocksDBContainer()
{
if (initialized)
{
rocksdb_ptr->Close();
rocksdb_ptr = nullptr;
std::filesystem::remove_all(rocksdb_dir);
}
}
std::vector<std::pair<std::string, Node>> getChildren(const std::string & key_)
{
rocksdb::ReadOptions read_options;
read_options.total_order_seek = true;
std::string key = key_;
if (!key.ends_with('/'))
key += '/';
size_t len = key.size() + 2;
auto iter = std::unique_ptr<rocksdb::Iterator>(rocksdb_ptr->NewIterator(read_options));
std::string encoded_string = getEncodedKey(key, true);
rocksdb::Slice prefix(encoded_string);
std::vector<std::pair<std::string, Node>> result;
for (iter->Seek(prefix); iter->Valid() && iter->key().starts_with(prefix); iter->Next())
{
Node node;
ReadBufferFromOwnString buffer(iter->value().ToStringView());
typename Node::Meta & meta = node;
/// We do not read data here
readPODBinary(meta, buffer);
std::string real_key(iter->key().data() + len, iter->key().size() - len);
// std::cout << "real key: " << real_key << std::endl;
result.emplace_back(std::move(real_key), std::move(node));
}
return result;
}
bool contains(const std::string & path)
{
const std::string & encoded_key = getEncodedKey(path);
std::string buffer_str;
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &buffer_str);
if (status.IsNotFound())
return false;
if (!status.ok())
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during executing contains. The error message is {}.", status.ToString());
return true;
}
const_iterator find(StringRef key_)
{
/// rocksdb::PinnableSlice slice;
const std::string & encoded_key = getEncodedKey(key_.toString());
std::string buffer_str;
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &buffer_str);
if (status.IsNotFound())
return end();
if (!status.ok())
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during executing find. The error message is {}.", status.ToString());
ReadBufferFromOwnString buffer(buffer_str);
auto kv = std::make_shared<KVPair>();
kv->key = key_;
typename Node::Meta & meta = kv->value;
readPODBinary(meta, buffer);
/// TODO: Sometimes we don't need to load data.
readVarUInt(kv->value.data_size, buffer);
if (kv->value.data_size)
{
kv->value.data = std::unique_ptr<char[]>(new char[kv->value.data_size]);
buffer.readStrict(kv->value.data.get(), kv->value.data_size);
}
return const_iterator(kv);
}
MockNode getValue(StringRef key)
{
auto it = find(key);
chassert(it != end());
return MockNode(it->value.numChildren(), it->value.getData());
}
const_iterator updateValue(StringRef key_, ValueUpdater updater)
{
/// rocksdb::PinnableSlice slice;
const std::string & key = key_.toString();
const std::string & encoded_key = getEncodedKey(key);
std::string buffer_str;
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &buffer_str);
if (!status.ok())
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during find. The error message is {}.", status.ToString());
auto kv = std::make_shared<KVPair>();
kv->key = key_;
kv->value.decodeFromString(buffer_str);
/// storage->removeDigest(node, key);
updater(kv->value);
insertOrReplace(key, kv->value);
return const_iterator(kv);
}
bool insert(const std::string & key, Node & value)
{
std::string value_str;
const std::string & encoded_key = getEncodedKey(key);
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &value_str);
if (status.ok())
{
return false;
}
else if (status.IsNotFound())
{
status = rocksdb_ptr->Put(write_options, encoded_key, value.getEncodedString());
if (status.ok())
{
counter++;
return true;
}
}
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during insert. The error message is {}.", status.ToString());
}
void insertOrReplace(const std::string & key, Node & value)
{
const std::string & encoded_key = getEncodedKey(key);
/// storage->addDigest(value, key);
std::string value_str;
rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &value_str);
bool increase_counter = false;
if (status.IsNotFound())
increase_counter = true;
else if (!status.ok())
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during get. The error message is {}.", status.ToString());
status = rocksdb_ptr->Put(write_options, encoded_key, value.getEncodedString());
if (status.ok())
counter += increase_counter;
else
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during insert. The error message is {}.", status.ToString());
}
using KeyPtr = std::unique_ptr<char[]>;
/// To be compatible with SnapshotableHashTable, will remove later;
KeyPtr allocateKey(size_t size)
{
return KeyPtr{new char[size]};
}
void insertOrReplace(KeyPtr key_data, size_t key_size, Node value)
{
std::string key(key_data.get(), key_size);
insertOrReplace(key, value);
}
bool erase(const std::string & key)
{
/// storage->removeDigest(value, key);
const std::string & encoded_key = getEncodedKey(key);
auto status = rocksdb_ptr->Delete(write_options, encoded_key);
if (status.IsNotFound())
return false;
if (status.ok())
{
counter--;
return true;
}
throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during erase. The error message is {}.", status.ToString());
}
void recalculateDataSize() {}
void reverse(size_t size_) {(void)size_;}
uint64_t getApproximateDataSize() const
{
/// use statistics from rocksdb
return counter * sizeof(Node);
}
void enableSnapshotMode(size_t version)
{
chassert(!snapshot_mode);
snapshot_mode = true;
snapshot_up_to_version = version;
snapshot_size = counter;
++current_version;
snapshot = rocksdb_ptr->GetSnapshot();
}
void disableSnapshotMode()
{
chassert(snapshot_mode);
snapshot_mode = false;
rocksdb_ptr->ReleaseSnapshot(snapshot);
}
void clearOutdatedNodes() {}
std::pair<size_t, size_t> snapshotSizeWithVersion() const
{
if (!snapshot_mode)
return std::make_pair(counter, current_version);
else
return std::make_pair(snapshot_size, current_version);
}
const_iterator begin() const
{
rocksdb::ReadOptions read_options;
read_options.total_order_seek = true;
if (snapshot_mode)
read_options.snapshot = snapshot;
auto * iter = rocksdb_ptr->NewIterator(read_options);
iter->SeekToFirst();
return const_iterator(iter);
}
const_iterator end() const
{
return end_ptr;
}
size_t size() const
{
return counter;
}
uint64_t getArenaDataSize() const
{
return 0;
}
uint64_t keyArenaSize() const
{
return 0;
}
private:
String rocksdb_dir;
std::unique_ptr<rocksdb::DB> rocksdb_ptr;
rocksdb::WriteOptions write_options;
const rocksdb::Snapshot * snapshot;
bool snapshot_mode{false};
size_t current_version{0};
size_t snapshot_up_to_version{0};
size_t snapshot_size{0};
size_t counter{0};
};
}

View File

@ -212,9 +212,9 @@ private:
updateDataSize(INSERT_OR_REPLACE, key.size, new_value_size, old_value_size, !snapshot_mode);
}
public:
using Node = V;
using iterator = typename List::iterator;
using const_iterator = typename List::const_iterator;
using ValueUpdater = std::function<void(V & value)>;
@ -364,6 +364,7 @@ public:
{
auto map_it = map.find(key);
if (map_it != map.end())
/// return std::make_shared<KVPair>(KVPair{map_it->getMapped()->key, map_it->getMapped()->value});
return map_it->getMapped();
return list.end();
}

View File

@ -43,7 +43,8 @@ void deserializeSnapshotMagic(ReadBuffer & in)
throw Exception(ErrorCodes::CORRUPTED_DATA, "Incorrect magic header in file, expected {}, got {}", SNP_HEADER, magic_header);
}
int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in)
template<typename Storage>
int64_t deserializeSessionAndTimeout(Storage & storage, ReadBuffer & in)
{
int32_t count;
Coordination::read(count, in);
@ -62,7 +63,8 @@ int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in)
return max_session_id;
}
void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in)
template<typename Storage>
void deserializeACLMap(Storage & storage, ReadBuffer & in)
{
int32_t count;
Coordination::read(count, in);
@ -90,7 +92,8 @@ void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in)
}
}
int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerPtr log)
template<typename Storage>
int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log)
{
int64_t max_zxid = 0;
std::string path;
@ -98,7 +101,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP
size_t count = 0;
while (path != "/")
{
KeeperStorage::Node node{};
typename Storage::Node node{};
String data;
Coordination::read(data, in);
node.setData(data);
@ -146,14 +149,15 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP
if (itr.key != "/")
{
auto parent_path = parentNodePath(itr.key);
storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); value.increaseNumChildren(); });
storage.container.updateValue(parent_path, [my_path = itr.key] (typename Storage::Node & value) { value.addChild(getBaseNodeName(my_path)); value.increaseNumChildren(); });
}
}
return max_zxid;
}
void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, LoggerPtr log)
template<typename Storage>
void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, LoggerPtr log)
{
LOG_INFO(log, "Deserializing storage snapshot {}", snapshot_path);
int64_t zxid = getZxidFromName(snapshot_path);
@ -192,9 +196,11 @@ void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::st
LOG_INFO(log, "Finished, snapshot ZXID {}", storage.zxid);
}
void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, LoggerPtr log)
namespace fs = std::filesystem;
template<typename Storage>
void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, LoggerPtr log)
{
namespace fs = std::filesystem;
std::map<int64_t, std::string> existing_snapshots;
for (const auto & p : fs::directory_iterator(path))
{
@ -480,7 +486,8 @@ bool hasErrorsInMultiRequest(Coordination::ZooKeeperRequestPtr request)
}
bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, LoggerPtr /*log*/)
template<typename Storage>
bool deserializeTxn(Storage & storage, ReadBuffer & in, LoggerPtr /*log*/)
{
int64_t checksum;
Coordination::read(checksum, in);
@ -535,7 +542,8 @@ bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, LoggerPtr /*log*/)
return true;
}
void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, LoggerPtr log)
template<typename Storage>
void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, LoggerPtr log)
{
ReadBufferFromFile reader(log_path);
@ -559,9 +567,9 @@ void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string
LOG_INFO(log, "Finished {} deserialization, totally read {} records", log_path, counter);
}
void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, LoggerPtr log)
template<typename Storage>
void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, LoggerPtr log)
{
namespace fs = std::filesystem;
std::map<int64_t, std::string> existing_logs;
for (const auto & p : fs::directory_iterator(path))
{
@ -595,4 +603,9 @@ void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string
}
}
template void deserializeKeeperStorageFromSnapshot<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & snapshot_path, LoggerPtr log);
template void deserializeKeeperStorageFromSnapshotsDir<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & path, LoggerPtr log);
template void deserializeLogAndApplyToStorage<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & log_path, LoggerPtr log);
template void deserializeLogsAndApplyToStorage<KeeperMemoryStorage>(KeeperMemoryStorage & storage, const std::string & path, LoggerPtr log);
}

View File

@ -5,12 +5,16 @@
namespace DB
{
void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, LoggerPtr log);
template<typename Storage>
void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, LoggerPtr log);
void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, LoggerPtr log);
template<typename Storage>
void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, LoggerPtr log);
void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, LoggerPtr log);
template<typename Storage>
void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, LoggerPtr log);
void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, LoggerPtr log);
template<typename Storage>
void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, LoggerPtr log);
}

File diff suppressed because it is too large Load Diff

View File

@ -786,7 +786,7 @@ struct SetupNodeCollector
if (snapshot_result.storage == nullptr)
{
std::cerr << "No initial snapshot found" << std::endl;
initial_storage = std::make_unique<Coordination::KeeperStorage>(
initial_storage = std::make_unique<Coordination::KeeperMemoryStorage>(
/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false);
initial_storage->initializeSystemNodes();
}
@ -932,7 +932,7 @@ struct SetupNodeCollector
std::cerr << "Generating snapshot with starting data" << std::endl;
DB::SnapshotMetadataPtr snapshot_meta = std::make_shared<DB::SnapshotMetadata>(initial_storage->getZXID(), 1, std::make_shared<nuraft::cluster_config>());
DB::KeeperStorageSnapshot snapshot(initial_storage.get(), snapshot_meta);
DB::KeeperStorageSnapshot<Coordination::KeeperMemoryStorage> snapshot(initial_storage.get(), snapshot_meta);
snapshot_manager->serializeSnapshotToDisk(snapshot);
new_nodes = false;
@ -940,9 +940,9 @@ struct SetupNodeCollector
std::mutex nodes_mutex;
DB::KeeperContextPtr keeper_context;
Coordination::KeeperStoragePtr initial_storage;
std::shared_ptr<Coordination::KeeperMemoryStorage> initial_storage;
std::unordered_set<std::string> nodes_created_during_replay;
std::optional<Coordination::KeeperSnapshotManager> snapshot_manager;
std::optional<Coordination::KeeperSnapshotManager<Coordination::KeeperMemoryStorage>> snapshot_manager;
bool new_nodes = false;
};

View File

@ -16,7 +16,7 @@
using namespace Coordination;
using namespace DB;
void dumpMachine(std::shared_ptr<KeeperStateMachine> machine)
void dumpMachine(std::shared_ptr<KeeperStateMachine<DB::KeeperMemoryStorage>> machine)
{
auto & storage = machine->getStorageUnsafe();
std::queue<std::string> keys;
@ -70,7 +70,7 @@ int main(int argc, char *argv[])
keeper_context->setLogDisk(std::make_shared<DB::DiskLocal>("LogDisk", argv[2]));
keeper_context->setSnapshotDisk(std::make_shared<DB::DiskLocal>("SnapshotDisk", argv[1]));
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, keeper_context, nullptr);
auto state_machine = std::make_shared<KeeperStateMachine<DB::KeeperMemoryStorage>>(queue, snapshots_queue, keeper_context, nullptr);
state_machine->init();
size_t last_commited_index = state_machine->last_commit_index();