diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 7cca262baca..34c9d94fca5 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -171,6 +171,7 @@ bool isUserError(Error zk_return_code) void CreateRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void RemoveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } +void RemoveRecursiveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void ExistsRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void GetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void SetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index f887e67bd92..a0d6ae54f56 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -238,16 +238,30 @@ struct RemoveRequest : virtual Request String path; int32_t version = -1; + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } + + size_t bytesSize() const override { return path.size() + sizeof(version); } +}; + +struct RemoveResponse : virtual Response +{ +}; + +struct RemoveRecursiveRequest : virtual Request +{ + String path; + /// strict limit for number of deleted nodes uint32_t remove_nodes_limit = 1; void addRootPath(const String & root_path) override; String getPath() const override { return path; } - size_t bytesSize() const override { return path.size() + sizeof(version) + sizeof(remove_nodes_limit); } + size_t bytesSize() const override { return path.size() + sizeof(remove_nodes_limit); } }; -struct RemoveResponse : virtual Response +struct RemoveRecursiveResponse : virtual Response { }; @@ -433,6 +447,7 @@ struct ErrorResponse : virtual Response using CreateCallback = std::function; using RemoveCallback = std::function; +using RemoveRecursiveCallback = std::function; using ExistsCallback = std::function; using GetCallback = std::function; using SetCallback = std::function; @@ -588,9 +603,13 @@ public: virtual void remove( const String & path, int32_t version, - uint32_t remove_nodes_limit, RemoveCallback callback) = 0; + virtual void removeRecursive( + const String & path, + uint32_t remove_nodes_limit, + RemoveRecursiveCallback callback) = 0; + virtual void exists( const String & path, ExistsCallback callback, diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 65a575549c4..1ac085d6050 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -759,7 +759,6 @@ void TestKeeper::create( void TestKeeper::remove( const String & path, int32_t version, - [[maybe_unused]] uint32_t remove_nodes_limit, // TODO(michicosun): enable RemoveCallback callback) { TestKeeperRemoveRequest request; @@ -772,6 +771,14 @@ void TestKeeper::remove( pushRequest(std::move(request_info)); } +void TestKeeper::removeRecursive( + [[maybe_unused]] const String & path, + [[maybe_unused]] uint32_t remove_nodes_limit, + [[maybe_unused]] RemoveRecursiveCallback callback) +{ + /// TODO(michicosun) implement +} + void TestKeeper::exists( const String & path, ExistsCallback callback, diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 14b37448a20..c32f0064dec 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -56,9 +56,13 @@ public: void remove( const String & path, int32_t version, - uint32_t remove_nodes_limit, RemoveCallback callback) override; + void removeRecursive( + const String & path, + uint32_t remove_nodes_limit, + RemoveRecursiveCallback callback) override; + void exists( const String & path, ExistsCallback callback, diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index eb4f5e0725a..93e833e87c8 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -981,16 +981,16 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab void ZooKeeper::removeRecursive(const std::string & path) // TODO(michicosun) rewrite { - auto promise = std::make_shared>(); + auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise](const Coordination::RemoveResponse & response) mutable + auto callback = [promise](const Coordination::RemoveRecursiveResponse & response) mutable { promise->set_value(response); }; - impl->remove(path, -1, /*remove_nodes_limit=*/ 100, std::move(callback)); - + impl->removeRecursive(path, 100, std::move(callback)); + if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) { impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::RemoveRecursive, path)); @@ -1385,7 +1385,7 @@ std::future ZooKeeper::asyncRemove(const std::stri promise->set_value(response); }; - impl->remove(path, version, /*remove_nodes_limit=*/ 1, std::move(callback)); + impl->remove(path, version, std::move(callback)); return future; } @@ -1408,7 +1408,7 @@ std::future ZooKeeper::asyncTryRemove(const std::s promise->set_value(response); }; - impl->remove(path, version, /*remove_nodes_limit=*/ 1, std::move(callback)); + impl->remove(path, version, std::move(callback)); return future; } @@ -1422,7 +1422,7 @@ std::future ZooKeeper::asyncTryRemoveNoThrow(const promise->set_value(response); }; - impl->remove(path, version, /*remove_nodes_limit=*/ 1, std::move(callback)); + impl->remove(path, version, std::move(callback)); return future; } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index fb37aa6ac22..3f9225e84dd 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -232,24 +232,24 @@ void ZooKeeperRemoveRequest::readImpl(ReadBuffer & in) Coordination::read(version, in); } -void ZooKeeperRemoveRecursiveRequest::writeImpl(WriteBuffer & out) const +void ZooKeeperRemoveRecursiveRequest::writeImpl(WriteBuffer & out) const { - ZooKeeperRemoveRequest::writeImpl(out); + Coordination::write(path, out); Coordination::write(remove_nodes_limit, out); } -void ZooKeeperRemoveRecursiveRequest::readImpl(ReadBuffer & in) +void ZooKeeperRemoveRecursiveRequest::readImpl(ReadBuffer & in) { - ZooKeeperRemoveRequest::readImpl(in); + Coordination::read(path, in); Coordination::read(remove_nodes_limit, in); } -std::string ZooKeeperRemoveRecursiveRequest::toStringImpl(bool short_format) const +std::string ZooKeeperRemoveRecursiveRequest::toStringImpl(bool /*short_format*/) const { return fmt::format( - "{}\n" + "path = {}\n" "remove_nodes_limit = {}", - ZooKeeperRemoveRequest::toStringImpl(short_format), + path, remove_nodes_limit); } @@ -531,7 +531,7 @@ ZooKeeperMultiRequest::ZooKeeperMultiRequest(std::span(*concrete_request_remove)); } - else if (const auto * concrete_request_remove_recursive = dynamic_cast(generic_request.get())) + else if (const auto * concrete_request_remove_recursive = dynamic_cast(generic_request.get())) { checkOperationType(Write); requests.push_back(std::make_shared(*concrete_request_remove_recursive)); diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index ef17d069b4c..66c075b277b 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -258,7 +258,7 @@ struct ZooKeeperCreateIfNotExistsResponse : ZooKeeperCreateResponse using ZooKeeperCreateResponse::ZooKeeperCreateResponse; }; -struct ZooKeeperRemoveRequest : RemoveRequest, ZooKeeperRequest +struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest { ZooKeeperRemoveRequest() = default; explicit ZooKeeperRemoveRequest(const RemoveRequest & base) : RemoveRequest(base) {} @@ -276,17 +276,7 @@ struct ZooKeeperRemoveRequest : RemoveRequest, ZooKeeperRequest void createLogElements(LogElements & elems) const override; }; -struct ZooKeeperRemoveRecursiveRequest : ZooKeeperRemoveRequest -{ - OpNum getOpNum() const override { return OpNum::RemoveRecursive; } - void writeImpl(WriteBuffer & out) const override; - void readImpl(ReadBuffer & in) override; - std::string toStringImpl(bool short_format) const override; - - ZooKeeperResponsePtr makeResponse() const override; -}; - -struct ZooKeeperRemoveResponse : RemoveResponse, ZooKeeperResponse +struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} @@ -295,9 +285,29 @@ struct ZooKeeperRemoveResponse : RemoveResponse, ZooKeeperResponse size_t bytesSize() const override { return RemoveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } }; -struct ZooKeeperRemoveRecursiveResponse : ZooKeeperRemoveResponse +struct ZooKeeperRemoveRecursiveRequest final : RemoveRecursiveRequest, ZooKeeperRequest { + ZooKeeperRemoveRecursiveRequest() = default; + explicit ZooKeeperRemoveRecursiveRequest(const RemoveRecursiveRequest & base) : RemoveRecursiveRequest(base) {} + OpNum getOpNum() const override { return OpNum::RemoveRecursive; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + std::string toStringImpl(bool short_format) const override; + + ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } + + size_t bytesSize() const override { return RemoveRecursiveRequest::bytesSize() + sizeof(xid); } +}; + +struct ZooKeeperRemoveRecursiveResponse : RemoveRecursiveResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer &) override {} + void writeImpl(WriteBuffer &) const override {} + OpNum getOpNum() const override { return OpNum::RemoveRecursive; } + + size_t bytesSize() const override { return RemoveRecursiveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } }; struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 5c4de98d5fc..a6dd9738e17 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1333,33 +1333,39 @@ void ZooKeeper::create( void ZooKeeper::remove( const String & path, int32_t version, - uint32_t remove_nodes_limit, RemoveCallback callback) { - std::shared_ptr request = nullptr; - - if (!isFeatureEnabled(KeeperFeatureFlag::REMOVE_RECURSIVE)) - { - if (remove_nodes_limit > 1) - throw Exception::fromMessage(Error::ZBADARGUMENTS, "RemoveRecursive request type cannot be used because it's not supported by the server"); - - request = std::make_shared(); - } - else - request = std::make_shared(); - - request->path = path; - request->version = version; - request->remove_nodes_limit = remove_nodes_limit; + ZooKeeperRemoveRequest request; + request.path = path; + request.version = version; RequestInfo request_info; - request_info.request = std::move(request); + request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperRemove); } +void ZooKeeper::removeRecursive( + const String &path, + uint32_t remove_nodes_limit, + RemoveRecursiveCallback callback) +{ + if (!isFeatureEnabled(KeeperFeatureFlag::REMOVE_RECURSIVE)) + throw Exception::fromMessage(Error::ZBADARGUMENTS, "RemoveRecursive request type cannot be used because it's not supported by the server"); + + ZooKeeperRemoveRecursiveRequest request; + request.path = path; + request.remove_nodes_limit = remove_nodes_limit; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + + pushRequest(std::move(request_info)); + ProfileEvents::increment(ProfileEvents::ZooKeeperRemove); +} void ZooKeeper::exists( const String & path, diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 6e98df546d9..47d2ab8f401 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -144,9 +144,13 @@ public: void remove( const String & path, int32_t version, - uint32_t remove_nodes_limit, RemoveCallback callback) override; + void removeRecursive( + const String &path, + uint32_t remove_nodes_limit, + RemoveRecursiveCallback callback) override; + void exists( const String & path, ExistsCallback callback, diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp index dfd3faf8d01..fc246e263d9 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.cpp @@ -587,7 +587,7 @@ zkutil::ZooKeeper::FutureRemove ZooKeeperWithFaultInjection::asyncTryRemove(std: promise->set_value(response); }; - keeper->impl->remove(path, version, /*remove_nodes_limit=*/ 1, std::move(callback)); + keeper->impl->remove(path, version, std::move(callback)); return future; } @@ -630,7 +630,7 @@ zkutil::ZooKeeper::FutureRemove ZooKeeperWithFaultInjection::asyncTryRemoveNoThr } }; - keeper->impl->remove(path, version, /*remove_nodes_limit=*/ 1, std::move(callback)); + keeper->impl->remove(path, version, std::move(callback)); return future; } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index fd80ee0a7e2..c91cdb53222 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -86,12 +86,17 @@ bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request break; } case Coordination::OpNum::Remove: - case Coordination::OpNum::RemoveRecursive: { Coordination::ZooKeeperRemoveRequest & remove_req = dynamic_cast(*sub_zk_request); memory_delta -= remove_req.bytesSize(); break; } + case Coordination::OpNum::RemoveRecursive: + { + Coordination::ZooKeeperRemoveRecursiveRequest & remove_req = dynamic_cast(*sub_zk_request); + memory_delta -= remove_req.bytesSize(); + break; + } default: break; } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index aa5b9d539b2..613714263ef 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1462,16 +1462,41 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } }; +namespace +{ + +template +void addUpdateParentPzxidDelta(Storage & storage, std::vector & deltas, int64_t zxid, StringRef path) +{ + auto parent_path = parentNodePath(path); + if (!storage.uncommitted_state.getNode(parent_path)) + return; + + deltas.emplace_back( + std::string{parent_path}, + zxid, + typename Storage::UpdateNodeDelta + { + [zxid](Storage::Node & parent) + { + parent.pzxid = std::max(parent.pzxid, zxid); + } + } + ); +} + +} + template struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor { + using KeeperStorageRequestProcessor::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::vector preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override { @@ -1488,62 +1513,35 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } - const auto update_parent_pzxid = [&]() - { - auto parent_path = parentNodePath(request.path); - if (!storage.uncommitted_state.getNode(parent_path)) - return; - - new_deltas.emplace_back( - std::string{parent_path}, - zxid, - typename Storage::UpdateNodeDelta - { - [zxid](Storage::Node & parent) - { - parent.pzxid = std::max(parent.pzxid, zxid); - } - } - ); - }; - auto node = storage.uncommitted_state.getNode(request.path); if (!node) { if (request.restored_from_zookeeper_log) - update_parent_pzxid(); - + addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; } - - if (request.version != -1 && request.version != node->version) + else if (request.version != -1 && request.version != node->version) return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; - - ToDeleteTreeCollector collector(storage, zxid, request.remove_nodes_limit); - bool limit_exceeded = collector.collect(request.path, *node); - - if (limit_exceeded) + else if (node->numChildren() != 0) return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; if (request.restored_from_zookeeper_log) - update_parent_pzxid(); + addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); - auto delete_deltas = collector.extractDeltas(); + new_deltas.emplace_back( + std::string{parentNodePath(request.path)}, + zxid, + typename Storage::UpdateNodeDelta{[](typename Storage::Node & parent) + { + ++parent.cversion; + parent.decreaseNumChildren(); + }}); - for (const auto & delta : delete_deltas) - std::visit( - Overloaded{ - [&](const typename Storage::RemoveNodeDelta & remove_delta) - { - if (remove_delta.ephemeral_owner) - storage.unregisterEphemeralPath(remove_delta.ephemeral_owner, delta.path); - }, - [](auto && /* delta */) {}, - }, - delta.operation); + new_deltas.emplace_back(request.path, zxid, typename Storage::RemoveNodeDelta{request.version, node->ephemeralOwner()}); - new_deltas.insert(new_deltas.end(), std::make_move_iterator(delete_deltas.begin()), std::make_move_iterator(delete_deltas.end())); + if (node->isEphemeral()) + storage.unregisterEphemeralPath(node->ephemeralOwner(), request.path); digest = storage.calculateNodesDigest(digest, new_deltas); @@ -1564,6 +1562,84 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr { return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); } +}; + +template +struct KeeperStorageRemoveRecursiveRequestProcessor final : public KeeperStorageRequestProcessor +{ + using KeeperStorageRequestProcessor::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); + } + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override + { + ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); + Coordination::ZooKeeperRemoveRecursiveRequest & request = dynamic_cast(*this->zk_request); + + std::vector new_deltas; + + if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) + { + auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path); + + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + } + + auto node = storage.uncommitted_state.getNode(request.path); + + if (!node) + { + if (request.restored_from_zookeeper_log) + addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); + + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + } + + ToDeleteTreeCollector collector(storage, zxid, request.remove_nodes_limit); + bool limit_exceeded = collector.collect(request.path, *node); + + if (limit_exceeded) + return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; + + if (request.restored_from_zookeeper_log) + addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); + + auto delete_deltas = collector.extractDeltas(); + + for (const auto & delta : delete_deltas) + { + const auto * remove_delta = std::get_if(&delta.operation); + if (remove_delta && remove_delta->ephemeral_owner) + storage.unregisterEphemeralPath(remove_delta->ephemeral_owner, delta.path); + } + + new_deltas.insert(new_deltas.end(), std::make_move_iterator(delete_deltas.begin()), std::make_move_iterator(delete_deltas.end())); + + digest = storage.calculateNodesDigest(digest, new_deltas); + + return new_deltas; + } + + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override + { + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); + Coordination::ZooKeeperRemoveRecursiveResponse & response = dynamic_cast(*response_ptr); + + response.error = storage.commit(zxid); + return response_ptr; + } + + KeeperStorageBase::ResponsesForSessions + processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override + { + /// TODO(michicosun) rewrite + return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); + } private: class ToDeleteTreeCollector @@ -2270,10 +2346,13 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Remove: - case Coordination::OpNum::RemoveRecursive: check_operation_type(OperationType::Write); concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; + case Coordination::OpNum::RemoveRecursive: + 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)); @@ -2543,7 +2622,7 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFa registerKeeperRequestProcessor>(*this); registerKeeperRequestProcessor>(*this); registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); }