From b43c3d75a2e21af9630152c335d40ff1f852310a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 4 Sep 2023 14:49:49 +0000 Subject: [PATCH 01/27] Initial implementation --- src/Coordination/KeeperSnapshotManager.cpp | 2 +- src/Coordination/KeeperStateMachine.cpp | 3 - src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStorage.cpp | 983 ++++++++++++--------- src/Coordination/KeeperStorage.h | 101 ++- src/Coordination/ZooKeeperDataReader.cpp | 6 +- 6 files changed, 638 insertions(+), 459 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a216272a9e1..c0c59b4e242 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -277,7 +277,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr } } -void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context) +void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context) TSA_NO_THREAD_SAFETY_ANALYSIS { uint8_t version; readBinary(version, in); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5a8cf88ea73..9d96a76e06a 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -271,8 +271,6 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; - std::lock_guard lock(storage_and_responses_lock); - if (storage->isFinalized()) return false; @@ -422,7 +420,6 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n parsed_request_cache.erase(request_for_session->session_id); } - std::lock_guard lock(storage_and_responses_lock); KeeperStorage::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) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index aad5d3aafd4..670b15394ce 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -142,7 +142,7 @@ private: CoordinationSettingsPtr coordination_settings; /// Main state machine logic - KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock); + KeeperStoragePtr storage; /// Save/Load and Serialize/Deserialize logic for snapshots. KeeperSnapshotManager snapshot_manager; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 95983dbcf3c..0978f5f965e 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -229,6 +229,7 @@ void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) { stat = other.stat; seq_num = other.seq_num; + acl_id = other.acl_id; setData(other.getData()); cached_digest = other.cached_digest; } @@ -316,6 +317,7 @@ Overloaded(Ts...) -> Overloaded; std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const { + std::lock_guard lock(storage.container_mutex); if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; @@ -330,18 +332,34 @@ std::shared_ptr KeeperStorage::UncommittedState::tryGetNode void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) { assert(!delta.path.empty()); - if (!nodes.contains(delta.path)) + UncommittedNode * uncommitted_node = nullptr; + + if (auto it = nodes.find(delta.path); it != nodes.end()) + { + uncommitted_node = &it->second; + } + else { if (auto storage_node = tryGetNodeFromStorage(delta.path)) - nodes.emplace(delta.path, UncommittedNode{.node = std::move(storage_node)}); + { + auto [emplaced_it, _] = nodes.emplace(delta.path, UncommittedNode{.node = std::move(storage_node)}); + uncommitted_node = &emplaced_it->second; + } else - nodes.emplace(delta.path, UncommittedNode{.node = nullptr}); + { + auto [emplaced_it, _] = nodes.emplace(delta.path, UncommittedNode{.node = nullptr}); + uncommitted_node = &emplaced_it->second; + } } std::visit( [&](const DeltaType & operation) { - auto & [node, acls, last_applied_zxid] = nodes.at(delta.path); + auto & [node, acls, applied_zxids] = *uncommitted_node; + + int64_t last_applied_zxid = 0; + if (!applied_zxids.empty()) + last_applied_zxid = applied_zxids.back(); if constexpr (std::same_as) { @@ -350,132 +368,193 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) node->stat = operation.stat; node->setData(operation.data); acls = operation.acls; - last_applied_zxid = delta.zxid; } else if constexpr (std::same_as) { assert(node); node = nullptr; - last_applied_zxid = delta.zxid; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { assert(node); node->invalidateDigestCache(); - operation.update_fn(*node); - last_applied_zxid = delta.zxid; + node->stat = operation.new_stats; + node->seq_num = operation.new_seq_num; + } + else if constexpr (std::same_as) + { + assert(node); + node->invalidateDigestCache(); + node->setData(operation.new_data); } else if constexpr (std::same_as) { - acls = operation.acls; - last_applied_zxid = delta.zxid; + acls = operation.new_acls; } + + if (last_applied_zxid != delta.zxid) + last_applied_zxid = applied_zxids.emplace_back(delta.zxid); }, delta.operation); } -void KeeperStorage::UncommittedState::addDelta(Delta new_delta) +void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) { - const auto & added_delta = deltas.emplace_back(std::move(new_delta)); + assert(!delta.path.empty()); - if (!added_delta.path.empty()) - { - deltas_for_path[added_delta.path].push_back(&added_delta); - applyDelta(added_delta); - } - else if (const auto * auth_delta = std::get_if(&added_delta.operation)) - { - auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; - uncommitted_auth.emplace_back(&auth_delta->auth_id); - } -} - -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) -{ - assert(deltas.empty() || deltas.front().zxid >= commit_zxid); - - // collect nodes that have no further modification in the current transaction - std::unordered_set modified_nodes; - - while (!deltas.empty() && deltas.front().zxid == commit_zxid) - { - if (std::holds_alternative(deltas.front().operation)) + std::visit( + [&](const DeltaType & operation) { - deltas.pop_front(); - break; - } + auto & [node, acls, applied_zxids] = nodes.at(delta.path); - auto & front_delta = deltas.front(); - - if (!front_delta.path.empty()) - { - auto & path_deltas = deltas_for_path.at(front_delta.path); - assert(path_deltas.front() == &front_delta); - path_deltas.pop_front(); - if (path_deltas.empty()) + if constexpr (std::same_as) { - deltas_for_path.erase(front_delta.path); - - // no more deltas for path -> no modification - modified_nodes.insert(std::move(front_delta.path)); + assert(node); + node = nullptr; } - else if (path_deltas.front()->zxid > commit_zxid) + else if constexpr (std::same_as) { - // next delta has a zxid from a different transaction -> no modification in this transaction - modified_nodes.insert(std::move(front_delta.path)); + assert(!node); + node = std::make_shared(); + node->stat = operation.stat; + node->setData(operation.data); + acls = operation.acls; + } + else if constexpr (std::same_as) + { + assert(node); + node->invalidateDigestCache(); + node->stat = operation.old_stats; + node->seq_num = operation.old_seq_num; + } + else if constexpr (std::same_as) + { + assert(node); + node->invalidateDigestCache(); + node->setData(operation.old_data); + } + else if constexpr (std::same_as) + { + acls = operation.old_acls; } - } - else if (auto * add_auth = std::get_if(&front_delta.operation)) - { - auto & uncommitted_auth = session_and_auth[add_auth->session_id]; - assert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id); - uncommitted_auth.pop_front(); - if (uncommitted_auth.empty()) - session_and_auth.erase(add_auth->session_id); - } + if (applied_zxids.back() != delta.zxid) + applied_zxids.pop_back(); + }, + delta.operation); +} - deltas.pop_front(); - } - - // delete all cached nodes that were not modified after the commit_zxid - // we only need to check the nodes that were modified in this transaction - for (const auto & node : modified_nodes) +void KeeperStorage::UncommittedState::applyDeltas(const std::list & new_deltas) +{ + for (const auto & delta : new_deltas) { - if (nodes[node].zxid == commit_zxid) - nodes.erase(node); + if (!delta.path.empty()) + { + applyDelta(delta); + } + else if (const auto * auth_delta = std::get_if(&delta.operation)) + { + auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; + uncommitted_auth.emplace_back(auth_delta->auth_id); + } } } +void KeeperStorage::UncommittedState::addDeltas(std::list new_deltas) +{ + std::lock_guard lock(deltas_mutex); + deltas.splice(deltas.end(), std::move(new_deltas)); +} + +//void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) +//{ +// assert(deltas.empty() || deltas.front().zxid >= commit_zxid); +// +// // collect nodes that have no further modification in the current transaction +// std::unordered_set modified_nodes; +// +// while (!deltas.empty() && deltas.front().zxid == commit_zxid) +// { +// if (std::holds_alternative(deltas.front().operation)) +// { +// deltas.pop_front(); +// break; +// } +// +// auto & front_delta = deltas.front(); +// +// if (!front_delta.path.empty()) +// { +// auto & path_deltas = deltas_for_path.at(front_delta.path); +// assert(path_deltas.front() == &front_delta); +// path_deltas.pop_front(); +// if (path_deltas.empty()) +// { +// deltas_for_path.erase(front_delta.path); +// +// // no more deltas for path -> no modification +// modified_nodes.insert(std::move(front_delta.path)); +// } +// else if (path_deltas.front()->zxid > commit_zxid) +// { +// // next delta has a zxid from a different transaction -> no modification in this transaction +// modified_nodes.insert(std::move(front_delta.path)); +// } +// } +// else if (auto * add_auth = std::get_if(&front_delta.operation)) +// { +// auto & uncommitted_auth = session_and_auth[add_auth->session_id]; +// assert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id); +// uncommitted_auth.pop_front(); +// if (uncommitted_auth.empty()) +// session_and_auth.erase(add_auth->session_id); +// +// } +// +// deltas.pop_front(); +// } +// +// // delete all cached nodes that were not modified after the commit_zxid +// // we only need to check the nodes that were modified in this transaction +// for (const auto & node : modified_nodes) +// { +// if (nodes[node].zxid == commit_zxid) +// nodes.erase(node); +// } +//} + 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 - if (!deltas.empty() && deltas.back().zxid > rollback_zxid) - throw DB::Exception( - DB::ErrorCodes::LOGICAL_ERROR, - "Invalid state of deltas found while trying to rollback request. Last ZXID ({}) is larger than the requested ZXID ({})", - deltas.back().zxid, - rollback_zxid); + std::list rollback_deltas; + { + std::lock_guard lock(deltas_mutex); + if (!deltas.empty() && deltas.back().zxid > rollback_zxid) + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Invalid state of deltas found while trying to rollback request. Last ZXID ({}) is larger than the requested ZXID ({})", + deltas.back().zxid, + rollback_zxid); - auto delta_it = deltas.rbegin(); + auto delta_it = deltas.rbegin(); + for (; delta_it != deltas.rend(); ++delta_it) + { + if (delta_it->zxid != rollback_zxid) + break; + } + + if (delta_it == deltas.rend()) + rollback_deltas = std::move(deltas); + else + rollback_deltas.splice(rollback_deltas.end(), deltas, delta_it.base(), deltas.end()); + } // we need to undo ephemeral mapping modifications // CreateNodeDelta added ephemeral for session id -> we need to remove it // RemoveNodeDelta removed ephemeral for session id -> we need to add it back - for (; delta_it != deltas.rend(); ++delta_it) + for (auto & delta : rollback_deltas) { - if (delta_it->zxid < rollback_zxid) - break; - - assert(delta_it->zxid == rollback_zxid); - if (!delta_it->path.empty()) + assert(delta.zxid == rollback_zxid); + if (!delta.path.empty()) { std::visit( [&](const DeltaType & operation) @@ -483,65 +562,28 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) if constexpr (std::same_as) { if (operation.stat.ephemeralOwner != 0) - storage.unregisterEphemeralPath(operation.stat.ephemeralOwner, delta_it->path); + storage.unregisterEphemeralPath(operation.stat.ephemeralOwner, delta.path); } else if constexpr (std::same_as) { - if (operation.ephemeral_owner != 0) - storage.ephemerals[operation.ephemeral_owner].emplace(delta_it->path); + if (operation.stat.ephemeralOwner != 0) + storage.ephemerals[operation.stat.ephemeralOwner].emplace(delta.path); } }, - delta_it->operation); - - auto & path_deltas = deltas_for_path.at(delta_it->path); - if (path_deltas.back() == &*delta_it) - { - path_deltas.pop_back(); - if (path_deltas.empty()) - deltas_for_path.erase(delta_it->path); - } + delta.operation); } - else if (auto * add_auth = std::get_if(&delta_it->operation)) + else if (auto * add_auth = std::get_if(&delta.operation)) { auto & uncommitted_auth = session_and_auth[add_auth->session_id]; - if (uncommitted_auth.back() == &add_auth->auth_id) + if (uncommitted_auth.back() == add_auth->auth_id) { uncommitted_auth.pop_back(); if (uncommitted_auth.empty()) session_and_auth.erase(add_auth->session_id); } } - } - if (delta_it == deltas.rend()) - deltas.clear(); - else - deltas.erase(delta_it.base(), deltas.end()); - - absl::flat_hash_set deleted_nodes; - std::erase_if( - nodes, - [&, rollback_zxid](const auto & node) - { - if (node.second.zxid == rollback_zxid) - { - deleted_nodes.emplace(std::move(node.first)); - return true; - } - return false; - }); - - // recalculate all the uncommitted deleted nodes - for (const auto & deleted_node : deleted_nodes) - { - auto path_delta_it = deltas_for_path.find(deleted_node); - if (path_delta_it != deltas_for_path.end()) - { - for (const auto & delta : path_delta_it->second) - { - applyDelta(*delta); - } - } + rollbackDelta(delta); } } @@ -550,19 +592,34 @@ std::shared_ptr KeeperStorage::UncommittedState::getNode(St if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.node; - return tryGetNodeFromStorage(path); + std::shared_ptr node = tryGetNodeFromStorage(path); + + if (node) + nodes.emplace(std::string{path}, UncommittedNode{node}); + + return node; } 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; + { + if (!node_it->second.acls.has_value()) + node_it->second.acls.emplace(storage.acl_map.convertNumber(node_it->second.node->acl_id)); - auto node_it = storage.container.find(path); - if (node_it == storage.container.end()) - return {}; + return *node_it->second.acls; + } - return storage.acl_map.convertNumber(node_it->value.acl_id); + std::shared_ptr node = tryGetNodeFromStorage(path); + + if (node) + { + auto [it, inserted] = nodes.emplace(std::string{path}, UncommittedNode{node}); + it->second.acls = storage.acl_map.convertNumber(node->acl_id); + return *it->second.acls; + } + + return {}; } void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, std::function func) const @@ -571,21 +628,25 @@ void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, s { for (const auto & auth : auth_ids) { - using TAuth = std::remove_reference_t; + using TAuth = std::remove_cvref_t; const AuthID * auth_ptr = nullptr; - if constexpr (std::is_pointer_v) - auth_ptr = auth; - else + if constexpr (std::same_as) auth_ptr = &auth; + else + auth_ptr = auth.get(); func(*auth_ptr); } }; - // for committed - if (storage.session_and_auth.contains(session_id)) - call_for_each_auth(storage.session_and_auth.at(session_id)); + { + std::lock_guard lock(storage.auth_mutex); + // for committed + if (storage.session_and_auth.contains(session_id)) + call_for_each_auth(storage.session_and_auth.at(session_id)); + } + // for uncommitted if (session_and_auth.contains(session_id)) call_for_each_auth(session_and_auth.at(session_id)); @@ -604,37 +665,55 @@ namespace } -void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_zxid) -{ - for (const auto & transaction : uncommitted_transactions) - { - if (transaction.zxid <= last_zxid) - continue; - other.uncommitted_transactions.push_back(transaction); - } - - auto it = uncommitted_state.deltas.begin(); - - for (; it != uncommitted_state.deltas.end(); ++it) - { - if (it->zxid <= last_zxid) - continue; - - other.uncommitted_state.addDelta(*it); - } +/// Get current committed zxid +int64_t KeeperStorage::getZXID() const +{ + std::lock_guard lock(transaction_mutex); + return zxid; } -Coordination::Error KeeperStorage::commit(int64_t commit_zxid) +int64_t KeeperStorage::getNextZXIDLocked(std::lock_guard &) const { + if (uncommitted_transactions.empty()) + return zxid + 1; + + return uncommitted_transactions.back().zxid + 1; +} + +int64_t KeeperStorage::getNextZXID() const +{ + std::lock_guard lock(transaction_mutex); + return getNextZXIDLocked(lock); +} + +void KeeperStorage::applyUncommittedState(KeeperStorage & /*other*/, int64_t /*last_zxid*/) +{ + //for (const auto & transaction : uncommitted_transactions) + //{ + // if (transaction.zxid <= last_zxid) + // continue; + // other.uncommitted_transactions.push_back(transaction); + //} + + //auto it = uncommitted_state.deltas.begin(); + + //for (; it != uncommitted_state.deltas.end(); ++it) + //{ + // if (it->zxid <= last_zxid) + // continue; + + // other.uncommitted_state.addDelta(*it); + //} +} + +Coordination::Error KeeperStorage::commit(std::list deltas) +{ + std::lock_guard lock(container_mutex); // Deltas are added with increasing ZXIDs // If there are no deltas for the commit_zxid (e.g. read requests), we instantly return // on first delta - for (auto & delta : uncommitted_state.deltas) + for (auto & delta : deltas) { - if (delta.zxid > commit_zxid) - break; - - bool finish_subdelta = false; auto result = std::visit( [&, &path = delta.path](DeltaType & operation) -> Coordination::Error { @@ -650,7 +729,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 || std::same_as) { auto node_it = container.find(path); if (node_it == container.end()) @@ -660,7 +739,17 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) onStorageInconsistency(); removeDigest(node_it->value, path); - auto updated_node = container.updateValue(path, operation.update_fn); + auto updated_node = container.updateValue(path, [&](auto & node) { + if constexpr (std::same_as) + { + node.stat = operation.new_stats; + node.seq_num = operation.new_seq_num; + } + else + { + node.setData(std::move(operation.new_data)); + } + }); addDigest(updated_node->value, path); return Coordination::Error::ZOK; @@ -683,7 +772,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) acl_map.removeUsage(node_it->value.acl_id); - uint64_t acl_id = acl_map.convertACLs(operation.acls); + uint64_t acl_id = acl_map.convertACLs(operation.new_acls); acl_map.addUsage(acl_id); container.updateValue(path, [acl_id](KeeperStorage::Node & node) { node.acl_id = acl_id; }); @@ -694,12 +783,12 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) return operation.error; else if constexpr (std::same_as) { - finish_subdelta = true; return Coordination::Error::ZOK; } else if constexpr (std::same_as) { - session_and_auth[operation.session_id].emplace_back(std::move(operation.auth_id)); + std::lock_guard auth_lock(auth_mutex); + session_and_auth[operation.session_id].emplace_back(std::move(*operation.auth_id)); return Coordination::Error::ZOK; } else @@ -712,9 +801,6 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) if (result != Coordination::Error::ZOK) return result; - - if (finish_subdelta) - return Coordination::Error::ZOK; } return Coordination::Error::ZOK; @@ -799,8 +885,8 @@ 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 + virtual Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const = 0; + virtual std::list preprocess(KeeperStorage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const { return {}; @@ -808,7 +894,7 @@ struct KeeperStorageRequestProcessor // process the request using locally committed data virtual Coordination::ZooKeeperResponsePtr - processLocal(KeeperStorage & /*storage*/, int64_t /*zxid*/) const + processLocal(KeeperStorage & /*storage*/, std::list /* deltas */) const { throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Cannot process the request locally"}; } @@ -827,7 +913,7 @@ struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageReques { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; Coordination::ZooKeeperResponsePtr - process(KeeperStorage & /* storage */, int64_t /* zxid */) const override + process(KeeperStorage & /* storage */, std::list /* deltas */) const override { return zk_request->makeResponse(); } @@ -837,7 +923,7 @@ struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProc { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; Coordination::ZooKeeperResponsePtr - process(KeeperStorage & /* storage */, int64_t /* zxid */) const override + process(KeeperStorage & /* storage */, std::list /* deltas */) const override { auto response = zk_request->makeResponse(); dynamic_cast(*response).path @@ -853,6 +939,7 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ { if (is_local) { + std::lock_guard lock(storage.container_mutex); auto node_it = storage.container.find(path); if (node_it == storage.container.end()) return {}; @@ -877,6 +964,13 @@ void handleSystemNodeModification(const KeeperContext & keeper_context, std::str } +KeeperStorage::UpdateNodeStatDelta::UpdateNodeStatDelta(const KeeperStorage::Node & node) + : old_stats(node.stat) + , new_stats(node.stat) + , old_seq_num(node.seq_num) + , new_seq_num(node.seq_num) +{} + bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session_id, bool is_local) { const auto node_acls = getNodeACLs(*this, path, is_local); @@ -931,13 +1025,13 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); } - std::vector + std::list preprocess(KeeperStorage & 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); - std::vector new_deltas; + std::list new_deltas; auto parent_path = parentNodePath(request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); @@ -982,21 +1076,21 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr int32_t parent_cversion = request.parent_cversion; - auto parent_update = [parent_cversion, zxid](KeeperStorage::Node & node) - { - /// Increment sequential number even if node is not sequential - ++node.seq_num; - if (parent_cversion == -1) - ++node.stat.cversion; - else if (parent_cversion > node.stat.cversion) - node.stat.cversion = parent_cversion; - if (zxid > node.stat.pzxid) - node.stat.pzxid = zxid; - ++node.stat.numChildren; - }; + KeeperStorage::UpdateNodeStatDelta update_parent_delta(*parent_node); + ++update_parent_delta.new_seq_num; - new_deltas.emplace_back(std::string{parent_path}, zxid, KeeperStorage::UpdateNodeDelta{std::move(parent_update)}); + if (parent_cversion == -1) + ++update_parent_delta.new_stats.cversion; + else if (parent_cversion > update_parent_delta.old_stats.cversion) + update_parent_delta.new_stats.cversion = parent_cversion; + + if (zxid > update_parent_delta.old_stats.pzxid) + update_parent_delta.new_stats.pzxid = zxid; + + ++update_parent_delta.new_stats.numChildren; + + new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(update_parent_delta)); Coordination::Stat stat; stat.czxid = zxid; @@ -1020,25 +1114,28 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + std::string created_path; + auto create_delta_it = std::find_if( + deltas.begin(), + deltas.end(), + [](const auto & delta) + { return std::holds_alternative(delta.operation); }); + + if (create_delta_it != deltas.end()) + created_path = create_delta_it->path; + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; } - const auto & deltas = storage.uncommitted_state.deltas; - auto create_delta_it = std::find_if( - deltas.begin(), - deltas.end(), - [zxid](const auto & delta) - { return delta.zxid == zxid && std::holds_alternative(delta.operation); }); - - response.path_created = create_delta_it->path; + response.path_created = std::move(created_path); response.error = Coordination::Error::ZOK; return response_ptr; } @@ -1053,7 +1150,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); @@ -1070,7 +1167,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); @@ -1078,7 +1175,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1113,15 +1210,15 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } }; @@ -1133,13 +1230,13 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & 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); - std::vector new_deltas; + std::list new_deltas; if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { @@ -1149,32 +1246,38 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } - const auto update_parent_pzxid = [&]() - { - auto parent_path = parentNodePath(request.path); - if (!storage.uncommitted_state.getNode(parent_path)) - return; + auto parent_path = parentNodePath(request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + KeeperStorage::UpdateNodeStatDelta update_parent_delta(*parent_node); + + const auto add_parent_update_delta = [&] + { new_deltas.emplace_back( std::string{parent_path}, zxid, - KeeperStorage::UpdateNodeDelta - { - [zxid](KeeperStorage::Node & parent) - { - if (parent.stat.pzxid < zxid) - parent.stat.pzxid = zxid; - } - } + std::move(update_parent_delta) ); }; + const auto update_parent_pzxid = [&]() + { + if (!parent_node) + return; + + if (update_parent_delta.old_stats.pzxid < zxid) + update_parent_delta.new_stats.pzxid = zxid; + }; + auto node = storage.uncommitted_state.getNode(request.path); if (!node) { if (request.restored_from_zookeeper_log) + { update_parent_pzxid(); + add_parent_update_delta(); + } return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; } else if (request.version != -1 && request.version != node->stat.version) @@ -1185,16 +1288,14 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr if (request.restored_from_zookeeper_log) update_parent_pzxid(); - new_deltas.emplace_back( - std::string{parentNodePath(request.path)}, - zxid, - KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent) - { - ++parent.stat.cversion; - --parent.stat.numChildren; - }}); + ++update_parent_delta.new_stats.cversion; + --update_parent_delta.new_stats.numChildren; + add_parent_update_delta(); - new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->stat.ephemeralOwner}); + new_deltas.emplace_back( + request.path, + zxid, + KeeperStorage::RemoveNodeDelta{request.version, node->stat, storage.uncommitted_state.getACLs(request.path), node->getData()}); if (node->stat.ephemeralOwner != 0) storage.unregisterEphemeralPath(node->stat.ephemeralOwner, request.path); @@ -1204,12 +1305,12 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); - response.error = storage.commit(zxid); + response.error = storage.commit(std::move(deltas)); return response_ptr; } @@ -1224,7 +1325,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); @@ -1237,7 +1338,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); @@ -1245,7 +1346,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1270,15 +1371,15 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } }; @@ -1290,13 +1391,13 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & 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); - std::vector new_deltas; + std::list new_deltas; if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { @@ -1314,37 +1415,31 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce if (request.version != -1 && request.version != node->stat.version) return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + KeeperStorage::UpdateNodeStatDelta node_delta(*node); + node_delta.version = request.version; + auto & new_stats = node_delta.new_stats; + new_stats.version++; + new_stats.mzxid = zxid; + new_stats.mtime = time; + new_stats.dataLength = static_cast(request.data.length()); + + new_deltas.emplace_back(request.path, zxid, std::move(node_delta)); new_deltas.emplace_back( request.path, zxid, - KeeperStorage::UpdateNodeDelta{ - [zxid, data = request.data, time](KeeperStorage::Node & value) - { - value.stat.version++; - value.stat.mzxid = zxid; - value.stat.mtime = time; - value.stat.dataLength = static_cast(data.length()); - value.setData(data); - }, - request.version}); + KeeperStorage::UpdateNodeDataDelta{.old_data = node->getData(), .new_data = request.data, .version = request.version}); - new_deltas.emplace_back( - parentNodePath(request.path).toString(), - zxid, - KeeperStorage::UpdateNodeDelta - { - [](KeeperStorage::Node & parent) - { - parent.stat.cversion++; - } - } - ); + auto parent_path = parentNodePath(request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + KeeperStorage::UpdateNodeStatDelta parent_delta(*parent_node); + ++parent_delta.new_stats.cversion; + new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(parent_delta)); digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { auto & container = storage.container; @@ -1352,7 +1447,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); Coordination::ZooKeeperSetRequest & request = dynamic_cast(*zk_request); - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1383,7 +1478,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); @@ -1397,7 +1492,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); @@ -1405,7 +1500,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1466,15 +1561,15 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } }; @@ -1492,7 +1587,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return storage.checkACL(check_not_exists ? parentNodePath(path) : path, Coordination::ACL::Read, session_id, is_local); } - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); @@ -1518,7 +1613,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); @@ -1526,7 +1621,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1564,15 +1659,15 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } private: @@ -1589,7 +1684,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & 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); @@ -1616,35 +1711,26 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr if (!fixupACL(request.acls, session_id, uncommitted_state, node_acls)) return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; - std::vector new_deltas - { - { - request.path, - zxid, - KeeperStorage::SetACLDelta{std::move(node_acls), request.version} - }, - { - request.path, - zxid, - KeeperStorage::UpdateNodeDelta - { - [](KeeperStorage::Node & n) { ++n.stat.aversion; } - } - } - }; + KeeperStorage::UpdateNodeStatDelta update_stat_delta(*node); + ++update_stat_delta.new_stats.aversion; + std::list new_deltas{ + {request.path, + zxid, + KeeperStorage::SetACLDelta{std::move(node_acls), uncommitted_state.getACLs(request.path), request.version}}, + {request.path, zxid, std::move(update_stat_delta)}}; digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*zk_request); - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1669,7 +1755,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & 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); @@ -1681,7 +1767,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); @@ -1689,7 +1775,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1714,14 +1800,14 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } }; @@ -1799,18 +1885,19 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro assert(request.requests.empty() || operation_type.has_value()); } - std::vector + std::list preprocess(KeeperStorage & 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; response_errors.reserve(concrete_requests.size()); uint64_t current_digest = digest; + std::list new_deltas; for (size_t i = 0; i < concrete_requests.size(); ++i) { - auto new_deltas = concrete_requests[i]->preprocess(storage, zxid, session_id, time, current_digest, keeper_context); + auto new_subdeltas = concrete_requests[i]->preprocess(storage, zxid, session_id, time, current_digest, keeper_context); - if (!new_deltas.empty()) + if (!new_subdeltas.empty()) { if (auto * error = std::get_if(&new_deltas.back().operation); error && *operation_type == OperationType::Write) @@ -1826,24 +1913,49 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return {KeeperStorage::Delta{zxid, KeeperStorage::FailedMultiDelta{std::move(response_errors)}}}; } } - new_deltas.emplace_back(zxid, KeeperStorage::SubDeltaEnd{}); + + new_subdeltas.emplace_back(zxid, KeeperStorage::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 - storage.uncommitted_state.addDeltas(std::move(new_deltas)); + storage.uncommitted_state.applyDeltas(new_subdeltas); + new_deltas.splice(new_deltas.end(), std::move(new_subdeltas)); } digest = current_digest; + storage.uncommitted_state.addDeltas(std::move(new_deltas)); + return {}; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + std::list getSubdeltas(std::list & deltas) const + { + std::list subdeltas; + auto it = deltas.begin(); + + for (; it != deltas.end(); ++it) + { + if (std::holds_alternative(it->operation)) + { + ++it; + break; + } + } + + if (it == deltas.end()) + subdeltas = std::move(deltas); + else + subdeltas.splice(subdeltas.end(), deltas, deltas.begin(), it); + + return subdeltas; + } + + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = 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)) @@ -1854,21 +1966,20 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro response.responses[i]->error = failed_multi->error_codes[i]; } - storage.uncommitted_state.commit(zxid); return response_ptr; } for (size_t i = 0; i < concrete_requests.size(); ++i) { - response.responses[i] = concrete_requests[i]->process(storage, zxid); - storage.uncommitted_state.commit(zxid); + std::list subdeltas = getSubdeltas(deltas); + response.responses[i] = concrete_requests[i]->process(storage, std::move(subdeltas)); } response.error = Coordination::Error::ZOK; return response_ptr; } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); @@ -1876,7 +1987,8 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro for (size_t i = 0; i < concrete_requests.size(); ++i) { - response.responses[i] = concrete_requests[i]->processLocal(storage, zxid); + std::list subdeltas = getSubdeltas(deltas); + response.responses[i] = concrete_requests[i]->processLocal(storage, std::move(subdeltas)); } response.error = Coordination::Error::ZOK; @@ -1899,7 +2011,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - Coordination::ZooKeeperResponsePtr process(KeeperStorage &, int64_t) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage &, std::list /* deltas */) const override { throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Called process on close request"); } @@ -1908,7 +2020,7 @@ struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestPro struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & 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); @@ -1917,29 +2029,32 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc if (auth_request.scheme != "digest" || std::count(auth_request.data.begin(), auth_request.data.end(), ':') != 1) return {KeeperStorage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; - std::vector new_deltas; + std::list new_deltas; auto auth_digest = KeeperStorage::generateDigest(auth_request.data); if (auth_digest == storage.superdigest) { - KeeperStorage::AuthID auth{"super", ""}; + auto auth = std::make_shared(); + auth->scheme = "super"; new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(auth)}); } else { - KeeperStorage::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; })) + auto new_auth = std::make_shared(); + new_auth->scheme = auth_request.scheme; + new_auth->id = std::move(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)}); } return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) auth_response.error = result; return response_ptr; @@ -2030,7 +2145,7 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() } -UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const +UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::list & new_deltas) const { if (!keeper_context->digestEnabled()) return current_digest; @@ -2058,7 +2173,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec updated_nodes.insert_or_assign(delta.path, nullptr); }, - [&](const UpdateNodeDelta & update_delta) + [&](const UpdateNodeStatDelta & update_delta) { std::shared_ptr node{nullptr}; @@ -2073,7 +2188,25 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec else node = updated_node_it->second; - update_delta.update_fn(*node); + node->stat = update_delta.new_stats; + node->seq_num = update_delta.new_seq_num; + }, + [&](const UpdateNodeDataDelta & update_delta) + { + std::shared_ptr node{nullptr}; + + auto updated_node_it = updated_nodes.find(delta.path); + if (updated_node_it == updated_nodes.end()) + { + node = std::make_shared(); + node->shallowCopy(*uncommitted_state.getNode(delta.path)); + current_digest -= node->getDigest(delta.path); + updated_nodes.emplace(delta.path, node); + } + else + node = updated_node_it->second; + + node->setData(update_delta.new_data); }, [](auto && /* delta */) {}}, delta.operation); @@ -2102,43 +2235,50 @@ void KeeperStorage::preprocessRequest( if (!initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage system nodes are not initialized"); - int64_t last_zxid = getNextZXID() - 1; + TransactionInfo * transaction; + uint64_t new_digest = 0; - if (uncommitted_transactions.empty()) { - // if we have no uncommitted transactions it means the last zxid is possibly loaded from snapshot - if (last_zxid != old_snapshot_zxid && new_last_zxid <= last_zxid) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", - new_last_zxid, last_zxid); - } - else - { - if (last_zxid == new_last_zxid && digest && checkDigest(*digest, getNodesDigest(false))) - // we found the preprocessed request with the same ZXID, we can skip it - return; + std::lock_guard lock(transaction_mutex); + int64_t last_zxid = getNextZXIDLocked(lock) - 1; - if (new_last_zxid <= last_zxid) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", - new_last_zxid, last_zxid); + if (uncommitted_transactions.empty()) + { + // if we have no uncommitted transactions it means the last zxid is possibly loaded from snapshot + if (last_zxid != old_snapshot_zxid && new_last_zxid <= last_zxid) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", + new_last_zxid, last_zxid); + } + else + { + if (last_zxid == new_last_zxid && digest && checkDigest(*digest, getNodesDigest(false))) + // we found the preprocessed request with the same ZXID, we can skip it + return; + + if (new_last_zxid <= last_zxid) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", + new_last_zxid, last_zxid); + } + + transaction = &uncommitted_transactions.emplace_back(TransactionInfo{.zxid = new_last_zxid, .nodes_digest = {}}); + new_digest = getNodesDigest(false).value; } - std::vector new_deltas; - TransactionInfo transaction{.zxid = new_last_zxid, .nodes_digest = {}}; - uint64_t new_digest = getNodesDigest(false).value; + std::list new_deltas; SCOPE_EXIT({ if (keeper_context->digestEnabled()) // if the version of digest we got from the leader is the same as the one this instances has, we can simply copy the value // and just check the digest on the commit // a mistake can happen while applying the changes to the uncommitted_state so for now let's just recalculate the digest here also - transaction.nodes_digest = Digest{CURRENT_DIGEST_VERSION, new_digest}; + transaction->nodes_digest = Digest{CURRENT_DIGEST_VERSION, new_digest}; else - transaction.nodes_digest = Digest{DigestVersion::NO_DIGEST}; + transaction->nodes_digest = Digest{DigestVersion::NO_DIGEST}; - uncommitted_transactions.emplace_back(transaction); + uncommitted_state.applyDeltas(new_deltas); uncommitted_state.addDeltas(std::move(new_deltas)); }); @@ -2146,30 +2286,30 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { - auto session_ephemerals = ephemerals.find(session_id); - if (session_ephemerals != ephemerals.end()) - { - for (const auto & ephemeral_path : session_ephemerals->second) - { - new_deltas.emplace_back - ( - parentNodePath(ephemeral_path).toString(), - new_last_zxid, - UpdateNodeDelta - { - [ephemeral_path](Node & parent) - { - ++parent.stat.cversion; - --parent.stat.numChildren; - } - } - ); + //auto session_ephemerals = ephemerals.find(session_id); + //if (session_ephemerals != ephemerals.end()) + //{ + // for (const auto & ephemeral_path : session_ephemerals->second) + // { + // new_deltas.emplace_back + // ( + // parentNodePath(ephemeral_path).toString(), + // new_last_zxid, + // UpdateNodeDelta + // { + // [ephemeral_path](Node & parent) + // { + // ++parent.stat.cversion; + // --parent.stat.numChildren; + // } + // } + // ); - new_deltas.emplace_back(ephemeral_path, transaction.zxid, RemoveNodeDelta{.ephemeral_owner = session_id}); - } + // new_deltas.emplace_back(ephemeral_path, transaction->zxid, RemoveNodeDelta{.ephemeral_owner = session_id}); + // } - ephemerals.erase(session_ephemerals); - } + // ephemerals.erase(session_ephemerals); + //} new_digest = calculateNodesDigest(new_digest, new_deltas); return; @@ -2177,11 +2317,11 @@ void KeeperStorage::preprocessRequest( if (check_acl && !request_processor->checkAuth(*this, session_id, false)) { - uncommitted_state.deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); + new_deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); return; } - new_deltas = request_processor->preprocess(*this, transaction.zxid, session_id, time, new_digest, *keeper_context); + new_deltas = request_processor->preprocess(*this, transaction->zxid, session_id, time, new_digest, *keeper_context); } KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( @@ -2194,20 +2334,26 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( if (!initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage system nodes are not initialized"); - if (new_last_zxid) + int64_t commit_zxid = 0; { - if (uncommitted_transactions.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to commit a ZXID ({}) which was not preprocessed", *new_last_zxid); + std::lock_guard lock(transaction_mutex); + if (new_last_zxid) + { + if (uncommitted_transactions.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to commit a ZXID ({}) which was not preprocessed", *new_last_zxid); - if (uncommitted_transactions.front().zxid != *new_last_zxid) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Trying to commit a ZXID {} while the next ZXID to commit is {}", - *new_last_zxid, - uncommitted_transactions.front().zxid); + if (uncommitted_transactions.front().zxid != *new_last_zxid) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Trying to commit a ZXID {} while the next ZXID to commit is {}", + *new_last_zxid, + uncommitted_transactions.front().zxid); - zxid = *new_last_zxid; - uncommitted_transactions.pop_front(); + zxid = *new_last_zxid; + uncommitted_transactions.pop_front(); + } + + commit_zxid = zxid; } KeeperStorage::ResponsesForSessions results; @@ -2215,15 +2361,20 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// ZooKeeper update sessions expirity for each request, not only for heartbeats session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); + std::list deltas; + { + std::lock_guard lock(uncommitted_state.deltas_mutex); + auto it = uncommitted_state.deltas.begin(); + for (; it != uncommitted_state.deltas.end() && it->zxid == commit_zxid; ++it) + ; + + deltas.splice(deltas.end(), uncommitted_state.deltas, uncommitted_state.deltas.begin(), it); + } + if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { - commit(zxid); - - for (const auto & delta : uncommitted_state.deltas) + for (const auto & delta : deltas) { - if (delta.zxid > zxid) - break; - if (std::holds_alternative(delta.operation)) { auto responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); @@ -2231,17 +2382,20 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } } - uncommitted_state.commit(zxid); + commit(std::move(deltas)); clearDeadWatches(session_id); - auto auth_it = session_and_auth.find(session_id); - if (auth_it != session_and_auth.end()) - session_and_auth.erase(auth_it); + { + std::lock_guard lock(auth_mutex); + auto auth_it = session_and_auth.find(session_id); + if (auth_it != session_and_auth.end()) + session_and_auth.erase(auth_it); + } /// Finish connection auto response = std::make_shared(); response->xid = zk_request->xid; - response->zxid = getZXID(); + response->zxid = commit_zxid; session_expiry_queue.remove(session_id); session_and_timeout.erase(session_id); results.push_back(ResponseForSession{session_id, response}); @@ -2249,9 +2403,9 @@ 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 response = storage_request->process(*this, zxid); + auto response = storage_request->process(*this, std::move(deltas)); response->xid = zk_request->xid; - response->zxid = getZXID(); + response->zxid = commit_zxid; results.push_back(ResponseForSession{session_id, response}); } @@ -2271,13 +2425,12 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } else { - response = request_processor->processLocal(*this, zxid); + response = request_processor->processLocal(*this, std::move(deltas)); } } else { - response = request_processor->process(*this, zxid); - uncommitted_state.commit(zxid); + response = request_processor->process(*this, std::move(deltas)); } /// Watches for this requests are added to the watches lists @@ -2310,7 +2463,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } response->xid = zk_request->xid; - response->zxid = getZXID(); + response->zxid = commit_zxid; results.push_back(ResponseForSession{session_id, response}); } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 844cbf85c1e..ab1ef201a4e 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -11,6 +11,8 @@ #include #include +#include + #include namespace DB @@ -138,11 +140,13 @@ public: int64_t session_id_counter{1}; - SessionAndAuth session_and_auth; + mutable std::mutex auth_mutex; + SessionAndAuth session_and_auth TSA_GUARDED_BY(auth_mutex); /// Main hashtable with nodes. Contain all information about data. /// All other structures expect session_and_timeout can be restored from /// container. + mutable std::mutex container_mutex; Container container; // Applying ZooKeeper request to storage consists of two steps: @@ -166,18 +170,33 @@ public: struct RemoveNodeDelta { int32_t version{-1}; - int64_t ephemeral_owner{0}; + Coordination::Stat stat; + Coordination::ACLs acls; + String data; }; - struct UpdateNodeDelta + struct UpdateNodeStatDelta { - std::function update_fn; + explicit UpdateNodeStatDelta(const KeeperStorage::Node & node); + + Coordination::Stat old_stats; + Coordination::Stat new_stats; + int32_t old_seq_num; + int32_t new_seq_num; + int32_t version{-1}; + }; + + struct UpdateNodeDataDelta + { + std::string old_data; + std::string new_data; int32_t version{-1}; }; struct SetACLDelta { - Coordination::ACLs acls; + Coordination::ACLs old_acls; + Coordination::ACLs new_acls; int32_t version{-1}; }; @@ -199,11 +218,19 @@ public: struct AddAuthDelta { int64_t session_id; - AuthID auth_id; + std::shared_ptr auth_id; }; - using Operation = std:: - variant; + using Operation = std::variant< + CreateNodeDelta, + RemoveNodeDelta, + UpdateNodeStatDelta, + UpdateNodeDataDelta, + SetACLDelta, + AddAuthDelta, + ErrorDelta, + SubDeltaEnd, + FailedMultiDelta>; struct Delta { @@ -222,15 +249,16 @@ public: { explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { } - void addDelta(Delta new_delta); - void addDeltas(std::vector new_deltas); + void addDeltas(std::list new_deltas); void commit(int64_t commit_zxid); void rollback(int64_t rollback_zxid); std::shared_ptr getNode(StringRef path) const; Coordination::ACLs getACLs(StringRef path) const; + void applyDeltas(const std::list & new_deltas); void applyDelta(const Delta & delta); + void rollbackDelta(const Delta & delta); bool hasACL(int64_t session_id, bool is_local, std::function predicate) { @@ -238,13 +266,13 @@ public: { for (const auto & auth : auth_ids) { - using TAuth = std::remove_reference_t; + using TAuth = std::remove_cvref_t; const AuthID * auth_ptr = nullptr; - if constexpr (std::is_pointer_v) - auth_ptr = auth; - else + if constexpr (std::same_as) auth_ptr = &auth; + else + auth_ptr = auth.get(); if (predicate(*auth_ptr)) return true; @@ -253,30 +281,34 @@ public: }; if (is_local) + { + std::lock_guard lock(storage.auth_mutex); return check_auth(storage.session_and_auth[session_id]); - - if (check_auth(storage.session_and_auth[session_id])) - return true; + } // check if there are uncommitted const auto auth_it = session_and_auth.find(session_id); if (auth_it == session_and_auth.end()) return false; - return check_auth(auth_it->second); + if (check_auth(auth_it->second)) + return true; + + std::lock_guard lock(storage.auth_mutex); + return check_auth(storage.session_and_auth[session_id]); } void forEachAuthInSession(int64_t session_id, std::function func) const; std::shared_ptr tryGetNodeFromStorage(StringRef path) const; - std::unordered_map> session_and_auth; + std::unordered_map>> session_and_auth; struct UncommittedNode { std::shared_ptr node{nullptr}; - Coordination::ACLs acls{}; - int64_t zxid{0}; + std::optional acls{}; + std::vector applied_zxids{}; }; struct Hash @@ -303,9 +335,9 @@ public: }; mutable std::unordered_map nodes; - std::unordered_map, Hash, Equal> deltas_for_path; - std::list deltas; + mutable std::mutex deltas_mutex; + std::list deltas TSA_GUARDED_BY(deltas_mutex); KeeperStorage & storage; }; @@ -315,7 +347,7 @@ public: // with zxid > last_zxid void applyUncommittedState(KeeperStorage & other, int64_t last_zxid); - Coordination::Error commit(int64_t zxid); + Coordination::Error commit(std::list deltas); // Create node in the storage // Returns false if it failed to create the node, true otherwise @@ -348,8 +380,10 @@ public: /// ACLMap for more compact ACLs storage inside nodes. ACLMap acl_map; + mutable std::mutex transaction_mutex; + /// Global id of all requests applied to storage - int64_t zxid{0}; + int64_t zxid = 0; // older Keeper node (pre V5 snapshots) can create snapshots and receive logs from newer Keeper nodes // this can lead to some inconsistencies, e.g. from snapshot it will use log_idx as zxid @@ -364,7 +398,7 @@ public: Digest nodes_digest; }; - std::deque uncommitted_transactions; + std::list uncommitted_transactions; uint64_t nodes_digest{0}; @@ -377,15 +411,10 @@ public: void clearDeadWatches(int64_t session_id); /// Get current committed zxid - int64_t getZXID() const { return zxid; } + int64_t getZXID() const; - int64_t getNextZXID() const - { - if (uncommitted_transactions.empty()) - return zxid + 1; - - return uncommitted_transactions.back().zxid + 1; - } + int64_t getNextZXID() const; + int64_t getNextZXIDLocked(std::lock_guard & lock) const; Digest getNodesDigest(bool committed) const; @@ -393,7 +422,7 @@ public: const String superdigest; - bool initialized{false}; + std::atomic initialized{false}; KeeperStorage(int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, bool initialize_system_nodes = true); @@ -415,7 +444,7 @@ public: session_expiry_queue.addNewSessionOrUpdate(session_id, session_timeout_ms); } - UInt64 calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const; + UInt64 calculateNodesDigest(UInt64 current_digest, const std::list & new_deltas) const; /// Process user request and return response. /// check_acl = false only when converting data from ZooKeeper. diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 79929c4e66e..d7ce097b795 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -147,7 +147,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L return max_zxid; } -void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log) +void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log) TSA_NO_THREAD_SAFETY_ANALYSIS { LOG_INFO(log, "Deserializing storage snapshot {}", snapshot_path); int64_t zxid = getZxidFromName(snapshot_path); @@ -474,7 +474,7 @@ bool hasErrorsInMultiRequest(Coordination::ZooKeeperRequestPtr request) } -bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*log*/) +bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*log*/) TSA_NO_THREAD_SAFETY_ANALYSIS { int64_t checksum; Coordination::read(checksum, in); @@ -553,7 +553,7 @@ 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) +void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log) TSA_NO_THREAD_SAFETY_ANALYSIS { namespace fs = std::filesystem; std::map existing_logs; From dd1bb579df57480f5df75e240e4c46f688c5204a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 5 Sep 2023 10:49:19 +0000 Subject: [PATCH 02/27] Better --- src/Coordination/KeeperStateMachine.cpp | 2 - src/Coordination/KeeperStorage.cpp | 223 +++++++++++------------- src/Coordination/KeeperStorage.h | 22 +-- 3 files changed, 116 insertions(+), 131 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 9d96a76e06a..c0f66f246d6 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -509,7 +509,6 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; - std::lock_guard lock(storage_and_responses_lock); storage->rollbackRequest(request_for_session.zxid, allow_missing); } @@ -756,7 +755,6 @@ std::vector KeeperStateMachine::getDeadSessions() int64_t KeeperStateMachine::getNextZxid() const { - std::lock_guard lock(storage_and_responses_lock); return storage->getNextZXID(); } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 0978f5f965e..f881b75a808 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -317,7 +317,7 @@ Overloaded(Ts...) -> Overloaded; std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const { - std::lock_guard lock(storage.container_mutex); + std::lock_guard lock(storage.storage_mutex); if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; @@ -455,7 +455,7 @@ void KeeperStorage::UncommittedState::applyDeltas(const std::list & new_d else if (const auto * auth_delta = std::get_if(&delta.operation)) { auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; - uncommitted_auth.emplace_back(auth_delta->auth_id); + uncommitted_auth.push_back(std::pair{delta.zxid, auth_delta->auth_id}); } } } @@ -466,62 +466,30 @@ void KeeperStorage::UncommittedState::addDeltas(std::list new_deltas) deltas.splice(deltas.end(), std::move(new_deltas)); } -//void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) -//{ -// assert(deltas.empty() || deltas.front().zxid >= commit_zxid); -// -// // collect nodes that have no further modification in the current transaction -// std::unordered_set modified_nodes; -// -// while (!deltas.empty() && deltas.front().zxid == commit_zxid) -// { -// if (std::holds_alternative(deltas.front().operation)) -// { -// deltas.pop_front(); -// break; -// } -// -// auto & front_delta = deltas.front(); -// -// if (!front_delta.path.empty()) -// { -// auto & path_deltas = deltas_for_path.at(front_delta.path); -// assert(path_deltas.front() == &front_delta); -// path_deltas.pop_front(); -// if (path_deltas.empty()) -// { -// deltas_for_path.erase(front_delta.path); -// -// // no more deltas for path -> no modification -// modified_nodes.insert(std::move(front_delta.path)); -// } -// else if (path_deltas.front()->zxid > commit_zxid) -// { -// // next delta has a zxid from a different transaction -> no modification in this transaction -// modified_nodes.insert(std::move(front_delta.path)); -// } -// } -// else if (auto * add_auth = std::get_if(&front_delta.operation)) -// { -// auto & uncommitted_auth = session_and_auth[add_auth->session_id]; -// assert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id); -// uncommitted_auth.pop_front(); -// if (uncommitted_auth.empty()) -// session_and_auth.erase(add_auth->session_id); -// -// } -// -// deltas.pop_front(); -// } -// -// // delete all cached nodes that were not modified after the commit_zxid -// // we only need to check the nodes that were modified in this transaction -// for (const auto & node : modified_nodes) -// { -// if (nodes[node].zxid == commit_zxid) -// nodes.erase(node); -// } -//} +void KeeperStorage::UncommittedState::cleanup(int64_t commit_zxid) +{ + for (auto it = nodes.begin(); it != nodes.end();) + { + auto & applied_zxids = it->second.applied_zxids; + std::erase_if(applied_zxids, [commit_zxid](auto current_zxid) { return current_zxid <= commit_zxid; }); + if (applied_zxids.empty()) + it = nodes.erase(it); + else + ++it; + } + + for (auto it = session_and_auth.begin(); it != session_and_auth.end();) + { + auto & auths = it->second; + std::erase_if(auths, [commit_zxid](auto auth_pair) { return auth_pair.first <= commit_zxid; }); + if (auths.empty()) + it = session_and_auth.erase(it); + else + ++it; + } + + +} void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) { @@ -575,7 +543,7 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) else if (auto * add_auth = std::get_if(&delta.operation)) { auto & uncommitted_auth = session_and_auth[add_auth->session_id]; - if (uncommitted_auth.back() == add_auth->auth_id) + if (uncommitted_auth.back().second == add_auth->auth_id) { uncommitted_auth.pop_back(); if (uncommitted_auth.empty()) @@ -634,22 +602,22 @@ void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, s if constexpr (std::same_as) auth_ptr = &auth; else - auth_ptr = auth.get(); + auth_ptr = auth.second.get(); func(*auth_ptr); } }; { - std::lock_guard lock(storage.auth_mutex); + std::lock_guard lock(storage.storage_mutex); // for committed - if (storage.session_and_auth.contains(session_id)) - call_for_each_auth(storage.session_and_auth.at(session_id)); + if (auto auth_it = storage.session_and_auth.find(session_id); auth_it != storage.session_and_auth.end()) + call_for_each_auth(auth_it->second); } // for uncommitted - if (session_and_auth.contains(session_id)) - call_for_each_auth(session_and_auth.at(session_id)); + if (auto auth_it = session_and_auth.find(session_id); auth_it != session_and_auth.end()) + call_for_each_auth(auth_it->second); } namespace @@ -686,29 +654,29 @@ int64_t KeeperStorage::getNextZXID() const return getNextZXIDLocked(lock); } -void KeeperStorage::applyUncommittedState(KeeperStorage & /*other*/, int64_t /*last_zxid*/) +void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_zxid) TSA_NO_THREAD_SAFETY_ANALYSIS { - //for (const auto & transaction : uncommitted_transactions) - //{ - // if (transaction.zxid <= last_zxid) - // continue; - // other.uncommitted_transactions.push_back(transaction); - //} + for (const auto & transaction : uncommitted_transactions) + { + if (transaction.zxid <= last_zxid) + continue; + other.uncommitted_transactions.push_back(transaction); + } - //auto it = uncommitted_state.deltas.begin(); + auto it = uncommitted_state.deltas.begin(); - //for (; it != uncommitted_state.deltas.end(); ++it) - //{ - // if (it->zxid <= last_zxid) - // continue; + for (; it != uncommitted_state.deltas.end(); ++it) + { + if (it->zxid <= last_zxid) + continue; - // other.uncommitted_state.addDelta(*it); - //} + other.uncommitted_state.applyDelta(*it); + other.uncommitted_state.deltas.push_back(*it); + } } Coordination::Error KeeperStorage::commit(std::list deltas) { - std::lock_guard lock(container_mutex); // Deltas are added with increasing ZXIDs // If there are no deltas for the commit_zxid (e.g. read requests), we instantly return // on first delta @@ -787,7 +755,6 @@ Coordination::Error KeeperStorage::commit(std::list deltas) } else if constexpr (std::same_as) { - std::lock_guard auth_lock(auth_mutex); session_and_auth[operation.session_id].emplace_back(std::move(*operation.auth_id)); return Coordination::Error::ZOK; } @@ -939,7 +906,6 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ { if (is_local) { - std::lock_guard lock(storage.container_mutex); auto node_it = storage.container.find(path); if (node_it == storage.container.end()) return {}; @@ -2286,30 +2252,33 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { - //auto session_ephemerals = ephemerals.find(session_id); - //if (session_ephemerals != ephemerals.end()) - //{ - // for (const auto & ephemeral_path : session_ephemerals->second) - // { - // new_deltas.emplace_back - // ( - // parentNodePath(ephemeral_path).toString(), - // new_last_zxid, - // UpdateNodeDelta - // { - // [ephemeral_path](Node & parent) - // { - // ++parent.stat.cversion; - // --parent.stat.numChildren; - // } - // } - // ); + auto session_ephemerals = ephemerals.find(session_id); + if (session_ephemerals != ephemerals.end()) + { + for (const auto & ephemeral_path : session_ephemerals->second) + { + auto parent_node_path = parentNodePath(ephemeral_path); + auto parent_node = uncommitted_state.getNode(parent_node_path); + UpdateNodeStatDelta parent_update_delta(*parent_node); + ++parent_update_delta.new_stats.cversion; + --parent_update_delta.new_stats.numChildren; - // new_deltas.emplace_back(ephemeral_path, transaction->zxid, RemoveNodeDelta{.ephemeral_owner = session_id}); - // } + new_deltas.emplace_back + ( + parent_node_path.toString(), + new_last_zxid, + std::move(parent_update_delta) + ); - // ephemerals.erase(session_ephemerals); - //} + auto node = uncommitted_state.getNode(ephemeral_path); + new_deltas.emplace_back( + ephemeral_path, + transaction->zxid, + RemoveNodeDelta{.stat = node->stat, .acls = uncommitted_state.getACLs(ephemeral_path), .data = node->getData()}); + } + + ephemerals.erase(session_ephemerals); + } new_digest = calculateNodesDigest(new_digest, new_deltas); return; @@ -2322,6 +2291,8 @@ void KeeperStorage::preprocessRequest( } new_deltas = request_processor->preprocess(*this, transaction->zxid, session_id, time, new_digest, *keeper_context); + + uncommitted_state.cleanup(getZXID()); } KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( @@ -2349,18 +2320,14 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( *new_last_zxid, uncommitted_transactions.front().zxid); - zxid = *new_last_zxid; - uncommitted_transactions.pop_front(); + commit_zxid = *new_last_zxid; + } + else + { + commit_zxid = zxid; } - - commit_zxid = zxid; } - KeeperStorage::ResponsesForSessions results; - - /// ZooKeeper update sessions expirity for each request, not only for heartbeats - session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); - std::list deltas; { std::lock_guard lock(uncommitted_state.deltas_mutex); @@ -2371,6 +2338,11 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( deltas.splice(deltas.end(), uncommitted_state.deltas, uncommitted_state.deltas.begin(), it); } + KeeperStorage::ResponsesForSessions results; + + /// ZooKeeper update sessions expirity for each request, not only for heartbeats + session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); + if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { for (const auto & delta : deltas) @@ -2382,16 +2354,16 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } } - commit(std::move(deltas)); - - clearDeadWatches(session_id); { - std::lock_guard lock(auth_mutex); + std::lock_guard lock(storage_mutex); + commit(std::move(deltas)); auto auth_it = session_and_auth.find(session_id); if (auth_it != session_and_auth.end()) session_and_auth.erase(auth_it); } + clearDeadWatches(session_id); + /// Finish connection auto response = std::make_shared(); response->xid = zk_request->xid; @@ -2403,7 +2375,11 @@ 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 response = storage_request->process(*this, std::move(deltas)); + Coordination::ZooKeeperResponsePtr response = nullptr; + { + std::lock_guard lock(storage_mutex); + response = storage_request->process(*this, std::move(deltas)); + } response->xid = zk_request->xid; response->zxid = commit_zxid; @@ -2416,6 +2392,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( if (is_local) { + std::lock_guard lock(storage_mutex); assert(zk_request->isReadRequest()); if (check_acl && !request_processor->checkAuth(*this, session_id, true)) { @@ -2430,6 +2407,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } else { + std::lock_guard lock(storage_mutex); response = request_processor->process(*this, std::move(deltas)); } @@ -2468,11 +2446,19 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( results.push_back(ResponseForSession{session_id, response}); } + { + std::lock_guard lock(transaction_mutex); + if (new_last_zxid) + uncommitted_transactions.pop_front(); + zxid = commit_zxid; + } + return results; } void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) { + std::unique_lock transaction_lock(transaction_mutex); if (allow_missing && (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid < rollback_zxid)) return; @@ -2488,6 +2474,7 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) try { uncommitted_transactions.pop_back(); + transaction_lock.unlock(); uncommitted_state.rollback(rollback_zxid); } catch (...) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index ab1ef201a4e..84bf7d297b4 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -138,15 +139,17 @@ public: using SessionAndAuth = std::unordered_map; using Watches = std::map; + mutable SharedMutex main_mutex; + + mutable std::mutex storage_mutex; + int64_t session_id_counter{1}; - mutable std::mutex auth_mutex; - SessionAndAuth session_and_auth TSA_GUARDED_BY(auth_mutex); + SessionAndAuth session_and_auth; /// Main hashtable with nodes. Contain all information about data. /// All other structures expect session_and_timeout can be restored from /// container. - mutable std::mutex container_mutex; Container container; // Applying ZooKeeper request to storage consists of two steps: @@ -250,7 +253,7 @@ public: explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { } void addDeltas(std::list new_deltas); - void commit(int64_t commit_zxid); + void cleanup(int64_t commit_zxid); void rollback(int64_t rollback_zxid); std::shared_ptr getNode(StringRef path) const; @@ -272,7 +275,7 @@ public: if constexpr (std::same_as) auth_ptr = &auth; else - auth_ptr = auth.get(); + auth_ptr = auth.second.get(); if (predicate(*auth_ptr)) return true; @@ -281,10 +284,7 @@ public: }; if (is_local) - { - std::lock_guard lock(storage.auth_mutex); return check_auth(storage.session_and_auth[session_id]); - } // check if there are uncommitted const auto auth_it = session_and_auth.find(session_id); @@ -294,7 +294,7 @@ public: if (check_auth(auth_it->second)) return true; - std::lock_guard lock(storage.auth_mutex); + std::lock_guard lock(storage.storage_mutex); return check_auth(storage.session_and_auth[session_id]); } @@ -302,7 +302,7 @@ public: std::shared_ptr tryGetNodeFromStorage(StringRef path) const; - std::unordered_map>> session_and_auth; + std::unordered_map>>> session_and_auth; struct UncommittedNode { @@ -402,7 +402,7 @@ public: uint64_t nodes_digest{0}; - bool finalized{false}; + std::atomic finalized{false}; /// Currently active watches (node_path -> subscribed sessions) Watches watches; From 6be1d0724a2edc71fc2af7510cd504289946c19f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 6 Sep 2023 13:04:08 +0000 Subject: [PATCH 03/27] More mutex --- src/Coordination/ACLMap.cpp | 5 + src/Coordination/ACLMap.h | 2 + src/Coordination/KeeperStateMachine.cpp | 66 +++--- src/Coordination/KeeperStateMachine.h | 7 +- src/Coordination/KeeperStorage.cpp | 277 +++++++++++++++++------ src/Coordination/KeeperStorage.h | 92 ++++---- src/Coordination/ZooKeeperDataReader.cpp | 2 +- 7 files changed, 299 insertions(+), 152 deletions(-) diff --git a/src/Coordination/ACLMap.cpp b/src/Coordination/ACLMap.cpp index 5c4a8f397ca..f92e497119e 100644 --- a/src/Coordination/ACLMap.cpp +++ b/src/Coordination/ACLMap.cpp @@ -45,6 +45,7 @@ uint64_t ACLMap::convertACLs(const Coordination::ACLs & acls) if (acls.empty()) return 0; + std::lock_guard lock(map_mutex); if (acl_to_num.contains(acls)) return acl_to_num[acls]; @@ -62,6 +63,7 @@ Coordination::ACLs ACLMap::convertNumber(uint64_t acls_id) const if (acls_id == 0) return Coordination::ACLs{}; + std::lock_guard lock(map_mutex); if (!num_to_acl.contains(acls_id)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown ACL id {}. It's a bug", acls_id); @@ -70,6 +72,7 @@ Coordination::ACLs ACLMap::convertNumber(uint64_t acls_id) const void ACLMap::addMapping(uint64_t acls_id, const Coordination::ACLs & acls) { + std::lock_guard lock(map_mutex); num_to_acl[acls_id] = acls; acl_to_num[acls] = acls_id; max_acl_id = std::max(acls_id + 1, max_acl_id); /// max_acl_id pointer next slot @@ -77,11 +80,13 @@ void ACLMap::addMapping(uint64_t acls_id, const Coordination::ACLs & acls) void ACLMap::addUsage(uint64_t acl_id) { + std::lock_guard lock(map_mutex); usage_counter[acl_id]++; } void ACLMap::removeUsage(uint64_t acl_id) { + std::lock_guard lock(map_mutex); if (!usage_counter.contains(acl_id)) return; diff --git a/src/Coordination/ACLMap.h b/src/Coordination/ACLMap.h index e1b2ce1eff6..7a635951346 100644 --- a/src/Coordination/ACLMap.h +++ b/src/Coordination/ACLMap.h @@ -32,6 +32,8 @@ private: NumToACLMap num_to_acl; UsageCounter usage_counter; uint64_t max_acl_id{1}; + + mutable std::mutex map_mutex; public: /// Convert ACL to number. If it's new ACL than adds it to map diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index c0f66f246d6..2d1d3a7658b 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -271,6 +271,7 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; + std::shared_lock storage_lock(storage_mutex); if (storage->isFinalized()) return false; @@ -291,14 +292,14 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req } if (keeper_context->digestEnabled() && request_for_session.digest) - assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false); + assertDigest(*request_for_session.digest, storage->getNodesDigest(false, /*lock_transaction_mutex=*/true), *request_for_session.request, false); return true; } void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session) { - std::lock_guard _(storage_and_responses_lock); + std::lock_guard lock(process_and_responses_lock); KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session); if (!responses_queue.push(response)) { @@ -393,6 +394,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n } }; + std::shared_lock storage_lock(storage_mutex); const auto op_num = request_for_session->request->getOpNum(); if (op_num == Coordination::OpNum::SessionID) { @@ -406,7 +408,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n response_for_session.session_id = -1; response_for_session.response = response; - std::lock_guard lock(storage_and_responses_lock); + std::lock_guard lock(process_and_responses_lock); session_id = storage->getSessionID(session_id_request.session_timeout_ms); LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); response->session_id = session_id; @@ -420,13 +422,14 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n parsed_request_cache.erase(request_for_session->session_id); } + std::lock_guard lock(process_and_responses_lock); KeeperStorage::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); if (keeper_context->digestEnabled() && request_for_session->digest) - assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, true); + assertDigest(*request_for_session->digest, storage->getNodesDigest(true, /*lock_transaction_mutex=*/true), *request_for_session->request, true); } ProfileEvents::increment(ProfileEvents::KeeperCommits); @@ -462,8 +465,6 @@ 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; if (latest_snapshot_ptr) snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr); @@ -471,6 +472,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx())); + std::unique_lock storage_lock(storage_mutex); /// maybe some logs were preprocessed with log idx larger than the snapshot idx /// we have to apply them to the new storage storage->applyUncommittedState(*snapshot_deserialization_result.storage, snapshot_deserialization_result.storage->getZXID()); @@ -509,14 +511,7 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; - storage->rollbackRequest(request_for_session.zxid, allow_missing); -} - -void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing) -{ - if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) - return; - + std::shared_lock lock(storage_mutex); storage->rollbackRequest(request_for_session.zxid, allow_missing); } @@ -535,7 +530,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res auto snapshot_meta_copy = nuraft::snapshot::deserialize(*snp_buf); 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); + std::unique_lock lock(storage_mutex); snapshot_task.snapshot = std::make_shared(storage.get(), snapshot_meta_copy, getClusterConfig()); } @@ -582,8 +577,6 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res } { - /// Destroy snapshot with lock - std::lock_guard lock(storage_and_responses_lock); LOG_TRACE(log, "Clearing garbage after snapshot"); /// Turn off "snapshot mode" and clear outdate part of storage state storage->clearGarbageAfterSnapshot(); @@ -732,8 +725,10 @@ int KeeperStateMachine::read_logical_snp_obj( void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session) { + std::shared_lock storage_lock(storage_mutex); + /// Pure local request, just process it with storage - std::lock_guard lock(storage_and_responses_lock); + std::lock_guard lock(process_and_responses_lock); auto responses = storage->processRequest( request_for_session.request, request_for_session.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); for (const auto & response : responses) @@ -743,96 +738,97 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi void KeeperStateMachine::shutdownStorage() { - std::lock_guard lock(storage_and_responses_lock); + std::unique_lock storage_lock(storage_mutex); storage->finalize(); } std::vector KeeperStateMachine::getDeadSessions() { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getDeadSessions(); } int64_t KeeperStateMachine::getNextZxid() const { + std::shared_lock storage_lock(storage_mutex); return storage->getNextZXID(); } KeeperStorage::Digest KeeperStateMachine::getNodesDigest() const { - std::lock_guard lock(storage_and_responses_lock); - return storage->getNodesDigest(false); + std::shared_lock storage_lock(storage_mutex); + return storage->getNodesDigest(false, /*lock_transaction_mutex=*/true); } uint64_t KeeperStateMachine::getLastProcessedZxid() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getZXID(); } uint64_t KeeperStateMachine::getNodesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getNodesCount(); } uint64_t KeeperStateMachine::getTotalWatchesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getTotalWatchesCount(); } uint64_t KeeperStateMachine::getWatchedPathsCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getWatchedPathsCount(); } uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getSessionsWithWatchesCount(); } uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getTotalEphemeralNodesCount(); } uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getSessionWithEphemeralNodesCount(); } void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); storage->dumpWatches(buf); } void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); storage->dumpWatchesByPath(buf); } void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); storage->dumpSessionsAndEphemerals(buf); } uint64_t KeeperStateMachine::getApproximateDataSize() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getApproximateDataSize(); } uint64_t KeeperStateMachine::getKeyArenaSize() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getArenaDataSize(); } @@ -858,7 +854,7 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const void KeeperStateMachine::recalculateStorageStats() { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); LOG_INFO(log, "Recalculating storage stats"); storage->recalculateStats(); LOG_INFO(log, "Done recalculating storage stats"); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 670b15394ce..39410044e20 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -132,6 +132,8 @@ public: void reconfigure(const KeeperStorage::RequestForSession& request_for_session); private: + mutable SharedMutex storage_mutex; + CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored /// in memory in compressed (serialized) format. @@ -161,7 +163,7 @@ private: /// we can get strange cases when, for example client send read request with /// watch and after that receive watch response and only receive response /// for request. - mutable std::mutex storage_and_responses_lock; + mutable std::mutex process_and_responses_lock; std::unordered_map>> parsed_request_cache; uint64_t min_request_size_to_cache{0}; @@ -189,7 +191,6 @@ private: KeeperSnapshotManagerS3 * snapshot_manager_s3; KeeperStorage::ResponseForSession processReconfiguration( - const KeeperStorage::RequestForSession& request_for_session) - TSA_REQUIRES(storage_and_responses_lock); + const KeeperStorage::RequestForSession& request_for_session); }; } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index f881b75a808..c44588680e2 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -535,7 +535,10 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) else if constexpr (std::same_as) { if (operation.stat.ephemeralOwner != 0) + { + std::lock_guard lock(storage.ephemerals_mutex); storage.ephemerals[operation.stat.ephemeralOwner].emplace(delta.path); + } } }, delta.operation); @@ -635,12 +638,12 @@ namespace /// Get current committed zxid int64_t KeeperStorage::getZXID() const -{ +{ std::lock_guard lock(transaction_mutex); return zxid; } -int64_t KeeperStorage::getNextZXIDLocked(std::lock_guard &) const +int64_t KeeperStorage::getNextZXIDLocked() const { if (uncommitted_transactions.empty()) return zxid + 1; @@ -651,7 +654,7 @@ int64_t KeeperStorage::getNextZXIDLocked(std::lock_guard &) const int64_t KeeperStorage::getNextZXID() const { std::lock_guard lock(transaction_mutex); - return getNextZXIDLocked(lock); + return getNextZXIDLocked(); } void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_zxid) TSA_NO_THREAD_SAFETY_ANALYSIS @@ -683,7 +686,7 @@ Coordination::Error KeeperStorage::commit(std::list deltas) for (auto & delta : deltas) { auto result = std::visit( - [&, &path = delta.path](DeltaType & operation) -> Coordination::Error + [&, &path = delta.path](DeltaType & operation) TSA_REQUIRES(storage_mutex) -> Coordination::Error { if constexpr (std::same_as) { @@ -906,6 +909,7 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ { if (is_local) { + std::lock_guard lock(storage.storage_mutex); auto node_it = storage.container.find(path); if (node_it == storage.container.end()) return {}; @@ -966,6 +970,7 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) { + std::lock_guard ephemerals_lock(ephemerals_mutex); auto ephemerals_it = ephemerals.find(session_id); if (ephemerals_it == ephemerals.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path"); @@ -1038,7 +1043,10 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; if (request.is_ephemeral) + { + std::lock_guard lock(storage.ephemerals_mutex); storage.ephemerals[session_id].emplace(path_created); + } int32_t parent_cversion = request.parent_cversion; @@ -1080,7 +1088,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); @@ -1133,7 +1141,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); @@ -1176,12 +1184,12 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); return processImpl(storage, std::move(deltas)); @@ -1271,7 +1279,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); @@ -1304,7 +1312,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); @@ -1337,12 +1345,12 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); return processImpl(storage, std::move(deltas)); @@ -1405,7 +1413,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { auto & container = storage.container; @@ -1458,7 +1466,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); @@ -1492,7 +1500,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc const auto & children = node_it->value.getChildren(); response.names.reserve(children.size()); - const auto add_child = [&](const auto child) + const auto add_child = [&](const auto child) TSA_REQUIRES(storage.storage_mutex) { using enum Coordination::ListRequestType; @@ -1527,12 +1535,12 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) TSA_REQUIRES(storage.storage_mutex) { ProfileEvents::increment(ProfileEvents::KeeperListRequest); return processImpl(storage, std::move(deltas)); @@ -1579,7 +1587,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); @@ -1625,12 +1633,12 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); return processImpl(storage, std::move(deltas)); @@ -1690,7 +1698,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); @@ -1733,7 +1741,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); @@ -1766,12 +1774,12 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } @@ -2015,7 +2023,7 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); @@ -2034,11 +2042,19 @@ void KeeperStorage::finalize() finalized = true; - ephemerals.clear(); + { + std::lock_guard lock(ephemerals_mutex); + ephemerals.clear(); + } - watches.clear(); - list_watches.clear(); - sessions_and_watchers.clear(); + { + std::lock_guard lock(watches_mutex); + watches.clear(); + list_watches.clear(); + sessions_and_watchers.clear(); + } + + std::lock_guard lock(session_mutex); session_expiry_queue.clear(); } @@ -2206,7 +2222,8 @@ void KeeperStorage::preprocessRequest( { std::lock_guard lock(transaction_mutex); - int64_t last_zxid = getNextZXIDLocked(lock) - 1; + int64_t last_zxid = getNextZXIDLocked() - 1; + auto current_digest = getNodesDigest(false, /*lock_transaction_mutex=*/false); if (uncommitted_transactions.empty()) { @@ -2219,7 +2236,7 @@ void KeeperStorage::preprocessRequest( } else { - if (last_zxid == new_last_zxid && digest && checkDigest(*digest, getNodesDigest(false))) + if (last_zxid == new_last_zxid && digest && checkDigest(*digest, current_digest)) // we found the preprocessed request with the same ZXID, we can skip it return; @@ -2230,8 +2247,8 @@ void KeeperStorage::preprocessRequest( new_last_zxid, last_zxid); } + new_digest = current_digest.value; transaction = &uncommitted_transactions.emplace_back(TransactionInfo{.zxid = new_last_zxid, .nodes_digest = {}}); - new_digest = getNodesDigest(false).value; } std::list new_deltas; @@ -2252,32 +2269,35 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { - auto session_ephemerals = ephemerals.find(session_id); - if (session_ephemerals != ephemerals.end()) { - for (const auto & ephemeral_path : session_ephemerals->second) + std::lock_guard lock(ephemerals_mutex); + auto session_ephemerals = ephemerals.find(session_id); + if (session_ephemerals != ephemerals.end()) { - auto parent_node_path = parentNodePath(ephemeral_path); - auto parent_node = uncommitted_state.getNode(parent_node_path); - UpdateNodeStatDelta parent_update_delta(*parent_node); - ++parent_update_delta.new_stats.cversion; - --parent_update_delta.new_stats.numChildren; + for (const auto & ephemeral_path : session_ephemerals->second) + { + auto parent_node_path = parentNodePath(ephemeral_path); + auto parent_node = uncommitted_state.getNode(parent_node_path); + UpdateNodeStatDelta parent_update_delta(*parent_node); + ++parent_update_delta.new_stats.cversion; + --parent_update_delta.new_stats.numChildren; - new_deltas.emplace_back - ( - parent_node_path.toString(), - new_last_zxid, - std::move(parent_update_delta) - ); + new_deltas.emplace_back + ( + parent_node_path.toString(), + new_last_zxid, + std::move(parent_update_delta) + ); - auto node = uncommitted_state.getNode(ephemeral_path); - new_deltas.emplace_back( - ephemeral_path, - transaction->zxid, - RemoveNodeDelta{.stat = node->stat, .acls = uncommitted_state.getACLs(ephemeral_path), .data = node->getData()}); + auto node = uncommitted_state.getNode(ephemeral_path); + new_deltas.emplace_back( + ephemeral_path, + transaction->zxid, + RemoveNodeDelta{.stat = node->stat, .acls = uncommitted_state.getACLs(ephemeral_path), .data = node->getData()}); + } + + ephemerals.erase(session_ephemerals); } - - ephemerals.erase(session_ephemerals); } new_digest = calculateNodesDigest(new_digest, new_deltas); @@ -2331,7 +2351,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( std::list deltas; { std::lock_guard lock(uncommitted_state.deltas_mutex); - auto it = uncommitted_state.deltas.begin(); + auto it = uncommitted_state.deltas.begin(); for (; it != uncommitted_state.deltas.end() && it->zxid == commit_zxid; ++it) ; @@ -2341,7 +2361,10 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( KeeperStorage::ResponsesForSessions results; /// ZooKeeper update sessions expirity for each request, not only for heartbeats - session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); + { + std::lock_guard lock(session_mutex); + session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); + } if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { @@ -2349,6 +2372,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( { if (std::holds_alternative(delta.operation)) { + std::lock_guard lock(watches_mutex); auto responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); results.insert(results.end(), responses.begin(), responses.end()); } @@ -2368,8 +2392,11 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( auto response = std::make_shared(); response->xid = zk_request->xid; response->zxid = commit_zxid; - session_expiry_queue.remove(session_id); - session_and_timeout.erase(session_id); + { + std::lock_guard lock(session_mutex); + session_expiry_queue.remove(session_id); + session_and_timeout.erase(session_id); + } results.push_back(ResponseForSession{session_id, response}); } else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special @@ -2392,7 +2419,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( if (is_local) { - std::lock_guard lock(storage_mutex); assert(zk_request->isReadRequest()); if (check_acl && !request_processor->checkAuth(*this, session_id, true)) { @@ -2414,6 +2440,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// Watches for this requests are added to the watches lists if (zk_request->has_watch) { + std::lock_guard lock(watches_mutex); if (response->error == Coordination::Error::ZOK) { static constexpr std::array list_requests{ @@ -2436,6 +2463,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { + std::lock_guard lock(watches_mutex); auto watch_responses = request_processor->processWatches(watches, list_watches); results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } @@ -2456,7 +2484,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( return results; } -void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) +void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS { std::unique_lock transaction_lock(transaction_mutex); if (allow_missing && (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid < rollback_zxid)) @@ -2484,13 +2512,28 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) } } -KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const +KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed, bool lock_transaction_mutex) const TSA_NO_THREAD_SAFETY_ANALYSIS { if (!keeper_context->digestEnabled()) return {.version = DigestVersion::NO_DIGEST}; - if (committed || uncommitted_transactions.empty()) + if (committed) + { + std::lock_guard storage_lock(storage_mutex); return {CURRENT_DIGEST_VERSION, nodes_digest}; + } + + std::unique_lock transaction_lock(transaction_mutex, std::defer_lock); + if (lock_transaction_mutex) + transaction_lock.lock(); + + if (uncommitted_transactions.empty()) + { + if (lock_transaction_mutex) + transaction_lock.unlock(); + std::lock_guard storage_lock(storage_mutex); + return {CURRENT_DIGEST_VERSION, nodes_digest}; + } return uncommitted_transactions.back().nodes_digest; } @@ -2510,8 +2553,89 @@ void KeeperStorage::addDigest(const Node & node, const std::string_view path) } } +/// Allocate new session id with the specified timeouts +int64_t KeeperStorage::getSessionID(int64_t session_timeout_ms) +{ + std::lock_guard lock(session_mutex); + auto result = session_id_counter++; + session_and_timeout.emplace(result, session_timeout_ms); + session_expiry_queue.addNewSessionOrUpdate(result, session_timeout_ms); + return result; +} + +/// Add session id. Used when restoring KeeperStorage from snapshot. +void KeeperStorage::addSessionID(int64_t session_id, int64_t session_timeout_ms) +{ + session_and_timeout.emplace(session_id, session_timeout_ms); + session_expiry_queue.addNewSessionOrUpdate(session_id, session_timeout_ms); +} + +std::vector KeeperStorage::getDeadSessions() const +{ + std::lock_guard lock(session_mutex); + return session_expiry_queue.getExpiredSessions(); +} + +const SessionAndTimeout & KeeperStorage::getActiveSessions() const +{ + return session_and_timeout; +} + +/// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. +void KeeperStorage::enableSnapshotMode(size_t up_to_version) +{ + std::lock_guard lock(storage_mutex); + container.enableSnapshotMode(up_to_version); +} + +/// Turn off snapshot mode. +void KeeperStorage::disableSnapshotMode() +{ + std::lock_guard lock(storage_mutex); + container.disableSnapshotMode(); +} + +KeeperStorage::Container::const_iterator KeeperStorage::getSnapshotIteratorBegin() const +{ + std::lock_guard lock(storage_mutex); + return container.begin(); +} + +/// Clear outdated data from internal container. +void KeeperStorage::clearGarbageAfterSnapshot() +{ + std::lock_guard lock(storage_mutex); + container.clearOutdatedNodes(); +} + +/// Introspection functions mostly used in 4-letter commands +uint64_t KeeperStorage::getNodesCount() const +{ + std::lock_guard lock(storage_mutex); + return container.size(); +} + +uint64_t KeeperStorage::getApproximateDataSize() const +{ + std::lock_guard lock(storage_mutex); + return container.getApproximateDataSize(); +} + +uint64_t KeeperStorage::getArenaDataSize() const +{ + std::lock_guard lock(storage_mutex); + return container.keyArenaSize(); +} + +uint64_t KeeperStorage::getWatchedPathsCount() const +{ + std::lock_guard lock(watches_mutex); + return watches.size() + list_watches.size(); +} + void KeeperStorage::clearDeadWatches(int64_t session_id) { + std::lock_guard lock(watches_mutex); /// Clear all watches for this session auto watches_it = sessions_and_watchers.find(session_id); if (watches_it != sessions_and_watchers.end()) @@ -2545,6 +2669,7 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const { + std::lock_guard lock(watches_mutex); for (const auto & [session_id, watches_paths] : sessions_and_watchers) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; @@ -2563,6 +2688,7 @@ void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const } }; + std::lock_guard lock(watches_mutex); for (const auto & [watch_path, sessions] : watches) { buf << watch_path << "\n"; @@ -2586,14 +2712,18 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co } }; - buf << "Sessions dump (" << session_and_timeout.size() << "):\n"; - - for (const auto & [session_id, _] : session_and_timeout) { - buf << "0x" << getHexUIntLowercase(session_id) << "\n"; + std::lock_guard lock(session_mutex); + buf << "Sessions dump (" << session_and_timeout.size() << "):\n"; + + for (const auto & [session_id, _] : session_and_timeout) + { + buf << "0x" << getHexUIntLowercase(session_id) << "\n"; + } } - buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCount() << "):\n"; + std::lock_guard ephemerals_lock(ephemerals_mutex); + buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCountLocked() << "):\n"; for (const auto & [session_id, ephemeral_paths] : ephemerals) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; @@ -2603,6 +2733,7 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co uint64_t KeeperStorage::getTotalWatchesCount() const { + std::lock_guard lock(watches_mutex); uint64_t ret = 0; for (const auto & [path, subscribed_sessions] : watches) ret += subscribed_sessions.size(); @@ -2613,9 +2744,21 @@ uint64_t KeeperStorage::getTotalWatchesCount() const return ret; } +uint64_t KeeperStorage::getSessionWithEphemeralNodesCount() const +{ + std::lock_guard ephemerals_lock(ephemerals_mutex); + return getSessionWithEphemeralNodesCountLocked(); +} + +uint64_t KeeperStorage::getSessionWithEphemeralNodesCountLocked() const +{ + return ephemerals.size(); +} + uint64_t KeeperStorage::getSessionsWithWatchesCount() const { std::unordered_set counter; + std::lock_guard lock(watches_mutex); for (const auto & [path, subscribed_sessions] : watches) counter.insert(subscribed_sessions.begin(), subscribed_sessions.end()); @@ -2627,6 +2770,7 @@ uint64_t KeeperStorage::getSessionsWithWatchesCount() const uint64_t KeeperStorage::getTotalEphemeralNodesCount() const { + std::lock_guard ephemerals_lock(ephemerals_mutex); uint64_t ret = 0; for (const auto & [session_id, nodes] : ephemerals) ret += nodes.size(); @@ -2636,6 +2780,7 @@ uint64_t KeeperStorage::getTotalEphemeralNodesCount() const void KeeperStorage::recalculateStats() { + std::lock_guard lock(storage_mutex); container.recalculateDataSize(); } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 84bf7d297b4..3ac5165e38b 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -139,18 +139,14 @@ public: using SessionAndAuth = std::unordered_map; using Watches = std::map; - mutable SharedMutex main_mutex; - mutable std::mutex storage_mutex; - int64_t session_id_counter{1}; - - SessionAndAuth session_and_auth; + SessionAndAuth session_and_auth TSA_GUARDED_BY(storage_mutex); /// Main hashtable with nodes. Contain all information about data. /// All other structures expect session_and_timeout can be restored from /// container. - Container container; + Container container TSA_GUARDED_BY(storage_mutex); // Applying ZooKeeper request to storage consists of two steps: // - preprocessing which, instead of applying the changes directly to storage, @@ -284,7 +280,10 @@ public: }; if (is_local) + { + std::lock_guard lock(storage.storage_mutex); return check_auth(storage.session_and_auth[session_id]); + } // check if there are uncommitted const auto auth_it = session_and_auth.find(session_id); @@ -347,7 +346,7 @@ public: // with zxid > last_zxid void applyUncommittedState(KeeperStorage & other, int64_t last_zxid); - Coordination::Error commit(std::list deltas); + Coordination::Error commit(std::list deltas) TSA_REQUIRES(storage_mutex); // Create node in the storage // Returns false if it failed to create the node, true otherwise @@ -357,25 +356,27 @@ public: String data, const Coordination::Stat & stat, bool is_sequental, - Coordination::ACLs node_acls); + Coordination::ACLs node_acls) TSA_REQUIRES(storage_mutex); // Remove node in the storage // Returns false if it failed to remove the node, true otherwise // We don't care about the exact failure because we should've caught it during preprocessing - bool removeNode(const std::string & path, int32_t version); + bool removeNode(const std::string & path, int32_t version) TSA_REQUIRES(storage_mutex); bool checkACL(StringRef path, int32_t permissions, int64_t session_id, bool is_local); void unregisterEphemeralPath(int64_t session_id, const std::string & path); + mutable std::mutex ephemerals_mutex; /// Mapping session_id -> set of ephemeral nodes paths - Ephemerals ephemerals; - /// Mapping session_id -> set of watched nodes paths - SessionAndWatcher sessions_and_watchers; + Ephemerals ephemerals TSA_GUARDED_BY(ephemerals_mutex); + + mutable std::mutex session_mutex; + int64_t session_id_counter TSA_GUARDED_BY(session_mutex) = 1; /// Expiration queue for session, allows to get dead sessions at some point of time - SessionExpiryQueue session_expiry_queue; + SessionExpiryQueue session_expiry_queue TSA_GUARDED_BY(session_mutex); /// All active sessions with timeout - SessionAndTimeout session_and_timeout; + SessionAndTimeout session_and_timeout TSA_GUARDED_BY(session_mutex); /// ACLMap for more compact ACLs storage inside nodes. ACLMap acl_map; @@ -383,7 +384,7 @@ public: mutable std::mutex transaction_mutex; /// Global id of all requests applied to storage - int64_t zxid = 0; + int64_t zxid TSA_GUARDED_BY(transaction_mutex) = 0; // older Keeper node (pre V5 snapshots) can create snapshots and receive logs from newer Keeper nodes // this can lead to some inconsistencies, e.g. from snapshot it will use log_idx as zxid @@ -398,15 +399,20 @@ public: Digest nodes_digest; }; - std::list uncommitted_transactions; + std::list uncommitted_transactions TSA_GUARDED_BY(transaction_mutex); - uint64_t nodes_digest{0}; + uint64_t nodes_digest TSA_GUARDED_BY(storage_mutex) = 0; std::atomic finalized{false}; + + mutable std::mutex watches_mutex; + /// Mapping session_id -> set of watched nodes paths + SessionAndWatcher sessions_and_watchers TSA_GUARDED_BY(watches_mutex); + /// Currently active watches (node_path -> subscribed sessions) - Watches watches; - Watches list_watches; /// Watches for 'list' request (watches on children). + Watches watches TSA_GUARDED_BY(watches_mutex); + Watches list_watches TSA_GUARDED_BY(watches_mutex); /// Watches for 'list' request (watches on children). void clearDeadWatches(int64_t session_id); @@ -414,9 +420,9 @@ public: int64_t getZXID() const; int64_t getNextZXID() const; - int64_t getNextZXIDLocked(std::lock_guard & lock) const; + int64_t getNextZXIDLocked() const TSA_REQUIRES(transaction_mutex); - Digest getNodesDigest(bool committed) const; + Digest getNodesDigest(bool committed, bool lock_transaction_mutex) const; KeeperContextPtr keeper_context; @@ -426,23 +432,13 @@ public: KeeperStorage(int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, bool initialize_system_nodes = true); - void initializeSystemNodes(); + void initializeSystemNodes() TSA_NO_THREAD_SAFETY_ANALYSIS; /// Allocate new session id with the specified timeouts - int64_t getSessionID(int64_t session_timeout_ms) - { - auto result = session_id_counter++; - session_and_timeout.emplace(result, session_timeout_ms); - session_expiry_queue.addNewSessionOrUpdate(result, session_timeout_ms); - return result; - } + int64_t getSessionID(int64_t session_timeout_ms); /// Add session id. Used when restoring KeeperStorage from snapshot. - void addSessionID(int64_t session_id, int64_t session_timeout_ms) - { - session_and_timeout.emplace(session_id, session_timeout_ms); - session_expiry_queue.addNewSessionOrUpdate(session_id, session_timeout_ms); - } + void addSessionID(int64_t session_id, int64_t session_timeout_ms) TSA_NO_THREAD_SAFETY_ANALYSIS; UInt64 calculateNodesDigest(UInt64 current_digest, const std::list & new_deltas) const; @@ -470,36 +466,36 @@ public: /// Set of methods for creating snapshots /// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. - void enableSnapshotMode(size_t up_to_version) { container.enableSnapshotMode(up_to_version); } + void enableSnapshotMode(size_t up_to_version); /// Turn off snapshot mode. - void disableSnapshotMode() { container.disableSnapshotMode(); } + void disableSnapshotMode(); - Container::const_iterator getSnapshotIteratorBegin() const { return container.begin(); } + Container::const_iterator getSnapshotIteratorBegin() const; /// Clear outdated data from internal container. - void clearGarbageAfterSnapshot() { container.clearOutdatedNodes(); } + void clearGarbageAfterSnapshot(); /// Get all active sessions - const SessionAndTimeout & getActiveSessions() const { return session_and_timeout; } + const SessionAndTimeout & getActiveSessions() const; /// Get all dead sessions - std::vector getDeadSessions() const { return session_expiry_queue.getExpiredSessions(); } + std::vector getDeadSessions() const; /// Introspection functions mostly used in 4-letter commands - uint64_t getNodesCount() const { return container.size(); } + uint64_t getNodesCount() const; - uint64_t getApproximateDataSize() const { return container.getApproximateDataSize(); } + uint64_t getApproximateDataSize() const; - uint64_t getArenaDataSize() const { return container.keyArenaSize(); } + uint64_t getArenaDataSize() const; uint64_t getTotalWatchesCount() const; - uint64_t getWatchedPathsCount() const { return watches.size() + list_watches.size(); } + uint64_t getWatchedPathsCount() const; uint64_t getSessionsWithWatchesCount() const; - uint64_t getSessionWithEphemeralNodesCount() const { return ephemerals.size(); } + uint64_t getSessionWithEphemeralNodesCount() const; uint64_t getTotalEphemeralNodesCount() const; void dumpWatches(WriteBufferFromOwnString & buf) const; @@ -508,8 +504,10 @@ public: void recalculateStats(); private: - void removeDigest(const Node & node, std::string_view path); - void addDigest(const Node & node, std::string_view path); + uint64_t getSessionWithEphemeralNodesCountLocked() const TSA_REQUIRES(ephemerals_mutex); + + void removeDigest(const Node & node, std::string_view path) TSA_REQUIRES(storage_mutex); + void addDigest(const Node & node, std::string_view path) TSA_REQUIRES(storage_mutex); }; using KeeperStoragePtr = std::unique_ptr; diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index d7ce097b795..54a917e03a5 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -90,7 +90,7 @@ void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in) } } -int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * log) +int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * log) TSA_NO_THREAD_SAFETY_ANALYSIS { int64_t max_zxid = 0; std::string path; From acc08c65d951175123d62e3e55ea8db87d385e5e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 22 May 2024 11:56:45 +0200 Subject: [PATCH 04/27] Add stopwatch --- src/Coordination/KeeperDispatcher.cpp | 8 ++++++++ src/Coordination/KeeperStateMachine.cpp | 5 +++-- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index c9eba87590e..94ce159f055 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -448,11 +448,19 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) { + Stopwatch watch; if (server->isLeaderAlive()) server->putLocalReadRequest(request_queue_it->second); else addErrorResponses(request_queue_it->second, Coordination::Error::ZCONNECTIONLOSS); + if (request_queue_it->second.size() > 500) + LOG_INFO( + getLogger("Speed"), + "It took {}ms for {} requests", + watch.elapsedMilliseconds(), + request_queue_it->second.size()); + xid_to_request_queue.erase(request_queue_it); } } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 8a94ef7347e..67ff316a6f3 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -768,14 +768,15 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestsForSess /// Pure local request, just process it with storage std::lock_guard lock(process_and_responses_lock); std::vector all_responses; - if (request_for_session.size() > 50) + if (request_for_session.size() > 100) { all_responses.resize(request_for_session.size()); //LOG_INFO(getLogger("Keeper"), "Has read requests {}", request_queue_it->second.size()); for (size_t i = 0; i < request_for_session.size(); ++i) { - read_pool.scheduleOrThrowOnError([&, i, read_request = request_for_session[i]] + read_pool.scheduleOrThrowOnError([&, i] { + const auto & read_request = request_for_session[i]; all_responses[i] = storage->processRequest( read_request.request, read_request.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); }); From f0e970338454e2a5e09968974ea0850f357ec43e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 09:45:07 +0200 Subject: [PATCH 05/27] Some small improvements --- src/Coordination/FourLetterCommand.cpp | 22 +++++++++++-------- src/Coordination/Keeper4LWInfo.h | 5 ----- .../KeeperAsynchronousMetrics.cpp | 17 +++++++------- src/Coordination/KeeperServer.cpp | 2 -- src/Coordination/KeeperStateMachine.cpp | 5 ++--- src/Coordination/KeeperStateMachine.h | 1 + src/Coordination/KeeperStorage.cpp | 18 +++++++++++++++ src/Coordination/KeeperStorage.h | 17 ++++++++++++++ 8 files changed, 60 insertions(+), 27 deletions(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 8de9f8dfa1c..18b25ae8e77 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -300,11 +300,13 @@ String MonitorCommand::run() print(ret, "server_state", keeper_info.getRole()); - print(ret, "znode_count", state_machine.getNodesCount()); - print(ret, "watch_count", state_machine.getTotalWatchesCount()); - print(ret, "ephemerals_count", state_machine.getTotalEphemeralNodesCount()); - print(ret, "approximate_data_size", state_machine.getApproximateDataSize()); - print(ret, "key_arena_size", state_machine.getKeyArenaSize()); + const auto & storage_stats = state_machine.getStorageStats(); + + print(ret, "znode_count", storage_stats.nodes_count.load(std::memory_order_relaxed)); + print(ret, "watch_count", storage_stats.total_watches_count.load(std::memory_order_relaxed)); + print(ret, "ephemerals_count", storage_stats.total_emphemeral_nodes_count.load(std::memory_order_relaxed)); + print(ret, "approximate_data_size", storage_stats.approximate_data_size.load(std::memory_order_relaxed)); + print(ret, "key_arena_size", 0); print(ret, "latest_snapshot_size", state_machine.getLatestSnapshotSize()); #if defined(OS_LINUX) || defined(OS_DARWIN) @@ -386,6 +388,7 @@ String ServerStatCommand::run() auto & stats = keeper_dispatcher.getKeeperConnectionStats(); Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); + const auto & storage_stats = keeper_dispatcher.getStateMachine().getStorageStats(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); @@ -397,9 +400,9 @@ String ServerStatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", formatZxid(keeper_info.last_zxid)); + write("Zxid", formatZxid(storage_stats.last_zxid.load(std::memory_order_relaxed))); write("Mode", keeper_info.getRole()); - write("Node count", toString(keeper_info.total_nodes_count)); + write("Node count", toString(storage_stats.nodes_count.load(std::memory_order_relaxed))); return buf.str(); } @@ -415,6 +418,7 @@ String StatCommand::run() auto & stats = keeper_dispatcher.getKeeperConnectionStats(); Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); + const auto & storage_stats = keeper_dispatcher.getStateMachine().getStorageStats(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); @@ -430,9 +434,9 @@ String StatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", formatZxid(keeper_info.last_zxid)); + write("Zxid", formatZxid(storage_stats.last_zxid.load(std::memory_order_relaxed))); write("Mode", keeper_info.getRole()); - write("Node count", toString(keeper_info.total_nodes_count)); + write("Node count", toString(storage_stats.nodes_count.load(std::memory_order_relaxed))); return buf.str(); } diff --git a/src/Coordination/Keeper4LWInfo.h b/src/Coordination/Keeper4LWInfo.h index 80b00b3f36e..d02c9a8f91c 100644 --- a/src/Coordination/Keeper4LWInfo.h +++ b/src/Coordination/Keeper4LWInfo.h @@ -1,7 +1,5 @@ #pragma once -#include - #include #include @@ -30,9 +28,6 @@ struct Keeper4LWInfo uint64_t follower_count; uint64_t synced_follower_count; - uint64_t total_nodes_count; - int64_t last_zxid; - String getRole() const { if (is_standalone) diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 86166ffe31b..1f317c4b303 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -38,15 +38,16 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM is_follower = static_cast(keeper_info.is_follower); is_exceeding_mem_soft_limit = static_cast(keeper_info.is_exceeding_mem_soft_limit); - zxid = keeper_info.last_zxid; const auto & state_machine = keeper_dispatcher.getStateMachine(); - znode_count = state_machine.getNodesCount(); - watch_count = state_machine.getTotalWatchesCount(); - ephemerals_count = state_machine.getTotalEphemeralNodesCount(); - approximate_data_size = state_machine.getApproximateDataSize(); - key_arena_size = state_machine.getKeyArenaSize(); - session_with_watches = state_machine.getSessionsWithWatchesCount(); - paths_watched = state_machine.getWatchedPathsCount(); + const auto & storage_stats = state_machine.getStorageStats(); + zxid = storage_stats.last_zxid.load(std::memory_order_relaxed); + znode_count = storage_stats.nodes_count.load(std::memory_order_relaxed); + watch_count = storage_stats.total_watches_count.load(std::memory_order_relaxed); + ephemerals_count = storage_stats.total_emphemeral_nodes_count.load(std::memory_order_relaxed); + approximate_data_size = storage_stats.approximate_data_size.load(std::memory_order_relaxed); + key_arena_size = 0; + session_with_watches = storage_stats.sessions_with_watches_count.load(std::memory_order_relaxed); + paths_watched = storage_stats.watched_paths_count.load(std::memory_order_relaxed); # if defined(__linux__) || defined(__APPLE__) open_file_descriptor_count = getCurrentProcessFDCount(); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 736a01443ce..02b05c6346c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -1126,8 +1126,6 @@ Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const result.synced_follower_count = getSyncedFollowerCount(); } result.is_exceeding_mem_soft_limit = isExceedingMemorySoftLimit(); - result.total_nodes_count = getKeeperStateMachine()->getNodesCount(); - result.last_zxid = getKeeperStateMachine()->getLastProcessedZxid(); return result; } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e4d661dfe17..cbb4c55de33 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -802,10 +802,9 @@ uint64_t KeeperStateMachine::getLastProcessedZxid() const return storage->getZXID(); } -uint64_t KeeperStateMachine::getNodesCount() const +const KeeperStorage::Stats & KeeperStateMachine::getStorageStats() const TSA_NO_THREAD_SAFETY_ANALYSIS { - std::lock_guard lock(storage_and_responses_lock); - return storage->getNodesCount(); + return storage->getStorageStats(); } uint64_t KeeperStateMachine::getTotalWatchesCount() const diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index ee6109f0a17..6a34ef7ddd8 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -111,6 +111,7 @@ public: /// Introspection functions for 4lw commands uint64_t getLastProcessedZxid() const; + const KeeperStorage::Stats & getStorageStats() const; uint64_t getNodesCount() const; uint64_t getTotalWatchesCount() const; uint64_t getWatchedPathsCount() const; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 9bcd0608bf7..c1dc38cec5d 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -2514,6 +2514,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( results.push_back(ResponseForSession{session_id, response}); } + updateStats(); return results; } @@ -2660,6 +2661,23 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co } } +void KeeperStorage::updateStats() +{ + stats.nodes_count.store(getNodesCount(), std::memory_order_relaxed); + stats.approximate_data_size.store(getApproximateDataSize(), std::memory_order_relaxed); + stats.total_watches_count.store(getTotalWatchesCount(), std::memory_order_relaxed); + stats.watched_paths_count.store(getWatchedPathsCount(), std::memory_order_relaxed); + stats.sessions_with_watches_count.store(getSessionsWithWatchesCount(), std::memory_order_relaxed); + stats.session_with_ephemeral_nodes_count.store(getSessionWithEphemeralNodesCount(), std::memory_order_relaxed); + stats.total_emphemeral_nodes_count.store(getTotalEphemeralNodesCount(), std::memory_order_relaxed); + stats.last_zxid.store(getZXID(), std::memory_order_relaxed); +} + +const KeeperStorage::Stats & KeeperStorage::getStorageStats() const +{ + return stats; +} + uint64_t KeeperStorage::getTotalWatchesCount() const { uint64_t ret = 0; diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index d9e67f799f8..2d20bb38532 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -539,6 +539,23 @@ public: /// Get all dead sessions std::vector getDeadSessions() const { return session_expiry_queue.getExpiredSessions(); } + struct Stats + { + std::atomic nodes_count = 0; + std::atomic approximate_data_size = 0; + std::atomic total_watches_count = 0; + std::atomic watched_paths_count = 0; + std::atomic sessions_with_watches_count = 0; + std::atomic session_with_ephemeral_nodes_count = 0; + std::atomic total_emphemeral_nodes_count = 0; + std::atomic last_zxid = 0; + }; + + Stats stats; + + void updateStats(); + const Stats & getStorageStats() const; + /// Introspection functions mostly used in 4-letter commands uint64_t getNodesCount() const { return container.size(); } From 1e97d73bd07e0d8ce398e02b47afdb6afe244aa6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Jun 2024 17:13:37 +0200 Subject: [PATCH 06/27] Squashed commit of the following: commit 27fe0439fad3987ca6769504c73713e24f3fb81f Merge: bfb1c4c7939 bb469e0d45a Author: Antonio Andelic Date: Thu Jun 6 14:36:02 2024 +0200 Merge branch 'master' into fix-global-trace-collector commit bfb1c4c79396bb234b63a0a963fc643af292956d Author: Antonio Andelic Date: Thu Jun 6 11:29:42 2024 +0200 better commit fcee260b259b7750f18389e26debc90de22017a5 Author: Antonio Andelic Date: Thu Jun 6 11:22:48 2024 +0200 Update src/Interpreters/TraceCollector.h Co-authored-by: alesapin commit 1d3cf170534b1ae8382397d8d2672868e237f33f Author: Antonio Andelic Date: Thu Jun 6 11:11:08 2024 +0200 Fix global trace collector --- programs/server/Server.cpp | 90 ++++++++++++++--------------- src/Common/QueryProfiler.cpp | 6 +- src/Interpreters/Context.cpp | 17 +++++- src/Interpreters/Context.h | 2 + src/Interpreters/TraceCollector.cpp | 30 ++++++++-- src/Interpreters/TraceCollector.h | 11 +++- 6 files changed, 96 insertions(+), 60 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8fcb9d87a93..84ff0e94cef 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -773,7 +773,27 @@ try LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info); #endif - bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log"); + bool has_trace_collector = false; + /// Disable it if we collect test coverage information, because it will work extremely slow. +#if !WITH_COVERAGE + /// Profilers cannot work reliably with any other libunwind or without PHDR cache. + has_trace_collector = hasPHDRCache() && config().has("trace_log"); +#endif + + /// Describe multiple reasons when query profiler cannot work. + +#if WITH_COVERAGE + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); +#endif + +#if defined(SANITIZER) + LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" + " when two different stack unwinding methods will interfere with each other."); +#endif + + if (!hasPHDRCache()) + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" + " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will @@ -782,8 +802,27 @@ try server_settings.max_thread_pool_size, server_settings.max_thread_pool_free_size, server_settings.thread_pool_queue_size, - will_have_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, - will_have_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + has_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, + has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + + if (has_trace_collector) + { + global_context->createTraceCollector(); + + /// Set up server-wide memory profiler (for total memory tracker). + if (server_settings.total_memory_profiler_step) + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + + if (server_settings.total_memory_tracker_sample_probability > 0.0) + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; @@ -1950,52 +1989,9 @@ try LOG_DEBUG(log, "Loaded metadata."); - /// Init trace collector only after trace_log system table was created - /// Disable it if we collect test coverage information, because it will work extremely slow. -#if !WITH_COVERAGE - /// Profilers cannot work reliably with any other libunwind or without PHDR cache. - if (hasPHDRCache()) - { + if (has_trace_collector) global_context->initializeTraceCollector(); - /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) - { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); - } - - if (server_settings.total_memory_tracker_sample_probability > 0.0) - { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); - } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } -#endif - - /// Describe multiple reasons when query profiler cannot work. - -#if WITH_COVERAGE - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); -#endif - -#if defined(SANITIZER) - LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" - " when two different stack unwinding methods will interfere with each other."); -#endif - - if (!hasPHDRCache()) - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" - " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); - #if defined(OS_LINUX) auto tasks_stats_provider = TasksStatsCounters::findBestAvailableProvider(); if (tasks_stats_provider == TasksStatsCounters::MetricsProvider::None) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index c3affbdd968..746010b5462 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -228,9 +228,9 @@ void Timer::cleanup() #endif template -QueryProfilerBase::QueryProfilerBase([[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) - : log(getLogger("QueryProfiler")) - , pause_signal(pause_signal_) +QueryProfilerBase::QueryProfilerBase( + [[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) + : log(getLogger("QueryProfiler")), pause_signal(pause_signal_) { #if defined(SANITIZER) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers"); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 06b3adb328d..b1fb6a68618 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -740,12 +740,18 @@ struct ContextSharedPart : boost::noncopyable void initializeTraceCollector(std::shared_ptr trace_log) { - if (!trace_log) - return; + if (!trace_collector.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector needs to be first created before initialization"); + + trace_collector->initialize(trace_log); + } + + void createTraceCollector() + { if (hasTraceCollector()) return; - trace_collector.emplace(std::move(trace_log)); + trace_collector.emplace(); } void addWarningMessage(const String & message) TSA_REQUIRES(mutex) @@ -3891,6 +3897,11 @@ void Context::initializeSystemLogs() }); } +void Context::createTraceCollector() +{ + shared->createTraceCollector(); +} + void Context::initializeTraceCollector() { shared->initializeTraceCollector(getTraceLog()); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 87a7baa0469..68f37377926 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1077,6 +1077,8 @@ public: void initializeSystemLogs(); /// Call after initialization before using trace collector. + void createTraceCollector(); + void initializeTraceCollector(); /// Call after unexpected crash happen. diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 8e9c397b7a1..77f70d754c8 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -1,5 +1,4 @@ -#include "TraceCollector.h" - +#include #include #include #include @@ -14,8 +13,12 @@ namespace DB { -TraceCollector::TraceCollector(std::shared_ptr trace_log_) - : trace_log(std::move(trace_log_)) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +TraceCollector::TraceCollector() { TraceSender::pipe.open(); @@ -28,6 +31,23 @@ TraceCollector::TraceCollector(std::shared_ptr trace_log_) thread = ThreadFromGlobalPool(&TraceCollector::run, this); } +void TraceCollector::initialize(std::shared_ptr trace_log_) +{ + if (is_trace_log_initialized) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector is already initialized"); + + trace_log_ptr = trace_log_; + is_trace_log_initialized.store(true, std::memory_order_release); +} + +std::shared_ptr TraceCollector::getTraceLog() +{ + if (!is_trace_log_initialized.load(std::memory_order_acquire)) + return nullptr; + + return trace_log_ptr; +} + void TraceCollector::tryClosePipe() { try @@ -120,7 +140,7 @@ void TraceCollector::run() ProfileEvents::Count increment; readPODBinary(increment, in); - if (trace_log) + if (auto trace_log = getTraceLog()) { // time and time_in_microseconds are both being constructed from the same timespec so that the // times will be equal up to the precision of a second. diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index 382e7511ac6..c2894394dd0 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -1,4 +1,5 @@ #pragma once +#include #include class StackTrace; @@ -16,11 +17,17 @@ class TraceLog; class TraceCollector { public: - explicit TraceCollector(std::shared_ptr trace_log_); + TraceCollector(); ~TraceCollector(); + void initialize(std::shared_ptr trace_log_); + private: - std::shared_ptr trace_log; + std::shared_ptr getTraceLog(); + + std::atomic is_trace_log_initialized = false; + std::shared_ptr trace_log_ptr; + ThreadFromGlobalPool thread; void tryClosePipe(); From 7ea3345e0da818a6f9ea3127c9cffc54dbe9fd8a Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Jun 2024 17:08:23 +0200 Subject: [PATCH 07/27] Use ThreadFromGlobalPool in Poco::ThreadPool --- base/poco/Foundation/CMakeLists.txt | 3 + base/poco/Foundation/src/ThreadPool.cpp | 79 ++++++++++++++++++------- programs/server/Server.cpp | 10 ++-- src/Server/KeeperTCPHandler.cpp | 1 - src/Server/TCPHandler.cpp | 1 - 5 files changed, 64 insertions(+), 30 deletions(-) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index dfb41a33fb1..fe59e72569c 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -168,6 +168,9 @@ set (SRCS add_library (_poco_foundation ${SRCS}) add_library (Poco::Foundation ALIAS _poco_foundation) +target_link_libraries (_poco_foundation PRIVATE clickhouse_common_io) +target_compile_definitions(_poco_foundation PUBLIC USE_CLICKHOUSE_THREADS=1) + # TODO: remove these warning exclusions target_compile_options (_poco_foundation PRIVATE diff --git a/base/poco/Foundation/src/ThreadPool.cpp b/base/poco/Foundation/src/ThreadPool.cpp index 6335ee82b47..a13e91d8b76 100644 --- a/base/poco/Foundation/src/ThreadPool.cpp +++ b/base/poco/Foundation/src/ThreadPool.cpp @@ -20,12 +20,15 @@ #include "Poco/ErrorHandler.h" #include #include +#if USE_CLICKHOUSE_THREADS +#include +#endif namespace Poco { -class PooledThread: public Runnable +class PooledThread : public Runnable { public: PooledThread(const std::string& name, int stackSize = POCO_THREAD_STACK_SIZE); @@ -46,7 +49,11 @@ private: volatile std::time_t _idleTime; Runnable* _pTarget; std::string _name; +#if USE_CLICKHOUSE_THREADS + ThreadFromGlobalPool _thread; +#else Thread _thread; +#endif Event _targetReady; Event _targetCompleted; Event _started; @@ -54,16 +61,20 @@ private: }; -PooledThread::PooledThread(const std::string& name, int stackSize): - _idle(true), - _idleTime(0), - _pTarget(0), - _name(name), +PooledThread::PooledThread(const std::string& name, [[maybe_unused]] int stackSize): + _idle(true), + _idleTime(0), + _pTarget(0), + _name(name), +#if !USE_CLICKHOUSE_THREADS _thread(name), +#endif _targetCompleted(false) { poco_assert_dbg (stackSize >= 0); +#if !USE_CLICKHOUSE_THREADS _thread.setStackSize(stackSize); +#endif _idleTime = std::time(NULL); } @@ -75,24 +86,32 @@ PooledThread::~PooledThread() void PooledThread::start() { +#if USE_CLICKHOUSE_THREADS + _thread = ThreadFromGlobalPool([this]() { run(); }); +#else _thread.start(*this); +#endif _started.wait(); } -void PooledThread::start(Thread::Priority priority, Runnable& target) +void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& target) { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_pTarget == 0); _pTarget = ⌖ + +#if !USE_CLICKHOUSE_THREADS _thread.setPriority(priority); +#endif + _targetReady.set(); } -void PooledThread::start(Thread::Priority priority, Runnable& target, const std::string& name) +void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& target, const std::string& name) { FastMutex::ScopedLock lock(_mutex); @@ -107,9 +126,12 @@ void PooledThread::start(Thread::Priority priority, Runnable& target, const std: fullName.append(_name); fullName.append(")"); } + +#if !USE_CLICKHOUSE_THREADS _thread.setName(fullName); _thread.setPriority(priority); - +#endif + poco_assert (_pTarget == 0); _pTarget = ⌖ @@ -145,7 +167,7 @@ void PooledThread::join() void PooledThread::activate() { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_idle); _idle = false; _targetCompleted.reset(); @@ -154,21 +176,30 @@ void PooledThread::activate() void PooledThread::release() { - const long JOIN_TIMEOUT = 10000; - _mutex.lock(); _pTarget = 0; _mutex.unlock(); + // In case of a statically allocated thread pool (such // as the default thread pool), Windows may have already // terminated the thread before we got here. +#if USE_CLICKHOUSE_THREADS + if (_thread.joinable()) +#else if (_thread.isRunning()) +#endif _targetReady.set(); +#if USE_CLICKHOUSE_THREADS + if (_thread.joinable()) + _thread.join(); +#else + const long JOIN_TIMEOUT = 10000; if (_thread.tryJoin(JOIN_TIMEOUT)) { delete this; } +#endif } @@ -205,8 +236,10 @@ void PooledThread::run() _idle = true; _targetCompleted.set(); ThreadLocalStorage::clear(); +#if !USE_CLICKHOUSE_THREADS _thread.setName(_name); _thread.setPriority(Thread::PRIO_NORMAL); +#endif } else { @@ -220,9 +253,9 @@ void PooledThread::run() ThreadPool::ThreadPool(int minCapacity, int maxCapacity, int idleTime, - int stackSize): - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + int stackSize): + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), @@ -245,8 +278,8 @@ ThreadPool::ThreadPool(const std::string& name, int idleTime, int stackSize): _name(name), - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), @@ -393,15 +426,15 @@ void ThreadPool::housekeep() ThreadVec activeThreads; idleThreads.reserve(_threads.size()); activeThreads.reserve(_threads.size()); - + for (ThreadVec::iterator it = _threads.begin(); it != _threads.end(); ++it) { if ((*it)->idle()) { if ((*it)->idleTime() < _idleTime) idleThreads.push_back(*it); - else - expiredThreads.push_back(*it); + else + expiredThreads.push_back(*it); } else activeThreads.push_back(*it); } @@ -481,7 +514,7 @@ public: ThreadPool* pool() { FastMutex::ScopedLock lock(_mutex); - + if (!_pPool) { _pPool = new ThreadPool("default"); @@ -490,7 +523,7 @@ public: } return _pPool; } - + private: ThreadPool* _pPool; FastMutex _mutex; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 84ff0e94cef..9d4ff00c8a1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -721,11 +721,6 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); - Poco::ThreadPool server_pool(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -823,6 +818,11 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } + Poco::ThreadPool server_pool(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 6709cd298e5..4612e2e9fa8 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -309,7 +309,6 @@ Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length, bool void KeeperTCPHandler::runImpl() { setThreadName("KeeperHandler"); - ThreadStatus thread_status; socket().setReceiveTimeout(receive_timeout); socket().setSendTimeout(send_timeout); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index e3a820340ad..b60339e9fd8 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -246,7 +246,6 @@ TCPHandler::~TCPHandler() void TCPHandler::runImpl() { setThreadName("TCPHandler"); - ThreadStatus thread_status; extractConnectionSettingsFromContext(server.context()); From e30fa1da4d9a51c5626bde79629981fb693c401a Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 Jun 2024 15:03:13 +0200 Subject: [PATCH 08/27] Fix ThreadStatus --- src/Server/HTTPHandler.cpp | 1 - src/Server/InterserverIOHTTPHandler.cpp | 2 -- src/Server/MySQLHandler.cpp | 2 -- src/Server/PostgreSQLHandler.cpp | 2 -- 4 files changed, 7 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 02d0959ff50..d6afa571e71 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1060,7 +1060,6 @@ void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("HTTPHandler"); - ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::HTTP, request.isSecure()); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 0d79aaa227b..45c28babe3a 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include @@ -81,7 +80,6 @@ void InterserverIOHTTPHandler::processQuery(HTTPServerRequest & request, HTTPSer void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("IntersrvHandler"); - ThreadStatus thread_status; /// In order to work keep-alive. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 6456f6d24ff..9471509ad4b 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include @@ -199,7 +198,6 @@ MySQLHandler::~MySQLHandler() = default; void MySQLHandler::run() { setThreadName("MySQLHandler"); - ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::MYSQL); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 473d681ddb2..8ba8421e6f0 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -59,7 +58,6 @@ void PostgreSQLHandler::changeIO(Poco::Net::StreamSocket & socket) void PostgreSQLHandler::run() { setThreadName("PostgresHandler"); - ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::POSTGRESQL); SCOPE_EXIT({ session.reset(); }); From 6604d94271a491772f9fbd42f129bd28afb8c950 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 Jun 2024 17:11:49 +0200 Subject: [PATCH 09/27] Ping CI: skip fast test to see all stateless runs From a11ba3f437c2262f1a1263ebe012bd3a85dc6ea3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 Jun 2024 15:19:03 +0200 Subject: [PATCH 10/27] Fix shutdown --- programs/server/Server.cpp | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9d4ff00c8a1..802c994e872 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -818,7 +818,7 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } - Poco::ThreadPool server_pool(3, server_settings.max_connections); + auto server_pool = std::make_unique(3, server_settings.max_connections); std::mutex servers_lock; std::vector servers; std::vector servers_to_start_before_tables; @@ -898,7 +898,8 @@ try global_context->shutdownKeeperDispatcher(); /// Wait server pool to avoid use-after-free of destroyed context in the handlers - server_pool.joinAll(); + server_pool->joinAll(); + server_pool.reset(); /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. * At this moment, no one could own shared part of Context. @@ -1629,7 +1630,7 @@ try if (global_context->isServerCompletelyStarted()) { std::lock_guard lock(servers_lock); - updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); + updateServers(*config, *server_pool, async_metrics, servers, servers_to_start_before_tables); } } @@ -1726,7 +1727,7 @@ try config_getter, global_context->getKeeperDispatcher(), global_context->getSettingsRef().receive_timeout.totalSeconds(), global_context->getSettingsRef().send_timeout.totalSeconds(), - false), server_pool, socket)); + false), *server_pool, socket)); }); const char * secure_port_name = "keeper_server.tcp_port_secure"; @@ -1748,7 +1749,7 @@ try new KeeperTCPHandlerFactory( config_getter, global_context->getKeeperDispatcher(), global_context->getSettingsRef().receive_timeout.totalSeconds(), - global_context->getSettingsRef().send_timeout.totalSeconds(), true), server_pool, socket)); + global_context->getSettingsRef().send_timeout.totalSeconds(), true), *server_pool, socket)); #else UNUSED(port); throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); @@ -1780,7 +1781,7 @@ try createKeeperHTTPControlMainHandlerFactory( config_getter(), global_context->getKeeperDispatcher(), - "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params)); + "KeeperHTTPControlHandler-factory"), *server_pool, socket, http_params)); }); } #else @@ -1804,7 +1805,7 @@ try config(), interserver_listen_hosts, listen_try, - server_pool, + *server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ false); @@ -1855,7 +1856,7 @@ try config(), listen_hosts, listen_try, - server_pool, + *server_pool, async_metrics, servers, /* start_servers= */ true, @@ -2027,7 +2028,7 @@ try { std::lock_guard lock(servers_lock); - createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); + createServers(config(), listen_hosts, listen_try, *server_pool, async_metrics, servers); if (servers.empty()) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " From c3d4b429d9b98b0629435cacbba59cc61a3e8ebd Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 Jun 2024 15:39:54 +0200 Subject: [PATCH 11/27] Fix merge --- programs/server/Server.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 802c994e872..dfbcd36bcb3 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -721,6 +721,11 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); + auto server_pool = std::make_unique(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -818,11 +823,6 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } - auto server_pool = std::make_unique(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; From 6514d72fea8e723683a614d258fc24bdc00f78ba Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 Jun 2024 18:53:51 +0200 Subject: [PATCH 12/27] Move servers pool back --- programs/server/Server.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index dfbcd36bcb3..802c994e872 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -721,11 +721,6 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); - auto server_pool = std::make_unique(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -823,6 +818,11 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } + auto server_pool = std::make_unique(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; From c802d7d58aa6d8e369497cc769dd0860bf929124 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 11 Jun 2024 14:35:26 +0200 Subject: [PATCH 13/27] Writing improvements --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 216 +++++++++++++++++++++-- src/Common/ZooKeeper/ZooKeeperCommon.h | 40 ++++- src/Common/ZooKeeper/ZooKeeperIO.cpp | 26 +++ src/Common/ZooKeeper/ZooKeeperIO.h | 30 ++++ 4 files changed, 296 insertions(+), 16 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 48bb510e589..602b05274e4 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -9,7 +9,6 @@ #include #include #include -#include namespace Coordination @@ -19,14 +18,16 @@ using namespace DB; void ZooKeeperResponse::write(WriteBuffer & out) const { - /// Excessive copy to calculate length. - WriteBufferFromOwnString buf; - Coordination::write(xid, buf); - Coordination::write(zxid, buf); - Coordination::write(error, buf); + auto response_size = Coordination::size(xid) + Coordination::size(zxid) + Coordination::size(error); if (error == Error::ZOK) - writeImpl(buf); - Coordination::write(buf.str(), out); + response_size += sizeImpl(); + + Coordination::write(static_cast(response_size), out); + Coordination::write(xid, out); + Coordination::write(zxid, out); + Coordination::write(error, out); + if (error == Error::ZOK) + writeImpl(out); } std::string ZooKeeperRequest::toString() const @@ -42,12 +43,12 @@ std::string ZooKeeperRequest::toString() const void ZooKeeperRequest::write(WriteBuffer & out) const { - /// Excessive copy to calculate length. - WriteBufferFromOwnString buf; - Coordination::write(xid, buf); - Coordination::write(getOpNum(), buf); - writeImpl(buf); - Coordination::write(buf.str(), out); + auto request_size = Coordination::size(xid) + Coordination::size(getOpNum()) + sizeImpl(); + + Coordination::write(static_cast(request_size), out); + Coordination::write(xid, out); + Coordination::write(getOpNum(), out); + writeImpl(out); } void ZooKeeperSyncRequest::writeImpl(WriteBuffer & out) const @@ -55,6 +56,11 @@ void ZooKeeperSyncRequest::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +size_t ZooKeeperSyncRequest::sizeImpl() const +{ + return Coordination::size(path); +} + void ZooKeeperSyncRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -75,6 +81,11 @@ void ZooKeeperSyncResponse::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +size_t ZooKeeperSyncResponse::sizeImpl() const +{ + return Coordination::size(path); +} + void ZooKeeperReconfigRequest::writeImpl(WriteBuffer & out) const { Coordination::write(joining, out); @@ -83,6 +94,11 @@ void ZooKeeperReconfigRequest::writeImpl(WriteBuffer & out) const Coordination::write(version, out); } +size_t ZooKeeperReconfigRequest::sizeImpl() const +{ + return Coordination::size(joining) + Coordination::size(leaving) + Coordination::size(new_members) + Coordination::size(version); +} + void ZooKeeperReconfigRequest::readImpl(ReadBuffer & in) { Coordination::read(joining, in); @@ -110,6 +126,11 @@ void ZooKeeperReconfigResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperReconfigResponse::sizeImpl() const +{ + return Coordination::size(value) + Coordination::size(stat); +} + void ZooKeeperWatchResponse::readImpl(ReadBuffer & in) { Coordination::read(type, in); @@ -124,6 +145,11 @@ void ZooKeeperWatchResponse::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +size_t ZooKeeperWatchResponse::sizeImpl() const +{ + return Coordination::size(type) + Coordination::size(state) + Coordination::size(path); +} + void ZooKeeperWatchResponse::write(WriteBuffer & out) const { if (error == Error::ZOK) @@ -138,6 +164,11 @@ void ZooKeeperAuthRequest::writeImpl(WriteBuffer & out) const Coordination::write(data, out); } +size_t ZooKeeperAuthRequest::sizeImpl() const +{ + return Coordination::size(type) + Coordination::size(scheme) + Coordination::size(data); +} + void ZooKeeperAuthRequest::readImpl(ReadBuffer & in) { Coordination::read(type, in); @@ -176,6 +207,19 @@ void ZooKeeperCreateRequest::writeImpl(WriteBuffer & out) const Coordination::write(flags, out); } +size_t ZooKeeperCreateRequest::sizeImpl() const +{ + /// See https://github.com/ClickHouse/clickhouse-private/issues/3029 + if (path.starts_with("/clickhouse/tables/") && path.find("/parts/") != std::string::npos) + { + LOG_TRACE(getLogger(__PRETTY_FUNCTION__), "Creating part at path {}", path); + } + + int32_t flags = 0; + + return Coordination::size(path) + Coordination::size(data) + Coordination::size(acls) + Coordination::size(flags); +} + void ZooKeeperCreateRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -212,12 +256,22 @@ void ZooKeeperCreateResponse::writeImpl(WriteBuffer & out) const Coordination::write(path_created, out); } +size_t ZooKeeperCreateResponse::sizeImpl() const +{ + return Coordination::size(path_created); +} + void ZooKeeperRemoveRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); Coordination::write(version, out); } +size_t ZooKeeperRemoveRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(version); +} + std::string ZooKeeperRemoveRequest::toStringImpl() const { return fmt::format( @@ -239,6 +293,11 @@ void ZooKeeperExistsRequest::writeImpl(WriteBuffer & out) const Coordination::write(has_watch, out); } +size_t ZooKeeperExistsRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(has_watch); +} + void ZooKeeperExistsRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -260,12 +319,22 @@ void ZooKeeperExistsResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperExistsResponse::sizeImpl() const +{ + return Coordination::size(stat); +} + void ZooKeeperGetRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); Coordination::write(has_watch, out); } +size_t ZooKeeperGetRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(has_watch); +} + void ZooKeeperGetRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -289,6 +358,11 @@ void ZooKeeperGetResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperGetResponse::sizeImpl() const +{ + return Coordination::size(data) + Coordination::size(stat); +} + void ZooKeeperSetRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); @@ -296,6 +370,11 @@ void ZooKeeperSetRequest::writeImpl(WriteBuffer & out) const Coordination::write(version, out); } +size_t ZooKeeperSetRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(data) + Coordination::size(version); +} + void ZooKeeperSetRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -322,12 +401,22 @@ void ZooKeeperSetResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperSetResponse::sizeImpl() const +{ + return Coordination::size(stat); +} + void ZooKeeperListRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); Coordination::write(has_watch, out); } +size_t ZooKeeperListRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(has_watch); +} + void ZooKeeperListRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -346,6 +435,11 @@ void ZooKeeperFilteredListRequest::writeImpl(WriteBuffer & out) const Coordination::write(static_cast(list_request_type), out); } +size_t ZooKeeperFilteredListRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(has_watch) + Coordination::size(static_cast(list_request_type)); +} + void ZooKeeperFilteredListRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -377,6 +471,11 @@ void ZooKeeperListResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperListResponse::sizeImpl() const +{ + return Coordination::size(names) + Coordination::size(stat); +} + void ZooKeeperSimpleListResponse::readImpl(ReadBuffer & in) { Coordination::read(names, in); @@ -387,6 +486,11 @@ void ZooKeeperSimpleListResponse::writeImpl(WriteBuffer & out) const Coordination::write(names, out); } +size_t ZooKeeperSimpleListResponse::sizeImpl() const +{ + return Coordination::size(names); +} + void ZooKeeperSetACLRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); @@ -394,6 +498,11 @@ void ZooKeeperSetACLRequest::writeImpl(WriteBuffer & out) const Coordination::write(version, out); } +size_t ZooKeeperSetACLRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(acls) + Coordination::size(version); +} + void ZooKeeperSetACLRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -411,6 +520,11 @@ void ZooKeeperSetACLResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperSetACLResponse::sizeImpl() const +{ + return Coordination::size(stat); +} + void ZooKeeperSetACLResponse::readImpl(ReadBuffer & in) { Coordination::read(stat, in); @@ -426,6 +540,11 @@ void ZooKeeperGetACLRequest::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +size_t ZooKeeperGetACLRequest::sizeImpl() const +{ + return Coordination::size(path); +} + std::string ZooKeeperGetACLRequest::toStringImpl() const { return fmt::format("path = {}", path); @@ -437,6 +556,11 @@ void ZooKeeperGetACLResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperGetACLResponse::sizeImpl() const +{ + return Coordination::size(acl) + Coordination::size(stat); +} + void ZooKeeperGetACLResponse::readImpl(ReadBuffer & in) { Coordination::read(acl, in); @@ -449,6 +573,11 @@ void ZooKeeperCheckRequest::writeImpl(WriteBuffer & out) const Coordination::write(version, out); } +size_t ZooKeeperCheckRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(version); +} + void ZooKeeperCheckRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -474,6 +603,11 @@ void ZooKeeperErrorResponse::writeImpl(WriteBuffer & out) const Coordination::write(error, out); } +size_t ZooKeeperErrorResponse::sizeImpl() const +{ + return Coordination::size(error); +} + void ZooKeeperMultiRequest::checkOperationType(OperationType type) { chassert(!operation_type.has_value() || *operation_type == type); @@ -571,6 +705,27 @@ void ZooKeeperMultiRequest::writeImpl(WriteBuffer & out) const Coordination::write(error, out); } +size_t ZooKeeperMultiRequest::sizeImpl() const +{ + size_t total_size = 0; + for (const auto & request : requests) + { + const auto & zk_request = dynamic_cast(*request); + + bool done = false; + int32_t error = -1; + + total_size + += Coordination::size(zk_request.getOpNum()) + Coordination::size(done) + Coordination::size(error) + zk_request.sizeImpl(); + } + + OpNum op_num = OpNum::Error; + bool done = true; + int32_t error = -1; + + return total_size + Coordination::size(op_num) + Coordination::size(done) + Coordination::size(error); +} + void ZooKeeperMultiRequest::readImpl(ReadBuffer & in) { while (true) @@ -701,6 +856,29 @@ void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const } } +size_t ZooKeeperMultiResponse::sizeImpl() const +{ + size_t total_size = 0; + for (const auto & response : responses) + { + const ZooKeeperResponse & zk_response = dynamic_cast(*response); + OpNum op_num = zk_response.getOpNum(); + bool done = false; + Error op_error = zk_response.error; + + total_size += Coordination::size(op_num) + Coordination::size(done) + Coordination::size(op_error); + if (op_error == Error::ZOK || op_num == OpNum::Error) + total_size += zk_response.sizeImpl(); + } + + /// Footer. + OpNum op_num = OpNum::Error; + bool done = true; + int32_t error_read = - 1; + + return total_size + Coordination::size(op_num) + Coordination::size(done) + Coordination::size(error_read); +} + ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return setTime(std::make_shared()); } @@ -748,6 +926,11 @@ void ZooKeeperSessionIDRequest::writeImpl(WriteBuffer & out) const Coordination::write(server_id, out); } +size_t ZooKeeperSessionIDRequest::sizeImpl() const +{ + return Coordination::size(internal_id) + Coordination::size(session_timeout_ms) + Coordination::size(server_id); +} + void ZooKeeperSessionIDRequest::readImpl(ReadBuffer & in) { Coordination::read(internal_id, in); @@ -774,6 +957,11 @@ void ZooKeeperSessionIDResponse::writeImpl(WriteBuffer & out) const Coordination::write(server_id, out); } +size_t ZooKeeperSessionIDResponse::sizeImpl() const +{ + return Coordination::size(internal_id) + Coordination::size(session_id) + Coordination::size(server_id); +} + void ZooKeeperRequest::createLogElements(LogElements & elems) const { diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 490c2dce4f8..7c318a97676 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -7,13 +7,11 @@ #include #include #include -#include #include #include #include #include #include -#include namespace Coordination @@ -32,6 +30,7 @@ struct ZooKeeperResponse : virtual Response ~ZooKeeperResponse() override; virtual void readImpl(ReadBuffer &) = 0; virtual void writeImpl(WriteBuffer &) const = 0; + virtual size_t sizeImpl() const = 0; virtual void write(WriteBuffer & out) const; virtual OpNum getOpNum() const = 0; virtual void fillLogElements(LogElements & elems, size_t idx) const; @@ -66,6 +65,7 @@ struct ZooKeeperRequest : virtual Request std::string toString() const; virtual void writeImpl(WriteBuffer &) const = 0; + virtual size_t sizeImpl() const = 0; virtual void readImpl(ReadBuffer &) = 0; virtual std::string toStringImpl() const { return ""; } @@ -86,6 +86,7 @@ struct ZooKeeperHeartbeatRequest final : ZooKeeperRequest String getPath() const override { return {}; } OpNum getOpNum() const override { return OpNum::Heartbeat; } void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } void readImpl(ReadBuffer &) override {} ZooKeeperResponsePtr makeResponse() const override; bool isReadRequest() const override { return false; } @@ -97,6 +98,7 @@ struct ZooKeeperSyncRequest final : ZooKeeperRequest String getPath() const override { return path; } OpNum getOpNum() const override { return OpNum::Sync; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -109,6 +111,7 @@ struct ZooKeeperSyncResponse final : SyncResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Sync; } }; @@ -122,6 +125,7 @@ struct ZooKeeperReconfigRequest final : ZooKeeperRequest String getPath() const override { return keeper_config_path; } OpNum getOpNum() const override { return OpNum::Reconfig; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -138,6 +142,7 @@ struct ZooKeeperReconfigResponse final : ReconfigResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Reconfig; } }; @@ -145,6 +150,7 @@ struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Heartbeat; } }; @@ -153,6 +159,7 @@ struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void write(WriteBuffer & out) const override; @@ -175,6 +182,7 @@ struct ZooKeeperAuthRequest final : ZooKeeperRequest String getPath() const override { return {}; } OpNum getOpNum() const override { return OpNum::Auth; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -189,6 +197,7 @@ struct ZooKeeperAuthResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Auth; } @@ -200,6 +209,7 @@ struct ZooKeeperCloseRequest final : ZooKeeperRequest String getPath() const override { return {}; } OpNum getOpNum() const override { return OpNum::Close; } void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } void readImpl(ReadBuffer &) override {} ZooKeeperResponsePtr makeResponse() const override; @@ -214,6 +224,7 @@ struct ZooKeeperCloseResponse final : ZooKeeperResponse } void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Close; } }; @@ -228,6 +239,7 @@ struct ZooKeeperCreateRequest final : public CreateRequest, ZooKeeperRequest OpNum getOpNum() const override { return not_exists ? OpNum::CreateIfNotExists : OpNum::Create; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -244,6 +256,7 @@ struct ZooKeeperCreateResponse : CreateResponse, ZooKeeperResponse void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Create; } @@ -265,6 +278,7 @@ struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::Remove; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -280,6 +294,7 @@ struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Remove; } size_t bytesSize() const override { return RemoveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -292,6 +307,7 @@ struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::Exists; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -305,6 +321,7 @@ struct ZooKeeperExistsResponse final : ExistsResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Exists; } size_t bytesSize() const override { return ExistsResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -319,6 +336,7 @@ struct ZooKeeperGetRequest final : GetRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::Get; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -332,6 +350,7 @@ struct ZooKeeperGetResponse final : GetResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Get; } size_t bytesSize() const override { return GetResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -346,6 +365,7 @@ struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::Set; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -360,6 +380,7 @@ struct ZooKeeperSetResponse final : SetResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Set; } size_t bytesSize() const override { return SetResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -374,6 +395,7 @@ struct ZooKeeperListRequest : ListRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::List; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -394,6 +416,7 @@ struct ZooKeeperFilteredListRequest final : ZooKeeperListRequest OpNum getOpNum() const override { return OpNum::FilteredList; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -404,6 +427,7 @@ struct ZooKeeperListResponse : ListResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::List; } size_t bytesSize() const override { return ListResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -415,6 +439,7 @@ struct ZooKeeperSimpleListResponse final : ZooKeeperListResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::SimpleList; } size_t bytesSize() const override { return ZooKeeperListResponse::bytesSize() - sizeof(stat); } @@ -427,6 +452,7 @@ struct ZooKeeperCheckRequest : CheckRequest, ZooKeeperRequest OpNum getOpNum() const override { return not_exists ? OpNum::CheckNotExists : OpNum::Check; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -442,6 +468,7 @@ struct ZooKeeperCheckResponse : CheckResponse, ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Check; } size_t bytesSize() const override { return CheckResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -458,6 +485,7 @@ struct ZooKeeperErrorResponse final : ErrorResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Error; } @@ -468,6 +496,7 @@ struct ZooKeeperSetACLRequest final : SetACLRequest, ZooKeeperRequest { OpNum getOpNum() const override { return OpNum::SetACL; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -480,6 +509,7 @@ struct ZooKeeperSetACLResponse final : SetACLResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::SetACL; } size_t bytesSize() const override { return SetACLResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -489,6 +519,7 @@ struct ZooKeeperGetACLRequest final : GetACLRequest, ZooKeeperRequest { OpNum getOpNum() const override { return OpNum::GetACL; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -501,6 +532,7 @@ struct ZooKeeperGetACLResponse final : GetACLResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::GetACL; } size_t bytesSize() const override { return GetACLResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -515,6 +547,7 @@ struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest ZooKeeperMultiRequest(std::span generic_requests, const ACLs & default_acls); void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -554,6 +587,7 @@ struct ZooKeeperMultiResponse : MultiResponse, ZooKeeperResponse void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; size_t bytesSize() const override { return MultiResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -584,6 +618,7 @@ struct ZooKeeperSessionIDRequest final : ZooKeeperRequest Coordination::OpNum getOpNum() const override { return OpNum::SessionID; } String getPath() const override { return {}; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; Coordination::ZooKeeperResponsePtr makeResponse() const override; @@ -602,6 +637,7 @@ struct ZooKeeperSessionIDResponse final : ZooKeeperResponse void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; Coordination::OpNum getOpNum() const override { return OpNum::SessionID; } }; diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp index 6a51ffb36fa..a1afe039a04 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.cpp +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -42,6 +42,32 @@ void write(const Error & x, WriteBuffer & out) write(static_cast(x), out); } +size_t size(OpNum x) +{ + return size(static_cast(x)); +} + +size_t size(const std::string & s) +{ + return size(static_cast(s.size())) + s.size(); +} + +size_t size(const ACL & acl) +{ + return size(acl.permissions) + size(acl.scheme) + size(acl.id); +} + +size_t size(const Stat & stat) +{ + return size(stat.czxid) + size(stat.mzxid) + size(stat.ctime) + size(stat.mtime) + size(stat.version) + size(stat.cversion) + + size(stat.aversion) + size(stat.ephemeralOwner) + size(stat.dataLength) + size(stat.numChildren) + size(stat.pzxid); +} + +size_t size(const Error & x) +{ + return size(static_cast(x)); +} + void read(OpNum & x, ReadBuffer & in) { int32_t raw_op_num; diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h index 83973c1ae22..242370d2d25 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.h +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -43,6 +43,36 @@ void write(const std::vector & arr, WriteBuffer & out) write(elem, out); } +template +requires is_arithmetic_v +size_t size(T x) +{ + return sizeof(x); +} + +size_t size(OpNum x); +size_t size(const std::string & s); +size_t size(const ACL & acl); +size_t size(const Stat & stat); +size_t size(const Error & x); + +template +size_t size(const std::array) +{ + return size(static_cast(N)) + N; +} + +template +size_t size(const std::vector & arr) +{ + size_t total_size = size(static_cast(arr.size())); + for (const auto & elem : arr) + total_size += size(elem); + + return total_size; +} + + template requires is_arithmetic_v void read(T & x, ReadBuffer & in) From 0fa45c39540129bd68b3e50c2c5084f6959d48b6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 11 Jun 2024 14:35:34 +0200 Subject: [PATCH 14/27] More parallel storage --- src/Coordination/KeeperDispatcher.cpp | 19 ++-- src/Coordination/KeeperServer.cpp | 5 +- src/Coordination/KeeperServer.h | 2 +- src/Coordination/KeeperStateMachine.cpp | 92 +++++++------------- src/Coordination/KeeperStateMachine.h | 5 +- src/Coordination/KeeperStorage.cpp | 111 +++++++----------------- src/Coordination/KeeperStorage.h | 19 ++-- 7 files changed, 87 insertions(+), 166 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 0fe0b8cce1f..b9518c51893 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -449,18 +449,13 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) { - Stopwatch watch; - if (server->isLeaderAlive()) - server->putLocalReadRequest(request_queue_it->second); - else - addErrorResponses(request_queue_it->second, Coordination::Error::ZCONNECTIONLOSS); - - if (request_queue_it->second.size() > 500) - LOG_INFO( - getLogger("Speed"), - "It took {}ms for {} requests", - watch.elapsedMilliseconds(), - request_queue_it->second.size()); + for (const auto & read_request : request_queue_it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } xid_to_request_queue.erase(request_queue_it); } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index c14658ddccf..02b05c6346c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -533,10 +533,9 @@ nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestFor } -void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestsForSessions & request_for_session) +void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & request_for_session) { - if (std::any_of( - request_for_session.begin(), request_for_session.end(), [](const auto & request) { return !request.request->isReadRequest(); })) + if (!request_for_session.request->isReadRequest()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot process non-read request locally"); state_machine->processReadRequest(request_for_session); diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index c43fb35efe1..5e45a552cba 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -84,7 +84,7 @@ public: /// Put local read request and execute in state machine directly and response into /// responses queue - void putLocalReadRequest(const KeeperStorage::RequestsForSessions & request); + void putLocalReadRequest(const KeeperStorage::RequestForSession & request); bool isRecovering() const { return is_recovering; } bool reconfigEnabled() const { return enable_reconfiguration; } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 3b5b7e07f06..a5402944701 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -278,12 +278,12 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; - std::shared_lock storage_lock(storage_mutex); if (storage->isFinalized()) return false; try { + std::shared_lock storage_lock(storage_mutex); storage->preprocessRequest( request_for_session.request, request_for_session.session_id, @@ -312,7 +312,7 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session) { - std::lock_guard lock(process_and_responses_lock); + std::lock_guard lock(storage_mutex); KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session); if (!responses_queue.push(response)) { @@ -414,7 +414,6 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n try { - std::shared_lock storage_lock(storage_mutex); const auto op_num = request_for_session->request->getOpNum(); if (op_num == Coordination::OpNum::SessionID) { @@ -428,7 +427,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n response_for_session.session_id = -1; response_for_session.response = response; - std::lock_guard lock(process_and_responses_lock); + std::unique_lock lock(storage_mutex); session_id = storage->getSessionID(session_id_request.session_timeout_ms); LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); response->session_id = session_id; @@ -438,15 +437,18 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n { if (op_num == Coordination::OpNum::Close) { - std::lock_guard lock(request_cache_mutex); + std::lock_guard cache_lock(request_cache_mutex); parsed_request_cache.erase(request_for_session->session_id); } - std::lock_guard lock(process_and_responses_lock); - KeeperStorage::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); + { + std::shared_lock lock(storage_mutex); + std::lock_guard response_lock(process_and_responses_lock); + KeeperStorage::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); + } if (keeper_context->digestEnabled() && request_for_session->digest) assertDigest( @@ -549,7 +551,7 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; - std::shared_lock lock(storage_mutex); + std::unique_lock lock(storage_mutex); storage->rollbackRequest(request_for_session.zxid, allow_missing); } @@ -767,42 +769,16 @@ int KeeperStateMachine::read_logical_snp_obj( return 1; } -void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestsForSessions & request_for_session) +void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session) { std::shared_lock storage_lock(storage_mutex); + std::lock_guard response_lock(process_and_responses_lock); + auto responses = storage->processRequest( + request_for_session.request, request_for_session.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); + for (const auto & response : responses) + if (!responses_queue.push(response)) + LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id); - /// Pure local request, just process it with storage - std::lock_guard lock(process_and_responses_lock); - std::vector all_responses; - if (request_for_session.size() > 100) - { - all_responses.resize(request_for_session.size()); - //LOG_INFO(getLogger("Keeper"), "Has read requests {}", request_queue_it->second.size()); - for (size_t i = 0; i < request_for_session.size(); ++i) - { - read_pool.scheduleOrThrowOnError([&, i] - { - const auto & read_request = request_for_session[i]; - all_responses[i] = storage->processRequest( - read_request.request, read_request.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); - }); - } - read_pool.wait(); - } - else - { - all_responses.reserve(request_for_session.size()); - for (const auto & read_request : request_for_session) - { - all_responses.push_back(storage->processRequest( - read_request.request, read_request.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/)); - } - } - - for (const auto & responses : all_responses) - for (const auto & response : responses) - if (!responses_queue.push(response)) - LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id); } void KeeperStateMachine::shutdownStorage() @@ -813,25 +789,23 @@ void KeeperStateMachine::shutdownStorage() std::vector KeeperStateMachine::getDeadSessions() { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getDeadSessions(); } int64_t KeeperStateMachine::getNextZxid() const { - std::shared_lock storage_lock(storage_mutex); return storage->getNextZXID(); } KeeperStorage::Digest KeeperStateMachine::getNodesDigest() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getNodesDigest(false, /*lock_transaction_mutex=*/true); } uint64_t KeeperStateMachine::getLastProcessedZxid() const { - std::shared_lock storage_lock(storage_mutex); return storage->getZXID(); } @@ -842,61 +816,61 @@ const KeeperStorage::Stats & KeeperStateMachine::getStorageStats() const TSA_NO_ uint64_t KeeperStateMachine::getTotalWatchesCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getTotalWatchesCount(); } uint64_t KeeperStateMachine::getWatchedPathsCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getWatchedPathsCount(); } uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getSessionsWithWatchesCount(); } uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getTotalEphemeralNodesCount(); } uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getSessionWithEphemeralNodesCount(); } void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); storage->dumpWatches(buf); } void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); storage->dumpWatchesByPath(buf); } void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); storage->dumpSessionsAndEphemerals(buf); } uint64_t KeeperStateMachine::getApproximateDataSize() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getApproximateDataSize(); } uint64_t KeeperStateMachine::getKeyArenaSize() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getArenaDataSize(); } @@ -937,7 +911,7 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const void KeeperStateMachine::recalculateStorageStats() { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); LOG_INFO(log, "Recalculating storage stats"); storage->recalculateStats(); LOG_INFO(log, "Done recalculating storage stats"); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 68b9366611d..c1702394754 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -100,7 +100,7 @@ public: ClusterConfigPtr getClusterConfig() const; /// Process local read request - void processReadRequest(const KeeperStorage::RequestsForSessions & request_for_session); + void processReadRequest(const KeeperStorage::RequestForSession & request_for_session); std::vector getDeadSessions(); @@ -132,8 +132,6 @@ public: void reconfigure(const KeeperStorage::RequestForSession& request_for_session); private: - mutable SharedMutex storage_mutex; - CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored /// in memory in compressed (serialized) format. @@ -156,6 +154,7 @@ private: /// Mutex for snapshots mutable std::mutex snapshots_lock; + mutable SharedMutex storage_mutex; /// Lock for storage and responses_queue. It's important to process requests /// and push them to the responses queue while holding this lock. Otherwise /// we can get strange cases when, for example client send read request with diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index e62be70ede5..238ab07a276 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -412,7 +412,7 @@ Overloaded(Ts...) -> Overloaded; std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const { - std::lock_guard lock(storage.storage_mutex); + std::shared_lock lock(storage.storage_mutex); if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; @@ -532,7 +532,7 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool is_local, void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) { - assert(!delta.path.empty()); + chassert(!delta.path.empty()); std::visit( [&](const DeltaType & operation) @@ -541,12 +541,12 @@ void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) if constexpr (std::same_as) { - assert(node); + chassert(node); node = nullptr; } else if constexpr (std::same_as) { - assert(!node); + chassert(!node); node = std::make_shared(); node->stats = operation.stat; node->setData(operation.data); @@ -554,13 +554,13 @@ void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) } else if constexpr (std::same_as) { - assert(node); + chassert(node); node->invalidateDigestCache(); node->stats = operation.old_stats; } else if constexpr (std::same_as) { - assert(node); + chassert(node); node->invalidateDigestCache(); node->setData(operation.old_data); } @@ -569,7 +569,7 @@ void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) acls = operation.old_acls; } - if (applied_zxids.back() != delta.zxid) + if (applied_zxids.back() == delta.zxid) applied_zxids.pop_back(); }, delta.operation); @@ -645,15 +645,16 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) rollback_deltas.splice(rollback_deltas.end(), deltas, delta_it.base(), deltas.end()); } + rollback(std::move(rollback_deltas)); +} + +void KeeperStorage::UncommittedState::rollback(std::list rollback_deltas) +{ // we need to undo ephemeral mapping modifications // CreateNodeDelta added ephemeral for session id -> we need to remove it // RemoveNodeDelta removed ephemeral for session id -> we need to add it back for (auto & delta : rollback_deltas) { - if (delta.zxid < rollback_zxid) - break; - - chassert(delta.zxid == rollback_zxid); if (!delta.path.empty()) { std::visit( @@ -674,6 +675,8 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) } }, delta.operation); + + rollbackDelta(delta); } else if (auto * add_auth = std::get_if(&delta.operation)) { @@ -685,8 +688,6 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) session_and_auth.erase(add_auth->session_id); } } - - rollbackDelta(delta); } } @@ -849,13 +850,9 @@ Coordination::Error KeeperStorage::commit(std::list deltas) removeDigest(node_it->value, path); auto updated_node = container.updateValue(path, [&](auto & node) { if constexpr (std::same_as) - { node.stats = operation.new_stats; - } else - { node.setData(std::move(operation.new_data)); - } }); addDigest(updated_node->value, path); @@ -1044,7 +1041,7 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ { if (is_local) { - std::lock_guard lock(storage.storage_mutex); + std::shared_lock lock(storage.storage_mutex); auto node_it = storage.container.find(path); if (node_it == storage.container.end()) return {}; @@ -1223,6 +1220,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr zxid, KeeperStorage::CreateNodeDelta{stat, std::move(node_acls), request.data}); + digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } @@ -2019,16 +2017,14 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro if (!new_subdeltas.empty()) { - if (auto * error = std::get_if(&new_deltas.back().operation); + if (auto * error = std::get_if(&new_subdeltas.back().operation); error && *operation_type == OperationType::Write) { - storage.uncommitted_state.rollback(zxid); + storage.uncommitted_state.rollback(std::move(new_deltas)); response_errors.push_back(error->error); for (size_t j = i + 1; j < concrete_requests.size(); ++j) - { response_errors.push_back(Coordination::Error::ZRUNTIMEINCONSISTENCY); - } return {KeeperStorage::Delta{zxid, KeeperStorage::FailedMultiDelta{std::move(response_errors)}}}; } @@ -2043,9 +2039,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro } digest = current_digest; - storage.uncommitted_state.addDeltas(std::move(new_deltas)); - return {}; } @@ -2188,19 +2182,12 @@ void KeeperStorage::finalize() finalized = true; - { - std::lock_guard lock(ephemerals_mutex); - ephemerals.clear(); - } + ephemerals.clear(); - { - std::lock_guard lock(watches_mutex); - watches.clear(); - list_watches.clear(); - sessions_and_watchers.clear(); - } + watches.clear(); + list_watches.clear(); + sessions_and_watchers.clear(); - std::lock_guard lock(session_mutex); session_expiry_queue.clear(); } @@ -2509,10 +2496,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( KeeperStorage::ResponsesForSessions results; /// ZooKeeper update sessions expirity for each request, not only for heartbeats - { - std::lock_guard lock(session_mutex); - session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); - } + session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { @@ -2520,7 +2504,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( { if (std::holds_alternative(delta.operation)) { - std::lock_guard lock(watches_mutex); auto responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); results.insert(results.end(), responses.begin(), responses.end()); } @@ -2540,11 +2523,8 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( auto response = std::make_shared(); response->xid = zk_request->xid; response->zxid = commit_zxid; - { - std::lock_guard lock(session_mutex); - session_expiry_queue.remove(session_id); - session_and_timeout.erase(session_id); - } + session_expiry_queue.remove(session_id); + session_and_timeout.erase(session_id); results.push_back(ResponseForSession{session_id, response}); } else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special @@ -2589,7 +2569,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// Watches for this requests are added to the watches lists if (zk_request->has_watch) { - std::lock_guard lock(watches_mutex); if (response->error == Coordination::Error::ZOK) { static constexpr std::array list_requests{ @@ -2614,7 +2593,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { - std::lock_guard lock(watches_mutex); auto watch_responses = request_processor->processWatches(watches, list_watches); results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } @@ -2631,6 +2609,8 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( std::lock_guard lock(transaction_mutex); if (new_last_zxid) uncommitted_transactions.pop_front(); + if (commit_zxid < zxid) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to commit smaller ZXID, commit ZXID: {}, current ZXID {}", commit_zxid, zxid); zxid = commit_zxid; } @@ -2639,7 +2619,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS { - std::unique_lock transaction_lock(transaction_mutex); if (allow_missing && (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid < rollback_zxid)) return; @@ -2655,7 +2634,6 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) T try { uncommitted_transactions.pop_back(); - transaction_lock.unlock(); uncommitted_state.rollback(rollback_zxid); } catch (...) @@ -2672,7 +2650,7 @@ KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed, bool lock_tr if (committed) { - std::lock_guard storage_lock(storage_mutex); + std::shared_lock storage_lock(storage_mutex); return {CURRENT_DIGEST_VERSION, nodes_digest}; } @@ -2684,7 +2662,7 @@ KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed, bool lock_tr { if (lock_transaction_mutex) transaction_lock.unlock(); - std::lock_guard storage_lock(storage_mutex); + std::shared_lock storage_lock(storage_mutex); return {CURRENT_DIGEST_VERSION, nodes_digest}; } @@ -2709,7 +2687,6 @@ void KeeperStorage::addDigest(const Node & node, const std::string_view path) /// Allocate new session id with the specified timeouts int64_t KeeperStorage::getSessionID(int64_t session_timeout_ms) { - std::lock_guard lock(session_mutex); auto result = session_id_counter++; session_and_timeout.emplace(result, session_timeout_ms); session_expiry_queue.addNewSessionOrUpdate(result, session_timeout_ms); @@ -2725,71 +2702,60 @@ void KeeperStorage::addSessionID(int64_t session_id, int64_t session_timeout_ms) std::vector KeeperStorage::getDeadSessions() const { - std::lock_guard lock(session_mutex); return session_expiry_queue.getExpiredSessions(); } SessionAndTimeout KeeperStorage::getActiveSessions() const { - std::lock_guard lock(session_mutex); return session_and_timeout; } /// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. void KeeperStorage::enableSnapshotMode(size_t up_to_version) { - std::lock_guard lock(storage_mutex); container.enableSnapshotMode(up_to_version); } /// Turn off snapshot mode. void KeeperStorage::disableSnapshotMode() { - std::lock_guard lock(storage_mutex); container.disableSnapshotMode(); } KeeperStorage::Container::const_iterator KeeperStorage::getSnapshotIteratorBegin() const { - std::lock_guard lock(storage_mutex); return container.begin(); } /// Clear outdated data from internal container. void KeeperStorage::clearGarbageAfterSnapshot() { - std::lock_guard lock(storage_mutex); container.clearOutdatedNodes(); } /// Introspection functions mostly used in 4-letter commands uint64_t KeeperStorage::getNodesCount() const { - std::lock_guard lock(storage_mutex); return container.size(); } uint64_t KeeperStorage::getApproximateDataSize() const { - std::lock_guard lock(storage_mutex); return container.getApproximateDataSize(); } uint64_t KeeperStorage::getArenaDataSize() const { - std::lock_guard lock(storage_mutex); return container.keyArenaSize(); } uint64_t KeeperStorage::getWatchedPathsCount() const { - std::lock_guard lock(watches_mutex); return watches.size() + list_watches.size(); } void KeeperStorage::clearDeadWatches(int64_t session_id) { - std::lock_guard lock(watches_mutex); /// Clear all watches for this session auto watches_it = sessions_and_watchers.find(session_id); if (watches_it != sessions_and_watchers.end()) @@ -2823,7 +2789,6 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const { - std::lock_guard lock(watches_mutex); for (const auto & [session_id, watches_paths] : sessions_and_watchers) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; @@ -2842,7 +2807,6 @@ void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const } }; - std::lock_guard lock(watches_mutex); for (const auto & [watch_path, sessions] : watches) { buf << watch_path << "\n"; @@ -2866,17 +2830,13 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co } }; - { - std::lock_guard lock(session_mutex); - buf << "Sessions dump (" << session_and_timeout.size() << "):\n"; + buf << "Sessions dump (" << session_and_timeout.size() << "):\n"; - for (const auto & [session_id, _] : session_and_timeout) - { - buf << "0x" << getHexUIntLowercase(session_id) << "\n"; - } + for (const auto & [session_id, _] : session_and_timeout) + { + buf << "0x" << getHexUIntLowercase(session_id) << "\n"; } - std::lock_guard ephemerals_lock(ephemerals_mutex); buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCountLocked() << "):\n"; for (const auto & [session_id, ephemeral_paths] : ephemerals) { @@ -2904,7 +2864,6 @@ const KeeperStorage::Stats & KeeperStorage::getStorageStats() const uint64_t KeeperStorage::getTotalWatchesCount() const { - std::lock_guard lock(watches_mutex); uint64_t ret = 0; for (const auto & [session, paths] : sessions_and_watchers) ret += paths.size(); @@ -2914,7 +2873,6 @@ uint64_t KeeperStorage::getTotalWatchesCount() const uint64_t KeeperStorage::getSessionWithEphemeralNodesCount() const { - std::lock_guard ephemerals_lock(ephemerals_mutex); return getSessionWithEphemeralNodesCountLocked(); } @@ -2925,13 +2883,11 @@ uint64_t KeeperStorage::getSessionWithEphemeralNodesCountLocked() const uint64_t KeeperStorage::getSessionsWithWatchesCount() const { - std::lock_guard lock(watches_mutex); return sessions_and_watchers.size(); } uint64_t KeeperStorage::getTotalEphemeralNodesCount() const { - std::lock_guard ephemerals_lock(ephemerals_mutex); uint64_t ret = 0; for (const auto & [session_id, nodes] : ephemerals) ret += nodes.size(); @@ -2941,7 +2897,6 @@ uint64_t KeeperStorage::getTotalEphemeralNodesCount() const void KeeperStorage::recalculateStats() { - std::lock_guard lock(storage_mutex); container.recalculateDataSize(); } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 0e773fe68ef..d72ae46dee2 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -370,6 +370,7 @@ public: void addDeltas(std::list new_deltas); void cleanup(int64_t commit_zxid); void rollback(int64_t rollback_zxid); + void rollback(std::list rollback_deltas); std::shared_ptr getNode(StringRef path) const; Coordination::ACLs getACLs(StringRef path) const; @@ -451,14 +452,13 @@ public: mutable std::mutex ephemerals_mutex; /// Mapping session_id -> set of ephemeral nodes paths - Ephemerals ephemerals TSA_GUARDED_BY(ephemerals_mutex); + Ephemerals ephemerals; - mutable std::mutex session_mutex; - int64_t session_id_counter TSA_GUARDED_BY(session_mutex) = 1; + int64_t session_id_counter = 1; /// Expiration queue for session, allows to get dead sessions at some point of time - SessionExpiryQueue session_expiry_queue TSA_GUARDED_BY(session_mutex); + SessionExpiryQueue session_expiry_queue; /// All active sessions with timeout - SessionAndTimeout session_and_timeout TSA_GUARDED_BY(session_mutex); + SessionAndTimeout session_and_timeout; /// ACLMap for more compact ACLs storage inside nodes. ACLMap acl_map; @@ -490,13 +490,12 @@ public: std::atomic finalized{false}; - mutable std::mutex watches_mutex; /// Mapping session_id -> set of watched nodes paths - SessionAndWatcher sessions_and_watchers TSA_GUARDED_BY(watches_mutex); + SessionAndWatcher sessions_and_watchers; /// Currently active watches (node_path -> subscribed sessions) - Watches watches TSA_GUARDED_BY(watches_mutex); - Watches list_watches TSA_GUARDED_BY(watches_mutex); /// Watches for 'list' request (watches on children). + Watches watches; + Watches list_watches; /// Watches for 'list' request (watches on children). void clearDeadWatches(int64_t session_id); @@ -606,7 +605,7 @@ public: void recalculateStats(); private: - uint64_t getSessionWithEphemeralNodesCountLocked() const TSA_REQUIRES(ephemerals_mutex); + uint64_t getSessionWithEphemeralNodesCountLocked() const; void removeDigest(const Node & node, std::string_view path); void addDigest(const Node & node, std::string_view path); From 7dca59da5644a76b48324b841a4ce6d8a0b1f559 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 17 Jun 2024 09:03:49 +0200 Subject: [PATCH 15/27] Revert "Merge branch 'use-thread-from-global-pool-in-poco-threadpool' into keeper-some-improvement" This reverts commit 737d7484c5b3b737eb1ca1033c489c927bcba1cc, reversing changes made to b3a742304eb8981a52569fce56208b33efe94b11. --- base/poco/Foundation/CMakeLists.txt | 3 - base/poco/Foundation/src/ThreadPool.cpp | 79 +++++++------------------ programs/server/Server.cpp | 27 ++++----- src/Server/HTTPHandler.cpp | 1 + src/Server/InterserverIOHTTPHandler.cpp | 2 + src/Server/KeeperTCPHandler.cpp | 1 + src/Server/MySQLHandler.cpp | 2 + src/Server/PostgreSQLHandler.cpp | 2 + src/Server/TCPHandler.cpp | 1 + 9 files changed, 45 insertions(+), 73 deletions(-) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index fe59e72569c..dfb41a33fb1 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -168,9 +168,6 @@ set (SRCS add_library (_poco_foundation ${SRCS}) add_library (Poco::Foundation ALIAS _poco_foundation) -target_link_libraries (_poco_foundation PRIVATE clickhouse_common_io) -target_compile_definitions(_poco_foundation PUBLIC USE_CLICKHOUSE_THREADS=1) - # TODO: remove these warning exclusions target_compile_options (_poco_foundation PRIVATE diff --git a/base/poco/Foundation/src/ThreadPool.cpp b/base/poco/Foundation/src/ThreadPool.cpp index a13e91d8b76..6335ee82b47 100644 --- a/base/poco/Foundation/src/ThreadPool.cpp +++ b/base/poco/Foundation/src/ThreadPool.cpp @@ -20,15 +20,12 @@ #include "Poco/ErrorHandler.h" #include #include -#if USE_CLICKHOUSE_THREADS -#include -#endif namespace Poco { -class PooledThread : public Runnable +class PooledThread: public Runnable { public: PooledThread(const std::string& name, int stackSize = POCO_THREAD_STACK_SIZE); @@ -49,11 +46,7 @@ private: volatile std::time_t _idleTime; Runnable* _pTarget; std::string _name; -#if USE_CLICKHOUSE_THREADS - ThreadFromGlobalPool _thread; -#else Thread _thread; -#endif Event _targetReady; Event _targetCompleted; Event _started; @@ -61,20 +54,16 @@ private: }; -PooledThread::PooledThread(const std::string& name, [[maybe_unused]] int stackSize): - _idle(true), - _idleTime(0), - _pTarget(0), - _name(name), -#if !USE_CLICKHOUSE_THREADS +PooledThread::PooledThread(const std::string& name, int stackSize): + _idle(true), + _idleTime(0), + _pTarget(0), + _name(name), _thread(name), -#endif _targetCompleted(false) { poco_assert_dbg (stackSize >= 0); -#if !USE_CLICKHOUSE_THREADS _thread.setStackSize(stackSize); -#endif _idleTime = std::time(NULL); } @@ -86,32 +75,24 @@ PooledThread::~PooledThread() void PooledThread::start() { -#if USE_CLICKHOUSE_THREADS - _thread = ThreadFromGlobalPool([this]() { run(); }); -#else _thread.start(*this); -#endif _started.wait(); } -void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& target) +void PooledThread::start(Thread::Priority priority, Runnable& target) { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_pTarget == 0); _pTarget = ⌖ - -#if !USE_CLICKHOUSE_THREADS _thread.setPriority(priority); -#endif - _targetReady.set(); } -void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& target, const std::string& name) +void PooledThread::start(Thread::Priority priority, Runnable& target, const std::string& name) { FastMutex::ScopedLock lock(_mutex); @@ -126,12 +107,9 @@ void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& t fullName.append(_name); fullName.append(")"); } - -#if !USE_CLICKHOUSE_THREADS _thread.setName(fullName); _thread.setPriority(priority); -#endif - + poco_assert (_pTarget == 0); _pTarget = ⌖ @@ -167,7 +145,7 @@ void PooledThread::join() void PooledThread::activate() { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_idle); _idle = false; _targetCompleted.reset(); @@ -176,30 +154,21 @@ void PooledThread::activate() void PooledThread::release() { + const long JOIN_TIMEOUT = 10000; + _mutex.lock(); _pTarget = 0; _mutex.unlock(); - // In case of a statically allocated thread pool (such // as the default thread pool), Windows may have already // terminated the thread before we got here. -#if USE_CLICKHOUSE_THREADS - if (_thread.joinable()) -#else if (_thread.isRunning()) -#endif _targetReady.set(); -#if USE_CLICKHOUSE_THREADS - if (_thread.joinable()) - _thread.join(); -#else - const long JOIN_TIMEOUT = 10000; if (_thread.tryJoin(JOIN_TIMEOUT)) { delete this; } -#endif } @@ -236,10 +205,8 @@ void PooledThread::run() _idle = true; _targetCompleted.set(); ThreadLocalStorage::clear(); -#if !USE_CLICKHOUSE_THREADS _thread.setName(_name); _thread.setPriority(Thread::PRIO_NORMAL); -#endif } else { @@ -253,9 +220,9 @@ void PooledThread::run() ThreadPool::ThreadPool(int minCapacity, int maxCapacity, int idleTime, - int stackSize): - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + int stackSize): + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), @@ -278,8 +245,8 @@ ThreadPool::ThreadPool(const std::string& name, int idleTime, int stackSize): _name(name), - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), @@ -426,15 +393,15 @@ void ThreadPool::housekeep() ThreadVec activeThreads; idleThreads.reserve(_threads.size()); activeThreads.reserve(_threads.size()); - + for (ThreadVec::iterator it = _threads.begin(); it != _threads.end(); ++it) { if ((*it)->idle()) { if ((*it)->idleTime() < _idleTime) idleThreads.push_back(*it); - else - expiredThreads.push_back(*it); + else + expiredThreads.push_back(*it); } else activeThreads.push_back(*it); } @@ -514,7 +481,7 @@ public: ThreadPool* pool() { FastMutex::ScopedLock lock(_mutex); - + if (!_pPool) { _pPool = new ThreadPool("default"); @@ -523,7 +490,7 @@ public: } return _pPool; } - + private: ThreadPool* _pPool; FastMutex _mutex; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 802c994e872..84ff0e94cef 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -721,6 +721,11 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); + Poco::ThreadPool server_pool(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -818,11 +823,6 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } - auto server_pool = std::make_unique(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; @@ -898,8 +898,7 @@ try global_context->shutdownKeeperDispatcher(); /// Wait server pool to avoid use-after-free of destroyed context in the handlers - server_pool->joinAll(); - server_pool.reset(); + server_pool.joinAll(); /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. * At this moment, no one could own shared part of Context. @@ -1630,7 +1629,7 @@ try if (global_context->isServerCompletelyStarted()) { std::lock_guard lock(servers_lock); - updateServers(*config, *server_pool, async_metrics, servers, servers_to_start_before_tables); + updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); } } @@ -1727,7 +1726,7 @@ try config_getter, global_context->getKeeperDispatcher(), global_context->getSettingsRef().receive_timeout.totalSeconds(), global_context->getSettingsRef().send_timeout.totalSeconds(), - false), *server_pool, socket)); + false), server_pool, socket)); }); const char * secure_port_name = "keeper_server.tcp_port_secure"; @@ -1749,7 +1748,7 @@ try new KeeperTCPHandlerFactory( config_getter, global_context->getKeeperDispatcher(), global_context->getSettingsRef().receive_timeout.totalSeconds(), - global_context->getSettingsRef().send_timeout.totalSeconds(), true), *server_pool, socket)); + global_context->getSettingsRef().send_timeout.totalSeconds(), true), server_pool, socket)); #else UNUSED(port); throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); @@ -1781,7 +1780,7 @@ try createKeeperHTTPControlMainHandlerFactory( config_getter(), global_context->getKeeperDispatcher(), - "KeeperHTTPControlHandler-factory"), *server_pool, socket, http_params)); + "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params)); }); } #else @@ -1805,7 +1804,7 @@ try config(), interserver_listen_hosts, listen_try, - *server_pool, + server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ false); @@ -1856,7 +1855,7 @@ try config(), listen_hosts, listen_try, - *server_pool, + server_pool, async_metrics, servers, /* start_servers= */ true, @@ -2028,7 +2027,7 @@ try { std::lock_guard lock(servers_lock); - createServers(config(), listen_hosts, listen_try, *server_pool, async_metrics, servers); + createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); if (servers.empty()) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index d6afa571e71..02d0959ff50 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1060,6 +1060,7 @@ void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("HTTPHandler"); + ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::HTTP, request.isSecure()); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 45c28babe3a..0d79aaa227b 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -80,6 +81,7 @@ void InterserverIOHTTPHandler::processQuery(HTTPServerRequest & request, HTTPSer void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("IntersrvHandler"); + ThreadStatus thread_status; /// In order to work keep-alive. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 4612e2e9fa8..6709cd298e5 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -309,6 +309,7 @@ Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length, bool void KeeperTCPHandler::runImpl() { setThreadName("KeeperHandler"); + ThreadStatus thread_status; socket().setReceiveTimeout(receive_timeout); socket().setSendTimeout(send_timeout); diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 9471509ad4b..6456f6d24ff 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -198,6 +199,7 @@ MySQLHandler::~MySQLHandler() = default; void MySQLHandler::run() { setThreadName("MySQLHandler"); + ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::MYSQL); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 8ba8421e6f0..473d681ddb2 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -58,6 +59,7 @@ void PostgreSQLHandler::changeIO(Poco::Net::StreamSocket & socket) void PostgreSQLHandler::run() { setThreadName("PostgresHandler"); + ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::POSTGRESQL); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b60339e9fd8..e3a820340ad 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -246,6 +246,7 @@ TCPHandler::~TCPHandler() void TCPHandler::runImpl() { setThreadName("TCPHandler"); + ThreadStatus thread_status; extractConnectionSettingsFromContext(server.context()); From b6c361954319957533ee5171a51742001b803ec8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 14:51:06 +0200 Subject: [PATCH 16/27] Whitespace --- src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStorage.cpp | 45 ++++++++++++++------------- src/Coordination/KeeperStorage.h | 2 +- 3 files changed, 25 insertions(+), 24 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index dd6e19a6118..8de14ba301b 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -223,7 +223,7 @@ public: uint64_t getLastProcessedZxid() const override; const KeeperStorageBase::Stats & getStorageStats() const override; - + uint64_t getNodesCount() const override; uint64_t getTotalWatchesCount() const override; uint64_t getWatchedPathsCount() const override; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 579bcd877b4..9361ce9a4fc 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -972,7 +972,8 @@ Coordination::Error KeeperStorage::commit(std::list deltas) if constexpr (!use_rocksdb) removeDigest(node_it->value, path); - auto updated_node = container.updateValue(path, [&](auto & node) { + auto updated_node = container.updateValue(path, [&](auto & node) + { if constexpr (std::same_as) node.stats = operation.new_stats; else @@ -1376,7 +1377,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); @@ -1438,7 +1439,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); @@ -1482,12 +1483,12 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); return processImpl(storage, std::move(deltas)); @@ -1578,7 +1579,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); @@ -1612,7 +1613,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); @@ -1645,12 +1646,12 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); return processImpl(storage, std::move(deltas)); @@ -1666,7 +1667,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - + std::list preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { @@ -1714,7 +1715,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { auto & container = storage.container; @@ -1768,7 +1769,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); @@ -1857,12 +1858,12 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); return processImpl(storage, std::move(deltas)); @@ -1910,7 +1911,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); @@ -1956,12 +1957,12 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); return processImpl(storage, std::move(deltas)); @@ -2022,7 +2023,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); @@ -2066,7 +2067,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); @@ -2099,12 +2100,12 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } @@ -2348,7 +2349,7 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 97f790abe3d..6326e1c76a9 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -360,7 +360,7 @@ public: struct UpdateNodeDataDelta { - + std::string old_data; std::string new_data; int32_t version{-1}; From c61fc591c42c80882e09e432dc51349686698f7d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Aug 2024 11:33:17 +0200 Subject: [PATCH 17/27] Use functions instead of classes --- src/Coordination/KeeperStorage.cpp | 2300 ++++++++++++++-------------- 1 file changed, 1149 insertions(+), 1151 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 9361ce9a4fc..80b917deb2f 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1127,65 +1127,54 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers return true; } -template -struct KeeperStorageRequestProcessor +template +auto callOnConcreteRequestType(const Coordination::ZooKeeperRequest & zk_request, F function) { - Coordination::ZooKeeperRequestPtr zk_request; - - explicit KeeperStorageRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) : zk_request(zk_request_) { } - virtual Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const = 0; - virtual std::list - preprocess(Storage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const + switch (zk_request.getOpNum()) { - return {}; + case Coordination::OpNum::Heartbeat: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Sync: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Get: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Create: + case Coordination::OpNum::CreateIfNotExists: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Remove: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Exists: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Set: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::List: + case Coordination::OpNum::FilteredList: + case Coordination::OpNum::SimpleList: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Check: + case Coordination::OpNum::CheckNotExists: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Multi: + case Coordination::OpNum::MultiRead: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Auth: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Close: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::SetACL: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::GetACL: + return function(dynamic_cast(zk_request)); + default: + throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Unexpected request type: {}", zk_request.getOpNum()}; } +} - // process the request using locally committed data - virtual Coordination::ZooKeeperResponsePtr - processLocal(Storage & /*storage*/, std::list /* deltas */) const - { - throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Cannot process the request locally"}; - } - - virtual KeeperStorageBase::ResponsesForSessions - processWatches(KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) const - { - return {}; - } - - virtual bool checkAuth(Storage & /*storage*/, int64_t /*session_id*/, bool /*is_local*/) const { return true; } - - virtual ~KeeperStorageRequestProcessor() = default; -}; - -template -struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageRequestProcessor +template +auto callOnConcreteRequestType(const Coordination::Request & request, F function) { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - Coordination::ZooKeeperResponsePtr - process(Storage & storage, std::list deltas) const override - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - response_ptr->error = storage.commit(std::move(deltas)); - return response_ptr; - } -}; - -template -struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - Coordination::ZooKeeperResponsePtr - process(Storage & /* storage */, std::list /* deltas */) const override - { - auto response = this->zk_request->makeResponse(); - dynamic_cast(*response).path - = dynamic_cast(*this->zk_request).path; - return response; - } -}; + return callOnConcreteRequestType(dynamic_cast(request), function); +} namespace { @@ -1261,1105 +1250,1174 @@ void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const ephemerals.erase(ephemerals_it); } -template -struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor +/// Default implementations /// +template T, typename Storage> +Coordination::ZooKeeperResponsePtr +processLocal(const T & zk_request, Storage & /*storage*/, std::list /*deltas*/) { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Local processing not supported for request with type {}", zk_request.getOpNum()); +} - KeeperStorageBase::ResponsesForSessions - processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override +template T, typename Storage> +std::list preprocess( + const T & /*zk_request*/, + Storage & /*storage*/, + int64_t /*zxid*/, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + return {}; +} + +template T> +KeeperStorageBase::ResponsesForSessions +processWatches(const T & /*zk_request*/, KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) +{ + return {}; +} + +template T, typename Storage> +bool checkAuth(const T & /*zk_request*/, Storage & /*storage*/, int64_t /*session_id*/, bool /*is_local*/) +{ + return true; +} +/// Default implementations /// + +/// HEARTBEAT Request /// + +template +Coordination::ZooKeeperResponsePtr process( + const Coordination::ZooKeeperHeartbeatRequest & zk_request, + Storage & storage, + std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + response_ptr->error = storage.commit(std::move(deltas)); + return response_ptr; +} + +/// HEARTBEAT Request /// + +/// SYNC Request /// + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* storage */, std::list /* deltas */) +{ + auto response = zk_request.makeResponse(); + dynamic_cast(*response).path = zk_request.path; + return response; +} + +/// SYNC Request /// + +/// CREATE Request /// +template +bool checkAuth(const Coordination::ZooKeeperCreateRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + auto path = zk_request.getPath(); + return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); +} + +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperCreateRequest & zk_request, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches) +{ + return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CREATED); +} + +template +std::list preprocess( + const Coordination::ZooKeeperCreateRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t session_id, + int64_t time, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + ProfileEvents::increment(ProfileEvents::KeeperCreateRequest); + + std::list new_deltas; + + auto parent_path = parentNodePath(zk_request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + if (parent_node == nullptr) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + else if (parent_node->stats.isEphemeral()) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}}; + + std::string path_created = zk_request.path; + if (zk_request.is_sequential) { - return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED); - } - - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override - { - auto path = this->zk_request->getPath(); - return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); - } - - std::list - 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(*this->zk_request); - - std::list new_deltas; - - auto parent_path = parentNodePath(request.path); - auto parent_node = storage.uncommitted_state.getNode(parent_path); - if (parent_node == nullptr) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - else if (parent_node->stats.isEphemeral()) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}}; - - std::string path_created = request.path; - if (request.is_sequential) - { - if (request.not_exists) - return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - - auto seq_num = parent_node->stats.seqNum(); - - std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - seq_num_str.exceptions(std::ios::failbit); - seq_num_str << std::setw(10) << std::setfill('0') << seq_num; - - path_created += seq_num_str.str(); - } - - if (Coordination::matchPath(path_created, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) - { - 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 {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - } - - if (storage.uncommitted_state.getNode(path_created)) - { - if (this->zk_request->getOpNum() == Coordination::OpNum::CreateIfNotExists) - return new_deltas; - - return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; - } - - if (getBaseNodeName(path_created).size == 0) + if (zk_request.not_exists) return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - Coordination::ACLs node_acls; - if (!fixupACL(request.acls, session_id, storage.uncommitted_state, node_acls)) - return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + auto seq_num = parent_node->stats.seqNum(); - if (request.is_ephemeral) - { - std::lock_guard lock(storage.ephemerals_mutex); - storage.ephemerals[session_id].emplace(path_created); - } + std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + seq_num_str.exceptions(std::ios::failbit); + seq_num_str << std::setw(10) << std::setfill('0') << seq_num; - int32_t parent_cversion = request.parent_cversion; - - KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); - update_parent_delta.new_stats.increaseSeqNum(); - - if (parent_cversion == -1) - ++update_parent_delta.new_stats.cversion; - else if (parent_cversion > update_parent_delta.old_stats.cversion) - update_parent_delta.new_stats.cversion = parent_cversion; - - if (zxid > update_parent_delta.old_stats.pzxid) - update_parent_delta.new_stats.pzxid = zxid; - - update_parent_delta.new_stats.increaseNumChildren(); - - new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(update_parent_delta)); - - Coordination::Stat stat; - stat.czxid = zxid; - stat.mzxid = zxid; - stat.pzxid = zxid; - stat.ctime = time; - stat.mtime = time; - stat.numChildren = 0; - stat.version = 0; - stat.aversion = 0; - stat.cversion = 0; - stat.ephemeralOwner = request.is_ephemeral ? session_id : 0; - - new_deltas.emplace_back( - std::move(path_created), - zxid, - typename Storage::CreateNodeDelta{stat, std::move(node_acls), request.data}); - - - digest = storage.calculateNodesDigest(digest, new_deltas); - return new_deltas; + path_created += seq_num_str.str(); } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + if (Coordination::matchPath(path_created, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); + 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); - if (deltas.empty()) - { - response.path_created = this->zk_request->getPath(); - response.error = Coordination::Error::ZOK; - return response_ptr; - } + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + } - std::string created_path; - auto create_delta_it = std::find_if( - deltas.begin(), - deltas.end(), - [](const auto & delta) - { return std::holds_alternative(delta.operation); }); + if (storage.uncommitted_state.getNode(path_created)) + { + if (zk_request.getOpNum() == Coordination::OpNum::CreateIfNotExists) + return new_deltas; - if (create_delta_it != deltas.end()) - created_path = create_delta_it->path; + return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; + } - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } + if (getBaseNodeName(path_created).size == 0) + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - response.path_created = std::move(created_path); + Coordination::ACLs node_acls; + if (!fixupACL(zk_request.acls, session_id, storage.uncommitted_state, node_acls)) + return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + + if (zk_request.is_ephemeral) + { + std::lock_guard lock(storage.ephemerals_mutex); + storage.ephemerals[session_id].emplace(path_created); + } + + int32_t parent_cversion = zk_request.parent_cversion; + + KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); + update_parent_delta.new_stats.increaseSeqNum(); + + if (parent_cversion == -1) + ++update_parent_delta.new_stats.cversion; + else if (parent_cversion > update_parent_delta.old_stats.cversion) + update_parent_delta.new_stats.cversion = parent_cversion; + + if (zxid > update_parent_delta.old_stats.pzxid) + update_parent_delta.new_stats.pzxid = zxid; + + update_parent_delta.new_stats.increaseNumChildren(); + + new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(update_parent_delta)); + + Coordination::Stat stat; + stat.czxid = zxid; + stat.mzxid = zxid; + stat.pzxid = zxid; + stat.ctime = time; + stat.mtime = time; + stat.numChildren = 0; + stat.version = 0; + stat.aversion = 0; + stat.cversion = 0; + stat.ephemeralOwner = zk_request.is_ephemeral ? session_id : 0; + + new_deltas.emplace_back( + std::move(path_created), + zxid, + typename Storage::CreateNodeDelta{stat, std::move(node_acls), zk_request.data}); + + + digest = storage.calculateNodesDigest(digest, new_deltas); + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperCreateRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); + + if (deltas.empty()) + { + response.path_created = zk_request.getPath(); response.error = Coordination::Error::ZOK; return response_ptr; } -}; -template -struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor -{ - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + std::string created_path; + auto create_delta_it = std::find_if( + deltas.begin(), + deltas.end(), + [](const auto & delta) + { return std::holds_alternative(delta.operation); }); + + if (create_delta_it != deltas.end()) + created_path = create_delta_it->path; + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); + response.error = result; + return response_ptr; } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + response.path_created = std::move(created_path); + response.error = Coordination::Error::ZOK; + return response_ptr; +} +/// CREATE Request /// - std::list - 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(*this->zk_request); +/// GET Request /// +template +bool checkAuth(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Read, session_id, is_local); +} - if (request.path == Coordination::keeper_api_feature_flags_path - || request.path == Coordination::keeper_config_path - || request.path == Coordination::keeper_availability_zone_path) - return {}; - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; +template +std::list preprocess( + const Coordination::ZooKeeperGetRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + ProfileEvents::increment(ProfileEvents::KeeperGetRequest); + if (zk_request.path == Coordination::keeper_api_feature_flags_path + || zk_request.path == Coordination::keeper_config_path + || zk_request.path == Coordination::keeper_availability_zone_path) return {}; - } - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + return {}; +} + +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperGetRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - } - - if (request.path == Coordination::keeper_config_path) - { - response.data = serializeClusterConfig( - storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); - response.error = Coordination::Error::ZOK; - return response_ptr; - } - - auto & container = storage.container; - auto node_it = container.find(request.path); - if (node_it == container.end()) - { - if constexpr (local) - response.error = Coordination::Error::ZNONODE; - else - onStorageInconsistency(); - } - else - { - node_it->value.setResponseStat(response.stat); - auto data = node_it->value.getData(); - response.data = std::string(data); - response.error = Coordination::Error::ZOK; - } - - return response_ptr; - } - - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - ProfileEvents::increment(ProfileEvents::KeeperGetRequest); - return processImpl(storage, std::move(deltas)); - } -}; - -template -struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor -{ - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override - { - return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); - } - - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::list - 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(*this->zk_request); - - std::list 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 {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - } - - auto parent_path = parentNodePath(request.path); - auto parent_node = storage.uncommitted_state.getNode(parent_path); - - KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); - - const auto add_parent_update_delta = [&] - { - new_deltas.emplace_back( - std::string{parent_path}, - zxid, - std::move(update_parent_delta) - ); - }; - - const auto update_parent_pzxid = [&]() - { - if (!parent_node) - return; - - if (update_parent_delta.old_stats.pzxid < zxid) - update_parent_delta.new_stats.pzxid = zxid; - }; - - auto node = storage.uncommitted_state.getNode(request.path); - - if (!node) - { - if (request.restored_from_zookeeper_log) - { - update_parent_pzxid(); - add_parent_update_delta(); - } - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNONODE}}; - } - else if (request.version != -1 && request.version != node->stats.version) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; - else if (node->stats.numChildren() != 0) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; - - if (request.restored_from_zookeeper_log) - update_parent_pzxid(); - - ++update_parent_delta.new_stats.cversion; - update_parent_delta.new_stats.decreaseNumChildren(); - add_parent_update_delta(); - - new_deltas.emplace_back( - request.path, - zxid, - KeeperStorageBase::RemoveNodeDelta{request.version, node->stats, storage.uncommitted_state.getACLs(request.path), std::string{node->getData()}}); - - if (node->stats.isEphemeral()) - storage.unregisterEphemeralPath(node->stats.ephemeralOwner(), request.path); - - digest = storage.calculateNodesDigest(digest, new_deltas); - - return new_deltas; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); - - response.error = storage.commit(std::move(deltas)); - return response_ptr; - } - - KeeperStorageBase::ResponsesForSessions - processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override - { - return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); - } -}; - -template -struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::list - 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(*this->zk_request); - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - } - - auto & container = storage.container; - auto node_it = container.find(request.path); - if (node_it == container.end()) - { - if constexpr (local) - response.error = Coordination::Error::ZNONODE; - else - onStorageInconsistency(); - } - else - { - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; - } - - return response_ptr; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); - return processImpl(storage, std::move(deltas)); - } -}; - -template -struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProcessor -{ - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override - { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Write, session_id, is_local); - } - - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::list - 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(*this->zk_request); - - std::list 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 {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - } - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - auto node = storage.uncommitted_state.getNode(request.path); - - if (request.version != -1 && request.version != node->stats.version) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; - - KeeperStorageBase::UpdateNodeStatDelta node_delta(*node); - node_delta.version = request.version; - auto & new_stats = node_delta.new_stats; - new_stats.version++; - new_stats.mzxid = zxid; - new_stats.mtime = time; - - new_deltas.emplace_back(request.path, zxid, std::move(node_delta)); - new_deltas.emplace_back( - request.path, - zxid, - KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = request.data, .version = request.version}); - - auto parent_path = parentNodePath(request.path); - auto parent_node = storage.uncommitted_state.getNode(parent_path); - KeeperStorageBase::UpdateNodeStatDelta parent_delta(*parent_node); - ++parent_delta.new_stats.cversion; - new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(parent_delta)); - - digest = storage.calculateNodesDigest(digest, new_deltas); - return new_deltas; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - auto & container = storage.container; - - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperSetRequest & request = dynamic_cast(*this->zk_request); - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; } + } - auto node_it = container.find(request.path); - if (node_it == container.end()) + if (zk_request.path == Coordination::keeper_config_path) + { + response.data = serializeClusterConfig( + storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); + response.error = Coordination::Error::ZOK; + return response_ptr; + } + + auto & container = storage.container; + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) + { + if constexpr (local) + response.error = Coordination::Error::ZNONODE; + else onStorageInconsistency(); + } + else + { + node_it->value.setResponseStat(response.stat); + auto data = node_it->value.getData(); + response.data = std::string(data); + response.error = Coordination::Error::ZOK; + } + return response_ptr; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) +{ + ProfileEvents::increment(ProfileEvents::KeeperGetRequest); + return processImpl(zk_request, storage, std::move(deltas)); +} +/// GET Request /// + +/// REMOVE Request /// +template +bool checkAuth(const Coordination::ZooKeeperRemoveRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(parentNodePath(zk_request.getPath()), Coordination::ACL::Delete, session_id, is_local); +} + +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperRemoveRequest & zk_request, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches) +{ + return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CREATED); +} + +template +std::list preprocess( + const Coordination::ZooKeeperRemoveRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /* session_id */, + int64_t /* time */, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); + + std::list new_deltas; + + if (Coordination::matchPath(zk_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", zk_request.path); + + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + } + + auto parent_path = parentNodePath(zk_request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + + KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); + + const auto add_parent_update_delta = [&] + { + new_deltas.emplace_back( + std::string{parent_path}, + zxid, + std::move(update_parent_delta) + ); + }; + + const auto update_parent_pzxid = [&]() + { + if (!parent_node) + return; + + if (update_parent_delta.old_stats.pzxid < zxid) + update_parent_delta.new_stats.pzxid = zxid; + }; + + auto node = storage.uncommitted_state.getNode(zk_request.path); + + if (!node) + { + if (zk_request.restored_from_zookeeper_log) + { + update_parent_pzxid(); + add_parent_update_delta(); + } + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNONODE}}; + } + else if (zk_request.version != -1 && zk_request.version != node->stats.version) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; + else if (node->stats.numChildren() != 0) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; + + if (zk_request.restored_from_zookeeper_log) + update_parent_pzxid(); + + ++update_parent_delta.new_stats.cversion; + update_parent_delta.new_stats.decreaseNumChildren(); + add_parent_update_delta(); + + new_deltas.emplace_back( + zk_request.path, + zxid, + KeeperStorageBase::RemoveNodeDelta{ + zk_request.version, node->stats, storage.uncommitted_state.getACLs(zk_request.path), std::string{node->getData()}}); + + if (node->stats.isEphemeral()) + storage.unregisterEphemeralPath(node->stats.ephemeralOwner(), zk_request.path); + + digest = storage.calculateNodesDigest(digest, new_deltas); + + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperRemoveRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); + + response.error = storage.commit(std::move(deltas)); + return response_ptr; +} + +/// REMOVE Request /// + +/// EXISTS Request /// +template +bool checkAuth(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Read, session_id, is_local); +} + +template +std::list preprocess( + const Coordination::ZooKeeperExistsRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); + + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + return {}; +} + +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) + { + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + { + response.error = result; + return response_ptr; + } + } + + auto & container = storage.container; + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) + { + if constexpr (local) + response.error = Coordination::Error::ZNONODE; + else + onStorageInconsistency(); + } + else + { node_it->value.setResponseStat(response.stat); response.error = Coordination::Error::ZOK; + } + return response_ptr; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) +{ + ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); + return processImpl(zk_request, storage, std::move(deltas)); +} +/// EXISTS Request /// + +/// SET Request /// +template +bool checkAuth(const Coordination::ZooKeeperSetRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Write, session_id, is_local); +} + +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperSetRequest & zk_request, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) +{ + return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CHANGED); +} + +template +std::list preprocess( + const Coordination::ZooKeeperSetRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /* session_id */, + int64_t time, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + ProfileEvents::increment(ProfileEvents::KeeperSetRequest); + + std::list new_deltas; + + if (Coordination::matchPath(zk_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", zk_request.path); + + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + } + + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + auto node = storage.uncommitted_state.getNode(zk_request.path); + + if (zk_request.version != -1 && zk_request.version != node->stats.version) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; + + KeeperStorageBase::UpdateNodeStatDelta node_delta(*node); + node_delta.version = zk_request.version; + auto & new_stats = node_delta.new_stats; + new_stats.version++; + new_stats.mzxid = zxid; + new_stats.mtime = time; + + new_deltas.emplace_back(zk_request.path, zxid, std::move(node_delta)); + new_deltas.emplace_back( + zk_request.path, + zxid, + KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = zk_request.data, .version = zk_request.version}); + + auto parent_path = parentNodePath(zk_request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + KeeperStorageBase::UpdateNodeStatDelta parent_delta(*parent_node); + ++parent_delta.new_stats.cversion; + new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(parent_delta)); + + digest = storage.calculateNodesDigest(digest, new_deltas); + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSetRequest & zk_request, Storage & storage, std::list deltas) +{ + auto & container = storage.container; + + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + { + response.error = result; return response_ptr; } - KeeperStorageBase::ResponsesForSessions - processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override - { - return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); - } -}; + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) + onStorageInconsistency(); -template -struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProcessor + node_it->value.setResponseStat(response.stat); + response.error = Coordination::Error::ZOK; + + return response_ptr; +} +/// SET Request /// + +/// LIST Request /// +template +bool checkAuth(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) { - 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); +} + +template +std::list preprocess( + const Coordination::ZooKeeperListRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + ProfileEvents::increment(ProfileEvents::KeeperListRequest); + + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + return {}; +} + +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + { + response.error = result; + return response_ptr; + } } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + auto & container = storage.container; - std::list - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) { - ProfileEvents::increment(ProfileEvents::KeeperListRequest); - Coordination::ZooKeeperListRequest & request = dynamic_cast(*this->zk_request); - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperListRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - } - - auto & container = storage.container; - - auto node_it = container.find(request.path); - if (node_it == container.end()) - { - if constexpr (local) - response.error = Coordination::Error::ZNONODE; - else - onStorageInconsistency(); - } + if constexpr (local) + response.error = Coordination::Error::ZNONODE; else - { - auto path_prefix = request.path; - if (path_prefix.empty()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty"); + onStorageInconsistency(); + } + else + { + auto path_prefix = zk_request.path; + if (path_prefix.empty()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty"); - const auto & get_children = [&]() + const auto & get_children = [&]() + { + if constexpr (Storage::use_rocksdb) + return container.getChildren(zk_request.path); + else + return node_it->value.getChildren(); + }; + const auto & children = get_children(); + response.names.reserve(children.size()); + + const auto add_child = [&](const auto & child) + { + using enum Coordination::ListRequestType; + + auto list_request_type = ALL; + if (const auto * filtered_list = dynamic_cast(&zk_request)) + { + list_request_type = filtered_list->list_request_type; + } + + if (list_request_type == ALL) + return true; + + bool is_ephemeral; + if constexpr (!Storage::use_rocksdb) + { + auto child_path = (std::filesystem::path(zk_request.path) / child.toView()).generic_string(); + auto child_it = container.find(child_path); + if (child_it == container.end()) + onStorageInconsistency(); + is_ephemeral = child_it->value.stats.isEphemeral(); + } + else + { + is_ephemeral = child.second.stats.isEphemeral(); + } + + return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); + }; + + for (const auto & child : children) + { + if (add_child(child)) { if constexpr (Storage::use_rocksdb) - return container.getChildren(request.path); + response.names.push_back(child.first); else - return node_it->value.getChildren(); - }; - const auto & children = get_children(); - response.names.reserve(children.size()); - - const auto add_child = [&](const auto & child) - { - using enum Coordination::ListRequestType; - - auto list_request_type = ALL; - if (auto * filtered_list = dynamic_cast(&request)) - { - list_request_type = filtered_list->list_request_type; - } - - if (list_request_type == ALL) - return true; - - bool is_ephemeral; - if constexpr (!Storage::use_rocksdb) - { - auto child_path = (std::filesystem::path(request.path) / child.toView()).generic_string(); - auto child_it = container.find(child_path); - if (child_it == container.end()) - onStorageInconsistency(); - is_ephemeral = child_it->value.stats.isEphemeral(); - } - else - { - is_ephemeral = child.second.stats.isEphemeral(); - } - - return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); - }; - - for (const auto & child : children) - { - if (add_child(child)) - { - if constexpr (Storage::use_rocksdb) - response.names.push_back(child.first); - else - response.names.push_back(child.toString()); - } - } - - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; - } - - return response_ptr; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - ProfileEvents::increment(ProfileEvents::KeeperListRequest); - return processImpl(storage, std::move(deltas)); - } -}; - -template -struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor -{ - explicit KeeperStorageCheckRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequestProcessor(zk_request_) - { - check_not_exists = this->zk_request->getOpNum() == Coordination::OpNum::CheckNotExists; - } - - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override - { - auto path = this->zk_request->getPath(); - return storage.checkACL(check_not_exists ? parentNodePath(path) : path, Coordination::ACL::Read, session_id, is_local); - } - - std::list - 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(*this->zk_request); - - auto node = storage.uncommitted_state.getNode(request.path); - if (check_not_exists) - { - if (node && (request.version == -1 || request.version == node->stats.version)) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; - } - else - { - if (!node) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - if (request.version != -1 && request.version != node->stats.version) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; - } - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; + response.names.push_back(child.toString()); } } - const auto on_error = [&]([[maybe_unused]] const auto error_code) - { - if constexpr (local) - response.error = error_code; - else - onStorageInconsistency(); - }; - - auto & container = storage.container; - auto node_it = container.find(request.path); - - if (check_not_exists) - { - if (node_it != container.end() && (request.version == -1 || request.version == node_it->value.stats.version)) - on_error(Coordination::Error::ZNODEEXISTS); - else - response.error = Coordination::Error::ZOK; - } - else - { - if (node_it == container.end()) - on_error(Coordination::Error::ZNONODE); - else if (request.version != -1 && request.version != node_it->value.stats.version) - on_error(Coordination::Error::ZBADVERSION); - else - response.error = Coordination::Error::ZOK; - } - - return response_ptr; + node_it->value.setResponseStat(response.stat); + response.error = Coordination::Error::ZOK; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } + return response_ptr; +} - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); - return processImpl(storage, std::move(deltas)); - } - -private: - bool check_not_exists; -}; - - -template -struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) { - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) +{ + ProfileEvents::increment(ProfileEvents::KeeperListRequest); + return processImpl(zk_request, storage, std::move(deltas)); +} +/// LIST Request /// + +/// CHECK Request /// +template +bool checkAuth(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + auto path = zk_request.getPath(); + return storage.checkACL( + zk_request.getOpNum() == Coordination::OpNum::CheckNotExists ? parentNodePath(path) : path, + Coordination::ACL::Read, + session_id, + is_local); +} + +template +std::list preprocess( + const Coordination::ZooKeeperCheckRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); + + auto node = storage.uncommitted_state.getNode(zk_request.path); + if (zk_request.getOpNum() == Coordination::OpNum::CheckNotExists) { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin, session_id, is_local); + if (node && (zk_request.version == -1 || zk_request.version == node->stats.version)) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; } - - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::list - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override + else { - 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 {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - } - - auto & uncommitted_state = storage.uncommitted_state; - if (!uncommitted_state.getNode(request.path)) + if (!node) return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - auto node = uncommitted_state.getNode(request.path); - - if (request.version != -1 && request.version != node->stats.aversion) + if (zk_request.version != -1 && zk_request.version != node->stats.version) return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; - - - Coordination::ACLs node_acls; - if (!fixupACL(request.acls, session_id, uncommitted_state, node_acls)) - return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; - - KeeperStorageBase::UpdateNodeStatDelta update_stat_delta(*node); - ++update_stat_delta.new_stats.aversion; - std::list new_deltas{ - {request.path, - zxid, - KeeperStorageBase::SetACLDelta{std::move(node_acls), uncommitted_state.getACLs(request.path), request.version}}, - {request.path, zxid, std::move(update_stat_delta)}}; - - digest = storage.calculateNodesDigest(digest, new_deltas); - - return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*this->zk_request); + return {}; +} +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) + { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; } + } - auto node_it = storage.container.find(request.path); - if (node_it == storage.container.end()) + const auto on_error = [&]([[maybe_unused]] const auto error_code) + { + if constexpr (local) + response.error = error_code; + else onStorageInconsistency(); - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + }; - return response_ptr; - } -}; + auto & container = storage.container; + auto node_it = container.find(zk_request.path); -template -struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor -{ - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + if (zk_request.getOpNum() == Coordination::OpNum::CheckNotExists) { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local); + if (node_it != container.end() && (zk_request.version == -1 || zk_request.version == node_it->value.stats.version)) + on_error(Coordination::Error::ZNODEEXISTS); + else + response.error = Coordination::Error::ZOK; } - - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::list - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + else { - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - } - - auto & container = storage.container; - auto node_it = container.find(request.path); if (node_it == container.end()) - { - if constexpr (local) - response.error = Coordination::Error::ZNONODE; - else - onStorageInconsistency(); - } + on_error(Coordination::Error::ZNONODE); + else if (zk_request.version != -1 && zk_request.version != node_it->value.stats.version) + on_error(Coordination::Error::ZBADVERSION); else + response.error = Coordination::Error::ZOK; + } + + return response_ptr; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, std::list deltas) +{ + ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); + return processImpl(zk_request, storage, std::move(deltas)); +} +/// CHECK Request /// + +/// MULTI Request /// +using OperationType = Coordination::ZooKeeperMultiRequest::OperationType; +template +bool checkAuth(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + for (const auto & concrete_request : zk_request.requests) + { + if (!callOnConcreteRequestType( + *concrete_request, [&](const auto & subrequest) { return checkAuth(subrequest, storage, session_id, is_local); })) + return false; + } + return true; +} + +template +std::list preprocess( + const Coordination::ZooKeeperMultiRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t session_id, + int64_t time, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + ProfileEvents::increment(ProfileEvents::KeeperMultiRequest); + std::vector response_errors; + const auto & subrequests = zk_request.requests; + response_errors.reserve(subrequests.size()); + uint64_t current_digest = digest; + std::list new_deltas; + for (size_t i = 0; i < subrequests.size(); ++i) + { + auto new_subdeltas = callOnConcreteRequestType( + *subrequests[i], + [&](const auto & subrequest) + { return preprocess(subrequest, storage, zxid, session_id, time, current_digest, keeper_context); }); + + if (!new_subdeltas.empty()) { - node_it->value.setResponseStat(response.stat); - response.acl = storage.acl_map.convertNumber(node_it->value.stats.acl_id); + if (auto * error = std::get_if(&new_subdeltas.back().operation); + error && zk_request.getOpNum() == Coordination::OpNum::Multi) + { + storage.uncommitted_state.rollback(std::move(new_deltas)); + response_errors.push_back(error->error); + + for (size_t j = i + 1; j < subrequests.size(); ++j) + response_errors.push_back(Coordination::Error::ZRUNTIMEINCONSISTENCY); + + return {typename Storage::Delta{zxid, typename Storage::FailedMultiDelta{std::move(response_errors)}}}; + } + } + + new_subdeltas.emplace_back(zxid, KeeperStorageBase::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 + storage.uncommitted_state.applyDeltas(new_subdeltas); + new_deltas.splice(new_deltas.end(), std::move(new_subdeltas)); + } + + digest = current_digest; + storage.uncommitted_state.addDeltas(std::move(new_deltas)); + return {}; +} + +std::list getSubdeltas(std::list & deltas) +{ + std::list subdeltas; + auto it = deltas.begin(); + + for (; it != deltas.end(); ++it) + { + if (std::holds_alternative(it->operation)) + { + ++it; + break; + } + } + + if (it == deltas.end()) + subdeltas = std::move(deltas); + else + subdeltas.splice(subdeltas.end(), deltas, deltas.begin(), it); + + return subdeltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); + + const auto & subrequests = zk_request.requests; + + // the deltas will have at least SubDeltaEnd or FailedMultiDelta + chassert(!deltas.empty()); + if (auto * failed_multi = std::get_if(&deltas.front().operation)) + { + for (size_t i = 0; i < subrequests.size(); ++i) + { + response.responses[i] = std::make_shared(); + response.responses[i]->error = failed_multi->error_codes[i]; } return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + for (size_t i = 0; i < subrequests.size(); ++i) { - return processImpl(storage, std::move(deltas)); + std::list subdeltas = getSubdeltas(deltas); + response.responses[i] = callOnConcreteRequestType( + *subrequests[i], [&](const auto & subrequest) { return process(subrequest, storage, std::move(subdeltas)); }); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } -}; + response.error = Coordination::Error::ZOK; + return response_ptr; +} -template -struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor +template +Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, std::list deltas) { - using OperationType = Coordination::ZooKeeperMultiRequest::OperationType; - std::optional operation_type; + ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + const auto & subrequests = zk_request.requests; + for (size_t i = 0; i < subrequests.size(); ++i) { - for (const auto & concrete_request : concrete_requests) - if (!concrete_request->checkAuth(storage, session_id, is_local)) - return false; - return true; + std::list subdeltas = getSubdeltas(deltas); + response.responses[i] = callOnConcreteRequestType( + *subrequests[i], [&](const auto & subrequest) { return processLocal(subrequest, storage, std::move(subdeltas)); }); } - std::vector>> concrete_requests; - explicit KeeperStorageMultiRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequestProcessor(zk_request_) + response.error = Coordination::Error::ZOK; + return response_ptr; +} + +template +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperMultiRequest & zk_request, typename Storage::Watches & watches, typename Storage::Watches & list_watches) +{ + typename Storage::ResponsesForSessions result; + + const auto & subrequests = zk_request.requests; + for (const auto & generic_request : subrequests) { - Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*this->zk_request); - concrete_requests.reserve(request.requests.size()); + auto responses = callOnConcreteRequestType( + generic_request, [&](const auto & subrequest) { return processWatches(subrequest, watches, list_watches); }); + result.insert(result.end(), responses.begin(), responses.end()); + } + return result; +} +/// MULTI Request /// - const auto check_operation_type = [&](OperationType type) - { - if (operation_type.has_value() && *operation_type != type) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal mixing of read and write operations in multi request"); - operation_type = type; - }; +/// AUTH Request /// +template +std::list preprocess( + const Coordination::ZooKeeperAuthRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t session_id, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - for (const auto & sub_request : request.requests) - { - auto sub_zk_request = std::dynamic_pointer_cast(sub_request); - switch (sub_zk_request->getOpNum()) - { - case Coordination::OpNum::Create: - case Coordination::OpNum::CreateIfNotExists: - check_operation_type(OperationType::Write); - 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)); - break; - case Coordination::OpNum::Set: - check_operation_type(OperationType::Write); - 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)); - break; - case Coordination::OpNum::Get: - check_operation_type(OperationType::Read); - 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)); - 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)); - break; - default: - throw DB::Exception( - ErrorCodes::BAD_ARGUMENTS, - "Illegal command as part of multi ZooKeeper request {}", - sub_zk_request->getOpNum()); - } - } + if (zk_request.scheme != "digest" || std::count(zk_request.data.begin(), zk_request.data.end(), ':') != 1) + return {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; - chassert(request.requests.empty() || operation_type.has_value()); + std::list new_deltas; + auto auth_digest = Storage::generateDigest(zk_request.data); + if (auth_digest == storage.superdigest) + { + auto auth = std::make_shared(); + auth->scheme = "super"; + new_deltas.emplace_back(zxid, KeeperStorageBase::AddAuthDelta{session_id, std::move(auth)}); + } + else + { + auto new_auth = std::make_shared(); + new_auth->scheme = zk_request.scheme; + new_auth->id = std::move(auth_digest); + if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return *new_auth == auth_id; })) + new_deltas.emplace_back(zxid, KeeperStorageBase::AddAuthDelta{session_id, std::move(new_auth)}); } - std::list - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperAuthRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + auth_response.error = result; + + return response_ptr; +} +/// AUTH Request /// + +/// CLOSE Request /// +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperCloseRequest & /* zk_request */, Storage &, std::list /* deltas */) +{ + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Called process on close request"); +} +/// CLOSE Request /// + +/// SETACL Request /// +template +bool checkAuth(const Coordination::ZooKeeperSetACLRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Admin, session_id, is_local); +} + +template +std::list preprocess( + const Coordination::ZooKeeperSetACLRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t session_id, + int64_t /*time*/, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + if (Coordination::matchPath(zk_request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { - ProfileEvents::increment(ProfileEvents::KeeperMultiRequest); - std::vector response_errors; - response_errors.reserve(concrete_requests.size()); - uint64_t current_digest = digest; - std::list new_deltas; - for (size_t i = 0; i < concrete_requests.size(); ++i) - { - auto new_subdeltas = concrete_requests[i]->preprocess(storage, zxid, session_id, time, current_digest, keeper_context); + auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", zk_request.path); - if (!new_subdeltas.empty()) - { - if (auto * error = std::get_if(&new_subdeltas.back().operation); - error && *operation_type == OperationType::Write) - { - storage.uncommitted_state.rollback(std::move(new_deltas)); - response_errors.push_back(error->error); - - for (size_t j = i + 1; j < concrete_requests.size(); ++j) - response_errors.push_back(Coordination::Error::ZRUNTIMEINCONSISTENCY); - - return {typename Storage::Delta{zxid, typename Storage::FailedMultiDelta{std::move(response_errors)}}}; - } - } - - new_subdeltas.emplace_back(zxid, KeeperStorageBase::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 - storage.uncommitted_state.applyDeltas(new_subdeltas); - new_deltas.splice(new_deltas.end(), std::move(new_subdeltas)); - } - - digest = current_digest; - storage.uncommitted_state.addDeltas(std::move(new_deltas)); - return {}; + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } - std::list getSubdeltas(std::list & deltas) const + auto & uncommitted_state = storage.uncommitted_state; + if (!uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + auto node = uncommitted_state.getNode(zk_request.path); + + if (zk_request.version != -1 && zk_request.version != node->stats.aversion) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; + + + Coordination::ACLs node_acls; + if (!fixupACL(zk_request.acls, session_id, uncommitted_state, node_acls)) + return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + + KeeperStorageBase::UpdateNodeStatDelta update_stat_delta(*node); + ++update_stat_delta.new_stats.aversion; + std::list new_deltas{ + {zk_request.path, + zxid, + KeeperStorageBase::SetACLDelta{std::move(node_acls), uncommitted_state.getACLs(zk_request.path), zk_request.version}}, + {zk_request.path, zxid, std::move(update_stat_delta)}}; + + digest = storage.calculateNodesDigest(digest, new_deltas); + + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperSetACLRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - std::list subdeltas; - auto it = deltas.begin(); - - for (; it != deltas.end(); ++it) - { - if (std::holds_alternative(it->operation)) - { - ++it; - break; - } - } - - if (it == deltas.end()) - subdeltas = std::move(deltas); - else - subdeltas.splice(subdeltas.end(), deltas, deltas.begin(), it); - - return subdeltas; + response.error = result; + return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + auto node_it = storage.container.find(zk_request.path); + if (node_it == storage.container.end()) + onStorageInconsistency(); + node_it->value.setResponseStat(response.stat); + response.error = Coordination::Error::ZOK; + + return response_ptr; +} +/// SETACL Request /// + +/// GETACL Request /// +template +bool checkAuth(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local); +} + +template +std::list preprocess( + const Coordination::ZooKeeperGetACLRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + return {}; +} + +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); - - // the deltas will have at least SubDeltaEnd or FailedMultiDelta - chassert(!deltas.empty()); - if (auto * failed_multi = std::get_if(&deltas.front().operation)) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - for (size_t i = 0; i < concrete_requests.size(); ++i) - { - response.responses[i] = std::make_shared(); - response.responses[i]->error = failed_multi->error_codes[i]; - } - + response.error = result; return response_ptr; } - - for (size_t i = 0; i < concrete_requests.size(); ++i) - { - std::list subdeltas = getSubdeltas(deltas); - response.responses[i] = concrete_requests[i]->process(storage, std::move(subdeltas)); - } - - response.error = Coordination::Error::ZOK; - return response_ptr; } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + auto & container = storage.container; + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) { - ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); - 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) - { - std::list subdeltas = getSubdeltas(deltas); - response.responses[i] = concrete_requests[i]->processLocal(storage, std::move(subdeltas)); - } - - response.error = Coordination::Error::ZOK; - return response_ptr; - } - - KeeperStorageBase::ResponsesForSessions - processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override - { - typename Storage::ResponsesForSessions result; - for (const auto & generic_request : concrete_requests) - { - auto responses = generic_request->processWatches(watches, list_watches); - result.insert(result.end(), responses.begin(), responses.end()); - } - return result; - } -}; - -template -struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - Coordination::ZooKeeperResponsePtr process(Storage &, std::list /* deltas */) const override - { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Called process on close request"); - } -}; - -template -struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::list - 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(*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 {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; - - std::list new_deltas; - auto auth_digest = Storage::generateDigest(auth_request.data); - if (auth_digest == storage.superdigest) - { - auto auth = std::make_shared(); - auth->scheme = "super"; - new_deltas.emplace_back(zxid, KeeperStorageBase::AddAuthDelta{session_id, std::move(auth)}); - } + if constexpr (local) + response.error = Coordination::Error::ZNONODE; else - { - auto new_auth = std::make_shared(); - new_auth->scheme = auth_request.scheme; - new_auth->id = std::move(auth_digest); - if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return *new_auth == auth_id; })) - new_deltas.emplace_back(zxid, KeeperStorageBase::AddAuthDelta{session_id, std::move(new_auth)}); - } - - return new_deltas; + onStorageInconsistency(); } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + else { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); - - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - auth_response.error = result; - - return response_ptr; + node_it->value.setResponseStat(response.stat); + response.acl = storage.acl_map.convertNumber(node_it->value.stats.acl_id); } -}; + + return response_ptr; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} +/// GETACL Request /// template void KeeperStorage::finalize() @@ -2384,72 +2442,6 @@ bool KeeperStorage::isFinalized() const return finalized; } -template -class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable -{ -public: - using Creator = std::function>(const Coordination::ZooKeeperRequestPtr &)>; - using OpNumToRequest = std::unordered_map; - - static KeeperStorageRequestProcessorsFactory & instance() - { - static KeeperStorageRequestProcessorsFactory factory; - return factory; - } - - 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()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", zk_request->getOpNum()); - - return request_it->second(zk_request); - } - - void registerRequest(Coordination::OpNum op_num, Creator creator) - { - if (!op_num_to_request.try_emplace(op_num, creator).second) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Request with op num {} already registered", op_num); - } - -private: - OpNumToRequest op_num_to_request; - KeeperStorageRequestProcessorsFactory(); -}; - -template -void registerKeeperRequestProcessor(Factory & factory) -{ - factory.registerRequest( - num, [](const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared(zk_request); }); -} - - -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); -} - - template UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::list & new_deltas) const { @@ -2604,8 +2596,6 @@ void KeeperStorage::preprocessRequest( uncommitted_state.addDeltas(std::move(new_deltas)); }); - auto request_processor = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); - if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { { @@ -2643,13 +2633,18 @@ void KeeperStorage::preprocessRequest( return; } - if (check_acl && !request_processor->checkAuth(*this, session_id, false)) + const auto preprocess_request = [&] T>(const T & concrete_zk_request) { - new_deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); - return; - } + if (check_acl && !checkAuth(concrete_zk_request, *this, session_id, false)) + { + new_deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); + return; + } - new_deltas = request_processor->preprocess(*this, transaction->zxid, session_id, time, new_digest, *keeper_context); + new_deltas = preprocess(concrete_zk_request, *this, transaction->zxid, session_id, time, new_digest, *keeper_context); + }; + + callOnConcreteRequestType(*zk_request, preprocess_request); uncommitted_state.cleanup(getZXID()); } @@ -2750,11 +2745,10 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process } else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special { - auto storage_request = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); Coordination::ZooKeeperResponsePtr response = nullptr; { std::lock_guard lock(storage_mutex); - response = storage_request->process(*this, std::move(deltas)); + response = process(dynamic_cast(*zk_request), *this, std::move(deltas)); } response->xid = zk_request->xid; response->zxid = commit_zxid; @@ -2763,65 +2757,69 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process } else /// normal requests proccession { - auto request_processor = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); - Coordination::ZooKeeperResponsePtr response; - - if (is_local) + const auto process_request = [&] T>(const T & concrete_zk_request) { - chassert(zk_request->isReadRequest()); - if (check_acl && !request_processor->checkAuth(*this, session_id, true)) + Coordination::ZooKeeperResponsePtr response; + + if (is_local) { - response = zk_request->makeResponse(); - /// Original ZooKeeper always throws no auth, even when user provided some credentials - response->error = Coordination::Error::ZNOAUTH; + chassert(zk_request->isReadRequest()); + if (check_acl && !checkAuth(concrete_zk_request, *this, session_id, true)) + { + response = zk_request->makeResponse(); + /// Original ZooKeeper always throws no auth, even when user provided some credentials + response->error = Coordination::Error::ZNOAUTH; + } + else + { + std::shared_lock lock(storage_mutex); + response = processLocal(concrete_zk_request, *this, std::move(deltas)); + } } else { - std::shared_lock lock(storage_mutex); - response = request_processor->processLocal(*this, std::move(deltas)); + std::lock_guard lock(storage_mutex); + response = process(concrete_zk_request, *this, std::move(deltas)); } - } - else - { - std::lock_guard lock(storage_mutex); - response = request_processor->process(*this, std::move(deltas)); - } - /// Watches for this requests are added to the watches lists - if (zk_request->has_watch) - { + /// Watches for this requests are added to the watches lists + if (zk_request->has_watch) + { + if (response->error == Coordination::Error::ZOK) + { + static constexpr std::array list_requests{ + Coordination::OpNum::List, Coordination::OpNum::SimpleList, Coordination::OpNum::FilteredList}; + + auto & watches_type = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end() + ? list_watches + : watches; + + auto add_watch_result = watches_type[zk_request->getPath()].emplace(session_id); + if (add_watch_result.second) + sessions_and_watchers[session_id].emplace(zk_request->getPath()); + } + else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists) + { + auto add_watch_result = watches[zk_request->getPath()].emplace(session_id); + if (add_watch_result.second) + sessions_and_watchers[session_id].emplace(zk_request->getPath()); + } + } + + /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { - static constexpr std::array list_requests{ - Coordination::OpNum::List, Coordination::OpNum::SimpleList, Coordination::OpNum::FilteredList}; - - auto & watches_type = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end() - ? list_watches - : watches; - - auto add_watch_result = watches_type[zk_request->getPath()].emplace(session_id); - if (add_watch_result.second) - sessions_and_watchers[session_id].emplace(zk_request->getPath()); + auto watch_responses = processWatches(concrete_zk_request, watches, list_watches); + results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } - else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists) - { - auto add_watch_result = watches[zk_request->getPath()].emplace(session_id); - if (add_watch_result.second) - sessions_and_watchers[session_id].emplace(zk_request->getPath()); - } - } - /// If this requests processed successfully we need to check watches - if (response->error == Coordination::Error::ZOK) - { - auto watch_responses = request_processor->processWatches(watches, list_watches); - results.insert(results.end(), watch_responses.begin(), watch_responses.end()); - } + response->xid = zk_request->xid; + response->zxid = commit_zxid; - response->xid = zk_request->xid; - response->zxid = commit_zxid; + results.push_back(ResponseForSession{session_id, response}); + }; - results.push_back(ResponseForSession{session_id, response}); + callOnConcreteRequestType(*zk_request, process_request); } updateStats(); From e968984d170817a0d9922be862a82411ded21e00 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 14 Aug 2024 16:38:59 +0200 Subject: [PATCH 18/27] More changes --- src/Common/ZooKeeper/IKeeper.cpp | 6 - src/Common/ZooKeeper/IKeeper.h | 10 +- src/Common/ZooKeeper/TestKeeper.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 72 ++--- src/Common/ZooKeeper/ZooKeeperCommon.h | 14 +- src/Coordination/KeeperSnapshotManager.cpp | 21 +- src/Coordination/KeeperStorage.cpp | 291 +++++++++--------- src/Coordination/KeeperStorage.h | 12 +- src/Coordination/ZooKeeperDataReader.cpp | 4 +- src/Coordination/tests/gtest_coordination.cpp | 17 +- 10 files changed, 217 insertions(+), 232 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 7cca262baca..7bf0951817d 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -180,12 +180,6 @@ void SetACLRequest::addRootPath(const String & root_path) { Coordination::addRoo void GetACLRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void SyncRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } -void MultiRequest::addRootPath(const String & root_path) -{ - for (auto & request : requests) - request->addRootPath(root_path); -} - void CreateResponse::removeRootPath(const String & root_path) { Coordination::removeRootPath(path_created, root_path); } void WatchResponse::removeRootPath(const String & root_path) { Coordination::removeRootPath(path, root_path); } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index ce7489a33e5..c95dbd7d7bd 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -391,11 +391,17 @@ struct ReconfigResponse : virtual Response size_t bytesSize() const override { return value.size() + sizeof(stat); } }; +template struct MultiRequest : virtual Request { - Requests requests; + std::vector requests; + + void addRootPath(const String & root_path) override + { + for (auto & request : requests) + request->addRootPath(root_path); + } - void addRootPath(const String & root_path) override; String getPath() const override { return {}; } size_t bytesSize() const override diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 16ea412eb77..af1a3a54593 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -154,7 +154,7 @@ struct TestKeeperReconfigRequest final : ReconfigRequest, TestKeeperRequest std::pair process(TestKeeper::Container & container, int64_t zxid) const override; }; -struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest +struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest { explicit TestKeeperMultiRequest(const Requests & generic_requests) : TestKeeperMultiRequest(std::span(generic_requests)) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 6948e6a3830..8214390a10f 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -882,30 +882,30 @@ size_t ZooKeeperMultiResponse::sizeImpl() const return total_size + Coordination::size(op_num) + Coordination::size(done) + Coordination::size(error_read); } -ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperReconfigRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperSimpleListRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperReconfigRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSimpleListRequest::makeResponse() const { return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperCreateRequest::makeResponse() const { if (not_exists) - return setTime(std::make_shared()); - return setTime(std::make_shared()); + return std::make_shared(); + return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperCheckRequest::makeResponse() const { if (not_exists) - return setTime(std::make_shared()); + return std::make_shared(); - return setTime(std::make_shared()); + return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const @@ -916,11 +916,12 @@ ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const else response = std::make_shared(requests); - return setTime(std::move(response)); + return std::move(response); } -ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperSetACLRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperGetACLRequest::makeResponse() const { return setTime(std::make_shared()); } + +ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSetACLRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperGetACLRequest::makeResponse() const { return std::make_shared(); } void ZooKeeperSessionIDRequest::writeImpl(WriteBuffer & out) const { @@ -1122,40 +1123,6 @@ std::shared_ptr ZooKeeperRequest::read(ReadBuffer & in) return request; } -ZooKeeperRequest::~ZooKeeperRequest() -{ - if (!request_created_time_ns) - return; - UInt64 elapsed_ns = clock_gettime_ns() - request_created_time_ns; - constexpr UInt64 max_request_time_ns = 1000000000ULL; /// 1 sec - if (max_request_time_ns < elapsed_ns) - { - LOG_TEST(getLogger(__PRETTY_FUNCTION__), "Processing of request xid={} took {} ms", xid, elapsed_ns / 1000000UL); - } -} - -ZooKeeperResponsePtr ZooKeeperRequest::setTime(ZooKeeperResponsePtr response) const -{ - if (request_created_time_ns) - { - response->response_created_time_ns = clock_gettime_ns(); - } - return response; -} - -ZooKeeperResponse::~ZooKeeperResponse() -{ - if (!response_created_time_ns) - return; - UInt64 elapsed_ns = clock_gettime_ns() - response_created_time_ns; - constexpr UInt64 max_request_time_ns = 1000000000ULL; /// 1 sec - if (max_request_time_ns < elapsed_ns) - { - LOG_TEST(getLogger(__PRETTY_FUNCTION__), "Processing of response xid={} took {} ms", xid, elapsed_ns / 1000000UL); - } -} - - ZooKeeperRequestPtr ZooKeeperRequestFactory::get(OpNum op_num) const { auto it = op_num_to_request.find(op_num); @@ -1177,7 +1144,6 @@ void registerZooKeeperRequest(ZooKeeperRequestFactory & factory) factory.registerRequest(num, [] { auto res = std::make_shared(); - res->request_created_time_ns = clock_gettime_ns(); if constexpr (num == OpNum::MultiRead) res->operation_type = ZooKeeperMultiRequest::OperationType::Read; diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index c5096e1e641..75b2ddb0c04 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -23,11 +23,8 @@ struct ZooKeeperResponse : virtual Response { XID xid = 0; - UInt64 response_created_time_ns = 0; - ZooKeeperResponse() = default; ZooKeeperResponse(const ZooKeeperResponse &) = default; - ~ZooKeeperResponse() override; virtual void readImpl(ReadBuffer &) = 0; virtual void writeImpl(WriteBuffer &) const = 0; virtual size_t sizeImpl() const = 0; @@ -50,13 +47,11 @@ struct ZooKeeperRequest : virtual Request bool restored_from_zookeeper_log = false; - UInt64 request_created_time_ns = 0; UInt64 thread_id = 0; String query_id; ZooKeeperRequest() = default; ZooKeeperRequest(const ZooKeeperRequest &) = default; - ~ZooKeeperRequest() override; virtual OpNum getOpNum() const = 0; @@ -73,7 +68,6 @@ struct ZooKeeperRequest : virtual Request static std::shared_ptr read(ReadBuffer & in); virtual ZooKeeperResponsePtr makeResponse() const = 0; - ZooKeeperResponsePtr setTime(ZooKeeperResponsePtr response) const; virtual bool isReadRequest() const = 0; virtual void createLogElements(LogElements & elems) const; @@ -538,7 +532,7 @@ struct ZooKeeperGetACLResponse final : GetACLResponse, ZooKeeperResponse size_t bytesSize() const override { return GetACLResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } }; -struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest +struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest { OpNum getOpNum() const override; ZooKeeperMultiRequest() = default; @@ -571,12 +565,14 @@ private: struct ZooKeeperMultiResponse : MultiResponse, ZooKeeperResponse { - explicit ZooKeeperMultiResponse(const Requests & requests) + ZooKeeperMultiResponse() = default; + + explicit ZooKeeperMultiResponse(const std::vector & requests) { responses.reserve(requests.size()); for (const auto & request : requests) - responses.emplace_back(dynamic_cast(*request).makeResponse()); + responses.emplace_back(request->makeResponse()); } explicit ZooKeeperMultiResponse(const Responses & responses_) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 54ef0a53cef..69ebaeff9ca 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -72,7 +72,7 @@ namespace writeBinary(node.getData(), out); /// Serialize ACL - writeBinary(node.stats.acl_id, out); + writeBinary(node.acl_id, out); /// Write is_sequential for backwards compatibility if (version < SnapshotVersion::V6) writeBinary(false, out); @@ -109,7 +109,7 @@ namespace if (version >= SnapshotVersion::V1) { - readBinary(node.stats.acl_id, in); + readBinary(node.acl_id, in); } else if (version == SnapshotVersion::V0) { @@ -125,14 +125,14 @@ namespace readBinary(acl.id, in); acls.push_back(acl); } - node.stats.acl_id = acl_map.convertACLs(acls); + node.acl_id = acl_map.convertACLs(acls); } /// Some strange ACLID during deserialization from ZooKeeper - if (node.stats.acl_id == std::numeric_limits::max()) - node.stats.acl_id = 0; + if (node.acl_id == std::numeric_limits::max()) + node.acl_id = 0; - acl_map.addUsage(node.stats.acl_id); + acl_map.addUsage(node.acl_id); if (version < SnapshotVersion::V6) { @@ -455,9 +455,12 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult(new char[stats.data_size]); @@ -522,20 +524,26 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) chassert(!delta.path.empty()); UncommittedNode * uncommitted_node = nullptr; + std::string_view node_path; if (auto it = nodes.find(delta.path); it != nodes.end()) { uncommitted_node = &it->second; + node_path = it->first; } else { if (auto storage_node = tryGetNodeFromStorage(delta.path)) { auto [emplaced_it, _] = nodes.emplace(delta.path, UncommittedNode{.node = std::move(storage_node)}); + node_path = emplaced_it->first; + zxid_to_nodes[0].insert(node_path); uncommitted_node = &emplaced_it->second; } else { auto [emplaced_it, _] = nodes.emplace(delta.path, UncommittedNode{.node = nullptr}); + node_path = emplaced_it->first; + zxid_to_nodes[0].insert(node_path); uncommitted_node = &emplaced_it->second; } } @@ -545,10 +553,6 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) { auto & [node, acls, applied_zxids] = *uncommitted_node; - int64_t last_applied_zxid = 0; - if (!applied_zxids.empty()) - last_applied_zxid = applied_zxids.back(); - if constexpr (std::same_as) { chassert(!node); @@ -579,8 +583,8 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) acls = operation.new_acls; } - if (last_applied_zxid != delta.zxid) - last_applied_zxid = applied_zxids.emplace_back(delta.zxid); + applied_zxids.insert(delta.zxid); + zxid_to_nodes[delta.zxid].insert(node_path); }, delta.operation); } @@ -671,8 +675,8 @@ void KeeperStorage::UncommittedState::rollbackDelta(const Delta & del acls = operation.old_acls; } - if (applied_zxids.back() == delta.zxid) - applied_zxids.pop_back(); + applied_zxids.erase(delta.zxid); + zxid_to_nodes.erase(delta.zxid); }, delta.operation); } @@ -708,14 +712,21 @@ void KeeperStorage::UncommittedState::addDeltas(std::list new_ template void KeeperStorage::UncommittedState::cleanup(int64_t commit_zxid) { - for (auto it = nodes.begin(); it != nodes.end();) + for (const auto & [transaction_zxid, transaction_nodes] : zxid_to_nodes) { - auto & applied_zxids = it->second.applied_zxids; - std::erase_if(applied_zxids, [commit_zxid](auto current_zxid) { return current_zxid <= commit_zxid; }); - if (applied_zxids.empty()) - it = nodes.erase(it); - else - ++it; + if (transaction_zxid > commit_zxid) + break; + + std::cout << transaction_zxid << std::endl; + for (const auto node : transaction_nodes) + { + std::cout << node << std::endl; + auto node_it = nodes.find(node); + chassert(node_it != nodes.end()); + node_it->second.applied_zxids.erase(transaction_zxid); + if (node_it->second.applied_zxids.empty()) + nodes.erase(node_it); + } } for (auto it = session_and_auth.begin(); it != session_and_auth.end();) @@ -815,7 +826,10 @@ std::shared_ptr KeeperStorage::UncommittedS std::shared_ptr node = tryGetNodeFromStorage(path); if (node) - nodes.emplace(std::string{path}, UncommittedNode{node}); + { + auto [node_it, _] = nodes.emplace(std::string{path}, UncommittedNode{.node = node}); + zxid_to_nodes[0].insert(node_it->first); + } return node; } @@ -826,7 +840,7 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) { if (!node_it->second.acls.has_value()) - node_it->second.acls.emplace(storage.acl_map.convertNumber(node_it->second.node->stats.acl_id)); + node_it->second.acls.emplace(storage.acl_map.convertNumber(node_it->second.node->acl_id)); return *node_it->second.acls; } @@ -835,8 +849,9 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef if (node) { - auto [it, inserted] = nodes.emplace(std::string{path}, UncommittedNode{node}); - it->second.acls = storage.acl_map.convertNumber(node->stats.acl_id); + auto [it, inserted] = nodes.emplace(std::string{path}, UncommittedNode{.node = node}); + zxid_to_nodes[0].insert(it->first); + it->second.acls = storage.acl_map.convertNumber(node->acl_id); return *it->second.acls; } @@ -1000,12 +1015,12 @@ Coordination::Error KeeperStorage::commit(std::list deltas) if (operation.version != -1 && operation.version != node_it->value.stats.aversion) onStorageInconsistency(); - acl_map.removeUsage(node_it->value.stats.acl_id); + acl_map.removeUsage(node_it->value.acl_id); uint64_t acl_id = acl_map.convertACLs(operation.new_acls); acl_map.addUsage(acl_id); - container.updateValue(path, [acl_id](Node & node) { node.stats.acl_id = acl_id; }); + container.updateValue(path, [acl_id](Node & node) { node.acl_id = acl_id; }); return Coordination::Error::ZOK; } @@ -1063,7 +1078,7 @@ bool KeeperStorage::createNode( uint64_t acl_id = acl_map.convertACLs(node_acls); acl_map.addUsage(acl_id); - created_node.stats.acl_id = acl_id; + created_node.acl_id = acl_id; created_node.copyStats(stat); created_node.setData(data); @@ -1105,7 +1120,7 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers KeeperStorage::Node prev_node; prev_node.shallowCopy(node_it->value); - acl_map.removeUsage(node_it->value.stats.acl_id); + acl_map.removeUsage(node_it->value.acl_id); if constexpr (use_rocksdb) container.erase(path); @@ -1170,12 +1185,6 @@ auto callOnConcreteRequestType(const Coordination::ZooKeeperRequest & zk_request } } -template -auto callOnConcreteRequestType(const Coordination::Request & request, F function) -{ - return callOnConcreteRequestType(dynamic_cast(request), function); -} - namespace { @@ -1189,7 +1198,7 @@ Coordination::ACLs getNodeACLs(Storage & storage, StringRef path, bool is_local) if (node_it == storage.container.end()) return {}; - return storage.acl_map.convertNumber(node_it->value.stats.acl_id); + return storage.acl_map.convertNumber(node_it->value.acl_id); } return storage.uncommitted_state.getACLs(path); @@ -1306,8 +1315,8 @@ template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* storage */, std::list /* deltas */) { - auto response = zk_request.makeResponse(); - dynamic_cast(*response).path = zk_request.path; + auto response = std::make_shared(); + response->path = zk_request.path; return response; } @@ -1437,14 +1446,16 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperCreateRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); + std::shared_ptr response = zk_request.not_exists + ? std::make_shared() + : std::make_shared(); if (deltas.empty()) { - response.path_created = zk_request.getPath(); - response.error = Coordination::Error::ZOK; - return response_ptr; + chassert(zk_request.not_exists); + response->path_created = zk_request.getPath(); + response->error = Coordination::Error::ZOK; + return response; } std::string created_path; @@ -1459,13 +1470,13 @@ Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperCreateRe if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } - response.path_created = std::move(created_path); - response.error = Coordination::Error::ZOK; - return response_ptr; + response->path_created = std::move(created_path); + response->error = Coordination::Error::ZOK; + return response; } /// CREATE Request /// @@ -1500,25 +1511,26 @@ std::list preprocess( } template -Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); +Coordination::ZooKeeperResponsePtr +processImpl(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) +{ + auto response = std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } if (zk_request.path == Coordination::keeper_config_path) { - response.data = serializeClusterConfig( + response->data = serializeClusterConfig( storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); - response.error = Coordination::Error::ZOK; - return response_ptr; + response->error = Coordination::Error::ZOK; + return response; } auto & container = storage.container; @@ -1526,19 +1538,19 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetR if (node_it == container.end()) { if constexpr (local) - response.error = Coordination::Error::ZNONODE; + response->error = Coordination::Error::ZNONODE; else onStorageInconsistency(); } else { - node_it->value.setResponseStat(response.stat); + node_it->value.setResponseStat(response->stat); auto data = node_it->value.getData(); - response.data = std::string(data); - response.error = Coordination::Error::ZOK; + response->data = std::string(data); + response->error = Coordination::Error::ZOK; } - return response_ptr; + return response; } template @@ -1654,13 +1666,13 @@ std::list preprocess( } template -Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperRemoveRequest & zk_request, Storage & storage, std::list deltas) +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperRemoveRequest & /*zk_request*/, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); - response.error = storage.commit(std::move(deltas)); - return response_ptr; + response->error = storage.commit(std::move(deltas)); + return response; } /// REMOVE Request /// @@ -1691,16 +1703,17 @@ std::list preprocess( } template -Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); +Coordination::ZooKeeperResponsePtr +processImpl(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) +{ + auto response = std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } @@ -1709,17 +1722,17 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperExis if (node_it == container.end()) { if constexpr (local) - response.error = Coordination::Error::ZNONODE; + response->error = Coordination::Error::ZNONODE; else onStorageInconsistency(); } else { - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; } - return response_ptr; + return response; } template @@ -1808,23 +1821,22 @@ Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSetReque { auto & container = storage.container; - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } auto node_it = container.find(zk_request.path); if (node_it == container.end()) onStorageInconsistency(); - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; - return response_ptr; + return response; } /// SET Request /// @@ -1856,15 +1868,14 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } @@ -1874,7 +1885,7 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList if (node_it == container.end()) { if constexpr (local) - response.error = Coordination::Error::ZNONODE; + response->error = Coordination::Error::ZNONODE; else onStorageInconsistency(); } @@ -1892,7 +1903,7 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList return node_it->value.getChildren(); }; const auto & children = get_children(); - response.names.reserve(children.size()); + response->names.reserve(children.size()); const auto add_child = [&](const auto & child) { @@ -1929,17 +1940,17 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList if (add_child(child)) { if constexpr (Storage::use_rocksdb) - response.names.push_back(child.first); + response->names.push_back(child.first); else - response.names.push_back(child.toString()); + response->names.push_back(child.toString()); } } - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; } - return response_ptr; + return response; } template @@ -2002,22 +2013,23 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); + std::shared_ptr response = zk_request.not_exists + ? std::make_shared() + : std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } const auto on_error = [&]([[maybe_unused]] const auto error_code) { if constexpr (local) - response.error = error_code; + response->error = error_code; else onStorageInconsistency(); }; @@ -2030,7 +2042,7 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperChec if (node_it != container.end() && (zk_request.version == -1 || zk_request.version == node_it->value.stats.version)) on_error(Coordination::Error::ZNODEEXISTS); else - response.error = Coordination::Error::ZOK; + response->error = Coordination::Error::ZOK; } else { @@ -2039,10 +2051,10 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperChec else if (zk_request.version != -1 && zk_request.version != node_it->value.stats.version) on_error(Coordination::Error::ZBADVERSION); else - response.error = Coordination::Error::ZOK; + response->error = Coordination::Error::ZOK; } - return response_ptr; + return response; } template @@ -2151,8 +2163,13 @@ template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); + std::shared_ptr response; + if (zk_request.getOpNum() == Coordination::OpNum::Multi) + response = std::make_shared(); + else + response = std::make_shared(); + + response->responses.reserve(zk_request.requests.size()); const auto & subrequests = zk_request.requests; @@ -2162,41 +2179,40 @@ process(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storag { for (size_t i = 0; i < subrequests.size(); ++i) { - response.responses[i] = std::make_shared(); - response.responses[i]->error = failed_multi->error_codes[i]; + response->responses.push_back(std::make_shared()); + response->responses[i]->error = failed_multi->error_codes[i]; } - return response_ptr; + return response; } - for (size_t i = 0; i < subrequests.size(); ++i) + for (const auto & multi_subrequest : subrequests) { std::list subdeltas = getSubdeltas(deltas); - response.responses[i] = callOnConcreteRequestType( - *subrequests[i], [&](const auto & subrequest) { return process(subrequest, storage, std::move(subdeltas)); }); + response->responses.push_back(callOnConcreteRequestType( + *multi_subrequest, [&](const auto & subrequest) { return process(subrequest, storage, std::move(subdeltas)); })); } - response.error = Coordination::Error::ZOK; - return response_ptr; + response->error = Coordination::Error::ZOK; + return response; } template Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, std::list deltas) { ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); + response->responses.reserve(zk_request.requests.size()); - const auto & subrequests = zk_request.requests; - for (size_t i = 0; i < subrequests.size(); ++i) + for (const auto & multi_subrequest : zk_request.requests) { std::list subdeltas = getSubdeltas(deltas); - response.responses[i] = callOnConcreteRequestType( - *subrequests[i], [&](const auto & subrequest) { return processLocal(subrequest, storage, std::move(subdeltas)); }); + response->responses.push_back(callOnConcreteRequestType( + *multi_subrequest, [&](const auto & subrequest) { return processLocal(subrequest, storage, std::move(subdeltas)); })); } - response.error = Coordination::Error::ZOK; - return response_ptr; + response->error = Coordination::Error::ZOK; + return response; } template @@ -2227,8 +2243,6 @@ std::list preprocess( uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - if (zk_request.scheme != "digest" || std::count(zk_request.data.begin(), zk_request.data.end(), ':') != 1) return {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; @@ -2254,15 +2268,14 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr -process(const Coordination::ZooKeeperAuthRequest & zk_request, Storage & storage, std::list deltas) +process(const Coordination::ZooKeeperAuthRequest & /*zk_request*/, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - auth_response.error = result; + response->error = result; - return response_ptr; + return response; } /// AUTH Request /// @@ -2319,7 +2332,7 @@ std::list preprocess( std::list new_deltas{ {zk_request.path, zxid, - KeeperStorageBase::SetACLDelta{std::move(node_acls), uncommitted_state.getACLs(zk_request.path), zk_request.version}}, + KeeperStorageBase::SetACLDelta{uncommitted_state.getACLs(zk_request.path), std::move(node_acls), zk_request.version}}, {zk_request.path, zxid, std::move(update_stat_delta)}}; digest = storage.calculateNodesDigest(digest, new_deltas); @@ -2331,22 +2344,21 @@ template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSetACLRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } auto node_it = storage.container.find(zk_request.path); if (node_it == storage.container.end()) onStorageInconsistency(); - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; - return response_ptr; + return response; } /// SETACL Request /// @@ -2376,15 +2388,14 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } @@ -2393,17 +2404,17 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetA if (node_it == container.end()) { if constexpr (local) - response.error = Coordination::Error::ZNONODE; + response->error = Coordination::Error::ZNONODE; else onStorageInconsistency(); } else { - node_it->value.setResponseStat(response.stat); - response.acl = storage.acl_map.convertNumber(node_it->value.stats.acl_id); + node_it->value.setResponseStat(response->stat); + response->acl = storage.acl_map.convertNumber(node_it->value.acl_id); } - return response_ptr; + return response; } template @@ -2594,6 +2605,8 @@ void KeeperStorage::preprocessRequest( uncommitted_state.applyDeltas(new_deltas); uncommitted_state.addDeltas(std::move(new_deltas)); + + uncommitted_state.cleanup(getZXID()); }); if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special @@ -2645,8 +2658,6 @@ void KeeperStorage::preprocessRequest( }; callOnConcreteRequestType(*zk_request, preprocess_request); - - uncommitted_state.cleanup(getZXID()); } template @@ -2761,6 +2772,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process { Coordination::ZooKeeperResponsePtr response; + uncommitted_state.tryGetNodeFromStorage("/node_with_acl"); if (is_local) { chassert(zk_request->isReadRequest()); @@ -2781,6 +2793,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process std::lock_guard lock(storage_mutex); response = process(concrete_zk_request, *this, std::move(deltas)); } + uncommitted_state.tryGetNodeFromStorage("/node_with_acl"); /// Watches for this requests are added to the watches lists if (zk_request->has_watch) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 6326e1c76a9..1c3c77e140b 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -41,8 +42,6 @@ struct NodeStats uint32_t data_size{0}; - uint64_t acl_id = 0; /// 0 -- no ACL by default - /// void copyStats(const Coordination::Stat & stat); bool isEphemeral() const @@ -146,6 +145,8 @@ private: struct KeeperRocksNodeInfo { NodeStats stats; + uint64_t acl_id = 0; /// 0 -- no ACL by default + /// dummy interface for test void addChild(StringRef) {} auto getChildren() const @@ -181,6 +182,7 @@ struct KeeperRocksNode : public KeeperRocksNodeInfo void shallowCopy(const KeeperRocksNode & other) { stats = other.stats; + acl_id = other.acl_id; if (stats.data_size != 0) { data = std::unique_ptr(new char[stats.data_size]); @@ -221,6 +223,8 @@ struct KeeperMemNode std::unique_ptr data{nullptr}; mutable uint64_t cached_digest = 0; + uint64_t acl_id = 0; /// 0 -- no ACL by default + KeeperMemNode() = default; KeeperMemNode & operator=(const KeeperMemNode & other); @@ -505,11 +509,12 @@ public: std::unordered_map>>> session_and_auth; std::unordered_set closed_sessions; + using ZxidToNodes = std::map>; struct UncommittedNode { std::shared_ptr node{nullptr}; std::optional acls{}; - std::vector applied_zxids{}; + std::unordered_set applied_zxids{}; }; struct Hash @@ -536,6 +541,7 @@ public: }; mutable std::unordered_map nodes; + mutable ZxidToNodes zxid_to_nodes; mutable std::mutex deltas_mutex; std::list deltas TSA_GUARDED_BY(deltas_mutex); diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 08f8cc67aac..2ac0009ace1 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -105,7 +105,7 @@ int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log String data; Coordination::read(data, in); node.setData(data); - Coordination::read(node.stats.acl_id, in); + Coordination::read(node.acl_id, in); /// Deserialize stat Coordination::read(node.stats.czxid, in); @@ -136,7 +136,7 @@ int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log if (ephemeral_owner != 0) storage.ephemerals[ephemeral_owner].insert(path); - storage.acl_map.addUsage(node.stats.acl_id); + storage.acl_map.addUsage(node.acl_id); } Coordination::read(path, in); count++; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index d39031773cd..52cd68131e6 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1,6 +1,7 @@ #include #include +#include "base/defines.h" #include "config.h" #if USE_NURAFT @@ -1540,7 +1541,7 @@ void addNode(Storage & storage, const std::string & path, const std::string & da using Node = typename Storage::Node; Node node{}; node.setData(data); - node.setEphemeralOwner(ephemeral_owner); + node.stats.setEphemeralOwner(ephemeral_owner); storage.container.insertOrReplace(path, node); auto child_it = storage.container.find(path); auto child_path = DB::getBaseNodeName(child_it->key); @@ -1549,7 +1550,7 @@ void addNode(Storage & storage, const std::string & path, const std::string & da [&](auto & parent) { parent.addChild(child_path); - parent.increaseNumChildren(); + parent.stats.increaseNumChildren(); }); } @@ -1570,7 +1571,7 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) addNode(storage, "/hello1", "world", 1); addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; - storage.zxid = 2; + TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; storage.ephemerals[3] = {"/hello2"}; storage.ephemerals[1] = {"/hello1"}; storage.getSessionID(130); @@ -1601,7 +1602,7 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) EXPECT_EQ(restored_storage->container.getValue("/hello1").getData(), "world"); EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); - EXPECT_EQ(restored_storage->zxid, 2); + EXPECT_EQ(restored_storage->getZXID(), 2); EXPECT_EQ(restored_storage->ephemerals.size(), 2); EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); @@ -2534,7 +2535,7 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) addNode(storage, "/hello1", "world", 1); addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; - storage.zxid = 2; + TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; storage.ephemerals[3] = {"/hello2"}; storage.ephemerals[1] = {"/hello1"}; storage.getSessionID(130); @@ -2561,7 +2562,7 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) EXPECT_EQ(restored_storage->container.getValue("/hello1").getData(), "world"); EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); - EXPECT_EQ(restored_storage->zxid, 2); + EXPECT_EQ(restored_storage->getZXID(), 2); EXPECT_EQ(restored_storage->ephemerals.size(), 2); EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); @@ -2755,7 +2756,7 @@ TYPED_TEST(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.getZXID()); auto buf = manager.serializeSnapshotToBuffer(snapshot); @@ -3257,7 +3258,7 @@ TYPED_TEST(CoordinationTest, TestCheckNotExistsRequest) create_path("/test_node"); auto node_it = storage.container.find("/test_node"); ASSERT_NE(node_it, storage.container.end()); - auto node_version = node_it->value.version; + auto node_version = node_it->value.stats.version; { SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS"); From 79fc8d67ad9665d011cee06855d9bd4ce1065597 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 2 Sep 2024 11:44:21 +0200 Subject: [PATCH 19/27] More fixes --- src/Coordination/KeeperSnapshotManager.cpp | 19 +++++++--- src/Coordination/KeeperStorage.cpp | 42 +++++++++++----------- src/Coordination/KeeperStorage.h | 2 -- utils/keeper-bench/Runner.cpp | 1 + 4 files changed, 35 insertions(+), 29 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 81c72c1beb0..a5cb8218f1f 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -471,12 +471,21 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult::UncommittedState::addDeltas(std::list new_ template void KeeperStorage::UncommittedState::cleanup(int64_t commit_zxid) { - for (const auto & [transaction_zxid, transaction_nodes] : zxid_to_nodes) + for (auto it = zxid_to_nodes.begin(); it != zxid_to_nodes.end(); it = zxid_to_nodes.erase(it)) { + const auto & [transaction_zxid, transaction_nodes] = *it; + if (transaction_zxid > commit_zxid) break; - std::cout << transaction_zxid << std::endl; for (const auto node : transaction_nodes) { - std::cout << node << std::endl; auto node_it = nodes.find(node); - chassert(node_it != nodes.end()); + if (node_it == nodes.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing expected uncommitted node '{}'", node); + node_it->second.applied_zxids.erase(transaction_zxid); if (node_it->second.applied_zxids.empty()) nodes.erase(node_it); @@ -1295,7 +1297,6 @@ bool checkAuth(const T & /*zk_request*/, Storage & /*storage*/, int64_t /*sessio /// Default implementations /// /// HEARTBEAT Request /// - template Coordination::ZooKeeperResponsePtr process( const Coordination::ZooKeeperHeartbeatRequest & zk_request, @@ -1306,11 +1307,9 @@ Coordination::ZooKeeperResponsePtr process( response_ptr->error = storage.commit(std::move(deltas)); return response_ptr; } - /// HEARTBEAT Request /// /// SYNC Request /// - template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* storage */, std::list /* deltas */) @@ -1319,7 +1318,6 @@ process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* stor response->path = zk_request.path; return response; } - /// SYNC Request /// /// CREATE Request /// @@ -1793,18 +1791,19 @@ std::list preprocess( if (zk_request.version != -1 && zk_request.version != node->stats.version) return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; + new_deltas.emplace_back( + zk_request.path, + zxid, + KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = zk_request.data, .version = zk_request.version}); + KeeperStorageBase::UpdateNodeStatDelta node_delta(*node); node_delta.version = zk_request.version; auto & new_stats = node_delta.new_stats; new_stats.version++; new_stats.mzxid = zxid; new_stats.mtime = time; - + new_stats.data_size = static_cast(zk_request.data.size()); new_deltas.emplace_back(zk_request.path, zxid, std::move(node_delta)); - new_deltas.emplace_back( - zk_request.path, - zxid, - KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = zk_request.data, .version = zk_request.version}); auto parent_path = parentNodePath(zk_request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); @@ -1868,7 +1867,9 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) { - auto response = std::make_shared(); + std::shared_ptr response = zk_request.getOpNum() == Coordination::OpNum::SimpleList + ? std::make_shared() + : std::make_shared(); if constexpr (!local) { @@ -2215,17 +2216,16 @@ Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperMul return response; } -template KeeperStorageBase::ResponsesForSessions processWatches( - const Coordination::ZooKeeperMultiRequest & zk_request, typename Storage::Watches & watches, typename Storage::Watches & list_watches) + const Coordination::ZooKeeperMultiRequest & zk_request, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) { - typename Storage::ResponsesForSessions result; + KeeperStorageBase::ResponsesForSessions result; const auto & subrequests = zk_request.requests; for (const auto & generic_request : subrequests) { auto responses = callOnConcreteRequestType( - generic_request, [&](const auto & subrequest) { return processWatches(subrequest, watches, list_watches); }); + *generic_request, [&](const auto & subrequest) { return processWatches(subrequest, watches, list_watches); }); result.insert(result.end(), responses.begin(), responses.end()); } return result; @@ -2506,7 +2506,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, con 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); @@ -2772,7 +2772,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process { Coordination::ZooKeeperResponsePtr response; - uncommitted_state.tryGetNodeFromStorage("/node_with_acl"); if (is_local) { chassert(zk_request->isReadRequest()); @@ -2793,7 +2792,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process std::lock_guard lock(storage_mutex); response = process(concrete_zk_request, *this, std::move(deltas)); } - uncommitted_state.tryGetNodeFromStorage("/node_with_acl"); /// Watches for this requests are added to the watches lists if (zk_request->has_watch) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 1c3c77e140b..684f75c4bdf 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -357,8 +357,6 @@ public: NodeStats old_stats; NodeStats new_stats; - int32_t old_seq_num; - int32_t new_seq_num; int32_t version{-1}; }; diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 59761d827e1..0584e9e34c8 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -1114,6 +1114,7 @@ void Runner::runBenchmarkFromLog() else { request_from_log->connection = get_zookeeper_connection(request_from_log->session_id); + request_from_log->executor_id %= concurrency; push_request(std::move(*request_from_log)); } From 9633563fbdbf4aa391c89e12dd5802b5640d0ca5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 3 Sep 2024 10:07:01 +0200 Subject: [PATCH 20/27] Fix --- src/Coordination/KeeperSnapshotManager.cpp | 4 +- src/Coordination/KeeperStateMachine.cpp | 6 +- src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStorage.cpp | 210 +++++++++++------- src/Coordination/KeeperStorage.h | 15 +- src/Coordination/ZooKeeperDataReader.cpp | 2 +- src/Coordination/tests/gtest_coordination.cpp | 28 +-- utils/keeper-data-dumper/main.cpp | 14 +- 8 files changed, 162 insertions(+), 119 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a5cb8218f1f..dd643ee4c06 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -441,7 +441,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult::deserialize(SnapshotDeserializationResultpos(0); deserialization_result.cluster_config = ClusterConfig::deserialize(*buffer); } + + storage.updateStats(); } template diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 526a38e4f9e..b3c5465fee1 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -182,11 +182,11 @@ void assertDigest( } template -struct TSA_SCOPED_LOCKABLE LockGuardWithStats final +struct LockGuardWithStats final { using LockType = std::conditional_t, std::unique_lock>; LockType lock; - explicit LockGuardWithStats(SharedMutex & mutex) TSA_ACQUIRE(mutex) + explicit LockGuardWithStats(SharedMutex & mutex) { Stopwatch watch; LockType l(mutex); @@ -194,7 +194,7 @@ struct TSA_SCOPED_LOCKABLE LockGuardWithStats final lock = std::move(l); } - ~LockGuardWithStats() TSA_RELEASE() = default; + ~LockGuardWithStats() = default; }; } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 8de14ba301b..7e6830ec1ba 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -203,7 +203,7 @@ public: // 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 + Storage & getStorageUnsafe() { return *storage; } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 60872e2fdc0..0add7787ee2 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -112,6 +112,22 @@ bool fixupACL( return valid_found; } +void unregisterEphemeralPath(KeeperStorageBase::Ephemerals & ephemerals, int64_t session_id, const std::string & path, bool throw_if_missing) +{ + auto ephemerals_it = ephemerals.find(session_id); + if (ephemerals_it == ephemerals.end()) + { + if (throw_if_missing) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path {}", session_id, path); + + return; + } + + ephemerals_it->second.erase(path); + if (ephemerals_it->second.empty()) + ephemerals.erase(ephemerals_it); +} + KeeperStorageBase::ResponsesForSessions processWatchesImpl( const String & path, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches, Coordination::Event event_type) { @@ -489,6 +505,7 @@ void KeeperStorage::initializeSystemNodes() } } + updateStats(); initialized = true; } @@ -612,7 +629,7 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool if (is_local) { - std::shared_lock lock(storage.storage_mutex); + std::shared_lock lock(storage.auth_mutex); return check_auth(storage.session_and_auth[session_id]); } @@ -620,6 +637,7 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool if (closed_sessions.contains(session_id)) return false; + std::shared_lock lock(storage.auth_mutex); if (check_auth(storage.session_and_auth[session_id])) return true; @@ -631,7 +649,6 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool if (check_auth(auth_it->second)) return true; - std::lock_guard lock(storage.storage_mutex); return check_auth(storage.session_and_auth[session_id]); } @@ -777,8 +794,9 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb // we need to undo ephemeral mapping modifications // CreateNodeDelta added ephemeral for session id -> we need to remove it // RemoveNodeDelta removed ephemeral for session id -> we need to add it back - for (auto & delta : rollback_deltas) + for (auto delta_it = rollback_deltas.rbegin(); delta_it != rollback_deltas.rend(); ++delta_it) { + const auto & delta = *delta_it; if (!delta.path.empty()) { std::visit( @@ -787,14 +805,17 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb if constexpr (std::same_as) { if (operation.stat.ephemeralOwner != 0) - storage.unregisterEphemeralPath(operation.stat.ephemeralOwner, delta.path); + { + std::lock_guard lock(storage.ephemeral_mutex); + unregisterEphemeralPath(storage.uncommitted_state.ephemerals, operation.stat.ephemeralOwner, delta.path, /*throw_if_missing=*/false); + } } else if constexpr (std::same_as) { if (operation.stat.ephemeralOwner() != 0) { - std::lock_guard lock(storage.ephemerals_mutex); - storage.ephemerals[operation.stat.ephemeralOwner()].emplace(delta.path); + std::lock_guard lock(storage.ephemeral_mutex); + storage.uncommitted_state.ephemerals[operation.stat.ephemeralOwner()].emplace(delta.path); } } }, @@ -802,7 +823,7 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb rollbackDelta(delta); } - else if (auto * add_auth = std::get_if(&delta.operation)) + else if (const auto * add_auth = std::get_if(&delta.operation)) { auto & uncommitted_auth = session_and_auth[add_auth->session_id]; if (uncommitted_auth.back().second == add_auth->auth_id) @@ -812,7 +833,7 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb session_and_auth.erase(add_auth->session_id); } } - else if (auto * close_session = std::get_if(&delta.operation)) + else if (const auto * close_session = std::get_if(&delta.operation)) { closed_sessions.erase(close_session->session_id); } @@ -875,16 +896,16 @@ void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t se else auth_ptr = auth.second.get(); - func(*auth_ptr); + if (!auth_ptr->scheme.empty()) + func(*auth_ptr); } }; - { - std::lock_guard lock(storage.storage_mutex); - // for committed - if (auto auth_it = storage.session_and_auth.find(session_id); auth_it != storage.session_and_auth.end()) - call_for_each_auth(auth_it->second); - } + /// both committed and uncommitted need to be under the lock to avoid fetching the same AuthID from both committed and uncommitted state + std::shared_lock lock(storage.auth_mutex); + // for committed + if (auto auth_it = storage.session_and_auth.find(session_id); auth_it != storage.session_and_auth.end()) + call_for_each_auth(auth_it->second); // for uncommitted if (auto auth_it = session_and_auth.find(session_id); auth_it != session_and_auth.end()) @@ -1034,6 +1055,7 @@ Coordination::Error KeeperStorage::commit(std::list deltas) } else if constexpr (std::same_as) { + std::lock_guard auth_lock{auth_mutex}; session_and_auth[operation.session_id].emplace_back(std::move(*operation.auth_id)); return Coordination::Error::ZOK; } @@ -1104,6 +1126,15 @@ bool KeeperStorage::createNode( addDigest(map_key->getMapped()->value, map_key->getKey().toView()); } + + if (stat.ephemeralOwner != 0) + { + ++committed_ephemeral_nodes; + std::lock_guard lock(ephemeral_mutex); + committed_ephemerals[stat.ephemeralOwner].emplace(path); + unregisterEphemeralPath(uncommitted_state.ephemerals, stat.ephemeralOwner, path, /*throw_if_missing=*/false); + } + return true; }; @@ -1133,7 +1164,6 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers [child_basename = getBaseNodeName(node_it->key)](KeeperMemNode & parent) { parent.removeChild(child_basename); - chassert(parent.stats.numChildren() == static_cast(parent.getChildren().size())); } ); @@ -1141,6 +1171,14 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers removeDigest(prev_node, path); } + + if (prev_node.stats.ephemeralOwner() != 0) + { + --committed_ephemeral_nodes; + std::lock_guard lock(ephemeral_mutex); + unregisterEphemeralPath(committed_ephemerals, prev_node.stats.ephemeralOwner(), path, /*throw_if_missing=*/true); + } + return true; } @@ -1248,19 +1286,6 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int6 return false; } -template -void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) -{ - std::lock_guard ephemerals_lock(ephemerals_mutex); - auto ephemerals_it = ephemerals.find(session_id); - if (ephemerals_it == ephemerals.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path", session_id); - - ephemerals_it->second.erase(path); - if (ephemerals_it->second.empty()) - ephemerals.erase(ephemerals_it); -} - /// Default implementations /// template T, typename Storage> Coordination::ZooKeeperResponsePtr @@ -1398,8 +1423,8 @@ std::list preprocess( if (zk_request.is_ephemeral) { - std::lock_guard lock(storage.ephemerals_mutex); - storage.ephemerals[session_id].emplace(path_created); + std::lock_guard lock(storage.ephemeral_mutex); + storage.uncommitted_state.ephemerals[session_id].emplace(path_created); } int32_t parent_cversion = zk_request.parent_cversion; @@ -1436,7 +1461,6 @@ std::list preprocess( zxid, typename Storage::CreateNodeDelta{stat, std::move(node_acls), zk_request.data}); - digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } @@ -1606,24 +1630,29 @@ std::list preprocess( auto parent_path = parentNodePath(zk_request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); - KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); + std::optional update_parent_delta; + if (parent_node) + update_parent_delta.emplace(*parent_node); const auto add_parent_update_delta = [&] { + if (!update_parent_delta) + return; + new_deltas.emplace_back( std::string{parent_path}, zxid, - std::move(update_parent_delta) + std::move(*update_parent_delta) ); }; const auto update_parent_pzxid = [&]() { - if (!parent_node) + if (!update_parent_delta) return; - if (update_parent_delta.old_stats.pzxid < zxid) - update_parent_delta.new_stats.pzxid = zxid; + if (update_parent_delta->old_stats.pzxid < zxid) + update_parent_delta->new_stats.pzxid = zxid; }; auto node = storage.uncommitted_state.getNode(zk_request.path); @@ -1645,8 +1674,9 @@ std::list preprocess( if (zk_request.restored_from_zookeeper_log) update_parent_pzxid(); - ++update_parent_delta.new_stats.cversion; - update_parent_delta.new_stats.decreaseNumChildren(); + chassert(update_parent_delta); + ++update_parent_delta->new_stats.cversion; + update_parent_delta->new_stats.decreaseNumChildren(); add_parent_update_delta(); new_deltas.emplace_back( @@ -1656,7 +1686,11 @@ std::list preprocess( zk_request.version, node->stats, storage.uncommitted_state.getACLs(zk_request.path), std::string{node->getData()}}); if (node->stats.isEphemeral()) - storage.unregisterEphemeralPath(node->stats.ephemeralOwner(), zk_request.path); + { + std::lock_guard ephemeral_lock(storage.ephemeral_mutex); + /// try deleting the ephemeral node from the uncommitted state + unregisterEphemeralPath(storage.uncommitted_state.ephemerals, node->stats.ephemeralOwner(), zk_request.path, /*throw_if_missing=*/false); + } digest = storage.calculateNodesDigest(digest, new_deltas); @@ -1676,12 +1710,6 @@ process(const Coordination::ZooKeeperRemoveRequest & /*zk_request*/, Storage & s /// REMOVE Request /// /// EXISTS Request /// -template -bool checkAuth(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) -{ - return storage.checkACL(zk_request.getPath(), Coordination::ACL::Read, session_id, is_local); -} - template std::list preprocess( const Coordination::ZooKeeperExistsRequest & zk_request, @@ -2438,7 +2466,7 @@ void KeeperStorage::finalize() finalized = true; - ephemerals.clear(); + committed_ephemerals.clear(); watches.clear(); list_watches.clear(); @@ -2612,33 +2640,54 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { { - std::lock_guard lock(ephemerals_mutex); - auto session_ephemerals = ephemerals.find(session_id); - if (session_ephemerals != ephemerals.end()) + const auto process_ephemerals_for_session = [&](auto & current_ephemerals) { - for (const auto & ephemeral_path : session_ephemerals->second) + auto session_ephemerals = current_ephemerals.find(session_id); + if (session_ephemerals != current_ephemerals.end()) { - auto parent_node_path = parentNodePath(ephemeral_path); - auto parent_node = uncommitted_state.getNode(parent_node_path); - UpdateNodeStatDelta parent_update_delta(*parent_node); - ++parent_update_delta.new_stats.cversion; - parent_update_delta.new_stats.decreaseNumChildren(); - new_deltas.emplace_back - ( - parent_node_path.toString(), - new_last_zxid, - std::move(parent_update_delta) - ); + std::unordered_map parent_updates; + for (const auto & ephemeral_path : session_ephemerals->second) + { + auto node = uncommitted_state.getNode(ephemeral_path); - auto node = uncommitted_state.getNode(ephemeral_path); - new_deltas.emplace_back( - ephemeral_path, - transaction->zxid, - RemoveNodeDelta{.stat = node->stats, .acls = uncommitted_state.getACLs(ephemeral_path), .data = std::string{node->getData()}}); + /// maybe the node is deleted or recreated with different session_id in the uncommitted state + if (!node || node->stats.ephemeralOwner() != session_id) + continue; + + auto parent_node_path = parentNodePath(ephemeral_path).toView(); + + auto parent_update_it = parent_updates.find(parent_node_path); + if (parent_update_it == parent_updates.end()) + { + auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}); + std::tie(parent_update_it, std::ignore) = parent_updates.emplace(parent_node_path, *parent_node); + } + + auto & parent_update_delta = parent_update_it->second; + ++parent_update_delta.new_stats.cversion; + parent_update_delta.new_stats.decreaseNumChildren(); + + new_deltas.emplace_back( + ephemeral_path, + transaction->zxid, + RemoveNodeDelta{.stat = node->stats, .acls = uncommitted_state.getACLs(ephemeral_path), .data = std::string{node->getData()}}); + } + + for (auto & [parent_path, parent_update_delta] : parent_updates) + { + new_deltas.emplace_back + ( + std::string{parent_path}, + new_last_zxid, + std::move(parent_update_delta) + ); + } } - - ephemerals.erase(session_ephemerals); - } + }; + std::lock_guard ephemeral_lock(ephemeral_mutex); + process_ephemerals_for_session(committed_ephemerals); + process_ephemerals_for_session(uncommitted_state.ephemerals); + uncommitted_state.ephemerals.erase(session_id); } new_deltas.emplace_back(transaction->zxid, CloseSessionDelta{session_id}); @@ -2739,6 +2788,9 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process { std::lock_guard lock(storage_mutex); commit(std::move(deltas)); + } + { + std::lock_guard lock(auth_mutex); auto auth_it = session_and_auth.find(session_id); if (auth_it != session_and_auth.end()) session_and_auth.erase(auth_it); @@ -3087,8 +3139,8 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnStrin buf << "0x" << getHexUIntLowercase(session_id) << "\n"; } - buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCountLocked() << "):\n"; - for (const auto & [session_id, ephemeral_paths] : ephemerals) + buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCount() << "):\n"; + for (const auto & [session_id, ephemeral_paths] : committed_ephemerals) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; write_str_set(ephemeral_paths); @@ -3127,13 +3179,7 @@ uint64_t KeeperStorage::getTotalWatchesCount() const template uint64_t KeeperStorage::getSessionWithEphemeralNodesCount() const { - return getSessionWithEphemeralNodesCountLocked(); -} - -template -uint64_t KeeperStorage::getSessionWithEphemeralNodesCountLocked() const -{ - return ephemerals.size(); + return committed_ephemerals.size(); } template @@ -3145,11 +3191,7 @@ uint64_t KeeperStorage::getSessionsWithWatchesCount() const template uint64_t KeeperStorage::getTotalEphemeralNodesCount() const { - uint64_t ret = 0; - for (const auto & [session_id, nodes] : ephemerals) - ret += nodes.size(); - - return ret; + return committed_ephemeral_nodes; } template diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 684f75c4bdf..13f9631d2c3 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -474,9 +474,10 @@ public: int64_t session_id_counter{1}; + mutable SharedMutex auth_mutex; SessionAndAuth session_and_auth; - mutable std::shared_mutex storage_mutex; + mutable SharedMutex storage_mutex; /// Main hashtable with nodes. Contain all information about data. /// All other structures expect session_and_timeout can be restored from /// container. @@ -541,6 +542,8 @@ public: mutable std::unordered_map nodes; mutable ZxidToNodes zxid_to_nodes; + Ephemerals ephemerals; + mutable std::mutex deltas_mutex; std::list deltas TSA_GUARDED_BY(deltas_mutex); KeeperStorage & storage; @@ -570,11 +573,10 @@ public: bool checkACL(StringRef path, int32_t permissions, int64_t session_id, bool is_local); - void unregisterEphemeralPath(int64_t session_id, const std::string & path); - - mutable std::mutex ephemerals_mutex; + std::mutex ephemeral_mutex; /// Mapping session_id -> set of ephemeral nodes paths - Ephemerals ephemerals; + Ephemerals committed_ephemerals; + size_t committed_ephemeral_nodes{0}; /// Expiration queue for session, allows to get dead sessions at some point of time SessionExpiryQueue session_expiry_queue; @@ -687,7 +689,6 @@ public: /// Get all dead sessions std::vector getDeadSessions() const; - void updateStats(); const Stats & getStorageStats() const; @@ -713,8 +714,6 @@ public: void recalculateStats(); private: - uint64_t getSessionWithEphemeralNodesCountLocked() const; - void removeDigest(const Node & node, std::string_view path); void addDigest(const Node & node, std::string_view path); }; diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 2ac0009ace1..5783a803aa8 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -134,7 +134,7 @@ int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log storage.container.insertOrReplace(path, node); if (ephemeral_owner != 0) - storage.ephemerals[ephemeral_owner].insert(path); + storage.committed_ephemerals[ephemeral_owner].insert(path); storage.acl_map.addUsage(node.acl_id); } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 52cd68131e6..b2ba80e5624 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1572,8 +1572,8 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; - storage.ephemerals[3] = {"/hello2"}; - storage.ephemerals[1] = {"/hello1"}; + storage.committed_ephemerals[3] = {"/hello2"}; + storage.committed_ephemerals[1] = {"/hello1"}; storage.getSessionID(130); storage.getSessionID(130); @@ -1603,9 +1603,9 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); EXPECT_EQ(restored_storage->getZXID(), 2); - EXPECT_EQ(restored_storage->ephemerals.size(), 2); - EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); - EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); + EXPECT_EQ(restored_storage->committed_ephemerals.size(), 2); + EXPECT_EQ(restored_storage->committed_ephemerals[3].size(), 1); + EXPECT_EQ(restored_storage->committed_ephemerals[1].size(), 1); EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); } @@ -2028,7 +2028,7 @@ TYPED_TEST(CoordinationTest, TestEphemeralNodeRemove) state_machine->commit(1, entry_c->get_buf()); const auto & storage = state_machine->getStorageUnsafe(); - EXPECT_EQ(storage.ephemerals.size(), 1); + EXPECT_EQ(storage.committed_ephemerals.size(), 1); std::shared_ptr request_d = std::make_shared(); request_d->path = "/hello"; /// Delete from other session @@ -2036,7 +2036,7 @@ TYPED_TEST(CoordinationTest, TestEphemeralNodeRemove) state_machine->pre_commit(2, entry_d->get_buf()); state_machine->commit(2, entry_d->get_buf()); - EXPECT_EQ(storage.ephemerals.size(), 0); + EXPECT_EQ(storage.committed_ephemerals.size(), 0); } @@ -2536,8 +2536,8 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; - storage.ephemerals[3] = {"/hello2"}; - storage.ephemerals[1] = {"/hello1"}; + storage.committed_ephemerals[3] = {"/hello2"}; + storage.committed_ephemerals[1] = {"/hello1"}; storage.getSessionID(130); storage.getSessionID(130); @@ -2563,9 +2563,9 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); EXPECT_EQ(restored_storage->getZXID(), 2); - EXPECT_EQ(restored_storage->ephemerals.size(), 2); - EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); - EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); + EXPECT_EQ(restored_storage->committed_ephemerals.size(), 2); + EXPECT_EQ(restored_storage->committed_ephemerals[3].size(), 1); + EXPECT_EQ(restored_storage->committed_ephemerals[1].size(), 1); EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); } @@ -2750,8 +2750,8 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotEqual) storage.session_id_counter = 5; - storage.ephemerals[3] = {"/hello"}; - storage.ephemerals[1] = {"/hello/somepath"}; + storage.committed_ephemerals[3] = {"/hello"}; + storage.committed_ephemerals[1] = {"/hello/somepath"}; for (size_t j = 0; j < 3333; ++j) storage.getSessionID(130 * j); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 60170d67a2e..0291b6e9239 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -28,13 +28,13 @@ void dumpMachine(std::shared_ptr> ma keys.pop(); std::cout << key << "\n"; auto value = storage.container.getValue(key); - std::cout << "\tStat: {version: " << value.version << - ", mtime: " << value.mtime << - ", emphemeralOwner: " << value.ephemeralOwner() << - ", czxid: " << value.czxid << - ", mzxid: " << value.mzxid << - ", numChildren: " << value.numChildren() << - ", dataLength: " << value.data_size << + std::cout << "\tStat: {version: " << value.stats.version << + ", mtime: " << value.stats.mtime << + ", emphemeralOwner: " << value.stats.ephemeralOwner() << + ", czxid: " << value.stats.czxid << + ", mzxid: " << value.stats.mzxid << + ", numChildren: " << value.stats.numChildren() << + ", dataLength: " << value.stats.data_size << "}" << std::endl; std::cout << "\tData: " << storage.container.getValue(key).getData() << std::endl; From a3e233a537e3b7eb54d62cd0f567d65ce95275e4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 4 Sep 2024 15:19:56 +0200 Subject: [PATCH 21/27] Fix watch --- src/Coordination/KeeperStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 0add7787ee2..f0701326215 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1602,7 +1602,7 @@ KeeperStorageBase::ResponsesForSessions processWatches( KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) { - return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CREATED); + return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::DELETED); } template From e388f6f99ba477eb08e14a6f4ded9152b1d6368c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 6 Sep 2024 09:35:02 +0200 Subject: [PATCH 22/27] Remove useless log --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 8214390a10f..16da3fe0fe9 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -209,14 +209,7 @@ void ZooKeeperCreateRequest::writeImpl(WriteBuffer & out) const size_t ZooKeeperCreateRequest::sizeImpl() const { - /// See https://github.com/ClickHouse/clickhouse-private/issues/3029 - if (path.starts_with("/clickhouse/tables/") && path.find("/parts/") != std::string::npos) - { - LOG_TRACE(getLogger(__PRETTY_FUNCTION__), "Creating part at path {}", path); - } - int32_t flags = 0; - return Coordination::size(path) + Coordination::size(data) + Coordination::size(acls) + Coordination::size(flags); } From 03c7f3817b869196780c68d67a1959e7564f0566 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 6 Sep 2024 15:41:04 +0200 Subject: [PATCH 23/27] Correct lock order --- src/Coordination/KeeperStorage.cpp | 17 +++++++++++------ src/Coordination/KeeperStorage.h | 4 ++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index f0701326215..35134064191 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -521,9 +521,11 @@ template Overloaded(Ts...) -> Overloaded; template -std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const +std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path, bool should_lock_storage) const { - std::shared_lock lock(storage.storage_mutex); + std::shared_lock lock(storage.storage_mutex, std::defer_lock); + if (should_lock_storage) + lock.lock(); if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; @@ -841,12 +843,12 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb } template -std::shared_ptr KeeperStorage::UncommittedState::getNode(StringRef path) const +std::shared_ptr KeeperStorage::UncommittedState::getNode(StringRef path, bool should_lock_storage) const { if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.node; - std::shared_ptr node = tryGetNodeFromStorage(path); + std::shared_ptr node = tryGetNodeFromStorage(path, should_lock_storage); if (node) { @@ -2648,7 +2650,7 @@ void KeeperStorage::preprocessRequest( std::unordered_map parent_updates; for (const auto & ephemeral_path : session_ephemerals->second) { - auto node = uncommitted_state.getNode(ephemeral_path); + auto node = uncommitted_state.getNode(ephemeral_path, /*should_lock_storage=*/false); /// maybe the node is deleted or recreated with different session_id in the uncommitted state if (!node || node->stats.ephemeralOwner() != session_id) @@ -2659,7 +2661,7 @@ void KeeperStorage::preprocessRequest( auto parent_update_it = parent_updates.find(parent_node_path); if (parent_update_it == parent_updates.end()) { - auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}); + auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}, /*should_lock_storage=*/false); std::tie(parent_update_it, std::ignore) = parent_updates.emplace(parent_node_path, *parent_node); } @@ -2684,6 +2686,9 @@ void KeeperStorage::preprocessRequest( } } }; + + /// storage lock should always be taken before ephemeral lock + std::shared_lock storage_lock(storage_mutex); std::lock_guard ephemeral_lock(ephemeral_mutex); process_ephemerals_for_session(committed_ephemerals); process_ephemerals_for_session(uncommitted_state.ephemerals); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 13f9631d2c3..87cb7140f0e 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -492,7 +492,7 @@ public: void rollback(int64_t rollback_zxid); void rollback(std::list rollback_deltas); - std::shared_ptr getNode(StringRef path) const; + std::shared_ptr getNode(StringRef path, bool should_lock_storage = true) const; Coordination::ACLs getACLs(StringRef path) const; void applyDeltas(const std::list & new_deltas); @@ -503,7 +503,7 @@ public: void forEachAuthInSession(int64_t session_id, std::function func) const; - std::shared_ptr tryGetNodeFromStorage(StringRef path) const; + std::shared_ptr tryGetNodeFromStorage(StringRef path, bool should_lock_storage = true) const; std::unordered_map>>> session_and_auth; std::unordered_set closed_sessions; From 190339c4e680582844d12f956cc352f0e908fa8f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 7 Sep 2024 17:34:59 +0200 Subject: [PATCH 24/27] Fix snapshot sync --- src/Coordination/KeeperStorage.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 35134064191..bae60139c20 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -2609,8 +2609,15 @@ void KeeperStorage::preprocessRequest( else { if (last_zxid == new_last_zxid && digest && checkDigest(*digest, current_digest)) - // we found the preprocessed request with the same ZXID, we can skip it + { + auto & last_transaction = uncommitted_transactions.back(); + // we found the preprocessed request with the same ZXID, we can get log_idx and skip preprocessing it + chassert(last_transaction.zxid == new_last_zxid && log_idx != 0); + /// initially leader preprocessed without knowing the log idx + /// on the second call we have that information and can set the log idx for the correct transaction + last_transaction.log_idx = log_idx; return; + } if (new_last_zxid <= last_zxid) throw Exception( From 676b6238d05941156089861fe1e0f4d48f56dbd7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 17 Sep 2024 10:30:39 +0200 Subject: [PATCH 25/27] Update comments --- src/Coordination/KeeperStateMachine.h | 5 ++++- src/Coordination/KeeperStorage.cpp | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 7e6830ec1ba..03204036e7e 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -126,8 +126,11 @@ protected: /// Mutex for snapshots mutable std::mutex snapshots_lock; + /// Lock for the storage + /// Storage works in thread-safe way ONLY for preprocessing/processing + /// In any other case, unique storage lock needs to be taken mutable SharedMutex storage_mutex; - /// Lock for storage and responses_queue. It's important to process requests + /// Lock for processing and responses_queue. It's important to process requests /// and push them to the responses queue while holding this lock. Otherwise /// we can get strange cases when, for example client send read request with /// watch and after that receive watch response and only receive response diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index fe7bcf71160..20049fd23e8 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -772,6 +772,7 @@ void KeeperStorage::UncommittedState::cleanup(int64_t commit_zxid) template void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) { + // we can only rollback the last zxid (if there is any) std::list rollback_deltas; { std::lock_guard lock(deltas_mutex); From 8db3dddb3dc38dc4b75a1a62ece44e2c382482a6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 17 Sep 2024 14:15:33 +0200 Subject: [PATCH 26/27] Fix watches count and list request --- src/Coordination/KeeperStorage.cpp | 207 +++++++++++++++++------------ src/Coordination/KeeperStorage.h | 1 + 2 files changed, 121 insertions(+), 87 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 397e5db1f71..28bc1bf112b 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -130,7 +130,11 @@ void unregisterEphemeralPath(KeeperStorageBase::Ephemerals & ephemerals, int64_t } KeeperStorageBase::ResponsesForSessions processWatchesImpl( - const String & path, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches, Coordination::Event event_type) + const String & path, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers, + Coordination::Event event_type) { KeeperStorageBase::ResponsesForSessions result; auto watch_it = watches.find(path); @@ -143,7 +147,11 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl( watch_response->type = event_type; watch_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) + { + [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase(path); + chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_response}); + } watches.erase(watch_it); } @@ -179,7 +187,11 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl( watch_list_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) + { + [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase(path_to_check); + chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_list_response}); + } list_watches.erase(watch_it); } @@ -1322,8 +1334,12 @@ std::list preprocess( } template T> -KeeperStorageBase::ResponsesForSessions -processWatches(const T & /*zk_request*/, KeeperStorageBase::DeltaRange /*deltas*/, KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) +KeeperStorageBase::ResponsesForSessions processWatches( + const T & /*zk_request*/, + KeeperStorageBase::DeltaRange /*deltas*/, + KeeperStorageBase::Watches & /*watches*/, + KeeperStorageBase::Watches & /*list_watches*/, + KeeperStorageBase::SessionAndWatcher & /*sessions_and_watchers*/) { return {}; } @@ -1371,9 +1387,10 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperCreateRequest & zk_request, KeeperStorageBase::DeltaRange /*deltas*/, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { - return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CREATED); + return processWatchesImpl(zk_request.getPath(), watches, list_watches, sessions_and_watchers, Coordination::Event::CREATED); } template @@ -1616,9 +1633,10 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperRemoveRequest & zk_request, KeeperStorageBase::DeltaRange /*deltas*/, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { - return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::DELETED); + return processWatchesImpl(zk_request.getPath(), watches, list_watches, sessions_and_watchers, Coordination::Event::DELETED); } template @@ -1903,7 +1921,8 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperRemoveRecursiveRequest & /*zk_request*/, KeeperStorageBase::DeltaRange deltas, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { KeeperStorageBase::ResponsesForSessions responses; for (const auto & delta : deltas) @@ -1911,7 +1930,7 @@ KeeperStorageBase::ResponsesForSessions processWatches( const auto * remove_delta = std::get_if(&delta.operation); if (remove_delta) { - auto new_responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); + auto new_responses = processWatchesImpl(delta.path, watches, list_watches, sessions_and_watchers, Coordination::Event::DELETED); responses.insert(responses.end(), std::make_move_iterator(new_responses.begin()), std::make_move_iterator(new_responses.end())); } } @@ -2110,9 +2129,10 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperSetRequest & zk_request, KeeperStorageBase::DeltaRange /*deltas*/, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { - return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CHANGED); + return processWatchesImpl(zk_request.getPath(), watches, list_watches, sessions_and_watchers, Coordination::Event::CHANGED); } template @@ -2250,15 +2270,18 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList if (path_prefix.empty()) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty"); - const auto & get_children = [&]() + const auto get_children = [&]() { if constexpr (Storage::use_rocksdb) - return container.getChildren(zk_request.path); + return std::optional{container.getChildren(zk_request.path)}; else - return node_it->value.getChildren(); + return &node_it->value.getChildren(); }; - const auto & children = get_children(); - response->names.reserve(children.size()); + + const auto children = get_children(); + response->names.reserve(children->size()); + + chassert(static_cast(node_it->value.stats.numChildren()) == children->size()); const auto add_child = [&](const auto & child) { @@ -2290,7 +2313,7 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); }; - for (const auto & child : children) + for (const auto & child : *children) { if (add_child(child)) { @@ -2572,7 +2595,8 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperMultiRequest & zk_request, KeeperStorageBase::DeltaRange deltas, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { KeeperStorageBase::ResponsesForSessions result; @@ -2581,7 +2605,7 @@ KeeperStorageBase::ResponsesForSessions processWatches( { auto subdeltas = extractSubdeltas(deltas); auto responses = callOnConcreteRequestType( - *generic_request, [&](const auto & subrequest) { return processWatches(subrequest, subdeltas, watches, list_watches); }); + *generic_request, [&](const auto & subrequest) { return processWatches(subrequest, subdeltas, watches, list_watches, sessions_and_watchers); }); result.insert(result.end(), responses.begin(), responses.end()); } return result; @@ -2974,52 +2998,47 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { + std::unordered_map< + std::string, + UpdateNodeStatDelta, + StringHashForHeterogeneousLookup, + StringHashForHeterogeneousLookup::transparent_key_equal> + parent_updates; + const auto process_ephemerals_for_session = [&](auto & current_ephemerals) { - const auto process_ephemerals_for_session = [&](auto & current_ephemerals) + auto session_ephemerals = current_ephemerals.find(session_id); + if (session_ephemerals != current_ephemerals.end()) { - auto session_ephemerals = current_ephemerals.find(session_id); - if (session_ephemerals != current_ephemerals.end()) + for (const auto & ephemeral_path : session_ephemerals->second) { - std::unordered_map parent_updates; - for (const auto & ephemeral_path : session_ephemerals->second) + auto node = uncommitted_state.getNode(ephemeral_path, /*should_lock_storage=*/false); + + /// maybe the node is deleted or recreated with different session_id in the uncommitted state + if (!node || node->stats.ephemeralOwner() != session_id) + continue; + + auto parent_node_path = parentNodePath(ephemeral_path).toView(); + + auto parent_update_it = parent_updates.find(parent_node_path); + if (parent_update_it == parent_updates.end()) { - auto node = uncommitted_state.getNode(ephemeral_path, /*should_lock_storage=*/false); - - /// maybe the node is deleted or recreated with different session_id in the uncommitted state - if (!node || node->stats.ephemeralOwner() != session_id) - continue; - - auto parent_node_path = parentNodePath(ephemeral_path).toView(); - - auto parent_update_it = parent_updates.find(parent_node_path); - if (parent_update_it == parent_updates.end()) - { - auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}, /*should_lock_storage=*/false); - std::tie(parent_update_it, std::ignore) = parent_updates.emplace(parent_node_path, *parent_node); - } - - auto & parent_update_delta = parent_update_it->second; - ++parent_update_delta.new_stats.cversion; - parent_update_delta.new_stats.decreaseNumChildren(); - - new_deltas.emplace_back( - ephemeral_path, - transaction->zxid, - RemoveNodeDelta{.stat = node->stats, .acls = uncommitted_state.getACLs(ephemeral_path), .data = std::string{node->getData()}}); + auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}, /*should_lock_storage=*/false); + std::tie(parent_update_it, std::ignore) = parent_updates.emplace(parent_node_path, *parent_node); } - for (auto & [parent_path, parent_update_delta] : parent_updates) - { - new_deltas.emplace_back - ( - std::string{parent_path}, - new_last_zxid, - std::move(parent_update_delta) - ); - } + auto & parent_update_delta = parent_update_it->second; + ++parent_update_delta.new_stats.cversion; + parent_update_delta.new_stats.decreaseNumChildren(); + + new_deltas.emplace_back( + ephemeral_path, + transaction->zxid, + RemoveNodeDelta{.stat = node->stats, .acls = uncommitted_state.getACLs(ephemeral_path), .data = std::string{node->getData()}}); } - }; + } + }; + { /// storage lock should always be taken before ephemeral lock std::shared_lock storage_lock(storage_mutex); std::lock_guard ephemeral_lock(ephemeral_mutex); @@ -3028,6 +3047,16 @@ void KeeperStorage::preprocessRequest( uncommitted_state.ephemerals.erase(session_id); } + for (auto & [parent_path, parent_update_delta] : parent_updates) + { + new_deltas.emplace_back + ( + parent_path, + new_last_zxid, + std::move(parent_update_delta) + ); + } + new_deltas.emplace_back(transaction->zxid, CloseSessionDelta{session_id}); new_digest = calculateNodesDigest(new_digest, new_deltas); return; @@ -3132,7 +3161,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process { if (std::holds_alternative(delta.operation)) { - auto responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); + auto responses = processWatchesImpl(delta.path, watches, list_watches, sessions_and_watchers, Coordination::Event::DELETED); results.insert(results.end(), responses.begin(), responses.end()); } } @@ -3211,20 +3240,27 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process auto add_watch_result = watches_type[zk_request->getPath()].emplace(session_id); if (add_watch_result.second) + { + ++total_watches_count; sessions_and_watchers[session_id].emplace(zk_request->getPath()); + } } else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists) { auto add_watch_result = watches[zk_request->getPath()].emplace(session_id); if (add_watch_result.second) + { + ++total_watches_count; sessions_and_watchers[session_id].emplace(zk_request->getPath()); + } } } /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { - auto watch_responses = processWatches(concrete_zk_request, deltas_range, watches, list_watches); + auto watch_responses = processWatches(concrete_zk_request, deltas_range, watches, list_watches, sessions_and_watchers); + total_watches_count -= watch_responses.size(); results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } @@ -3409,33 +3445,34 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) { /// Clear all watches for this session auto watches_it = sessions_and_watchers.find(session_id); - if (watches_it != sessions_and_watchers.end()) - { - for (const auto & watch_path : watches_it->second) - { - /// Maybe it's a normal watch - auto watch = watches.find(watch_path); - if (watch != watches.end()) - { - auto & watches_for_path = watch->second; - watches_for_path.erase(session_id); - if (watches_for_path.empty()) - watches.erase(watch); - } + if (watches_it == sessions_and_watchers.end()) + return; - /// Maybe it's a list watch - auto list_watch = list_watches.find(watch_path); - if (list_watch != list_watches.end()) - { - auto & list_watches_for_path = list_watch->second; - list_watches_for_path.erase(session_id); - if (list_watches_for_path.empty()) - list_watches.erase(list_watch); - } + for (const auto & watch_path : watches_it->second) + { + /// Maybe it's a normal watch + auto watch = watches.find(watch_path); + if (watch != watches.end()) + { + auto & watches_for_path = watch->second; + watches_for_path.erase(session_id); + if (watches_for_path.empty()) + watches.erase(watch); } - sessions_and_watchers.erase(watches_it); + /// Maybe it's a list watch + auto list_watch = list_watches.find(watch_path); + if (list_watch != list_watches.end()) + { + auto & list_watches_for_path = list_watch->second; + list_watches_for_path.erase(session_id); + if (list_watches_for_path.empty()) + list_watches.erase(list_watch); + } } + + total_watches_count -= watches_it->second.size(); + sessions_and_watchers.erase(watches_it); } template @@ -3521,11 +3558,7 @@ const KeeperStorageBase::Stats & KeeperStorage::getStorageStats() con template uint64_t KeeperStorage::getTotalWatchesCount() const { - uint64_t ret = 0; - for (const auto & [session, paths] : sessions_and_watchers) - ret += paths.size(); - - return ret; + return total_watches_count; } template diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 9509ac79f2b..993ed626f96 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -636,6 +636,7 @@ public: /// Mapping session_id -> set of watched nodes paths SessionAndWatcher sessions_and_watchers; + size_t total_watches_count = 0; /// Currently active watches (node_path -> subscribed sessions) Watches watches; From 31066538527d36b5e78c1226445f51a252a61e29 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Sep 2024 09:52:19 +0200 Subject: [PATCH 27/27] Fix watches --- src/Coordination/KeeperStorage.cpp | 104 +++++++++++++++++------------ src/Coordination/KeeperStorage.h | 28 +++++++- 2 files changed, 87 insertions(+), 45 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 28bc1bf112b..493841fc50e 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -148,7 +148,8 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl( watch_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) { - [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase(path); + [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase( + KeeperStorageBase::WatchInfo{.path = path, .is_list_watch = false}); chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_response}); } @@ -188,7 +189,8 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl( watch_list_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) { - [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase(path_to_check); + [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase( + KeeperStorageBase::WatchInfo{.path = path_to_check, .is_list_watch = true}); chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_list_response}); } @@ -828,18 +830,12 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb if constexpr (std::same_as) { if (operation.stat.ephemeralOwner != 0) - { - std::lock_guard lock(storage.ephemeral_mutex); unregisterEphemeralPath(storage.uncommitted_state.ephemerals, operation.stat.ephemeralOwner, delta.path, /*throw_if_missing=*/false); - } } else if constexpr (std::same_as) { if (operation.stat.ephemeralOwner() != 0) - { - std::lock_guard lock(storage.ephemeral_mutex); storage.uncommitted_state.ephemerals[operation.stat.ephemeralOwner()].emplace(delta.path); - } } }, delta.operation); @@ -1159,7 +1155,6 @@ bool KeeperStorage::createNode( ++committed_ephemeral_nodes; std::lock_guard lock(ephemeral_mutex); committed_ephemerals[stat.ephemeralOwner].emplace(path); - unregisterEphemeralPath(uncommitted_state.ephemerals, stat.ephemeralOwner, path, /*throw_if_missing=*/false); } return true; @@ -1454,10 +1449,7 @@ std::list preprocess( return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; if (zk_request.is_ephemeral) - { - std::lock_guard lock(storage.ephemeral_mutex); storage.uncommitted_state.ephemerals[session_id].emplace(path_created); - } int32_t parent_cversion = zk_request.parent_cversion; @@ -1721,7 +1713,6 @@ std::list preprocess( if (node->stats.isEphemeral()) { - std::lock_guard ephemeral_lock(storage.ephemeral_mutex); /// try deleting the ephemeral node from the uncommitted state unregisterEphemeralPath(storage.uncommitted_state.ephemerals, node->stats.ephemeralOwner(), zk_request.path, /*throw_if_missing=*/false); } @@ -2027,7 +2018,6 @@ std::list preprocess( const auto * remove_delta = std::get_if(&delta.operation); if (remove_delta && remove_delta->stat.ephemeralOwner()) { - std::lock_guard lock(storage.ephemeral_mutex); unregisterEphemeralPath( storage.uncommitted_state.ephemerals, remove_delta->stat.ephemeralOwner(), delta.path, /*throw_if_missing=*/false); } @@ -2281,7 +2271,17 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList const auto children = get_children(); response->names.reserve(children->size()); - chassert(static_cast(node_it->value.stats.numChildren()) == children->size()); +#ifdef DEBUG_OR_SANITIZER_BUILD + if (!zk_request.path.starts_with(keeper_system_path) && static_cast(node_it->value.stats.numChildren()) != children->size()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Difference between numChildren ({}) and actual children size ({}) for '{}'", + node_it->value.stats.numChildren(), + children->size(), + zk_request.path); + } +#endif const auto add_child = [&](const auto & child) { @@ -3004,13 +3004,25 @@ void KeeperStorage::preprocessRequest( StringHashForHeterogeneousLookup, StringHashForHeterogeneousLookup::transparent_key_equal> parent_updates; - const auto process_ephemerals_for_session = [&](auto & current_ephemerals) + + const auto process_ephemerals_for_session + = [&](const auto & current_ephemerals, auto & processed_ephemeral_nodes, bool check_processed_nodes) { auto session_ephemerals = current_ephemerals.find(session_id); if (session_ephemerals != current_ephemerals.end()) { for (const auto & ephemeral_path : session_ephemerals->second) { + if (check_processed_nodes) + { + if (processed_ephemeral_nodes.contains(ephemeral_path)) + continue; + } + else + { + processed_ephemeral_nodes.insert(ephemeral_path); + } + auto node = uncommitted_state.getNode(ephemeral_path, /*should_lock_storage=*/false); /// maybe the node is deleted or recreated with different session_id in the uncommitted state @@ -3041,12 +3053,16 @@ void KeeperStorage::preprocessRequest( { /// storage lock should always be taken before ephemeral lock std::shared_lock storage_lock(storage_mutex); + + std::unordered_set processed_ephemeral_nodes; + process_ephemerals_for_session(uncommitted_state.ephemerals, processed_ephemeral_nodes, /*check_processed_nodes=*/false); + std::lock_guard ephemeral_lock(ephemeral_mutex); - process_ephemerals_for_session(committed_ephemerals); - process_ephemerals_for_session(uncommitted_state.ephemerals); - uncommitted_state.ephemerals.erase(session_id); + process_ephemerals_for_session(committed_ephemerals, processed_ephemeral_nodes, /*check_processed_nodes=*/true); } + uncommitted_state.ephemerals.erase(session_id); + for (auto & [parent_path, parent_update_delta] : parent_updates) { new_deltas.emplace_back @@ -3234,24 +3250,27 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process static constexpr std::array list_requests{ Coordination::OpNum::List, Coordination::OpNum::SimpleList, Coordination::OpNum::FilteredList}; - auto & watches_type = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end() - ? list_watches - : watches; + auto is_list_watch + = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end(); - auto add_watch_result = watches_type[zk_request->getPath()].emplace(session_id); - if (add_watch_result.second) + auto & watches_type = is_list_watch ? list_watches : watches; + + auto [watch_it, path_inserted] = watches_type.try_emplace(zk_request->getPath()); + auto [path_it, session_inserted] = watch_it->second.emplace(session_id); + if (session_inserted) { ++total_watches_count; - sessions_and_watchers[session_id].emplace(zk_request->getPath()); + sessions_and_watchers[session_id].emplace(WatchInfo{.path = watch_it->first, .is_list_watch = is_list_watch}); } } else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists) { - auto add_watch_result = watches[zk_request->getPath()].emplace(session_id); - if (add_watch_result.second) + auto [watch_it, path_inserted] = watches.try_emplace(zk_request->getPath()); + auto session_insert_info = watch_it->second.emplace(session_id); + if (session_insert_info.second) { ++total_watches_count; - sessions_and_watchers[session_id].emplace(zk_request->getPath()); + sessions_and_watchers[session_id].emplace(WatchInfo{.path = watch_it->first, .is_list_watch = false}); } } } @@ -3448,27 +3467,26 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) if (watches_it == sessions_and_watchers.end()) return; - for (const auto & watch_path : watches_it->second) + for (const auto [watch_path, is_list_watch] : watches_it->second) { - /// Maybe it's a normal watch - auto watch = watches.find(watch_path); - if (watch != watches.end()) - { - auto & watches_for_path = watch->second; - watches_for_path.erase(session_id); - if (watches_for_path.empty()) - watches.erase(watch); - } - - /// Maybe it's a list watch - auto list_watch = list_watches.find(watch_path); - if (list_watch != list_watches.end()) + if (is_list_watch) { + auto list_watch = list_watches.find(watch_path); + chassert(list_watch != list_watches.end()); auto & list_watches_for_path = list_watch->second; list_watches_for_path.erase(session_id); if (list_watches_for_path.empty()) list_watches.erase(list_watch); } + else + { + auto watch = watches.find(watch_path); + chassert(watch != watches.end()); + auto & watches_for_path = watch->second; + watches_for_path.erase(session_id); + if (watches_for_path.empty()) + watches.erase(watch); + } } total_watches_count -= watches_it->second.size(); @@ -3481,7 +3499,7 @@ void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const for (const auto & [session_id, watches_paths] : sessions_and_watchers) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; - for (const String & path : watches_paths) + for (const auto [path, is_list_watch] : watches_paths) buf << "\t" << path << "\n"; } } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 993ed626f96..03f2084df7c 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -6,6 +6,7 @@ #include #include #include +#include "Common/StringHashForHeterogeneousLookup.h" #include #include @@ -314,13 +315,36 @@ public: }; using Ephemerals = std::unordered_map>; - using SessionAndWatcher = std::unordered_map>; + struct WatchInfo + { + std::string_view path; + bool is_list_watch; + + bool operator==(const WatchInfo &) const = default; + }; + + struct WatchInfoHash + { + auto operator()(WatchInfo info) const + { + SipHash hash; + hash.update(info.path); + hash.update(info.is_list_watch); + return hash.get64(); + } + }; + + 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; + using Watches = std::unordered_map< + String /* path, relative of root_path */, + SessionIDs, + StringHashForHeterogeneousLookup, + StringHashForHeterogeneousLookup::transparent_key_equal>; // Applying ZooKeeper request to storage consists of two steps: // - preprocessing which, instead of applying the changes directly to storage,