From 3bccce2eae52f5f746600c31f83d89f71af80a67 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 11 Nov 2023 19:37:30 +0100 Subject: [PATCH] some preparement to support RocksDB as alternative keeper storage --- programs/keeper-converter/KeeperConverter.cpp | 7 +- src/Coordination/KeeperDispatcher.cpp | 30 +- src/Coordination/KeeperDispatcher.h | 10 +- src/Coordination/KeeperServer.cpp | 24 +- src/Coordination/KeeperServer.h | 10 +- src/Coordination/KeeperSnapshotManager.cpp | 101 ++- src/Coordination/KeeperSnapshotManager.h | 36 +- src/Coordination/KeeperStateMachine.cpp | 172 +++-- src/Coordination/KeeperStateMachine.h | 201 ++++-- src/Coordination/KeeperStorage.cpp | 672 ++++++++++-------- src/Coordination/KeeperStorage.h | 174 ++--- src/Coordination/SnapshotableHashTable.h | 1 + src/Coordination/ZooKeeperDataReader.cpp | 33 +- src/Coordination/ZooKeeperDataReader.h | 12 +- src/Coordination/tests/gtest_coordination.cpp | 86 +-- 15 files changed, 900 insertions(+), 669 deletions(-) diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index 20448aafa2f..a426f67e429 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -44,16 +44,17 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) keeper_context->setDigestEnabled(true); keeper_context->setSnapshotDisk(std::make_shared("Keeper-snapshots", options["output-dir"].as())); - 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(), logger); storage.initializeSystemNodes(); DB::deserializeLogsAndApplyToStorage(storage, options["zookeeper-logs-dir"].as(), logger); DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); - DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); + DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); - DB::KeeperSnapshotManager manager(1, keeper_context); + DB::KeeperSnapshotManager 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; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9e0cdbd6cd3..f97fc448532 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -66,13 +66,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; auto & shutdown_called = keeper_context->shutdown_called; 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(); @@ -93,7 +93,7 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; - KeeperStorage::RequestsForSessions current_batch; + KeeperStorageBase::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; bool has_read_request = false; @@ -244,7 +244,7 @@ void KeeperDispatcher::responseThread() auto & shutdown_called = keeper_context->shutdown_called; 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(); @@ -340,7 +340,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(system_clock::now().time_since_epoch()).count(); @@ -386,7 +386,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 @@ -486,7 +486,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)) @@ -497,7 +497,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); @@ -511,7 +511,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(system_clock::now().time_since_epoch()).count(), @@ -609,7 +609,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(system_clock::now().time_since_epoch()).count(), @@ -657,16 +657,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, @@ -675,7 +675,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession } } -nuraft::ptr KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions) +nuraft::ptr KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions) { if (!result->has_result()) result->get(); @@ -697,7 +697,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 request = std::make_shared(); /// Internal session id. It's a temporary number which is unique for each client on this server /// but can be same on different servers. diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 39941f55d5e..f8dc8503380 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -27,7 +27,7 @@ using ZooKeeperResponseCallback = std::function; + using RequestsQueue = ConcurrentBoundedQueue; using SessionToResponseCallback = std::unordered_map; using ClusterUpdateQueue = ConcurrentBoundedQueue; @@ -96,11 +96,11 @@ 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 forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions); + nuraft::ptr forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions); public: std::mutex read_request_queue_mutex; @@ -108,7 +108,7 @@ public: std::atomic our_last_committed_log_idx = 0; /// queue of read requests that can be processed after a request with specific session ID and XID is committed - std::unordered_map> read_request_queue; + std::unordered_map> read_request_queue; /// Just allocate some objects, real initialization is done by `intialize method` KeeperDispatcher(); @@ -189,7 +189,7 @@ public: Keeper4LWInfo getKeeper4LWInfo() const; - const KeeperStateMachine & getStateMachine() const + const IKeeperStateMachine & getStateMachine() const { return *server->getKeeperStateMachine(); } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index fd3db6f1032..b25d5e85431 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -116,7 +116,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) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) @@ -128,7 +128,7 @@ KeeperServer::KeeperServer( if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - state_machine = nuraft::cs_new( + state_machine = nuraft::cs_new>( responses_queue_, snapshots_queue_, coordination_settings, @@ -478,7 +478,7 @@ namespace { // Serialize the request for the log entry -nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestForSession & request_for_session) +nuraft::ptr getZooKeeperLogEntry(const KeeperStorageBase::RequestForSession & request_for_session) { DB::WriteBufferFromNuraftBuffer write_buf; DB::writeIntBinary(request_for_session.session_id, write_buf); @@ -486,7 +486,7 @@ nuraft::ptr 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(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(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(); @@ -494,7 +494,7 @@ nuraft::ptr 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"); @@ -502,7 +502,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> entries; for (const auto & request_for_session : requests_for_sessions) @@ -725,7 +725,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)) @@ -735,10 +735,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) @@ -752,19 +752,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(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(); diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index ed58418fe5f..402dcb38742 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -24,7 +24,7 @@ private: CoordinationSettingsPtr coordination_settings; - nuraft::ptr state_machine; + nuraft::ptr state_machine; nuraft::ptr 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 getDeadSessions(); - nuraft::ptr getKeeperStateMachine() const { return state_machine; } + nuraft::ptr getKeeperStateMachine() const { return state_machine; } void forceRecovery(); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a216272a9e1..739a31d89e7 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -68,7 +68,8 @@ namespace return base; } - void writeNode(const KeeperStorage::Node & node, SnapshotVersion version, WriteBuffer & out) + template + void writeNode(const Node & node, SnapshotVersion version, WriteBuffer & out) { writeBinary(node.getData(), out); @@ -96,7 +97,8 @@ namespace } } - void readNode(KeeperStorage::Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map) + template + void readNode(Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map) { String new_data; readBinary(new_data, in); @@ -169,7 +171,8 @@ namespace } } -void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context) +template +void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context) { writeBinary(static_cast(snapshot.version), out); serializeSnapshotMetadata(snapshot.snapshot_meta, out); @@ -179,11 +182,11 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr writeBinary(snapshot.zxid, out); if (keeper_context->digestEnabled()) { - writeBinary(static_cast(KeeperStorage::CURRENT_DIGEST_VERSION), out); + writeBinary(static_cast(Storage::CURRENT_DIGEST_VERSION), out); writeBinary(snapshot.nodes_digest, out); } else - writeBinary(static_cast(KeeperStorage::NO_DIGEST), out); + writeBinary(static_cast(Storage::NO_DIGEST), out); } writeBinary(snapshot.session_id, out); @@ -256,7 +259,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); @@ -277,7 +280,8 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr } } -void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context) +template +void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context) { uint8_t version; readBinary(version, in); @@ -286,7 +290,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) @@ -294,11 +298,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; @@ -361,7 +365,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { std::string path; readBinary(path, in); - KeeperStorage::Node node{}; + typename Storage::Node node{}; readNode(node, in, current_version, storage.acl_map); using enum Coordination::PathMatchResult; @@ -389,7 +393,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT) { LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); - node = KeeperStorage::Node{}; + node = typename Storage::Node{}; } else throw Exception( @@ -417,7 +421,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { auto parent_path = parentNodePath(itr.key); storage.container.updateValue( - parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); }); + parent_path, [version, path = itr.key](typename Storage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); }); } } @@ -457,14 +461,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++; } @@ -487,7 +491,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial } } -KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_) +template +KeeperStorageSnapshot::KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_) : storage(storage_) , snapshot_meta(std::make_shared(up_to_log_idx_, 0, std::make_shared())) , session_id(storage->session_id_counter) @@ -504,8 +509,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 +KeeperStorageSnapshot::KeeperStorageSnapshot( + Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_) : storage(storage_) , snapshot_meta(snapshot_meta_) , session_id(storage->session_id_counter) @@ -522,12 +528,14 @@ KeeperStorageSnapshot::KeeperStorageSnapshot( session_and_auth = storage->session_and_auth; } -KeeperStorageSnapshot::~KeeperStorageSnapshot() +template +KeeperStorageSnapshot::~KeeperStorageSnapshot() { storage->disableSnapshotMode(); } -KeeperSnapshotManager::KeeperSnapshotManager( +template +KeeperSnapshotManager::KeeperSnapshotManager( size_t snapshots_to_keep_, const KeeperContextPtr & keeper_context_, bool compress_snapshots_zstd_, @@ -606,7 +614,8 @@ KeeperSnapshotManager::KeeperSnapshotManager( moveSnapshotsIfNeeded(); } -SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) +template +SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) { ReadBufferFromNuraftBuffer reader(buffer); @@ -634,7 +643,8 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu return {snapshot_file_name, disk}; } -nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk() +template +nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk() { while (!existing_snapshots.empty()) { @@ -655,7 +665,8 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff return nullptr; } -nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const +template +nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const { const auto & [snapshot_path, snapshot_disk] = existing_snapshots.at(up_to_log_idx); WriteBufferFromNuraftBuffer writer; @@ -664,7 +675,8 @@ nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFrom return writer.getBuffer(); } -nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const +template +nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const { std::unique_ptr writer = std::make_unique(); auto * buffer_raw_ptr = writer.get(); @@ -674,13 +686,13 @@ nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(con else compressed_writer = std::make_unique(*writer); - KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context); + KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context); compressed_writer->finalize(); return buffer_raw_ptr->getBuffer(); } - -bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) +template +bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) { static constexpr unsigned char ZSTD_COMPRESSED_MAGIC[4] = {0x28, 0xB5, 0x2F, 0xFD}; @@ -691,7 +703,8 @@ bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) return memcmp(magic_from_buffer, ZSTD_COMPRESSED_MAGIC, 4) == 0; } -SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr buffer) const +template +SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr buffer) const { bool is_zstd_compressed = isZstdCompressed(buffer); @@ -703,14 +716,15 @@ SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuff else compressed_reader = std::make_unique(*reader); - SnapshotDeserializationResult result; - result.storage = std::make_unique(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false); - KeeperStorageSnapshot::deserialize(result, *compressed_reader, keeper_context); + SnapshotDeserializationResult result; + result.storage = std::make_unique(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false); + KeeperStorageSnapshot::deserialize(result, *compressed_reader, keeper_context); result.storage->initializeSystemNodes(); return result; } -SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() +template +SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() { if (existing_snapshots.empty()) return {}; @@ -721,23 +735,27 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() return deserializeSnapshotFromBuffer(buffer); } -DiskPtr KeeperSnapshotManager::getDisk() const +template +DiskPtr KeeperSnapshotManager::getDisk() const { return keeper_context->getSnapshotDisk(); } -DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const +template +DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const { return keeper_context->getLatestSnapshotDisk(); } -void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() +template +void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() { while (existing_snapshots.size() > snapshots_to_keep) removeSnapshot(existing_snapshots.begin()->first); } -void KeeperSnapshotManager::moveSnapshotsIfNeeded() +template +void KeeperSnapshotManager::moveSnapshotsIfNeeded() { /// move snapshots to correct disks @@ -767,7 +785,8 @@ void KeeperSnapshotManager::moveSnapshotsIfNeeded() } -void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) +template +void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) { auto itr = existing_snapshots.find(log_idx); if (itr == existing_snapshots.end()) @@ -777,7 +796,8 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) existing_snapshots.erase(itr); } -SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot) +template +SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & 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); @@ -796,7 +816,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor else compressed_writer = std::make_unique(*writer); - KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context); + KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context); compressed_writer->finalize(); compressed_writer->sync(); @@ -809,4 +829,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor return {snapshot_file_name, disk}; } +template struct KeeperStorageSnapshot; +template class KeeperSnapshotManager; + } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 9bb287b9276..b0a9425881f 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -29,16 +29,19 @@ enum SnapshotVersion : uint8_t static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V5; /// What is stored in binary snapshot +template struct SnapshotDeserializationResult { /// Storage - KeeperStoragePtr storage; + std::unique_ptr storage; /// Snapshot metadata (up_to_log_idx and so on) SnapshotMetadataPtr snapshot_meta; /// Cluster config ClusterConfigPtr cluster_config; }; +struct IKeeperStorageSnapshot {}; + /// In memory keeper snapshot. Keeper Storage based on a hash map which can be /// turned into snapshot mode. This operation is fast and KeeperStorageSnapshot /// class do it in constructor. It also copies iterators from storage hash table @@ -47,21 +50,22 @@ struct SnapshotDeserializationResult /// /// This representation of snapshot have to be serialized into NuRaft /// buffer and send over network or saved to file. -struct KeeperStorageSnapshot +template +struct KeeperStorageSnapshot : IKeeperStorageSnapshot { 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(); - static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context); + static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context); - static void deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context); + static void deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context); - KeeperStorage * storage; + Storage * storage; SnapshotVersion version = CURRENT_SNAPSHOT_VERSION; /// Snapshot metadata @@ -72,11 +76,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 acl_map; /// Cluster config from snapshot, can be empty @@ -93,14 +97,12 @@ struct SnapshotFileInfo DiskPtr disk; }; -using KeeperStorageSnapshotPtr = std::shared_ptr; +using KeeperStorageSnapshotPtr = std::shared_ptr; using CreateSnapshotCallback = std::function; - -using SnapshotMetaAndStorage = std::pair; - /// Class responsible for snapshots serialization and deserialization. Each snapshot /// has it's path on disk and log index. +template class KeeperSnapshotManager { public: @@ -112,18 +114,18 @@ public: size_t storage_tick_time_ = 500); /// Restore storage from latest available snapshot - SnapshotDeserializationResult restoreFromLatestSnapshot(); + SnapshotDeserializationResult restoreFromLatestSnapshot(); /// Compress snapshot and serialize it to buffer - nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const; + nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const; /// Serialize already compressed snapshot to disk (return path) SnapshotFileInfo serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); /// Serialize snapshot directly to disk - SnapshotFileInfo serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot); + SnapshotFileInfo serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot); - SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; + SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; /// Deserialize snapshot with log index up_to_log_idx from disk into compressed nuraft buffer. nuraft::ptr deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index b6185fa2e36..a3c9378a173 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -40,7 +40,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -KeeperStateMachine::KeeperStateMachine( +IKeeperStateMachine::IKeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const CoordinationSettingsPtr & coordination_settings_, @@ -50,12 +50,6 @@ KeeperStateMachine::KeeperStateMachine( const std::string & superdigest_) : commit_callback(commit_callback_) , coordination_settings(coordination_settings_) - , snapshot_manager( - coordination_settings->snapshots_to_keep, - keeper_context_, - coordination_settings->compress_snapshots_with_zstd_format, - superdigest_, - coordination_settings->dead_session_check_period_ms.totalMilliseconds()) , responses_queue(responses_queue_) , snapshots_queue(snapshots_queue_) , min_request_size_to_cache(coordination_settings_->min_request_size_for_cache) @@ -67,6 +61,32 @@ KeeperStateMachine::KeeperStateMachine( { } +template +KeeperStateMachine::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( + coordination_settings->snapshots_to_keep, + keeper_context_, + coordination_settings->compress_snapshots_with_zstd_format, + superdigest_, + coordination_settings->dead_session_check_period_ms.totalMilliseconds()) +{ +} + namespace { @@ -77,7 +97,8 @@ bool isLocalDisk(const IDisk & disk) } -void KeeperStateMachine::init() +template +void KeeperStateMachine::init() { /// Do everything without mutexes, no other threads exist. LOG_DEBUG(log, "Totally have {} snapshots", snapshot_manager.totalSnapshots()); @@ -129,7 +150,7 @@ void KeeperStateMachine::init() } if (!storage) - storage = std::make_unique( + storage = std::make_unique( coordination_settings->dead_session_check_period_ms.totalMilliseconds(), superdigest, keeper_context); } @@ -137,12 +158,12 @@ namespace { void assertDigest( - const KeeperStorage::Digest & first, - const KeeperStorage::Digest & second, + const KeeperStorageBase::Digest & first, + const KeeperStorageBase::Digest & second, const Coordination::ZooKeeperRequest & request, bool committing) { - if (!KeeperStorage::checkDigest(first, second)) + if (!KeeperStorageBase::checkDigest(first, second)) { LOG_FATAL( &Poco::Logger::get("KeeperStateMachine"), @@ -160,7 +181,8 @@ void assertDigest( } -nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data) +template +nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data, /*final=*/false); if (!request_for_session->zxid) @@ -175,10 +197,10 @@ nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nur return result; } -std::shared_ptr KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version) +std::shared_ptr IKeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version) { ReadBufferFromNuraftBuffer buffer(data); - auto request_for_session = std::make_shared(); + auto request_for_session = std::make_shared(); readIntBinary(request_for_session->session_id, buffer); int32_t length; @@ -251,7 +273,7 @@ std::shared_ptr 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); } @@ -267,7 +289,8 @@ std::shared_ptr KeeperStateMachine::parseReque return request_for_session; } -bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session) +template +bool KeeperStateMachine::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) @@ -301,10 +324,11 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req return true; } -void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session) +template +void KeeperStateMachine::reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) { std::lock_guard _(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); @@ -314,8 +338,9 @@ void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& req } } -KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( - const KeeperStorage::RequestForSession & request_for_session) +template +KeeperStorageBase::ResponseForSession KeeperStateMachine::processReconfiguration( + const KeeperStorageBase::RequestForSession & request_for_session) { ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); @@ -324,7 +349,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(); res->xid = request.xid; @@ -381,7 +406,8 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( return { session_id, std::move(response) }; } -nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) +template +nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data, true); if (!request_for_session->zxid) @@ -392,7 +418,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n if (!keeper_context->local_logs_preprocessed) preprocess(*request_for_session); - auto try_push = [this](const KeeperStorage::ResponseForSession& response) + auto try_push = [this](const KeeperStorageBase::ResponseForSession& response) { if (!responses_queue.push(response)) { @@ -412,7 +438,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n std::shared_ptr response = std::make_shared(); 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; @@ -431,7 +457,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n } std::lock_guard 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) try_push(response_for_session); @@ -448,7 +474,8 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n return nullptr; } -bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) +template +bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx()); nuraft::ptr latest_snapshot_ptr; @@ -475,7 +502,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { /// deserialize and apply snapshot to storage std::lock_guard lock(storage_and_responses_lock); - SnapshotDeserializationResult snapshot_deserialization_result; + SnapshotDeserializationResult snapshot_deserialization_result; if (latest_snapshot_ptr) snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr); else @@ -496,7 +523,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) } -void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr & new_conf) +void IKeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr & new_conf) { std::lock_guard lock(cluster_config_lock); auto tmp = new_conf->serialize(); @@ -504,7 +531,7 @@ void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr +void KeeperStateMachine::rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) { if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; @@ -525,7 +553,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 +void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) { if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; @@ -533,14 +562,15 @@ void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSe storage->rollbackRequest(request_for_session.zxid, allow_missing); } -nuraft::ptr KeeperStateMachine::last_snapshot() +nuraft::ptr 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::handler_type & when_done) +template +void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_result::handler_type & when_done) { LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx()); @@ -549,12 +579,13 @@ 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. std::lock_guard lock(storage_and_responses_lock); - snapshot_task.snapshot = std::make_shared(storage.get(), snapshot_meta_copy, getClusterConfig()); + snapshot_task.snapshot = std::make_shared>(storage.get(), snapshot_meta_copy, getClusterConfig()); } /// create snapshot task for background execution (in snapshot thread) - snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot) + snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot_) { + auto * snapshot = typeid_cast *>(snapshot_.get()); nuraft::ptr exception(nullptr); bool ret = true; try @@ -601,7 +632,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res /// Turn off "snapshot mode" and clear outdate part of storage state storage->clearGarbageAfterSnapshot(); LOG_TRACE(log, "Cleared garbage after snapshot"); - snapshot.reset(); + snapshot_.reset(); } } catch (...) @@ -638,7 +669,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 +void KeeperStateMachine::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); @@ -703,7 +735,7 @@ static int bufferFromFile(Poco::Logger * log, const std::string & path, nuraft:: 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 & data_out, bool & is_last_obj) { LOG_DEBUG(log, "Reading snapshot {} obj_id {}", s.get_last_log_idx(), obj_id); @@ -743,7 +775,8 @@ int KeeperStateMachine::read_logical_snp_obj( return 1; } -void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session) +template +void KeeperStateMachine::processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) { /// Pure local request, just process it with storage std::lock_guard lock(storage_and_responses_lock); @@ -754,103 +787,120 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id); } -void KeeperStateMachine::shutdownStorage() +template +void KeeperStateMachine::shutdownStorage() { std::lock_guard lock(storage_and_responses_lock); storage->finalize(); } -std::vector KeeperStateMachine::getDeadSessions() +template +std::vector KeeperStateMachine::getDeadSessions() { std::lock_guard lock(storage_and_responses_lock); return storage->getDeadSessions(); } -int64_t KeeperStateMachine::getNextZxid() const +template +int64_t KeeperStateMachine::getNextZxid() const { std::lock_guard lock(storage_and_responses_lock); return storage->getNextZXID(); } -KeeperStorage::Digest KeeperStateMachine::getNodesDigest() const +template +KeeperStorageBase::Digest KeeperStateMachine::getNodesDigest() const { std::lock_guard lock(storage_and_responses_lock); return storage->getNodesDigest(false); } -uint64_t KeeperStateMachine::getLastProcessedZxid() const +template +uint64_t KeeperStateMachine::getLastProcessedZxid() const { std::lock_guard lock(storage_and_responses_lock); return storage->getZXID(); } -uint64_t KeeperStateMachine::getNodesCount() const +template +uint64_t KeeperStateMachine::getNodesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getNodesCount(); } -uint64_t KeeperStateMachine::getTotalWatchesCount() const +template +uint64_t KeeperStateMachine::getTotalWatchesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getTotalWatchesCount(); } -uint64_t KeeperStateMachine::getWatchedPathsCount() const +template +uint64_t KeeperStateMachine::getWatchedPathsCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getWatchedPathsCount(); } -uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const +template +uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getSessionsWithWatchesCount(); } -uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const +template +uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getTotalEphemeralNodesCount(); } -uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const +template +uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getSessionWithEphemeralNodesCount(); } -void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const +template +void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const { std::lock_guard lock(storage_and_responses_lock); storage->dumpWatches(buf); } -void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const +template +void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { std::lock_guard lock(storage_and_responses_lock); storage->dumpWatchesByPath(buf); } -void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const +template +void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { std::lock_guard lock(storage_and_responses_lock); storage->dumpSessionsAndEphemerals(buf); } -uint64_t KeeperStateMachine::getApproximateDataSize() const +template +uint64_t KeeperStateMachine::getApproximateDataSize() const { std::lock_guard lock(storage_and_responses_lock); return storage->getApproximateDataSize(); } -uint64_t KeeperStateMachine::getKeyArenaSize() const +template +uint64_t KeeperStateMachine::getKeyArenaSize() const { std::lock_guard lock(storage_and_responses_lock); return storage->getArenaDataSize(); } -uint64_t KeeperStateMachine::getLatestSnapshotBufSize() const +template +uint64_t KeeperStateMachine::getLatestSnapshotBufSize() const { std::lock_guard lock(snapshots_lock); if (latest_snapshot_buf) @@ -858,7 +908,7 @@ uint64_t KeeperStateMachine::getLatestSnapshotBufSize() const return 0; } -ClusterConfigPtr KeeperStateMachine::getClusterConfig() const +ClusterConfigPtr IKeeperStateMachine::getClusterConfig() const { std::lock_guard lock(cluster_config_lock); if (cluster_config) @@ -870,11 +920,15 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const return nullptr; } -void KeeperStateMachine::recalculateStorageStats() +template +void KeeperStateMachine::recalculateStorageStats() { std::lock_guard lock(storage_and_responses_lock); LOG_INFO(log, "Recalculating storage stats"); storage->recalculateStats(); LOG_INFO(log, "Done recalculating storage stats"); } + +template class KeeperStateMachine; + } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index aad5d3aafd4..9ae3905cddd 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -12,27 +12,25 @@ namespace DB { -using ResponsesQueue = ConcurrentBoundedQueue; +using ResponsesQueue = ConcurrentBoundedQueue; using SnapshotsQueue = ConcurrentBoundedQueue; -/// 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; + using CommitCallback = std::function; - KeeperStateMachine( + IKeeperStateMachine( 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_ = ""); + CommitCallback commit_callback_, + const std::string & superdisgest_); /// Read state from the latest snapshot - void init(); + virtual void init() = 0; enum ZooKeeperLogSerializationVersion { @@ -49,89 +47,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 parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr); + std::shared_ptr 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 pre_commit(uint64_t log_idx, nuraft::buffer & data) override; - - nuraft::ptr 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 & 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 last_committed_idx; } - /// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state. - bool apply_snapshot(nuraft::snapshot & s) override; - nuraft::ptr last_snapshot() override; /// Create new snapshot from current state. - void create_snapshot(nuraft::snapshot & s, nuraft::async_result::handler_type & when_done) override; + void create_snapshot(nuraft::snapshot & s, nuraft::async_result::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 & 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 getDeadSessions(); + virtual std::vector 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 getLatestSnapshotBufSize() 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 getLatestSnapshotBufSize() 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. @@ -141,12 +116,7 @@ private: CoordinationSettingsPtr coordination_settings; - /// Main state machine logic - KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock); - /// Save/Load and Serialize/Deserialize logic for snapshots. - KeeperSnapshotManager snapshot_manager; - /// Put processed responses into this queue ResponsesQueue & responses_queue; @@ -163,7 +133,7 @@ private: /// for request. mutable std::mutex storage_and_responses_lock; - std::unordered_map>> parsed_request_cache; + std::unordered_map>> 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 @@ -188,8 +158,105 @@ 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 +class KeeperStateMachine : public IKeeperStateMachine +{ +public: + /// using CommitCallback = std::function; + + 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 pre_commit(uint64_t log_idx, nuraft::buffer & data) override; + + nuraft::ptr 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::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 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 getLatestSnapshotBufSize() const override; + + void recalculateStorageStats() override; + + void reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) override; + +private: + /// Main state machine logic + std::unique_ptr storage; //TSA_PT_GUARDED_BY(storage_and_responses_lock); + + /// Save/Load and Serialize/Deserialize logic for snapshots. + KeeperSnapshotManager snapshot_manager; + + KeeperStorageBase::ResponseForSession processReconfiguration( + const KeeperStorageBase::RequestForSession& request_for_session) + TSA_REQUIRES(storage_and_responses_lock) override; }; } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index a4bb65a2a72..e925067b6a3 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -61,10 +61,11 @@ String getSHA1(const String & userdata) return String{digest_id.begin(), digest_id.end()}; } +template bool fixupACL( const std::vector & request_acls, int64_t session_id, - const KeeperStorage::UncommittedState & uncommitted_state, + const UncommittedState & uncommitted_state, std::vector & result_acls) { if (request_acls.empty()) @@ -77,7 +78,7 @@ bool fixupACL( { uncommitted_state.forEachAuthInSession( session_id, - [&](const KeeperStorage::AuthID & auth_id) + [&](const KeeperStorageBase::AuthID & auth_id) { valid_found = true; Coordination::ACL new_acl = request_acl; @@ -108,10 +109,10 @@ bool fixupACL( return valid_found; } -KeeperStorage::ResponsesForSessions processWatchesImpl( - const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type) +KeeperStorageBase::ResponsesForSessions processWatchesImpl( + const String & path, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches, Coordination::Event event_type) { - KeeperStorage::ResponsesForSessions result; + KeeperStorageBase::ResponsesForSessions result; auto watch_it = watches.find(path); if (watch_it != watches.end()) { @@ -122,7 +123,7 @@ KeeperStorage::ResponsesForSessions processWatchesImpl( watch_response->type = event_type; watch_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) - result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_response}); + result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_response}); watches.erase(watch_it); } @@ -158,7 +159,7 @@ KeeperStorage::ResponsesForSessions processWatchesImpl( watch_list_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) - result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_list_response}); + result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_list_response}); list_watches.erase(watch_it); } @@ -193,31 +194,31 @@ uint64_t calculateDigest(std::string_view path, std::string_view data, const Coo } -void KeeperStorage::Node::setData(String new_data) +void KeeperMemNode::setData(String new_data) { size_bytes = size_bytes - data.size() + new_data.size(); data = std::move(new_data); } -void KeeperStorage::Node::addChild(StringRef child_path, bool update_size) +void KeeperMemNode::addChild(StringRef child_path, bool update_size) { if (update_size) [[likely]] size_bytes += sizeof child_path; children.insert(child_path); } -void KeeperStorage::Node::removeChild(StringRef child_path) +void KeeperMemNode::removeChild(StringRef child_path) { size_bytes -= sizeof child_path; children.erase(child_path); } -void KeeperStorage::Node::invalidateDigestCache() const +void KeeperMemNode::invalidateDigestCache() const { cached_digest.reset(); } -UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const +UInt64 KeeperMemNode::getDigest(const std::string_view path) const { if (!cached_digest) cached_digest = calculateDigest(path, data, stat); @@ -225,7 +226,7 @@ UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const return *cached_digest; }; -void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) +void KeeperMemNode::shallowCopy(const KeeperMemNode & other) { stat = other.stat; seq_num = other.seq_num; @@ -233,14 +234,15 @@ void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) cached_digest = other.cached_digest; } -void KeeperStorage::Node::recalculateSize() +void KeeperMemNode::recalculateSize() { - size_bytes = sizeof(Node); + size_bytes = sizeof(KeeperMemNode); size_bytes += children.size() * sizeof(decltype(children)::value_type); size_bytes += data.size(); } -KeeperStorage::KeeperStorage( +template +KeeperStorage::KeeperStorage( int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, const bool initialize_system_nodes) : session_expiry_queue(tick_time_ms), keeper_context(keeper_context_), superdigest(superdigest_) { @@ -252,7 +254,8 @@ KeeperStorage::KeeperStorage( initializeSystemNodes(); } -void KeeperStorage::initializeSystemNodes() +template +void KeeperStorage::initializeSystemNodes() { if (initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage system nodes initialized twice"); @@ -314,12 +317,13 @@ struct Overloaded : Ts... template Overloaded(Ts...) -> Overloaded; -std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const +template +std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const { if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; - auto node = std::make_shared(); + auto node = std::make_shared::Node>(); node->shallowCopy(committed_node); return node; } @@ -327,7 +331,8 @@ std::shared_ptr KeeperStorage::UncommittedState::tryGetNode return nullptr; } -void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) +template +void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) { assert(!delta.path.empty()); if (!nodes.contains(delta.path)) @@ -374,7 +379,8 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) delta.operation); } -void KeeperStorage::UncommittedState::addDelta(Delta new_delta) +template +void KeeperStorage::UncommittedState::addDelta(Delta new_delta) { const auto & added_delta = deltas.emplace_back(std::move(new_delta)); @@ -390,13 +396,15 @@ void KeeperStorage::UncommittedState::addDelta(Delta new_delta) } } -void KeeperStorage::UncommittedState::addDeltas(std::vector new_deltas) +template +void KeeperStorage::UncommittedState::addDeltas(std::vector new_deltas) { for (auto & delta : new_deltas) addDelta(std::move(delta)); } -void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) +template +void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) { assert(deltas.empty() || deltas.front().zxid >= commit_zxid); @@ -453,7 +461,8 @@ void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) } } -void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) +template +void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) { // we can only rollback the last zxid (if there is any) // if there is a delta with a larger zxid, we have invalid state @@ -545,7 +554,8 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) } } -std::shared_ptr KeeperStorage::UncommittedState::getNode(StringRef path) const +template +std::shared_ptr KeeperStorage::UncommittedState::getNode(StringRef path) const { if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.node; @@ -553,7 +563,8 @@ std::shared_ptr KeeperStorage::UncommittedState::getNode(St return tryGetNodeFromStorage(path); } -Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const +template +Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const { if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.acls; @@ -565,7 +576,8 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) cons return storage.acl_map.convertNumber(node_it->value.acl_id); } -void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, std::function func) const +template +void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, std::function func) const { const auto call_for_each_auth = [&func](const auto & auth_ids) { @@ -604,7 +616,8 @@ namespace } -void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_log_idx) +template +void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_log_idx) { std::unordered_set zxids_to_apply; for (const auto & transaction : uncommitted_transactions) @@ -630,7 +643,8 @@ void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_lo } } -Coordination::Error KeeperStorage::commit(int64_t commit_zxid) +template +Coordination::Error KeeperStorage::commit(int64_t commit_zxid) { // Deltas are added with increasing ZXIDs // If there are no deltas for the commit_zxid (e.g. read requests), we instantly return @@ -644,7 +658,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) auto result = std::visit( [&, &path = delta.path](DeltaType & operation) -> Coordination::Error { - if constexpr (std::same_as) + if constexpr (std::same_as) { if (!createNode( path, @@ -656,7 +670,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { auto node_it = container.find(path); if (node_it == container.end()) @@ -671,14 +685,14 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { if (!removeNode(path, operation.version)) onStorageInconsistency(); return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { auto node_it = container.find(path); if (node_it == container.end()) @@ -692,18 +706,18 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) uint64_t acl_id = acl_map.convertACLs(operation.acls); acl_map.addUsage(acl_id); - container.updateValue(path, [acl_id](KeeperStorage::Node & node) { node.acl_id = acl_id; }); + container.updateValue(path, [acl_id](Node & node) { node.acl_id = acl_id; }); return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) return operation.error; - else if constexpr (std::same_as) + else if constexpr (std::same_as) { finish_subdelta = true; return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { session_and_auth[operation.session_id].emplace_back(std::move(operation.auth_id)); return Coordination::Error::ZOK; @@ -726,7 +740,8 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) return Coordination::Error::ZOK; } -bool KeeperStorage::createNode( +template +bool KeeperStorage::createNode( const std::string & path, String data, const Coordination::Stat & stat, @@ -745,7 +760,7 @@ bool KeeperStorage::createNode( if (container.contains(path)) return false; - KeeperStorage::Node created_node; + Node created_node; uint64_t acl_id = acl_map.convertACLs(node_acls); acl_map.addUsage(acl_id); @@ -759,7 +774,7 @@ bool KeeperStorage::createNode( auto child_path = getBaseNodeName(map_key->getKey()); container.updateValue( parent_path, - [child_path](KeeperStorage::Node & parent) + [child_path](Node & parent) { parent.addChild(child_path); chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); @@ -770,7 +785,8 @@ bool KeeperStorage::createNode( return true; }; -bool KeeperStorage::removeNode(const std::string & path, int32_t version) +template +bool KeeperStorage::removeNode(const std::string & path, int32_t version) { auto node_it = container.find(path); if (node_it == container.end()) @@ -787,7 +803,7 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t version) container.updateValue( parentNodePath(path), - [child_basename = getBaseNodeName(node_it->key)](KeeperStorage::Node & parent) + [child_basename = getBaseNodeName(node_it->key)](Node & parent) { parent.removeChild(child_basename); chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); @@ -800,54 +816,62 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t version) return true; } +template struct KeeperStorageRequestProcessor { Coordination::ZooKeeperRequestPtr zk_request; explicit KeeperStorageRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) : zk_request(zk_request_) { } - virtual Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const = 0; - virtual std::vector - preprocess(KeeperStorage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const + + virtual Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const = 0; + + virtual std::vector + preprocess(Storage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const { return {}; } // process the request using locally committed data virtual Coordination::ZooKeeperResponsePtr - processLocal(KeeperStorage & /*storage*/, int64_t /*zxid*/) const + processLocal(Storage & /*storage*/, int64_t /*zxid*/) const { throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Cannot process the request locally"}; } - virtual KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & /*watches*/, KeeperStorage::Watches & /*list_watches*/) const + virtual Storage::ResponsesForSessions + processWatches(Storage::Watches & /*watches*/, Storage::Watches & /*list_watches*/) const { return {}; } - virtual bool checkAuth(KeeperStorage & /*storage*/, int64_t /*session_id*/, bool /*is_local*/) const { return true; } + + virtual bool checkAuth(Storage & /*storage*/, int64_t /*session_id*/, bool /*is_local*/) const { return true; } virtual ~KeeperStorageRequestProcessor() = default; }; -struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + Coordination::ZooKeeperResponsePtr - process(KeeperStorage & /* storage */, int64_t /* zxid */) const override + process(Storage & /* storage */, int64_t /* zxid */) const override { - return zk_request->makeResponse(); + return this->zk_request->makeResponse(); } }; -struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + Coordination::ZooKeeperResponsePtr - process(KeeperStorage & /* storage */, int64_t /* zxid */) const override + process(Storage & /* storage */, int64_t /* zxid */) const override { - auto response = zk_request->makeResponse(); + auto response = this->zk_request->makeResponse(); dynamic_cast(*response).path - = dynamic_cast(*zk_request).path; + = dynamic_cast(*this->zk_request).path; return response; } }; @@ -855,7 +879,8 @@ struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProc namespace { -Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_local) +template +Coordination::ACLs getNodeACLs(Storage & storage, StringRef path, bool is_local) { if (is_local) { @@ -883,7 +908,8 @@ void handleSystemNodeModification(const KeeperContext & keeper_context, std::str } -bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session_id, bool is_local) +template +bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session_id, bool is_local) { const auto node_acls = getNodeACLs(*this, path, is_local); if (node_acls.empty()) @@ -910,7 +936,8 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session return false; } -void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) +template +void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) { auto ephemerals_it = ephemerals.find(session_id); if (ephemerals_it == ephemerals.end()) @@ -921,43 +948,44 @@ void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::strin ephemerals.erase(ephemerals_it); } -struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorageBase::ResponsesForSessions + processWatches(Storage::Watches & watches, Storage::Watches & list_watches) const override { - return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED); + return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED); } - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - auto path = zk_request->getPath(); + auto path = this->zk_request->getPath(); return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); } - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperCreateRequest); - Coordination::ZooKeeperCreateRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperCreateRequest & request = dynamic_cast(*this->zk_request); - std::vector new_deltas; + std::vector new_deltas; auto parent_path = parentNodePath(request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); if (parent_node == nullptr) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; else if (parent_node->stat.ephemeralOwner != 0) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}}; std::string path_created = request.path; if (request.is_sequential) { if (request.not_exists) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; auto seq_num = parent_node->seq_num; @@ -973,30 +1001,30 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr auto error_msg = fmt::format("Trying to create a node inside the internal Keeper path ({}) which is not allowed. Path: {}", keeper_system_path, path_created); handleSystemNodeModification(keeper_context, error_msg); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } if (storage.uncommitted_state.getNode(path_created)) { - if (zk_request->getOpNum() == Coordination::OpNum::CreateIfNotExists) + if (this->zk_request->getOpNum() == Coordination::OpNum::CreateIfNotExists) return new_deltas; - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; } if (getBaseNodeName(path_created).size == 0) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; Coordination::ACLs node_acls; if (!fixupACL(request.acls, session_id, storage.uncommitted_state, node_acls)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; if (request.is_ephemeral) storage.ephemerals[session_id].emplace(path_created); int32_t parent_cversion = request.parent_cversion; - auto parent_update = [parent_cversion, zxid](KeeperStorage::Node & node) + auto parent_update = [parent_cversion, zxid](Storage::Node & node) { /// Increment sequential number even if node is not sequential ++node.seq_num; @@ -1010,7 +1038,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr ++node.stat.numChildren; }; - new_deltas.emplace_back(std::string{parent_path}, zxid, KeeperStorage::UpdateNodeDelta{std::move(parent_update)}); + new_deltas.emplace_back(std::string{parent_path}, zxid, typename Storage::UpdateNodeDelta{std::move(parent_update)}); Coordination::Stat stat; stat.czxid = zxid; @@ -1028,20 +1056,20 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr new_deltas.emplace_back( std::move(path_created), zxid, - KeeperStorage::CreateNodeDelta{stat, request.is_sequential, std::move(node_acls), request.data}); + typename Storage::CreateNodeDelta{stat, request.is_sequential, std::move(node_acls), request.data}); digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); if (storage.uncommitted_state.deltas.begin()->zxid != zxid) { - response.path_created = zk_request->getPath(); + response.path_created = this->zk_request->getPath(); response.error = Coordination::Error::ZOK; return response_ptr; } @@ -1057,7 +1085,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr deltas.begin(), deltas.end(), [zxid](const auto & delta) - { return delta.zxid == zxid && std::holds_alternative(delta.operation); }); + { return delta.zxid == zxid && std::holds_alternative(delta.operation); }); response.path_created = create_delta_it->path; response.error = Coordination::Error::ZOK; @@ -1065,37 +1093,38 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr } }; -struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); - Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperGetRequest & request = dynamic_cast(*this->zk_request); if (request.path == Coordination::keeper_api_feature_flags_path || request.path == Coordination::keeper_config_path) return {}; if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; return {}; } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperGetRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1134,40 +1163,42 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); return processImpl(storage, zxid); } }; -struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(parentNodePath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); + return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); - Coordination::ZooKeeperRemoveRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperRemoveRequest & request = dynamic_cast(*this->zk_request); - std::vector new_deltas; + std::vector new_deltas; if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path); handleSystemNodeModification(keeper_context, error_msg); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } const auto update_parent_pzxid = [&]() @@ -1179,9 +1210,9 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr new_deltas.emplace_back( std::string{parent_path}, zxid, - KeeperStorage::UpdateNodeDelta + typename Storage::UpdateNodeDelta { - [zxid](KeeperStorage::Node & parent) + [zxid](Storage::Node & parent) { if (parent.stat.pzxid < zxid) parent.stat.pzxid = zxid; @@ -1196,12 +1227,12 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr { if (request.restored_from_zookeeper_log) update_parent_pzxid(); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; } else if (request.version != -1 && request.version != node->stat.version) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; else if (node->stat.numChildren != 0) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; if (request.restored_from_zookeeper_log) update_parent_pzxid(); @@ -1209,13 +1240,13 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr new_deltas.emplace_back( std::string{parentNodePath(request.path)}, zxid, - KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent) + typename Storage::UpdateNodeDelta{[](typename Storage::Node & parent) { ++parent.stat.cversion; --parent.stat.numChildren; }}); - new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->stat.ephemeralOwner}); + new_deltas.emplace_back(request.path, zxid, typename Storage::RemoveNodeDelta{request.version, node->stat.ephemeralOwner}); if (node->stat.ephemeralOwner != 0) storage.unregisterEphemeralPath(node->stat.ephemeralOwner, request.path); @@ -1225,44 +1256,45 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); response.error = storage.commit(zxid); return response_ptr; } - KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorageBase::ResponsesForSessions + processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override { - return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); + return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); } }; -struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); - Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*this->zk_request); if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; return {}; } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1291,55 +1323,57 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); return processImpl(storage, zxid); } }; -struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Write, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Write, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperSetRequest); - Coordination::ZooKeeperSetRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperSetRequest & request = dynamic_cast(*this->zk_request); - std::vector new_deltas; + std::vector new_deltas; if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path); handleSystemNodeModification(keeper_context, error_msg); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; auto node = storage.uncommitted_state.getNode(request.path); if (request.version != -1 && request.version != node->stat.version) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; new_deltas.emplace_back( request.path, zxid, - KeeperStorage::UpdateNodeDelta{ - [zxid, data = request.data, time](KeeperStorage::Node & value) + typename Storage::UpdateNodeDelta{ + [zxid, data = request.data, time](typename Storage::Node & value) { value.stat.version++; value.stat.mzxid = zxid; @@ -1352,9 +1386,9 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce new_deltas.emplace_back( parentNodePath(request.path).toString(), zxid, - KeeperStorage::UpdateNodeDelta + typename Storage::UpdateNodeDelta { - [](KeeperStorage::Node & parent) + [](Storage::Node & parent) { parent.stat.cversion++; } @@ -1365,13 +1399,13 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { auto & container = storage.container; - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperSetRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperSetRequest & request = dynamic_cast(*this->zk_request); if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) { @@ -1389,40 +1423,41 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce return response_ptr; } - KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorageBase::ResponsesForSessions + processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override { - return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); + return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); } }; -struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); - Coordination::ZooKeeperListRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperListRequest & request = dynamic_cast(*this->zk_request); if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; return {}; } - template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperListRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperListRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1487,63 +1522,64 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); return processImpl(storage, zxid); } }; -struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor { explicit KeeperStorageCheckRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequestProcessor(zk_request_) + : KeeperStorageRequestProcessor(zk_request_) { - check_not_exists = zk_request->getOpNum() == Coordination::OpNum::CheckNotExists; + check_not_exists = this->zk_request->getOpNum() == Coordination::OpNum::CheckNotExists; } - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - auto path = zk_request->getPath(); + auto path = this->zk_request->getPath(); return storage.checkACL(check_not_exists ? parentNodePath(path) : path, Coordination::ACL::Read, session_id, is_local); } - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); - Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*this->zk_request); auto node = storage.uncommitted_state.getNode(request.path); if (check_not_exists) { if (node && (request.version == -1 || request.version == node->stat.version)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; } else { if (!node) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; if (request.version != -1 && request.version != node->stat.version) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; } return {}; } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1585,12 +1621,12 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); return processImpl(storage, zxid); @@ -1601,55 +1637,56 @@ private: }; -struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Admin, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override { - Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*this->zk_request); if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path); handleSystemNodeModification(keeper_context, error_msg); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } auto & uncommitted_state = storage.uncommitted_state; if (!uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; auto node = uncommitted_state.getNode(request.path); if (request.version != -1 && request.version != node->stat.aversion) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; Coordination::ACLs node_acls; if (!fixupACL(request.acls, session_id, uncommitted_state, node_acls)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; - std::vector new_deltas + std::vector new_deltas { { request.path, zxid, - KeeperStorage::SetACLDelta{std::move(node_acls), request.version} + typename Storage::SetACLDelta{std::move(node_acls), request.version} }, { request.path, zxid, - KeeperStorage::UpdateNodeDelta + typename Storage::UpdateNodeDelta { - [](KeeperStorage::Node & n) { ++n.stat.aversion; } + [](typename Storage::Node & n) { ++n.stat.aversion; } } } }; @@ -1659,11 +1696,11 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*this->zk_request); if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) { @@ -1681,32 +1718,33 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr } }; -struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; return {}; } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1735,23 +1773,24 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } }; -struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor { using OperationType = Coordination::ZooKeeperMultiRequest::OperationType; std::optional operation_type; - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { for (const auto & concrete_request : concrete_requests) if (!concrete_request->checkAuth(storage, session_id, is_local)) @@ -1759,11 +1798,11 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return true; } - std::vector concrete_requests; + std::vector>> concrete_requests; explicit KeeperStorageMultiRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequestProcessor(zk_request_) + : KeeperStorageRequestProcessor(zk_request_) { - Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*this->zk_request); concrete_requests.reserve(request.requests.size()); const auto check_operation_type = [&](OperationType type) @@ -1781,34 +1820,34 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro case Coordination::OpNum::Create: case Coordination::OpNum::CreateIfNotExists: check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Remove: check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Set: check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Check: case Coordination::OpNum::CheckNotExists: check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Get: check_operation_type(OperationType::Read); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Exists: check_operation_type(OperationType::Read); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::List: case Coordination::OpNum::FilteredList: case Coordination::OpNum::SimpleList: check_operation_type(OperationType::Read); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; default: throw DB::Exception( @@ -1821,8 +1860,8 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro assert(request.requests.empty() || operation_type.has_value()); } - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperMultiRequest); std::vector response_errors; @@ -1834,7 +1873,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro if (!new_deltas.empty()) { - if (auto * error = std::get_if(&new_deltas.back().operation); + if (auto * error = std::get_if(&new_deltas.back().operation); error && *operation_type == OperationType::Write) { storage.uncommitted_state.rollback(zxid); @@ -1845,10 +1884,10 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro response_errors.push_back(Coordination::Error::ZRUNTIMEINCONSISTENCY); } - return {KeeperStorage::Delta{zxid, KeeperStorage::FailedMultiDelta{std::move(response_errors)}}}; + return {typename Storage::Delta{zxid, typename Storage::FailedMultiDelta{std::move(response_errors)}}}; } } - new_deltas.emplace_back(zxid, KeeperStorage::SubDeltaEnd{}); + new_deltas.emplace_back(zxid, typename Storage::SubDeltaEnd{}); response_errors.push_back(Coordination::Error::ZOK); // manually add deltas so that the result of previous request in the transaction is used in the next request @@ -1860,15 +1899,15 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return {}; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); auto & deltas = storage.uncommitted_state.deltas; // the deltas will have at least SubDeltaEnd or FailedMultiDelta assert(!deltas.empty()); - if (auto * failed_multi = std::get_if(&deltas.front().operation)) + if (auto * failed_multi = std::get_if(&deltas.front().operation)) { for (size_t i = 0; i < concrete_requests.size(); ++i) { @@ -1890,10 +1929,10 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); for (size_t i = 0; i < concrete_requests.size(); ++i) @@ -1905,10 +1944,10 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorageBase::ResponsesForSessions + processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override { - KeeperStorage::ResponsesForSessions result; + typename Storage::ResponsesForSessions result; for (const auto & generic_request : concrete_requests) { auto responses = generic_request->processWatches(watches, list_watches); @@ -1918,47 +1957,50 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro } }; -struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - Coordination::ZooKeeperResponsePtr process(KeeperStorage &, int64_t) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + Coordination::ZooKeeperResponsePtr process(Storage &, int64_t) const override { throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Called process on close request"); } }; -struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { - Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast(*zk_request); - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast(*this->zk_request); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); if (auth_request.scheme != "digest" || std::count(auth_request.data.begin(), auth_request.data.end(), ':') != 1) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; - std::vector new_deltas; - auto auth_digest = KeeperStorage::generateDigest(auth_request.data); + std::vector new_deltas; + auto auth_digest = Storage::generateDigest(auth_request.data); if (auth_digest == storage.superdigest) { - KeeperStorage::AuthID auth{"super", ""}; - new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(auth)}); + typename Storage::AuthID auth{"super", ""}; + new_deltas.emplace_back(zxid, typename Storage::AddAuthDelta{session_id, std::move(auth)}); } else { - KeeperStorage::AuthID new_auth{auth_request.scheme, auth_digest}; + typename Storage::AuthID new_auth{auth_request.scheme, auth_digest}; if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return new_auth == auth_id; })) - new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(new_auth)}); + new_deltas.emplace_back(zxid, typename Storage::AddAuthDelta{session_id, std::move(new_auth)}); } return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) @@ -1968,7 +2010,8 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc } }; -void KeeperStorage::finalize() +template +void KeeperStorage::finalize() { if (finalized) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage already finalized"); @@ -1983,16 +2026,17 @@ void KeeperStorage::finalize() session_expiry_queue.clear(); } -bool KeeperStorage::isFinalized() const +template +bool KeeperStorage::isFinalized() const { return finalized; } - +template class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable { public: - using Creator = std::function; + using Creator = std::function>(const Coordination::ZooKeeperRequestPtr &)>; using OpNumToRequest = std::unordered_map; static KeeperStorageRequestProcessorsFactory & instance() @@ -2001,7 +2045,7 @@ public: return factory; } - KeeperStorageRequestProcessorPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const + std::shared_ptr> get(const Coordination::ZooKeeperRequestPtr & zk_request) const { auto request_it = op_num_to_request.find(zk_request->getOpNum()); if (request_it == op_num_to_request.end()) @@ -2021,39 +2065,41 @@ private: KeeperStorageRequestProcessorsFactory(); }; -template -void registerKeeperRequestProcessor(KeeperStorageRequestProcessorsFactory & factory) +template +void registerKeeperRequestProcessor(Factory & factory) { factory.registerRequest( num, [](const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared(zk_request); }); } -KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() +template +KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() { - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); } -UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const +template +UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const { if (!keeper_context->digestEnabled()) return current_digest; @@ -2088,7 +2134,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec auto updated_node_it = updated_nodes.find(delta.path); if (updated_node_it == updated_nodes.end()) { - node = std::make_shared(); + node = std::make_shared(); node->shallowCopy(*uncommitted_state.getNode(delta.path)); current_digest -= node->getDigest(delta.path); updated_nodes.emplace(delta.path, node); @@ -2114,7 +2160,8 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec return current_digest; } -void KeeperStorage::preprocessRequest( +template +void KeeperStorage::preprocessRequest( const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id, int64_t time, @@ -2173,7 +2220,7 @@ void KeeperStorage::preprocessRequest( uncommitted_state.addDeltas(std::move(new_deltas)); }); - KeeperStorageRequestProcessorPtr request_processor = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); + auto request_processor = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { @@ -2215,7 +2262,8 @@ void KeeperStorage::preprocessRequest( new_deltas = request_processor->preprocess(*this, transaction.zxid, session_id, time, new_digest, *keeper_context); } -KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( +template +KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id, std::optional new_last_zxid, @@ -2242,7 +2290,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( uncommitted_transactions.pop_front(); } - KeeperStorage::ResponsesForSessions results; + ResponsesForSessions results; /// ZooKeeper update sessions expirity for each request, not only for heartbeats session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); @@ -2280,7 +2328,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special { - KeeperStorageRequestProcessorPtr storage_request = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); + auto storage_request = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); auto response = storage_request->process(*this, zxid); response->xid = zk_request->xid; response->zxid = getZXID(); @@ -2289,7 +2337,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } else /// normal requests proccession { - KeeperStorageRequestProcessorPtr request_processor = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); + auto request_processor = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); Coordination::ZooKeeperResponsePtr response; if (is_local) @@ -2352,7 +2400,8 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( return results; } -void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) +template +void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) { if (allow_missing && (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid < rollback_zxid)) return; @@ -2378,7 +2427,8 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) } } -KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const +template +KeeperStorageBase::Digest KeeperStorage::getNodesDigest(bool committed) const { if (!keeper_context->digestEnabled()) return {.version = DigestVersion::NO_DIGEST}; @@ -2389,13 +2439,15 @@ KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const return uncommitted_transactions.back().nodes_digest; } -void KeeperStorage::removeDigest(const Node & node, const std::string_view path) +template +void KeeperStorage::removeDigest(const Node & node, const std::string_view path) { if (keeper_context->digestEnabled()) nodes_digest -= node.getDigest(path); } -void KeeperStorage::addDigest(const Node & node, const std::string_view path) +template +void KeeperStorage::addDigest(const Node & node, const std::string_view path) { if (keeper_context->digestEnabled()) { @@ -2404,7 +2456,8 @@ void KeeperStorage::addDigest(const Node & node, const std::string_view path) } } -void KeeperStorage::clearDeadWatches(int64_t session_id) +template +void KeeperStorage::clearDeadWatches(int64_t session_id) { /// Clear all watches for this session auto watches_it = sessions_and_watchers.find(session_id); @@ -2437,7 +2490,8 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) } } -void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const +template +void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const { for (const auto & [session_id, watches_paths] : sessions_and_watchers) { @@ -2447,7 +2501,8 @@ void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const } } -void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const +template +void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { auto write_int_container = [&buf](const auto & session_ids) { @@ -2470,7 +2525,8 @@ void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const } } -void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const +template +void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { auto write_str_set = [&buf](const std::unordered_set & ephemeral_paths) { @@ -2495,7 +2551,8 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co } } -uint64_t KeeperStorage::getTotalWatchesCount() const +template +uint64_t KeeperStorage::getTotalWatchesCount() const { uint64_t ret = 0; for (const auto & [session, paths] : sessions_and_watchers) @@ -2504,12 +2561,14 @@ uint64_t KeeperStorage::getTotalWatchesCount() const return ret; } -uint64_t KeeperStorage::getSessionsWithWatchesCount() const +template +uint64_t KeeperStorage::getSessionsWithWatchesCount() const { return sessions_and_watchers.size(); } -uint64_t KeeperStorage::getTotalEphemeralNodesCount() const +template +uint64_t KeeperStorage::getTotalEphemeralNodesCount() const { uint64_t ret = 0; for (const auto & [session_id, nodes] : ephemerals) @@ -2518,17 +2577,20 @@ uint64_t KeeperStorage::getTotalEphemeralNodesCount() const return ret; } -void KeeperStorage::recalculateStats() +template +void KeeperStorage::recalculateStats() { container.recalculateDataSize(); } -String KeeperStorage::generateDigest(const String & userdata) +template +String KeeperStorage::generateDigest(const String & userdata) { std::vector user_password; boost::split(user_password, userdata, [](char character) { return character == ':'; }); return user_password[0] + ":" + base64Encode(getSHA1(userdata)); } +template class KeeperStorage>; } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index ec5df74efb6..2eff9f6651b 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -16,62 +16,55 @@ namespace DB { -struct KeeperStorageRequestProcessor; -using KeeperStorageRequestProcessorPtr = std::shared_ptr; using ResponseCallback = std::function; using ChildrenSet = absl::flat_hash_set; using SessionAndTimeout = std::unordered_map; -struct KeeperStorageSnapshot; +struct KeeperMemNode +{ + uint64_t acl_id = 0; /// 0 -- no ACL by default + bool is_sequental = false; + Coordination::Stat stat{}; + int32_t seq_num = 0; + uint64_t size_bytes; // save size to avoid calculate every time -/// 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 + KeeperMemNode() : size_bytes(sizeof(KeeperMemNode)) { } + + /// Object memory size + uint64_t sizeInBytes() const { return size_bytes; } + + void setData(String new_data); + + const auto & getData() const noexcept { return data; } + + void addChild(StringRef child_path, bool update_size = true); + + void removeChild(StringRef child_path); + + const auto & getChildren() const noexcept { 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); + + void recalculateSize(); + +private: + String data; + ChildrenSet children{}; + mutable std::optional cached_digest; +}; + +class KeeperStorageBase { public: - struct Node - { - uint64_t acl_id = 0; /// 0 -- no ACL by default - bool is_sequental = false; - Coordination::Stat stat{}; - int32_t seq_num = 0; - uint64_t size_bytes; // save size to avoid calculate every time - - Node() : size_bytes(sizeof(Node)) { } - - /// Object memory size - uint64_t sizeInBytes() const { return size_bytes; } - - void setData(String new_data); - - const auto & getData() const noexcept { return data; } - - void addChild(StringRef child_path, bool update_size = true); - - void removeChild(StringRef child_path); - - const auto & getChildren() const noexcept { 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); - - void recalculateSize(); - - private: - String data; - ChildrenSet children{}; - mutable std::optional cached_digest; - }; - enum DigestVersion : uint8_t { NO_DIGEST = 0, @@ -79,7 +72,11 @@ public: V2 = 2 // added system nodes that modify the digest on startup so digest from V0 is invalid }; - static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V2; + struct Digest + { + DigestVersion version{DigestVersion::NO_DIGEST}; + uint64_t value{0}; + }; struct ResponseForSession { @@ -88,11 +85,34 @@ public: }; using ResponsesForSessions = std::vector; - struct Digest + struct RequestForSession { - DigestVersion version{DigestVersion::NO_DIGEST}; - uint64_t value{0}; + int64_t session_id; + int64_t time{0}; + Coordination::ZooKeeperRequestPtr request; + int64_t zxid{0}; + std::optional digest; + int64_t log_idx{0}; }; + using RequestsForSessions = std::vector; + + struct AuthID + { + std::string scheme; + std::string id; + + bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; } + }; + + // using Container = SnapshotableHashTable; + using Ephemerals = std::unordered_map>; + using SessionAndWatcher = std::unordered_map>; + using SessionIDs = std::unordered_set; + + /// Just vector of SHA1 from user:password + using AuthIDs = std::vector; + using SessionAndAuth = std::unordered_map; + using Watches = std::unordered_map; static bool checkDigest(const Digest & first, const Digest & second) { @@ -105,38 +125,22 @@ public: return first.value == second.value; } +}; + +/// 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 +class KeeperStorage : public KeeperStorageBase +{ +public: + using Container = Container_; + using Node = Container::Node; + + static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V2; + static String generateDigest(const String & userdata); - struct RequestForSession - { - int64_t session_id; - int64_t time{0}; - Coordination::ZooKeeperRequestPtr request; - int64_t zxid{0}; - std::optional digest; - int64_t log_idx{0}; - }; - - struct AuthID - { - std::string scheme; - std::string id; - - bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; } - }; - - using RequestsForSessions = std::vector; - - using Container = SnapshotableHashTable; - using Ephemerals = std::unordered_map>; - using SessionAndWatcher = std::unordered_map>; - using SessionIDs = std::unordered_set; - - /// Just vector of SHA1 from user:password - using AuthIDs = std::vector; - using SessionAndAuth = std::unordered_map; - using Watches = std::unordered_map; - int64_t session_id_counter{1}; SessionAndAuth session_and_auth; @@ -307,7 +311,7 @@ public: std::unordered_map, Hash, Equal> deltas_for_path; std::list deltas; - KeeperStorage & storage; + KeeperStorage & storage; }; UncommittedState uncommitted_state{*this}; @@ -487,6 +491,6 @@ private: void addDigest(const Node & node, std::string_view path); }; -using KeeperStoragePtr = std::unique_ptr; +using KeeperMemoryStorage = KeeperStorage>; } diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 093126237ef..2bdef189e6b 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -121,6 +121,7 @@ private: public: + using Node = V; using iterator = typename List::iterator; using const_iterator = typename List::const_iterator; using ValueUpdater = std::function; diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 3c1550f08c8..e783582af23 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -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 +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 +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, Poco::Logger * log) +template +int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, Poco::Logger * log) { int64_t max_zxid = 0; std::string path; @@ -98,7 +101,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L size_t count = 0; while (path != "/") { - KeeperStorage::Node node{}; + typename Storage::Node node{}; String data; Coordination::read(data, in); node.setData(std::move(data)); @@ -140,14 +143,15 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L 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.stat.numChildren; }); + storage.container.updateValue(parent_path, [my_path = itr.key] (typename Storage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; }); } } return max_zxid; } -void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log) +template +void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, Poco::Logger * log) { LOG_INFO(log, "Deserializing storage snapshot {}", snapshot_path); int64_t zxid = getZxidFromName(snapshot_path); @@ -186,7 +190,8 @@ void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::st LOG_INFO(log, "Finished, snapshot ZXID {}", storage.zxid); } -void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, Poco::Logger * log) +template +void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, Poco::Logger * log) { namespace fs = std::filesystem; std::map existing_snapshots; @@ -474,7 +479,8 @@ bool hasErrorsInMultiRequest(Coordination::ZooKeeperRequestPtr request) } -bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*log*/) +template +bool deserializeTxn(Storage & storage, ReadBuffer & in, Poco::Logger * /*log*/) { int64_t checksum; Coordination::read(checksum, in); @@ -529,7 +535,8 @@ bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*l return true; } -void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, Poco::Logger * log) +template +void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, Poco::Logger * log) { ReadBufferFromFile reader(log_path); @@ -553,7 +560,8 @@ 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, Poco::Logger * log) +template +void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, Poco::Logger * log) { namespace fs = std::filesystem; std::map existing_logs; @@ -589,4 +597,9 @@ void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string } } +template void deserializeKeeperStorageFromSnapshot(KeeperMemoryStorage & storage, const std::string & snapshot_path, Poco::Logger * log); +template void deserializeKeeperStorageFromSnapshotsDir(KeeperMemoryStorage & storage, const std::string & path, Poco::Logger * log); +template void deserializeLogAndApplyToStorage(KeeperMemoryStorage & storage, const std::string & log_path, Poco::Logger * log); +template void deserializeLogsAndApplyToStorage(KeeperMemoryStorage & storage, const std::string & path, Poco::Logger * log); + } diff --git a/src/Coordination/ZooKeeperDataReader.h b/src/Coordination/ZooKeeperDataReader.h index 8fd86ba99e2..928e560926b 100644 --- a/src/Coordination/ZooKeeperDataReader.h +++ b/src/Coordination/ZooKeeperDataReader.h @@ -5,12 +5,16 @@ namespace DB { -void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log); +template +void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, Poco::Logger * log); -void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, Poco::Logger * log); +template +void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, Poco::Logger * log); -void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, Poco::Logger * log); +template +void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, Poco::Logger * log); -void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log); +template +void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, Poco::Logger * log); } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 418b5225fa4..517b55af866 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1320,7 +1320,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) EXPECT_EQ(hello.getApproximateDataSize(), 0); /// Node - using Node = DB::KeeperStorage::Node; + using Node = DB::KeeperMemoryStorage::Node; DB::SnapshotableHashTable world; Node n1; n1.setData("1234"); @@ -1359,9 +1359,9 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) EXPECT_EQ(world.getApproximateDataSize(), 0); } -void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) +void addNode(DB::KeeperMemoryStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) { - using Node = DB::KeeperStorage::Node; + using Node = DB::KeeperMemoryStorage::Node; Node node{}; node.setData(data); node.stat.ephemeralOwner = ephemeral_owner; @@ -1383,9 +1383,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); addNode(storage, "/hello", "world", 1); addNode(storage, "/hello/somepath", "somedata", 3); storage.session_id_counter = 5; @@ -1395,7 +1395,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) storage.getSessionID(130); storage.getSessionID(130); - DB::KeeperStorageSnapshot snapshot(&storage, 2); + DB::KeeperStorageSnapshot snapshot(&storage, 2); EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2); EXPECT_EQ(snapshot.session_id, 7); @@ -1433,9 +1433,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); storage.getSessionID(130); for (size_t i = 0; i < 50; ++i) @@ -1443,7 +1443,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } - DB::KeeperStorageSnapshot snapshot(&storage, 50); + DB::KeeperStorageSnapshot snapshot(&storage, 50); EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50); EXPECT_EQ(snapshot.snapshot_container_size, 54); @@ -1476,9 +1476,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); storage.getSessionID(130); for (size_t j = 1; j <= 5; ++j) @@ -1488,7 +1488,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } - DB::KeeperStorageSnapshot snapshot(&storage, j * 50); + DB::KeeperStorageSnapshot snapshot(&storage, j * 50); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, j * 50); EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin" + params.extension)); @@ -1517,15 +1517,15 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperMemoryStorage storage(500, "", keeper_context); for (size_t i = 0; i < 50; ++i) { addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } { - DB::KeeperStorageSnapshot snapshot(&storage, 50); + DB::KeeperStorageSnapshot snapshot(&storage, 50); for (size_t i = 0; i < 50; ++i) { addNode(storage, "/hello_" + std::to_string(i), "wlrd_" + std::to_string(i)); @@ -1571,14 +1571,14 @@ TEST_P(CoordinationTest, TestStorageSnapshotBroken) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperMemoryStorage storage(500, "", keeper_context); for (size_t i = 0; i < 50; ++i) { addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } { - DB::KeeperStorageSnapshot snapshot(&storage, 50); + DB::KeeperStorageSnapshot snapshot(&storage, 50); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); } @@ -1602,7 +1602,7 @@ nuraft::ptr getBufferFromZKRequest(int64_t session_id, int64_t z auto time = duration_cast(system_clock::now().time_since_epoch()).count(); DB::writeIntBinary(time, buf); DB::writeIntBinary(zxid, buf); - DB::writeIntBinary(DB::KeeperStorage::DigestVersion::NO_DIGEST, buf); + DB::writeIntBinary(DB::KeeperMemoryStorage::DigestVersion::NO_DIGEST, buf); return buf.getBuffer(); } @@ -1629,7 +1629,7 @@ void testLogAndStateMachine( ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); DB::KeeperLogStore changelog( DB::LogFileSettings{ @@ -1672,7 +1672,7 @@ void testLogAndStateMachine( } SnapshotsQueue snapshots_queue1{1}; - auto restore_machine = std::make_shared(queue, snapshots_queue1, settings, keeper_context, nullptr); + auto restore_machine = std::make_shared>(queue, snapshots_queue1, settings, keeper_context, nullptr); restore_machine->init(); EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance); @@ -1791,7 +1791,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); std::shared_ptr request_c = std::make_shared(); @@ -1825,11 +1825,11 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; - String digest = KeeperStorage::generateDigest(user_auth_data); + String digest = KeeperMemoryStorage::generateDigest(user_auth_data); std::shared_ptr auth_req = std::make_shared(); auth_req->scheme = "digest"; @@ -1877,11 +1877,11 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; - String digest = KeeperStorage::generateDigest(user_auth_data); + String digest = KeeperMemoryStorage::generateDigest(user_auth_data); std::shared_ptr auth_req = std::make_shared(); auth_req->scheme = "digest"; @@ -2104,9 +2104,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); addNode(storage, "/hello", "world", 1); addNode(storage, "/hello/somepath", "somedata", 3); storage.session_id_counter = 5; @@ -2116,13 +2116,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) storage.getSessionID(130); storage.getSessionID(130); - DB::KeeperStorageSnapshot snapshot(&storage, 2); + DB::KeeperStorageSnapshot snapshot(&storage, 2); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 2); EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension)); - DB::KeeperSnapshotManager new_manager(3, keeper_context, !params.enable_compression); + DB::KeeperSnapshotManager new_manager(3, keeper_context, !params.enable_compression); auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); @@ -2308,9 +2308,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual) std::optional snapshot_hash; for (size_t i = 0; i < 15; ++i) { - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); addNode(storage, "/hello", ""); for (size_t j = 0; j < 5000; ++j) { @@ -2326,7 +2326,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual) for (size_t j = 0; j < 3333; ++j) storage.getSessionID(130 * j); - DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); + DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); auto buf = manager.serializeSnapshotToBuffer(snapshot); @@ -2389,7 +2389,7 @@ TEST_P(CoordinationTest, TestUncommittedStateBasicCrud) using namespace DB; using namespace Coordination; - DB::KeeperStorage storage{500, "", keeper_context}; + DB::KeeperMemoryStorage storage{500, "", keeper_context}; constexpr std::string_view path = "/test"; @@ -2506,7 +2506,7 @@ TEST_P(CoordinationTest, TestListRequestTypes) using namespace DB; using namespace Coordination; - KeeperStorage storage{500, "", keeper_context}; + KeeperMemoryStorage storage{500, "", keeper_context}; int32_t zxid = 0; @@ -2660,7 +2660,7 @@ TEST_P(CoordinationTest, TestDurableState) TEST_P(CoordinationTest, TestFeatureFlags) { using namespace Coordination; - KeeperStorage storage{500, "", keeper_context}; + KeeperMemoryStorage storage{500, "", keeper_context}; auto request = std::make_shared(); request->path = DB::keeper_api_feature_flags_path; auto responses = storage.processRequest(request, 0, std::nullopt, true, true); @@ -2679,7 +2679,7 @@ TEST_P(CoordinationTest, TestSystemNodeModify) // On INIT we abort when a system path is modified keeper_context->setServerState(KeeperContext::Phase::RUNNING); - KeeperStorage storage{500, "", keeper_context}; + KeeperMemoryStorage storage{500, "", keeper_context}; const auto assert_create = [&](const std::string_view path, const auto expected_code) { auto request = std::make_shared(); @@ -2771,7 +2771,7 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest) using namespace DB; using namespace Coordination; - KeeperStorage storage{500, "", keeper_context}; + KeeperMemoryStorage storage{500, "", keeper_context}; int32_t zxid = 0; @@ -2850,7 +2850,7 @@ TEST_P(CoordinationTest, TestReapplyingDeltas) create_request->path = "/test/data"; create_request->is_sequential = true; - const auto process_create = [](KeeperStorage & storage, const auto & request, int64_t zxid) + const auto process_create = [](KeeperMemoryStorage & storage, const auto & request, int64_t zxid) { storage.preprocessRequest(request, 1, 0, zxid); auto responses = storage.processRequest(request, 1, zxid); @@ -2871,19 +2871,19 @@ TEST_P(CoordinationTest, TestReapplyingDeltas) process_create(storage, create_request, zxid); }; - KeeperStorage storage1{500, "", keeper_context}; + KeeperMemoryStorage storage1{500, "", keeper_context}; commit_initial_data(storage1); for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) storage1.preprocessRequest(create_request, 1, 0, zxid, /*check_acl=*/true, /*digest=*/std::nullopt, /*log_idx=*/zxid); /// create identical new storage - KeeperStorage storage2{500, "", keeper_context}; + KeeperMemoryStorage storage2{500, "", keeper_context}; commit_initial_data(storage2); storage1.applyUncommittedState(storage2, initial_zxid); - const auto commit_unprocessed = [&](KeeperStorage & storage) + const auto commit_unprocessed = [&](KeeperMemoryStorage & storage) { for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) { @@ -2896,7 +2896,7 @@ TEST_P(CoordinationTest, TestReapplyingDeltas) commit_unprocessed(storage1); commit_unprocessed(storage2); - const auto get_children = [&](KeeperStorage & storage) + const auto get_children = [&](KeeperMemoryStorage & storage) { const auto list_request = std::make_shared(); list_request->path = "/test";