diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 34c9d94fca5..1021ed6985c 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -181,12 +181,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 a0d6ae54f56..7274cdfb9f1 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -408,11 +408,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 2fbe9110b6b..610c21a0599 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -184,7 +184,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 3f9225e84dd..cd2cefb37bd 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -18,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(bool short_format) const @@ -41,12 +43,12 @@ std::string ZooKeeperRequest::toString(bool short_format) 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 @@ -54,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); @@ -74,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); @@ -82,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); @@ -109,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); @@ -123,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) @@ -137,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); @@ -175,6 +207,12 @@ void ZooKeeperCreateRequest::writeImpl(WriteBuffer & out) const Coordination::write(flags, out); } +size_t ZooKeeperCreateRequest::sizeImpl() const +{ + 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); @@ -211,12 +249,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(bool /*short_format*/) const { return fmt::format( @@ -244,6 +292,11 @@ void ZooKeeperRemoveRecursiveRequest::readImpl(ReadBuffer & in) Coordination::read(remove_nodes_limit, in); } +size_t ZooKeeperRemoveRecursiveRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(remove_nodes_limit); +} + std::string ZooKeeperRemoveRecursiveRequest::toStringImpl(bool /*short_format*/) const { return fmt::format( @@ -259,6 +312,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); @@ -280,12 +338,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); @@ -309,6 +377,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); @@ -316,6 +389,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); @@ -342,12 +420,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); @@ -366,6 +454,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); @@ -397,6 +490,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); @@ -407,6 +505,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); @@ -414,6 +517,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); @@ -431,6 +539,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); @@ -446,6 +559,11 @@ void ZooKeeperGetACLRequest::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +size_t ZooKeeperGetACLRequest::sizeImpl() const +{ + return Coordination::size(path); +} + std::string ZooKeeperGetACLRequest::toStringImpl(bool /*short_format*/) const { return fmt::format("path = {}", path); @@ -457,6 +575,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); @@ -469,6 +592,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); @@ -494,6 +622,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); @@ -596,6 +729,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) @@ -729,31 +883,54 @@ void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const } } -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 ZooKeeperRemoveRecursiveRequest::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()); } +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 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 ZooKeeperRemoveRecursiveRequest::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 @@ -764,11 +941,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 { @@ -777,6 +955,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); @@ -803,6 +986,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 { @@ -960,40 +1148,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); @@ -1015,7 +1169,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 66c075b277b..cd1b2ff250b 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 @@ -25,13 +23,11 @@ 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; virtual void write(WriteBuffer & out) const; virtual OpNum getOpNum() const = 0; virtual void fillLogElements(LogElements & elems, size_t idx) const; @@ -51,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; @@ -66,6 +60,7 @@ struct ZooKeeperRequest : virtual Request std::string toString(bool short_format = false) const; virtual void writeImpl(WriteBuffer &) const = 0; + virtual size_t sizeImpl() const = 0; virtual void readImpl(ReadBuffer &) = 0; virtual std::string toStringImpl(bool /*short_format*/) const { return ""; } @@ -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; @@ -86,6 +80,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 +92,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(bool short_format) const override; ZooKeeperResponsePtr makeResponse() const override; @@ -109,6 +105,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 +119,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(bool short_format) const override; ZooKeeperResponsePtr makeResponse() const override; @@ -138,6 +136,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 +144,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 +153,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 +176,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(bool short_format) const override; @@ -189,6 +191,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 +203,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 +218,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 +233,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(bool short_format) const override; @@ -244,6 +250,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 +272,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(bool short_format) const override; @@ -280,6 +288,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); } @@ -293,6 +302,7 @@ struct ZooKeeperRemoveRecursiveRequest final : RemoveRecursiveRequest, ZooKeeper OpNum getOpNum() const override { return OpNum::RemoveRecursive; } void writeImpl(WriteBuffer & out) const override; void readImpl(ReadBuffer & in) override; + size_t sizeImpl() const override; std::string toStringImpl(bool short_format) const override; ZooKeeperResponsePtr makeResponse() const override; @@ -305,6 +315,7 @@ struct ZooKeeperRemoveRecursiveResponse : RemoveRecursiveResponse, ZooKeeperResp { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::RemoveRecursive; } size_t bytesSize() const override { return RemoveRecursiveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -317,6 +328,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(bool short_format) const override; @@ -330,6 +342,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); } @@ -344,6 +357,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(bool short_format) const override; @@ -357,6 +371,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); } @@ -371,6 +386,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(bool short_format) const override; ZooKeeperResponsePtr makeResponse() const override; @@ -385,6 +401,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); } @@ -399,6 +416,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(bool short_format) const override; ZooKeeperResponsePtr makeResponse() const override; @@ -419,6 +437,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(bool short_format) const override; @@ -429,6 +448,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); } @@ -440,6 +460,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); } @@ -452,6 +473,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(bool short_format) const override; @@ -467,6 +489,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); } @@ -483,6 +506,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; } @@ -493,6 +517,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(bool short_format) const override; ZooKeeperResponsePtr makeResponse() const override; @@ -505,6 +530,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); } @@ -514,6 +540,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(bool short_format) const override; ZooKeeperResponsePtr makeResponse() const override; @@ -526,12 +553,13 @@ 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); } }; -struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest +struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest { OpNum getOpNum() const override; ZooKeeperMultiRequest() = default; @@ -540,6 +568,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(bool short_format) const override; @@ -563,12 +592,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_) @@ -579,6 +610,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); } @@ -609,6 +641,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; @@ -627,6 +660,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) 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/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index d0ecbe744a0..30dfaee23c8 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -301,11 +301,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) @@ -387,6 +389,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); @@ -398,9 +401,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(); } @@ -416,6 +419,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); @@ -431,9 +435,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 157858f3c44..3f7fcbf4289 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/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 142bd3b7c71..8d7c6543c6e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -305,7 +305,7 @@ void KeeperDispatcher::requestThread() if (has_read_request) { if (server->isLeaderAlive()) - server->putLocalReadRequest(request); + server->putLocalReadRequest({request}); else addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 2eada508e22..bfe50288a40 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -1207,8 +1207,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/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 2ed89c414ff..19335835041 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -78,20 +78,20 @@ namespace writeBinary(false, out); /// Serialize stat - writeBinary(node.czxid, out); - writeBinary(node.mzxid, out); - writeBinary(node.ctime(), out); - writeBinary(node.mtime, out); - writeBinary(node.version, out); - writeBinary(node.cversion, out); - writeBinary(node.aversion, out); - writeBinary(node.ephemeralOwner(), out); + writeBinary(node.stats.czxid, out); + writeBinary(node.stats.mzxid, out); + writeBinary(node.stats.ctime(), out); + writeBinary(node.stats.mtime, out); + writeBinary(node.stats.version, out); + writeBinary(node.stats.cversion, out); + writeBinary(node.stats.aversion, out); + writeBinary(node.stats.ephemeralOwner(), out); if (version < SnapshotVersion::V6) - writeBinary(static_cast(node.getData().size()), out); - writeBinary(node.numChildren(), out); - writeBinary(node.pzxid, out); + writeBinary(static_cast(node.stats.data_size), out); + writeBinary(node.stats.numChildren(), out); + writeBinary(node.stats.pzxid, out); - writeBinary(node.seqNum(), out); + writeBinary(node.stats.seqNum(), out); if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) writeBinary(node.sizeInBytes(), out); @@ -100,11 +100,11 @@ namespace template void readNode(Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map) { - readVarUInt(node.data_size, in); - if (node.data_size != 0) + readVarUInt(node.stats.data_size, in); + if (node.stats.data_size != 0) { - node.data = std::unique_ptr(new char[node.data_size]); - in.readStrict(node.data.get(), node.data_size); + node.data = std::unique_ptr(new char[node.stats.data_size]); + in.readStrict(node.data.get(), node.stats.data_size); } if (version >= SnapshotVersion::V1) @@ -141,19 +141,19 @@ namespace } /// Deserialize stat - readBinary(node.czxid, in); - readBinary(node.mzxid, in); + readBinary(node.stats.czxid, in); + readBinary(node.stats.mzxid, in); int64_t ctime; readBinary(ctime, in); - node.setCtime(ctime); - readBinary(node.mtime, in); - readBinary(node.version, in); - readBinary(node.cversion, in); - readBinary(node.aversion, in); + node.stats.setCtime(ctime); + readBinary(node.stats.mtime, in); + readBinary(node.stats.version, in); + readBinary(node.stats.cversion, in); + readBinary(node.stats.aversion, in); int64_t ephemeral_owner = 0; readBinary(ephemeral_owner, in); if (ephemeral_owner != 0) - node.setEphemeralOwner(ephemeral_owner); + node.stats.setEphemeralOwner(ephemeral_owner); if (version < SnapshotVersion::V6) { @@ -163,14 +163,14 @@ namespace int32_t num_children = 0; readBinary(num_children, in); if (ephemeral_owner == 0) - node.setNumChildren(num_children); + node.stats.setNumChildren(num_children); - readBinary(node.pzxid, in); + readBinary(node.stats.pzxid, in); int32_t seq_num = 0; readBinary(seq_num, in); if (ephemeral_owner == 0) - node.setSeqNum(seq_num); + node.stats.setSeqNum(seq_num); if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) { @@ -256,7 +256,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot snapshot.zxid) + if (node.stats.mzxid > snapshot.zxid) break; writeBinary(path, out); writeNode(node, snapshot.version, out); @@ -306,7 +306,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot -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); @@ -435,13 +435,13 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult 0) - node.getChildren().reserve(node.numChildren()); + if (!node.stats.isEphemeral() && node.stats.numChildren() > 0) + node.getChildren().reserve(node.stats.numChildren()); if (ephemeral_owner != 0) - storage.ephemerals[node.ephemeralOwner()].insert(std::string{path}); + storage.committed_ephemerals[node.stats.ephemeralOwner()].insert(std::string{path}); if (recalculate_digest) storage.nodes_digest += node.getDigest(path); @@ -467,16 +467,25 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult(itr.value.getChildren().size())) + if (itr.value.stats.numChildren() != static_cast(itr.value.getChildren().size())) { #ifdef NDEBUG /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. - LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key); + LOG_ERROR( + getLogger("KeeperSnapshotManager"), + "Children counter in stat.numChildren {}" + " is different from actual children size {} for node {}", + itr.value.stats.numChildren(), + itr.value.getChildren().size(), + itr.key); #else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", - itr.value.numChildren(), itr.value.getChildren().size(), itr.key); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Children counter in stat.numChildren {}" + " is different from actual children size {} for node {}", + itr.value.stats.numChildren(), + itr.value.getChildren().size(), + itr.key); #endif } } @@ -511,7 +520,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult::deserialize(SnapshotDeserializationResultpos(0); deserialization_result.cluster_config = ClusterConfig::deserialize(*buffer); } + + storage.updateStats(); } template @@ -544,7 +555,7 @@ KeeperStorageSnapshot::KeeperStorageSnapshot(Storage * storage_, uint64 begin = storage->getSnapshotIteratorBegin(); session_and_timeout = storage->getActiveSessions(); acl_map = storage->acl_map.getMapping(); - session_and_auth = storage->session_and_auth; + session_and_auth = storage->committed_session_and_auth; } template @@ -563,7 +574,7 @@ KeeperStorageSnapshot::KeeperStorageSnapshot( begin = storage->getSnapshotIteratorBegin(); session_and_timeout = storage->getActiveSessions(); acl_map = storage->acl_map.getMapping(); - session_and_auth = storage->session_and_auth; + session_and_auth = storage->committed_session_and_auth; } template diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index a4aa1b18746..b3c5465fee1 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -36,6 +36,11 @@ namespace ProfileEvents extern const Event KeeperStorageLockWaitMicroseconds; } +namespace CurrentMetrics +{ + extern const Metric KeeperAliveConnections; +} + namespace DB { @@ -56,6 +61,7 @@ IKeeperStateMachine::IKeeperStateMachine( , snapshots_queue(snapshots_queue_) , min_request_size_to_cache(keeper_context_->getCoordinationSettings()->min_request_size_for_cache) , log(getLogger("KeeperStateMachine")) + , read_pool(CurrentMetrics::KeeperAliveConnections, CurrentMetrics::KeeperAliveConnections, CurrentMetrics::KeeperAliveConnections, 100, 10000, 10000) , superdigest(superdigest_) , keeper_context(keeper_context_) , snapshot_manager_s3(snapshot_manager_s3_) @@ -175,18 +181,20 @@ void assertDigest( } } -struct TSA_SCOPED_LOCKABLE LockGuardWithStats final +template +struct LockGuardWithStats final { - std::unique_lock lock; - explicit LockGuardWithStats(std::mutex & mutex) TSA_ACQUIRE(mutex) + using LockType = std::conditional_t, std::unique_lock>; + LockType lock; + explicit LockGuardWithStats(SharedMutex & mutex) { Stopwatch watch; - std::unique_lock l(mutex); + LockType l(mutex); ProfileEvents::increment(ProfileEvents::KeeperStorageLockWaitMicroseconds, watch.elapsedMicroseconds()); lock = std::move(l); } - ~LockGuardWithStats() TSA_RELEASE() = default; + ~LockGuardWithStats() = default; }; } @@ -312,13 +320,12 @@ bool KeeperStateMachine::preprocess(const KeeperStorageBase::RequestFor if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; - LockGuardWithStats lock(storage_and_responses_lock); - if (storage->isFinalized()) return false; try { + LockGuardWithStats lock(storage_mutex); storage->preprocessRequest( request_for_session.request, request_for_session.session_id, @@ -335,7 +342,12 @@ bool KeeperStateMachine::preprocess(const KeeperStorageBase::RequestFor } if (keeper_context->digestEnabled() && request_for_session.digest) - assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, request_for_session.log_idx, false); + assertDigest( + *request_for_session.digest, + storage->getNodesDigest(false, /*lock_transaction_mutex=*/true), + *request_for_session.request, + request_for_session.log_idx, + false); return true; } @@ -343,7 +355,7 @@ bool KeeperStateMachine::preprocess(const KeeperStorageBase::RequestFor template void KeeperStateMachine::reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); KeeperStorageBase::ResponseForSession response = processReconfiguration(request_for_session); if (!responses_queue.push(response)) { @@ -461,7 +473,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t l response_for_session.response = response; response_for_session.request = request_for_session->request; - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats 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; @@ -472,24 +484,31 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t l 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); } - LockGuardWithStats lock(storage_and_responses_lock); - KeeperStorageBase::ResponsesForSessions responses_for_sessions - = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); - - for (auto & response_for_session : responses_for_sessions) { - if (response_for_session.response->xid != Coordination::WATCH_XID) - response_for_session.request = request_for_session->request; + LockGuardWithStats lock(storage_mutex); + std::lock_guard response_lock(process_and_responses_lock); + KeeperStorageBase::ResponsesForSessions responses_for_sessions + = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); + for (auto & response_for_session : responses_for_sessions) + { + if (response_for_session.response->xid != Coordination::WATCH_XID) + response_for_session.request = request_for_session->request; - try_push(response_for_session); + 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, request_for_session->log_idx, true); + assertDigest( + *request_for_session->digest, + storage->getNodesDigest(true, /*lock_transaction_mutex=*/true), + *request_for_session->request, + request_for_session->log_idx, + true); } ProfileEvents::increment(ProfileEvents::KeeperCommits); @@ -534,8 +553,6 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) } { /// deserialize and apply snapshot to storage - LockGuardWithStats lock(storage_and_responses_lock); - SnapshotDeserializationResult snapshot_deserialization_result; if (latest_snapshot_ptr) snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr); @@ -543,6 +560,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx())); + LockGuardWithStats 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.snapshot_meta->get_last_log_idx()); @@ -587,16 +605,7 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorageBase::Reque if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; - LockGuardWithStats lock(storage_and_responses_lock); - storage->rollbackRequest(request_for_session.zxid, allow_missing); -} - -template -void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) -{ - if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) - return; - + LockGuardWithStats lock(storage_mutex); storage->rollbackRequest(request_for_session.zxid, allow_missing); } @@ -616,7 +625,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft:: 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. - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); snapshot_task.snapshot = std::make_shared>(storage.get(), snapshot_meta_copy, getClusterConfig()); } @@ -681,7 +690,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft:: } { /// Destroy snapshot with lock - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); LOG_TRACE(log, "Clearing garbage after snapshot"); /// Turn off "snapshot mode" and clear outdate part of storage state storage->clearGarbageAfterSnapshot(); @@ -824,10 +833,10 @@ template void KeeperStateMachine::processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) { /// Pure local request, just process it with storage - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats 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 (auto & response_for_session : responses) { if (response_for_session.response->xid != Coordination::WATCH_XID) @@ -840,112 +849,116 @@ void KeeperStateMachine::processReadRequest(const KeeperStorageBase::Re template void KeeperStateMachine::shutdownStorage() { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); storage->finalize(); } template std::vector KeeperStateMachine::getDeadSessions() { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); return storage->getDeadSessions(); } template int64_t KeeperStateMachine::getNextZxid() const { - LockGuardWithStats lock(storage_and_responses_lock); return storage->getNextZXID(); } template KeeperStorageBase::Digest KeeperStateMachine::getNodesDigest() const { - LockGuardWithStats lock(storage_and_responses_lock); - return storage->getNodesDigest(false); + LockGuardWithStats lock(storage_mutex); + return storage->getNodesDigest(false, /*lock_transaction_mutex=*/true); } template uint64_t KeeperStateMachine::getLastProcessedZxid() const { - LockGuardWithStats lock(storage_and_responses_lock); return storage->getZXID(); } +template +const KeeperStorageBase::Stats & KeeperStateMachine::getStorageStats() const TSA_NO_THREAD_SAFETY_ANALYSIS +{ + return storage->getStorageStats(); +} + template uint64_t KeeperStateMachine::getNodesCount() const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); return storage->getNodesCount(); } template uint64_t KeeperStateMachine::getTotalWatchesCount() const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); return storage->getTotalWatchesCount(); } template uint64_t KeeperStateMachine::getWatchedPathsCount() const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); return storage->getWatchedPathsCount(); } template uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); return storage->getSessionsWithWatchesCount(); } template uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); return storage->getTotalEphemeralNodesCount(); } template uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); return storage->getSessionWithEphemeralNodesCount(); } template void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); storage->dumpWatches(buf); } template void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); storage->dumpWatchesByPath(buf); } template void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); storage->dumpSessionsAndEphemerals(buf); } template uint64_t KeeperStateMachine::getApproximateDataSize() const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); return storage->getApproximateDataSize(); } template uint64_t KeeperStateMachine::getKeyArenaSize() const { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats lock(storage_mutex); return storage->getArenaDataSize(); } @@ -988,7 +1001,7 @@ ClusterConfigPtr IKeeperStateMachine::getClusterConfig() const template void KeeperStateMachine::recalculateStorageStats() { - LockGuardWithStats lock(storage_and_responses_lock); + LockGuardWithStats 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 6afd413d782..03204036e7e 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -85,6 +85,8 @@ public: /// Introspection functions for 4lw commands virtual uint64_t getLastProcessedZxid() const = 0; + virtual const KeeperStorageBase::Stats & getStorageStats() const = 0; + virtual uint64_t getNodesCount() const = 0; virtual uint64_t getTotalWatchesCount() const = 0; virtual uint64_t getWatchedPathsCount() const = 0; @@ -124,12 +126,16 @@ protected: /// Mutex for snapshots mutable std::mutex snapshots_lock; - /// Lock for storage and responses_queue. It's important to process requests + /// 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 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 /// 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}; @@ -146,6 +152,7 @@ protected: mutable std::mutex cluster_config_lock; ClusterConfigPtr cluster_config; + ThreadPool read_pool; /// Special part of ACL system -- superdigest specified in server config. const std::string superdigest; @@ -153,10 +160,8 @@ protected: KeeperSnapshotManagerS3 * snapshot_manager_s3; - virtual KeeperStorageBase::ResponseForSession processReconfiguration( - const KeeperStorageBase::RequestForSession& request_for_session) - TSA_REQUIRES(storage_and_responses_lock) = 0; - + virtual KeeperStorageBase::ResponseForSession processReconfiguration(const KeeperStorageBase::RequestForSession & request_for_session) + = 0; }; /// ClickHouse Keeper state machine. Wrapper for KeeperStorage. @@ -189,10 +194,6 @@ public: // (can happen in case of exception during preprocessing) void rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) override; - void rollbackRequestNoLock( - const KeeperStorageBase::RequestForSession & request_for_session, - bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS; - /// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state. bool apply_snapshot(nuraft::snapshot & s) override; @@ -205,7 +206,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; } @@ -224,6 +225,8 @@ public: /// Introspection functions for 4lw commands 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; @@ -245,12 +248,12 @@ public: private: /// Main state machine logic - std::unique_ptr storage; //TSA_PT_GUARDED_BY(storage_and_responses_lock); + std::unique_ptr storage; /// Save/Load and Serialize/Deserialize logic for snapshots. KeeperSnapshotManager snapshot_manager; - KeeperStorageBase::ResponseForSession processReconfiguration(const KeeperStorageBase::RequestForSession & request_for_session) - TSA_REQUIRES(storage_and_responses_lock) override; + KeeperStorageBase::ResponseForSession processReconfiguration(const KeeperStorageBase::RequestForSession & request_for_session) override; }; + } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index a3ba7402296..016e926ad54 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1,4 +1,4 @@ -// NOLINTBEGIN(clang-analyzer-optin.core.EnumCastOutOfRange) +/// NOLINTBEGIN(clang-analyzer-optin.core.EnumCastOutOfRange) #include #include @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -52,7 +53,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; } namespace @@ -113,8 +113,28 @@ 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) + 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); @@ -127,7 +147,12 @@ 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( + KeeperStorageBase::WatchInfo{.path = path, .is_list_watch = false}); + chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_response}); + } watches.erase(watch_it); } @@ -144,7 +169,7 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl( paths_to_check_for_list_watches.push_back(path); /// Trigger both list watches for this path paths_to_check_for_list_watches.push_back(parent_path.toString()); /// And for parent path } - /// CHANGED event never trigger list wathes + /// CHANGED event never trigger list watches for (const auto & path_to_check : paths_to_check_for_list_watches) { @@ -163,7 +188,12 @@ 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( + KeeperStorageBase::WatchInfo{.path = path_to_check, .is_list_watch = true}); + chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_list_response}); + } list_watches.erase(watch_it); } @@ -186,16 +216,16 @@ uint64_t calculateDigest(std::string_view path, const Node & node) hash.update(data); } - hash.update(node.czxid); - hash.update(node.mzxid); - hash.update(node.ctime()); - hash.update(node.mtime); - hash.update(node.version); - hash.update(node.cversion); - hash.update(node.aversion); - hash.update(node.ephemeralOwner()); - hash.update(node.numChildren()); - hash.update(node.pzxid); + hash.update(node.stats.czxid); + hash.update(node.stats.mzxid); + hash.update(node.stats.ctime()); + hash.update(node.stats.mtime); + hash.update(node.stats.version); + hash.update(node.stats.cversion); + hash.update(node.stats.aversion); + hash.update(node.stats.ephemeralOwner()); + hash.update(node.stats.numChildren()); + hash.update(node.stats.pzxid); auto digest = hash.get64(); @@ -208,7 +238,7 @@ uint64_t calculateDigest(std::string_view path, const Node & node) } -void KeeperRocksNodeInfo::copyStats(const Coordination::Stat & stat) +void NodeStats::copyStats(const Coordination::Stat & stat) { czxid = stat.czxid; mzxid = stat.mzxid; @@ -232,18 +262,23 @@ void KeeperRocksNodeInfo::copyStats(const Coordination::Stat & stat) } } +void KeeperRocksNodeInfo::copyStats(const Coordination::Stat & stat) +{ + stats.copyStats(stat); +} + void KeeperRocksNode::invalidateDigestCache() const { if (serialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "We modify node after serialized it"); - digest = 0; + cached_digest = 0; } UInt64 KeeperRocksNode::getDigest(std::string_view path) const { - if (!digest) - digest = calculateDigest(path, *this); - return digest; + if (!cached_digest) + cached_digest = calculateDigest(path, *this); + return cached_digest; } String KeeperRocksNode::getEncodedString() @@ -264,35 +299,41 @@ void KeeperRocksNode::decodeFromString(const String &buffer_str) ReadBufferFromOwnString buffer(buffer_str); KeeperRocksNodeInfo & node_info = *this; readPODBinary(node_info, buffer); - readVarUInt(data_size, buffer); - if (data_size) + readVarUInt(stats.data_size, buffer); + if (stats.data_size) { - data = std::unique_ptr(new char[data_size]); - buffer.readStrict(data.get(), data_size); + data = std::unique_ptr(new char[stats.data_size]); + buffer.readStrict(data.get(), stats.data_size); } } +void KeeperRocksNode::setResponseStat(Coordination::Stat & response_stat) const +{ + response_stat.czxid = stats.czxid; + response_stat.mzxid = stats.mzxid; + response_stat.ctime = stats.ctime(); + response_stat.mtime = stats.mtime; + response_stat.version = stats.version; + response_stat.cversion = stats.cversion; + response_stat.aversion = stats.aversion; + response_stat.ephemeralOwner = stats.ephemeralOwner(); + response_stat.dataLength = static_cast(stats.data_size); + response_stat.numChildren = stats.numChildren(); + response_stat.pzxid = stats.pzxid; +} + KeeperMemNode & KeeperMemNode::operator=(const KeeperMemNode & other) { if (this == &other) return *this; - czxid = other.czxid; - mzxid = other.mzxid; - pzxid = other.pzxid; + stats = other.stats; acl_id = other.acl_id; - mtime = other.mtime; - is_ephemeral_and_ctime = other.is_ephemeral_and_ctime; - ephemeral_or_children_data = other.ephemeral_or_children_data; - data_size = other.data_size; - version = other.version; - cversion = other.cversion; - aversion = other.aversion; - if (data_size != 0) + if (stats.data_size != 0) { - data = std::unique_ptr(new char[data_size]); - memcpy(data.get(), other.data.get(), data_size); + data = std::unique_ptr(new char[stats.data_size]); + memcpy(data.get(), other.data.get(), stats.data_size); } children = other.children; @@ -310,21 +351,12 @@ KeeperMemNode & KeeperMemNode::operator=(KeeperMemNode && other) noexcept if (this == &other) return *this; - czxid = other.czxid; - mzxid = other.mzxid; - pzxid = other.pzxid; + stats = other.stats; acl_id = other.acl_id; - mtime = other.mtime; - is_ephemeral_and_ctime = other.is_ephemeral_and_ctime; - ephemeral_or_children_data = other.ephemeral_or_children_data; - version = other.version; - cversion = other.cversion; - aversion = other.aversion; - data_size = other.data_size; data = std::move(other.data); - other.data_size = 0; + other.stats.data_size = 0; static_assert(std::is_nothrow_move_assignable_v); children = std::move(other.children); @@ -339,61 +371,41 @@ KeeperMemNode::KeeperMemNode(KeeperMemNode && other) noexcept bool KeeperMemNode::empty() const { - return data_size == 0 && mzxid == 0; + return stats.data_size == 0 && stats.mzxid == 0; } void KeeperMemNode::copyStats(const Coordination::Stat & stat) { - czxid = stat.czxid; - mzxid = stat.mzxid; - pzxid = stat.pzxid; - - mtime = stat.mtime; - setCtime(stat.ctime); - - version = stat.version; - cversion = stat.cversion; - aversion = stat.aversion; - - if (stat.ephemeralOwner == 0) - { - is_ephemeral_and_ctime.is_ephemeral = false; - setNumChildren(stat.numChildren); - } - else - { - setEphemeralOwner(stat.ephemeralOwner); - } + stats.copyStats(stat); } void KeeperMemNode::setResponseStat(Coordination::Stat & response_stat) const { - response_stat.czxid = czxid; - response_stat.mzxid = mzxid; - response_stat.ctime = ctime(); - response_stat.mtime = mtime; - response_stat.version = version; - response_stat.cversion = cversion; - response_stat.aversion = aversion; - response_stat.ephemeralOwner = ephemeralOwner(); - response_stat.dataLength = static_cast(data_size); - response_stat.numChildren = numChildren(); - response_stat.pzxid = pzxid; - + response_stat.czxid = stats.czxid; + response_stat.mzxid = stats.mzxid; + response_stat.ctime = stats.ctime(); + response_stat.mtime = stats.mtime; + response_stat.version = stats.version; + response_stat.cversion = stats.cversion; + response_stat.aversion = stats.aversion; + response_stat.ephemeralOwner = stats.ephemeralOwner(); + response_stat.dataLength = static_cast(stats.data_size); + response_stat.numChildren = stats.numChildren(); + response_stat.pzxid = stats.pzxid; } uint64_t KeeperMemNode::sizeInBytes() const { - return sizeof(KeeperMemNode) + children.size() * sizeof(StringRef) + data_size; + return sizeof(KeeperMemNode) + children.size() * sizeof(StringRef) + stats.data_size; } void KeeperMemNode::setData(const String & new_data) { - data_size = static_cast(new_data.size()); - if (data_size != 0) + stats.data_size = static_cast(new_data.size()); + if (stats.data_size != 0) { - data = std::unique_ptr(new char[new_data.size()]); - memcpy(data.get(), new_data.data(), data_size); + data = std::unique_ptr(new char[stats.data_size]); + memcpy(data.get(), new_data.data(), stats.data_size); } } @@ -422,28 +434,14 @@ UInt64 KeeperMemNode::getDigest(const std::string_view path) const void KeeperMemNode::shallowCopy(const KeeperMemNode & other) { - czxid = other.czxid; - mzxid = other.mzxid; - pzxid = other.pzxid; - acl_id = other.acl_id; /// 0 -- no ACL by default - - mtime = other.mtime; - - is_ephemeral_and_ctime = other.is_ephemeral_and_ctime; - - ephemeral_or_children_data = other.ephemeral_or_children_data; - - data_size = other.data_size; - if (data_size != 0) + stats = other.stats; + acl_id = other.acl_id; + if (stats.data_size != 0) { - data = std::unique_ptr(new char[data_size]); - memcpy(data.get(), other.data.get(), data_size); + data = std::unique_ptr(new char[stats.data_size]); + memcpy(data.get(), other.data.get(), stats.data_size); } - version = other.version; - cversion = other.cversion; - aversion = other.aversion; - cached_digest = other.cached_digest; } @@ -489,7 +487,7 @@ void KeeperStorage::initializeSystemNodes() "/", [](KeeperStorage::Node & node) { - node.increaseNumChildren(); + node.stats.increaseNumChildren(); if constexpr (!use_rocksdb) node.addChild(getBaseNodeName(keeper_system_path)); } @@ -522,6 +520,7 @@ void KeeperStorage::initializeSystemNodes() } } + updateStats(); initialized = true; } @@ -537,8 +536,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::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; @@ -550,22 +552,47 @@ std::shared_ptr KeeperStorage::UncommittedS return nullptr; } +template +void KeeperStorage::UncommittedState::UncommittedNode::materializeACL(const ACLMap & current_acl_map) +{ + if (!acls.has_value()) + acls.emplace(current_acl_map.convertNumber(node->acl_id)); +} + template void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) { chassert(!delta.path.empty()); - if (!nodes.contains(delta.path)) + 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)) - nodes.emplace(delta.path, UncommittedNode{.node = std::move(storage_node)}); + { + 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 - nodes.emplace(delta.path, UncommittedNode{.node = nullptr}); + { + 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; + } } std::visit( [&](const DeltaType & operation) { - auto & [node, acls, last_applied_zxid] = nodes.at(delta.path); + auto & [node, acls, applied_zxids] = *uncommitted_node; if constexpr (std::same_as) { @@ -574,26 +601,31 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) node->copyStats(operation.stat); node->setData(operation.data); acls = operation.acls; - last_applied_zxid = delta.zxid; } else if constexpr (std::same_as) { chassert(node); node = nullptr; - last_applied_zxid = delta.zxid; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { chassert(node); node->invalidateDigestCache(); - operation.update_fn(*node); - last_applied_zxid = delta.zxid; + node->stats = operation.new_stats; + } + 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; } + + applied_zxids.insert(delta.zxid); + zxid_to_nodes[delta.zxid].insert(node_path); }, delta.operation); } @@ -605,13 +637,13 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool { 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.second.get(); if (predicate(*auth_ptr)) return true; @@ -620,13 +652,17 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool }; if (is_local) - return check_auth(storage.session_and_auth[session_id]); + { + std::shared_lock lock(storage.auth_mutex); + return check_auth(storage.committed_session_and_auth[session_id]); + } /// we want to close the session and with that we will remove all the auth related to the session if (closed_sessions.contains(session_id)) return false; - if (check_auth(storage.session_and_auth[session_id])) + std::shared_lock lock(storage.auth_mutex); + if (check_auth(storage.committed_session_and_auth[session_id])) return true; // check if there are uncommitted @@ -634,95 +670,116 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool if (auth_it == session_and_auth.end()) return false; - return check_auth(auth_it->second); + if (check_auth(auth_it->second)) + return true; + + return check_auth(storage.committed_session_and_auth[session_id]); } template -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)); + chassert(!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); - } - else if (const auto * close_session_delta = std::get_if(&added_delta.operation)) - { - closed_sessions.insert(close_session_delta->session_id); - } -} - -template -void KeeperStorage::UncommittedState::addDeltas(std::vector new_deltas) -{ - for (auto & delta : new_deltas) - addDelta(std::move(delta)); -} - -template -void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) -{ - chassert(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(); + auto & [node, acls, applied_zxids] = nodes.at(delta.path); + + if constexpr (std::same_as) + { + chassert(node); + node = nullptr; + } + else if constexpr (std::same_as) + { + chassert(!node); + node = std::make_shared(); + node->stats = operation.stat; + node->setData(operation.data); + acls = operation.acls; + } + else if constexpr (std::same_as) + { + chassert(node); + node->invalidateDigestCache(); + node->stats = operation.old_stats; + } + else if constexpr (std::same_as) + { + chassert(node); + node->invalidateDigestCache(); + node->setData(operation.old_data); + } + else if constexpr (std::same_as) + { + acls = operation.old_acls; + } + + applied_zxids.erase(delta.zxid); + zxid_to_nodes.erase(delta.zxid); + }, + delta.operation); +} + +template +void KeeperStorage::UncommittedState::applyDeltas(const std::list & new_deltas) +{ + for (const auto & delta : new_deltas) + { + 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.push_back(std::pair{delta.zxid, auth_delta->auth_id}); + } + else if (const auto * close_session_delta = std::get_if(&delta.operation)) + { + closed_sessions.insert(close_session_delta->session_id); + } + } +} + +template +void KeeperStorage::UncommittedState::addDeltas(std::list new_deltas) +{ + std::lock_guard lock(deltas_mutex); + deltas.splice(deltas.end(), std::move(new_deltas)); +} + +template +void KeeperStorage::UncommittedState::cleanup(int64_t commit_zxid) +{ + 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; - } - auto & front_delta = deltas.front(); - - if (!front_delta.path.empty()) + for (const auto node : transaction_nodes) { - auto & path_deltas = deltas_for_path.at(front_delta.path); - chassert(path_deltas.front() == &front_delta); - path_deltas.pop_front(); - if (path_deltas.empty()) - { - deltas_for_path.erase(front_delta.path); + auto node_it = nodes.find(node); + if (node_it == nodes.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing expected uncommitted node '{}'", node); - // 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)); - } + node_it->second.applied_zxids.erase(transaction_zxid); + if (node_it->second.applied_zxids.empty()) + nodes.erase(node_it); } - else if (auto * add_auth = std::get_if(&front_delta.operation)) - { - auto & uncommitted_auth = session_and_auth[add_auth->session_id]; - chassert(!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); - } - else if (auto * close_session = std::get_if(&front_delta.operation)) - { - closed_sessions.erase(close_session->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) + for (auto it = session_and_auth.begin(); it != session_and_auth.end();) { - if (nodes[node].zxid == commit_zxid) - nodes.erase(node); + 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; } } @@ -730,26 +787,42 @@ template void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) { // we can only rollback the last zxid (if there is any) - // if there is a delta with a larger zxid, we have invalid state - 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()); + } + + rollback(std::move(rollback_deltas)); +} + +template +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 (; delta_it != deltas.rend(); ++delta_it) + for (auto delta_it = rollback_deltas.rbegin(); delta_it != rollback_deltas.rend(); ++delta_it) { - if (delta_it->zxid < rollback_zxid) - break; - - chassert(delta_it->zxid == rollback_zxid); - if (!delta_it->path.empty()) + const auto & delta = *delta_it; + if (!delta.path.empty()) { std::visit( [&](const DeltaType & operation) @@ -757,79 +830,50 @@ 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); + unregisterEphemeralPath(storage.uncommitted_state.ephemerals, operation.stat.ephemeralOwner, delta.path, /*throw_if_missing=*/false); } 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.uncommitted_state.ephemerals[operation.stat.ephemeralOwner()].emplace(delta.path); } }, - delta_it->operation); + delta.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); - } + rollbackDelta(delta); } - else if (auto * add_auth = std::get_if(&delta_it->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() == &add_auth->auth_id) + if (uncommitted_auth.back().second == add_auth->auth_id) { uncommitted_auth.pop_back(); if (uncommitted_auth.empty()) session_and_auth.erase(add_auth->session_id); } } - else if (auto * close_session = std::get_if(&delta_it->operation)) + else if (const auto * close_session = std::get_if(&delta.operation)) { closed_sessions.erase(close_session->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); - } - } - } } 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; - return tryGetNodeFromStorage(path); + std::shared_ptr node = tryGetNodeFromStorage(path, should_lock_storage); + + if (node) + { + auto [node_it, _] = nodes.emplace(std::string{path}, UncommittedNode{.node = node}); + zxid_to_nodes[0].insert(node_it->first); + } + + return node; } template @@ -845,13 +889,22 @@ template Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const { if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) - return node_it->second.acls; + { + node_it->second.materializeACL(storage.acl_map); + return *node_it->second.acls; + } - auto node_it = storage.container.find(path); - if (node_it == storage.container.end()) - return {}; + std::shared_ptr node = tryGetNodeFromStorage(path); - return storage.acl_map.convertNumber(node_it->value.acl_id); + if (node) + { + 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; + } + + return {}; } template @@ -861,41 +914,69 @@ void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t se { 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.second.get(); - func(*auth_ptr); + if (!auth_ptr->scheme.empty()) + func(*auth_ptr); } }; + /// 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 (storage.session_and_auth.contains(session_id)) - call_for_each_auth(storage.session_and_auth.at(session_id)); + if (auto auth_it = storage.committed_session_and_auth.find(session_id); auth_it != storage.committed_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 { -[[noreturn]] void onStorageInconsistency() +[[noreturn]] void onStorageInconsistency(std::string_view message) { LOG_ERROR( getLogger("KeeperStorage"), - "Inconsistency found between uncommitted and committed data. Keeper will terminate to avoid undefined behaviour."); + "Inconsistency found between uncommitted and committed data ({}). Keeper will terminate to avoid undefined behaviour.", message); std::terminate(); } } +/// Get current committed zxid template -void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_log_idx) +int64_t KeeperStorage::getZXID() const +{ + std::lock_guard lock(transaction_mutex); + return zxid; +} + +template +int64_t KeeperStorage::getNextZXIDLocked() const +{ + if (uncommitted_transactions.empty()) + return zxid + 1; + + return uncommitted_transactions.back().zxid + 1; +} + +template +int64_t KeeperStorage::getNextZXID() const +{ + std::lock_guard lock(transaction_mutex); + return getNextZXIDLocked(); +} + +template +void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_log_idx) TSA_NO_THREAD_SAFETY_ANALYSIS { std::unordered_set zxids_to_apply; for (const auto & transaction : uncommitted_transactions) @@ -917,48 +998,48 @@ void KeeperStorage::applyUncommittedState(KeeperStorage & other, int6 if (!zxids_to_apply.contains(it->zxid)) continue; - other.uncommitted_state.addDelta(*it); + other.uncommitted_state.applyDelta(*it); + other.uncommitted_state.deltas.push_back(*it); } } template -Coordination::Error KeeperStorage::commit(int64_t commit_zxid) +Coordination::Error KeeperStorage::commit(KeeperStorageBase::DeltaRange deltas) { - // 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 (const auto & delta : deltas) { - if (delta.zxid > commit_zxid) - break; - - bool finish_subdelta = false; auto result = std::visit( - [&, &path = delta.path](DeltaType & operation) -> Coordination::Error + [&, &path = delta.path](const DeltaType & operation) -> Coordination::Error { if constexpr (std::same_as) { if (!createNode( path, - std::move(operation.data), + operation.data, operation.stat, - std::move(operation.acls))) - onStorageInconsistency(); + operation.acls)) + onStorageInconsistency("Failed to create a node"); 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()) - onStorageInconsistency(); + onStorageInconsistency("Node to be updated is missing"); - if (operation.version != -1 && operation.version != node_it->value.version) - onStorageInconsistency(); + if (operation.version != -1 && operation.version != node_it->value.stats.version) + onStorageInconsistency("Node to be updated has invalid version"); if constexpr (!use_rocksdb) 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.stats = operation.new_stats; + else + node.setData(std::move(operation.new_data)); + }); if constexpr (!use_rocksdb) addDigest(updated_node->value, path); @@ -967,7 +1048,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) else if constexpr (std::same_as) { if (!removeNode(path, operation.version)) - onStorageInconsistency(); + onStorageInconsistency("Failed to remove node"); return Coordination::Error::ZOK; } @@ -975,14 +1056,14 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) { auto node_it = container.find(path); if (node_it == container.end()) - onStorageInconsistency(); + onStorageInconsistency("Failed to set ACL because node is missing"); - if (operation.version != -1 && operation.version != node_it->value.aversion) - onStorageInconsistency(); + if (operation.version != -1 && operation.version != node_it->value.stats.aversion) + onStorageInconsistency("Failed to set ACL because version of the node is invalid"); 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](Node & node) { node.acl_id = acl_id; }); @@ -993,12 +1074,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}; + committed_session_and_auth[operation.session_id].emplace_back(std::move(*operation.auth_id)); return Coordination::Error::ZOK; } else if constexpr (std::same_as) @@ -1008,16 +1089,13 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) else { // shouldn't be called in any process functions - onStorageInconsistency(); + onStorageInconsistency("Invalid delta operation"); } }, delta.operation); if (result != Coordination::Error::ZOK) return result; - - if (finish_subdelta) - return Coordination::Error::ZOK; } return Coordination::Error::ZOK; @@ -1036,7 +1114,7 @@ bool KeeperStorage::createNode( if (node_it == container.end()) return false; - if (node_it->value.isEphemeral()) + if (node_it->value.stats.isEphemeral()) return false; if (container.contains(path)) @@ -1050,6 +1128,7 @@ bool KeeperStorage::createNode( created_node.acl_id = acl_id; created_node.copyStats(stat); created_node.setData(data); + if constexpr (use_rocksdb) { container.insert(path, created_node); @@ -1064,12 +1143,20 @@ bool KeeperStorage::createNode( [child_path](KeeperMemNode & parent) { parent.addChild(child_path); - chassert(parent.numChildren() == static_cast(parent.getChildren().size())); + chassert(parent.stats.numChildren() == static_cast(parent.getChildren().size())); } ); 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); + } + return true; }; @@ -1080,10 +1167,7 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers if (node_it == container.end()) return false; - if (version != -1 && version != node_it->value.version) - return false; - - if (node_it->value.numChildren()) + if (version != -1 && version != node_it->value.stats.version) return false; KeeperStorage::Node prev_node; @@ -1099,7 +1183,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.numChildren() == static_cast(parent.getChildren().size())); } ); @@ -1107,70 +1190,61 @@ 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; } -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, int64_t zxid) const = 0; - - virtual std::vector - preprocess(Storage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const + 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::RemoveRecursive: + 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*/, int64_t /*zxid*/) const - { - throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Cannot process the request locally"}; - } - - virtual KeeperStorageBase::ResponsesForSessions - processWatches(const Storage & /*storage*/, int64_t /*zxid*/, 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 -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - Coordination::ZooKeeperResponsePtr - process(Storage & storage, int64_t zxid) const override - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - response_ptr->error = storage.commit(zxid); - return response_ptr; - } -}; - -template -struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - Coordination::ZooKeeperResponsePtr - process(Storage & /* storage */, int64_t /* zxid */) const override - { - auto response = this->zk_request->makeResponse(); - dynamic_cast(*response).path - = dynamic_cast(*this->zk_request).path; - return response; - } -}; +} namespace { @@ -1180,6 +1254,7 @@ Coordination::ACLs getNodeACLs(Storage & storage, StringRef path, bool is_local) { if (is_local) { + std::shared_lock lock(storage.storage_mutex); auto node_it = storage.container.find(path); if (node_it == storage.container.end()) return {}; @@ -1232,1416 +1307,1509 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int6 return false; } -template -void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) +/// Default implementations /// +template T, typename Storage> +Coordination::ZooKeeperResponsePtr +processLocal(const T & zk_request, Storage & /*storage*/, KeeperStorageBase::DeltaRange /*deltas*/) { - 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); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Local processing not supported for request with type {}", zk_request.getOpNum()); } -template -struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor +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*/) { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + return {}; +} - KeeperStorageBase::ResponsesForSessions - processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override +template T> +KeeperStorageBase::ResponsesForSessions processWatches( + const T & /*zk_request*/, + KeeperStorageBase::DeltaRange /*deltas*/, + KeeperStorageBase::Watches & /*watches*/, + KeeperStorageBase::Watches & /*list_watches*/, + KeeperStorageBase::SessionAndWatcher & /*sessions_and_watchers*/) +{ + 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, + KeeperStorageBase::DeltaRange deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + response_ptr->error = storage.commit(deltas); + return response_ptr; +} +/// HEARTBEAT Request /// + +/// SYNC Request /// +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* storage */, KeeperStorageBase::DeltaRange /* deltas */) +{ + auto response = std::make_shared(); + 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::DeltaRange /*deltas*/, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) +{ + return processWatchesImpl(zk_request.getPath(), watches, list_watches, sessions_and_watchers, 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::vector - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override - { - ProfileEvents::increment(ProfileEvents::KeeperCreateRequest); - Coordination::ZooKeeperCreateRequest & request = dynamic_cast(*this->zk_request); - - std::vector 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->isEphemeral()) - return {typename Storage::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->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) - 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; + path_created += seq_num_str.str(); + } - auto parent_update = [parent_cversion, zxid](Storage::Node & node) + 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 (zk_request.getOpNum() == Coordination::OpNum::CreateIfNotExists) + return new_deltas; + + return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; + } + + if (getBaseNodeName(path_created).size == 0) + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + + 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) + storage.uncommitted_state.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, KeeperStorageBase::DeltaRange deltas) +{ + std::shared_ptr response = zk_request.not_exists + ? std::make_shared() + : std::make_shared(); + + if (deltas.empty()) + { + chassert(zk_request.not_exists); + response->path_created = zk_request.getPath(); + response->error = Coordination::Error::ZOK; + return response; + } + + 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; + } + + response->path_created = std::move(created_path); + response->error = Coordination::Error::ZOK; + return response; +} +/// CREATE 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); +} + +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 {}; + + 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, KeeperStorageBase::DeltaRange deltas) +{ + auto response = std::make_shared(); + + if constexpr (!local) + { + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - /// Increment sequential number even if node is not sequential - node.increaseSeqNum(); - if (parent_cversion == -1) - ++node.cversion; - else if (parent_cversion > node.cversion) - node.cversion = parent_cversion; + response->error = result; + return response; + } + } - node.pzxid = std::max(zxid, node.pzxid); - node.increaseNumChildren(); - }; + if (zk_request.path == Coordination::keeper_config_path) + { + response->data = serializeClusterConfig( + storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); + response->error = Coordination::Error::ZOK; + return response; + } - new_deltas.emplace_back(std::string{parent_path}, zxid, typename Storage::UpdateNodeDelta{std::move(parent_update)}); + 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("Failed to get node because it's missing"); + } + else + { + node_it->value.setResponseStat(response->stat); + auto data = node_it->value.getData(); + response->data = std::string(data); + response->error = Coordination::Error::ZOK; + } - 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; + return response; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange 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::DeltaRange /*deltas*/, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) +{ + return processWatchesImpl(zk_request.getPath(), watches, list_watches, sessions_and_watchers, Coordination::Event::DELETED); +} + +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); + + 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::move(path_created), + std::string{parent_path}, zxid, - typename Storage::CreateNodeDelta{stat, std::move(node_acls), request.data}); + std::move(*update_parent_delta) + ); + }; - digest = storage.calculateNodesDigest(digest, new_deltas); - return new_deltas; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override + const auto update_parent_pzxid = [&]() { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); + if (!update_parent_delta) + return; - if (storage.uncommitted_state.deltas.begin()->zxid != 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); + + if (!node) + { + if (zk_request.restored_from_zookeeper_log) { - response.path_created = this->zk_request->getPath(); - response.error = Coordination::Error::ZOK; - return response_ptr; + update_parent_pzxid(); + add_parent_update_delta(); } - - if (const auto result = storage.commit(zxid); 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.error = Coordination::Error::ZOK; - return response_ptr; + 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}}; -template -struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor + if (zk_request.restored_from_zookeeper_log) + update_parent_pzxid(); + + chassert(update_parent_delta); + ++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()) + { + /// 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); + + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperRemoveRequest & /*zk_request*/, Storage & storage, KeeperStorageBase::DeltaRange deltas) { - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override - { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); - } + auto response = std::make_shared(); - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + response->error = storage.commit(std::move(deltas)); + return response; +} - std::vector - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override - { - ProfileEvents::increment(ProfileEvents::KeeperGetRequest); - Coordination::ZooKeeperGetRequest & request = dynamic_cast(*this->zk_request); +/// REMOVE Request /// - 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}}; - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const - { - 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(zxid); 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, int64_t zxid) const override - { - return processImpl(storage, zxid); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override - { - ProfileEvents::increment(ProfileEvents::KeeperGetRequest); - return processImpl(storage, zxid); - } -}; +/// REMOVERECURSIVE Request /// namespace { template -void addUpdateParentPzxidDelta(Storage & storage, std::vector & deltas, int64_t zxid, StringRef path) +class ToDeleteTreeCollector { - auto parent_path = parentNodePath(path); - if (!storage.uncommitted_state.getNode(parent_path)) - return; + Storage & storage; + int64_t zxid; + int64_t session_id; + uint32_t limit; - deltas.emplace_back( - std::string{parent_path}, - zxid, - typename Storage::UpdateNodeDelta - { - [zxid](Storage::Node & parent) - { - parent.pzxid = std::max(parent.pzxid, zxid); - } - } - ); -} + uint32_t nodes_observed = 1; /// root node -} + std::list deltas; + using UncommittedChildren + = std::unordered_set; -template -struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override +public: + enum class CollectStatus + { + Ok, + NoAuth, + LimitExceeded, + }; + + friend struct KeeperStorageRemoveRecursiveRequestProcessor; + + ToDeleteTreeCollector(Storage & storage_, int64_t zxid_, int64_t session_id_, uint32_t limit_) + : storage(storage_) + , zxid(zxid_) + , session_id(session_id_) + , limit(limit_) { - 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 + CollectStatus collect(StringRef root_path, const Storage::Node & root_node) { - ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); - Coordination::ZooKeeperRemoveRequest & request = dynamic_cast(*this->zk_request); + if (checkLimits(root_node)) + return CollectStatus::LimitExceeded; - std::vector new_deltas; + addDelta(root_path, root_node.stats, storage.uncommitted_state.getACLs(root_path), std::string{root_node.getData()}); - if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) + for (auto current_delta_it = deltas.rbegin(); current_delta_it != deltas.rend(); ++current_delta_it) { - auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path); + const auto & current_path = current_delta_it->path; + chassert(!current_path.empty()); - handleSystemNodeModification(keeper_context, error_msg); - return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - } + if (!storage.checkACL(current_path, Coordination::ACL::Delete, session_id, /*is_local=*/false)) + return CollectStatus::NoAuth; - auto node = storage.uncommitted_state.getNode(request.path); + UncommittedChildren uncommitted_children; + if (auto status = visitUncommitted(current_path, uncommitted_children); status != CollectStatus::Ok) + return status; - if (!node) - { - if (request.restored_from_zookeeper_log) - addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - } - else if (request.version != -1 && request.version != node->version) - return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; - else if (node->numChildren() != 0) - return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; - - if (request.restored_from_zookeeper_log) - addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); - - new_deltas.emplace_back( - std::string{parentNodePath(request.path)}, - zxid, - typename Storage::UpdateNodeDelta{[](typename Storage::Node & parent) - { - ++parent.cversion; - parent.decreaseNumChildren(); - }}); - - new_deltas.emplace_back(request.path, zxid, typename Storage::RemoveNodeDelta{request.version, node->ephemeralOwner()}); - - if (node->isEphemeral()) - storage.unregisterEphemeralPath(node->ephemeralOwner(), request.path); - - 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::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); - - response.error = storage.commit(zxid); - return response_ptr; - } - - KeeperStorageBase::ResponsesForSessions - processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override - { - 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, session_id, request.remove_nodes_limit); - auto collect_status = collector.collect(request.path, *node); - - if (collect_status == ToDeleteTreeCollector::CollectStatus::NoAuth) - return {typename Storage::Delta{zxid, Coordination::Error::ZNOAUTH}}; - - if (collect_status == ToDeleteTreeCollector::CollectStatus::LimitExceeded) - 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(const Storage & storage, int64_t zxid, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override - { - /// need to iterate over zxid deltas and update watches for deleted tree. - const auto & deltas = storage.uncommitted_state.deltas; - - KeeperStorageBase::ResponsesForSessions responses; - for (auto it = deltas.rbegin(); it != deltas.rend() && it->zxid == zxid; ++it) - { - const auto * remove_delta = std::get_if(&it->operation); - if (remove_delta) - { - auto new_responses = processWatchesImpl(it->path, watches, list_watches, Coordination::Event::DELETED); - responses.insert(responses.end(), std::make_move_iterator(new_responses.begin()), std::make_move_iterator(new_responses.end())); - } - } - - return responses; - } - -private: - using SNode = typename Storage::Node; - - class ToDeleteTreeCollector - { - Storage & storage; - int64_t zxid; - int64_t session_id; - uint32_t limit; - - uint32_t max_level = 0; - uint32_t nodes_observed = 1; /// root node - std::unordered_map> by_level_deltas; - - struct Step - { - String path; - std::variant node; - uint32_t level; - }; - - enum class CollectStatus - { - Ok, - NoAuth, - LimitExceeded, - }; - - friend struct KeeperStorageRemoveRecursiveRequestProcessor; - - public: - ToDeleteTreeCollector(Storage & storage_, int64_t zxid_, int64_t session_id_, uint32_t limit_) - : storage(storage_) - , zxid(zxid_) - , session_id(session_id_) - , limit(limit_) - { - } - - CollectStatus collect(StringRef root_path, const SNode & root_node) - { - std::deque steps; - - if (checkLimits(&root_node)) - return CollectStatus::LimitExceeded; - - steps.push_back(Step{root_path.toString(), &root_node, 0}); - - while (!steps.empty()) - { - Step step = std::move(steps.front()); - steps.pop_front(); - - StringRef path = step.path; - uint32_t level = step.level; - const SNode * node_ptr = nullptr; - - if (auto * rdb = std::get_if(&step.node)) - node_ptr = rdb; - else - node_ptr = std::get(step.node); - - chassert(!path.empty()); - chassert(node_ptr != nullptr); - - const auto & node = *node_ptr; - auto actual_node_ptr = storage.uncommitted_state.getActualNodeView(path, node); - chassert(actual_node_ptr != nullptr); /// explicitly check that node is not deleted - - if (actual_node_ptr->numChildren() > 0 && !storage.checkACL(path, Coordination::ACL::Delete, session_id, /*is_local=*/false)) - return CollectStatus::NoAuth; - - if (auto status = visitRocksDBNode(steps, path, level); status != CollectStatus::Ok) - return status; - - if (auto status = visitMemNode(steps, path, level); status != CollectStatus::Ok) - return status; - - if (auto status = visitRootAndUncommitted(steps, path, node, level); status != CollectStatus::Ok) - return status; - } - - return CollectStatus::Ok; - } - - std::vector extractDeltas() - { - std::vector deltas; - - for (ssize_t level = max_level; level >= 0; --level) - { - auto & level_deltas = by_level_deltas[static_cast(level)]; - deltas.insert(deltas.end(), std::make_move_iterator(level_deltas.begin()), std::make_move_iterator(level_deltas.end())); - } - - return std::move(deltas); - } - - private: - CollectStatus visitRocksDBNode(std::deque & steps, StringRef root_path, uint32_t level) - { if constexpr (Storage::use_rocksdb) { - std::filesystem::path root_fs_path(root_path.toString()); - auto children = storage.container.getChildren(root_path.toString()); - - for (auto && [child_name, child_node] : children) - { - auto child_path = (root_fs_path / child_name).generic_string(); - const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node); - - /// if node was changed in previous step of multi transaction - skip until the uncommitted state visit - if (actual_child_node_ptr != &child_node) - continue; - - if (checkLimits(actual_child_node_ptr)) - return CollectStatus::LimitExceeded; - - steps.push_back(Step{std::move(child_path), std::move(child_node), level + 1}); - } + if (auto status = visitRocksDBNode(current_path, uncommitted_children); status != CollectStatus::Ok) + return status; } - - return CollectStatus::Ok; - } - - CollectStatus visitMemNode(std::deque & steps, StringRef root_path, uint32_t level) - { - if constexpr (!Storage::use_rocksdb) - { - auto node_it = storage.container.find(root_path); - if (node_it == storage.container.end()) - return CollectStatus::Ok; - - std::filesystem::path root_fs_path(root_path.toString()); - const auto & children = node_it->value.getChildren(); - - for (const auto & child_name : children) - { - auto child_path = (root_fs_path / child_name.toView()).generic_string(); - - auto child_it = storage.container.find(child_path); - chassert(child_it != storage.container.end()); - const auto & child_node = child_it->value; - - const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node); - - /// if node was changed in previous step of multi transaction - skip until the uncommitted state visit - if (actual_child_node_ptr != &child_node) - continue; - - if (checkLimits(actual_child_node_ptr)) - return CollectStatus::LimitExceeded; - - steps.push_back(Step{std::move(child_path), &child_node, level + 1}); - } - } - - return CollectStatus::Ok; - } - - CollectStatus visitRootAndUncommitted(std::deque & steps, StringRef root_path, const SNode & root_node, uint32_t level) - { - const auto & nodes = storage.uncommitted_state.nodes; - - /// nodes are sorted by paths with level locality - auto it = nodes.upper_bound(root_path.toString() + "/"); - - for (; it != nodes.end() && parentNodePath(it->first) == root_path; ++it) - { - const auto actual_child_node_ptr = it->second.node.get(); - - if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction - continue; - - if (checkLimits(actual_child_node_ptr)) - return CollectStatus::LimitExceeded; - - const String & child_path = it->first; - const SNode & child_node = *it->second.node; - - steps.push_back(Step{child_path, &child_node, level + 1}); - } - - addDelta(root_path, root_node, level); - - return CollectStatus::Ok; - } - - void addDelta(StringRef root_path, const SNode & root_node, uint32_t level) - { - max_level = std::max(max_level, level); - - by_level_deltas[level].emplace_back( - parentNodePath(root_path).toString(), - zxid, - typename Storage::UpdateNodeDelta{ - [](SNode & parent) - { - ++parent.cversion; - parent.decreaseNumChildren(); - } - }); - - by_level_deltas[level].emplace_back(root_path.toString(), zxid, typename Storage::RemoveNodeDelta{root_node.version, root_node.ephemeralOwner()}); - } - - bool checkLimits(const SNode * node) - { - chassert(node != nullptr); - nodes_observed += node->numChildren(); - return nodes_observed > limit; - } - }; -}; - -template -struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::vector - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override - { - ProfileEvents::increment(ProfileEvents::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, int64_t zxid) 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(zxid); 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, int64_t zxid) const override - { - return processImpl(storage, zxid); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override - { - ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); - return processImpl(storage, zxid); - } -}; - -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::vector - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override - { - ProfileEvents::increment(ProfileEvents::KeeperSetRequest); - Coordination::ZooKeeperSetRequest & request = dynamic_cast(*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 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->version) - return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; - - new_deltas.emplace_back( - request.path, - zxid, - typename Storage::UpdateNodeDelta{ - [zxid, data = request.data, time](typename Storage::Node & value) - { - value.version++; - value.mzxid = zxid; - value.mtime = time; - value.setData(data); - }, - request.version}); - - new_deltas.emplace_back( - parentNodePath(request.path).toString(), - zxid, - typename Storage::UpdateNodeDelta - { - [](Storage::Node & parent) - { - parent.cversion++; - } - } - ); - - digest = storage.calculateNodesDigest(digest, new_deltas); - return new_deltas; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) 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(zxid); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - - auto node_it = container.find(request.path); - if (node_it == container.end()) - onStorageInconsistency(); - - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; - - return response_ptr; - } - - KeeperStorageBase::ResponsesForSessions - processWatches(const Storage & /*storage*/, int64_t /*zxid*/, typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override - { - return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); - } -}; - -template -struct KeeperStorageListRequestProcessor 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::Read, 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 - { - 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, int64_t zxid) 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(zxid); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + if (auto status = visitMemNode(current_path, uncommitted_children); status != CollectStatus::Ok) + return status; } } - 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 - { - auto path_prefix = request.path; - if (path_prefix.empty()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty"); - - const auto & get_children = [&]() - { - if constexpr (Storage::use_rocksdb) - return container.getChildren(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 (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.isEphemeral(); - } - else - { - is_ephemeral = child.second.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; + return CollectStatus::Ok; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override + std::list extractDeltas() { - return processImpl(storage, zxid); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override - { - ProfileEvents::increment(ProfileEvents::KeeperListRequest); - return processImpl(storage, zxid); - } -}; - -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::vector - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override - { - ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); - - Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*this->zk_request); - - auto node = storage.uncommitted_state.getNode(request.path); - if (check_not_exists) - { - if (node && (request.version == -1 || request.version == node->version)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; - } - else - { - if (!node) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - if (request.version != -1 && request.version != node->version) - return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; - } - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) 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(zxid); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - } - - 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.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.version) - on_error(Coordination::Error::ZBADVERSION); - else - response.error = Coordination::Error::ZOK; - } - - return response_ptr; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override - { - return processImpl(storage, zxid); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override - { - ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); - return processImpl(storage, zxid); + return std::move(deltas); } private: - bool check_not_exists; -}; - - -template -struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor -{ - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + CollectStatus visitRocksDBNode(StringRef current_path, const UncommittedChildren & uncommitted_children) requires Storage::use_rocksdb { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin, session_id, is_local); - } + std::filesystem::path current_path_fs(current_path.toString()); - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::vector - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override - { - Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*this->zk_request); - - if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) + std::vector> children; { - 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}}; + std::lock_guard lock(storage.storage_mutex); + children = storage.container.getChildren(current_path.toString(), /*read_data=*/true); } - auto & uncommitted_state = storage.uncommitted_state; - if (!uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - auto node = uncommitted_state.getNode(request.path); - - if (request.version != -1 && request.version != node->aversion) - return {typename Storage::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}}; - - std::vector new_deltas + for (auto && [child_name, child_node] : children) { - { - request.path, - zxid, - typename Storage::SetACLDelta{std::move(node_acls), request.version} - }, - { - request.path, - zxid, - typename Storage::UpdateNodeDelta - { - [](typename Storage::Node & n) { ++n.aversion; } - } - } - }; + auto child_path = (current_path_fs / child_name).generic_string(); - digest = storage.calculateNodesDigest(digest, new_deltas); + if (uncommitted_children.contains(child_path)) + continue; - return new_deltas; - } + if (checkLimits(child_node)) + return CollectStatus::LimitExceeded; - Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) 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); - - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; + addDelta(child_path, child_node.stats, storage.acl_map.convertNumber(child_node.acl_id), std::string{child_node.getData()}); } - auto node_it = storage.container.find(request.path); + return CollectStatus::Ok; + } + + CollectStatus visitMemNode(StringRef current_path, const UncommittedChildren & uncommitted_children) requires (!Storage::use_rocksdb) + { + std::lock_guard lock(storage.storage_mutex); + auto node_it = storage.container.find(current_path); if (node_it == storage.container.end()) - onStorageInconsistency(); - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + return CollectStatus::Ok; - return response_ptr; + std::filesystem::path current_path_fs(current_path.toString()); + const auto & children = node_it->value.getChildren(); + + for (const auto & child_name : children) + { + auto child_path = (current_path_fs / child_name.toView()).generic_string(); + + if (uncommitted_children.contains(child_path)) + continue; + + auto child_it = storage.container.find(child_path); + chassert(child_it != storage.container.end()); + const auto & child_node = child_it->value; + + if (checkLimits(child_node)) + return CollectStatus::LimitExceeded; + + addDelta(child_path, child_node.stats, storage.acl_map.convertNumber(child_node.acl_id), std::string{child_node.getData()}); + } + + return CollectStatus::Ok; + } + + CollectStatus visitUncommitted(const std::string & path, UncommittedChildren & uncommitted_children) + { + auto & nodes = storage.uncommitted_state.nodes; + + for (auto nodes_it = nodes.upper_bound(path + "/"); nodes_it != nodes.end() && parentNodePath(nodes_it->first) == path; ++nodes_it) + { + const auto actual_child_node_ptr = nodes_it->second.node.get(); + + uncommitted_children.insert(nodes_it->first); + + if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction + continue; + + if (checkLimits(*actual_child_node_ptr)) + return CollectStatus::LimitExceeded; + + nodes_it->second.materializeACL(storage.acl_map); + addDelta(nodes_it->first, actual_child_node_ptr->stats, *nodes_it->second.acls, std::string{actual_child_node_ptr->getData()}); + } + + return CollectStatus::Ok; + } + + void addDelta(StringRef path, const NodeStats & stats, Coordination::ACLs acls, std::string data) + { + deltas.emplace_front(std::string{path}, zxid, KeeperStorageBase::RemoveNodeDelta{/*version=*/-1, stats, std::move(acls), std::move(data)}); + } + + bool checkLimits(const Storage::Node & node) + { + nodes_observed += node.stats.numChildren(); + return nodes_observed > limit; } }; +} -template -struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor +template +bool checkAuth(const Coordination::ZooKeeperRemoveRecursiveRequest & 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(parentNodePath(zk_request.getPath()), Coordination::ACL::Delete, session_id, is_local); +} + +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperRemoveRecursiveRequest & /*zk_request*/, + KeeperStorageBase::DeltaRange deltas, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) +{ + KeeperStorageBase::ResponsesForSessions responses; + for (const auto & delta : deltas) { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local); + const auto * remove_delta = std::get_if(&delta.operation); + if (remove_delta) + { + 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())); + } } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + return responses; +} - std::vector - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override +template +std::list preprocess( + const Coordination::ZooKeeperRemoveRecursiveRequest & 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) { - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); + auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", zk_request.path); - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - return {}; + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const + auto node = storage.uncommitted_state.getNode(zk_request.path); + + auto parent_path = parentNodePath(zk_request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + + std::optional update_parent_delta; + if (parent_node) + update_parent_delta.emplace(*parent_node); + + const auto add_parent_update_delta = [&] { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); + if (!update_parent_delta) + return; - if constexpr (!local) + new_deltas.emplace_back( + std::string{parent_path}, + zxid, + std::move(*update_parent_delta) + ); + }; + + const auto update_parent_pzxid = [&]() + { + if (!update_parent_delta) + return; + + if (update_parent_delta->old_stats.pzxid < zxid) + update_parent_delta->new_stats.pzxid = zxid; + }; + + if (!node) + { + if (zk_request.restored_from_zookeeper_log) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } + update_parent_pzxid(); + add_parent_update_delta(); } - auto & container = storage.container; - auto node_it = container.find(request.path); - if (node_it == container.end()) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + } + + ToDeleteTreeCollector collector(storage, zxid, session_id, zk_request.remove_nodes_limit); + auto collect_status = collector.collect(zk_request.path, *node); + + if (collect_status == ToDeleteTreeCollector::CollectStatus::NoAuth) + return {typename Storage::Delta{zxid, Coordination::Error::ZNOAUTH}}; + + if (collect_status == ToDeleteTreeCollector::CollectStatus::LimitExceeded) + return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; + + if (zk_request.restored_from_zookeeper_log) + { + update_parent_pzxid(); + } + + chassert(update_parent_delta); + ++update_parent_delta->new_stats.cversion; + update_parent_delta->new_stats.decreaseNumChildren(); + add_parent_update_delta(); + + 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->stat.ephemeralOwner()) { - if constexpr (local) - response.error = Coordination::Error::ZNONODE; - else - onStorageInconsistency(); + unregisterEphemeralPath( + storage.uncommitted_state.ephemerals, remove_delta->stat.ephemeralOwner(), delta.path, /*throw_if_missing=*/false); } + } + + new_deltas.splice(new_deltas.end(), std::move(delete_deltas)); + + digest = storage.calculateNodesDigest(digest, new_deltas); + + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperRemoveRecursiveRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + response_ptr->error = storage.commit(std::move(deltas)); + return response_ptr; +} + +/// REMOVERECURSIVE Request /// + +/// EXISTS Request /// +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, KeeperStorageBase::DeltaRange 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; + } + } + + 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 - { - node_it->value.setResponseStat(response.stat); - response.acl = storage.acl_map.convertNumber(node_it->value.acl_id); - } - - return response_ptr; + onStorageInconsistency("Node unexpectedly missing while checking if it exists"); } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override + else { - return processImpl(storage, zxid); + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override - { - return processImpl(storage, zxid); - } -}; + return response; +} -template -struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) { - using OperationType = Coordination::ZooKeeperMultiRequest::OperationType; - std::optional operation_type; + return processImpl(zk_request, storage, std::move(deltas)); +} - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange 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::DeltaRange /*deltas*/, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) +{ + return processWatchesImpl(zk_request.getPath(), watches, list_watches, sessions_and_watchers, 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) { - for (const auto & concrete_request : concrete_requests) - if (!concrete_request->checkAuth(storage, session_id, is_local)) - return false; - return true; + 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}}; } - std::vector>> concrete_requests; - explicit KeeperStorageMultiRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequestProcessor(zk_request_) - { - Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*this->zk_request); - concrete_requests.reserve(request.requests.size()); + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - const auto check_operation_type = [&](OperationType type) + 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}}; + + 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)); + + 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, KeeperStorageBase::DeltaRange deltas) +{ + auto & container = storage.container; + + auto response = std::make_shared(); + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + { + response->error = result; + return response; + } + + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) + onStorageInconsistency("Node to be updated is missing"); + + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; + + return response; +} +/// SET Request /// + +/// LIST Request /// +template +bool checkAuth(const Coordination::ZooKeeperListRequest & 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::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, KeeperStorageBase::DeltaRange deltas) +{ + std::shared_ptr response = zk_request.getOpNum() == Coordination::OpNum::SimpleList + ? std::make_shared() + : std::make_shared(); + + if constexpr (!local) + { + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - 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; + response->error = result; + return response; + } + } + + 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("Failed to list children of node because it's missing"); + } + 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 = [&]() + { + if constexpr (Storage::use_rocksdb) + return std::optional{container.getChildren(zk_request.path)}; + else + return &node_it->value.getChildren(); }; - for (const auto & sub_request : request.requests) + const auto children = get_children(); + response->names.reserve(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()) { - auto sub_zk_request = std::dynamic_pointer_cast(sub_request); - switch (sub_zk_request->getOpNum()) + 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) + { + using enum Coordination::ListRequestType; + + auto list_request_type = ALL; + if (const auto * filtered_list = dynamic_cast(&zk_request)) { - 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::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)); - 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()); + 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("Failed to find a child"); + 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()); } } - chassert(request.requests.empty() || operation_type.has_value()); + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; } - std::vector - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override - { - ProfileEvents::increment(ProfileEvents::KeeperMultiRequest); - std::vector response_errors; - response_errors.reserve(concrete_requests.size()); - uint64_t current_digest = digest; - 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); + return response; +} - if (!new_deltas.empty()) - { - if (auto * error = std::get_if(&new_deltas.back().operation); - error && *operation_type == OperationType::Write) - { - storage.uncommitted_state.rollback(zxid); - 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_deltas.emplace_back(zxid, typename Storage::SubDeltaEnd{}); - response_errors.push_back(Coordination::Error::ZOK); - - // manually add deltas so that the result of previous request in the transaction is used in the next request - storage.uncommitted_state.addDeltas(std::move(new_deltas)); - } - - digest = current_digest; - - return {}; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); - - auto & deltas = storage.uncommitted_state.deltas; - // the deltas will have at least SubDeltaEnd or FailedMultiDelta - chassert(!deltas.empty()); - if (auto * failed_multi = std::get_if(&deltas.front().operation)) - { - 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 = failed_multi->global_error; - 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); - } - - response.error = Coordination::Error::ZOK; - return response_ptr; - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override - { - 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) - { - response.responses[i] = concrete_requests[i]->processLocal(storage, zxid); - } - - response.error = Coordination::Error::ZOK; - return response_ptr; - } - - KeeperStorageBase::ResponsesForSessions - processWatches(const Storage & storage, int64_t zxid, 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(storage, zxid, watches, list_watches); - result.insert(result.end(), responses.begin(), responses.end()); - } - return result; - } -}; - -template -struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - Coordination::ZooKeeperResponsePtr process(Storage &, int64_t) const override - { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Called process on close request"); - } -}; + return processImpl(zk_request, storage, std::move(deltas)); +} -template -struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + ProfileEvents::increment(ProfileEvents::KeeperListRequest); + return processImpl(zk_request, storage, std::move(deltas)); +} +/// LIST Request /// - std::vector - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override +/// 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) { - Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast(*this->zk_request); - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); + if (node && (zk_request.version == -1 || zk_request.version == node->stats.version)) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; + } + else + { + if (!node) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - if (auth_request.scheme != "digest" || std::count(auth_request.data.begin(), auth_request.data.end(), ':') != 1) - return {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; + if (zk_request.version != -1 && zk_request.version != node->stats.version) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; + } - std::vector new_deltas; - auto auth_digest = Storage::generateDigest(auth_request.data); - if (auth_digest == storage.superdigest) + return {}; +} + +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + 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) { - typename Storage::AuthID auth{"super", ""}; - new_deltas.emplace_back(zxid, typename Storage::AddAuthDelta{session_id, std::move(auth)}); + response->error = result; + return response; } + } + + const auto on_error = [&]([[maybe_unused]] const auto error_code) + { + if constexpr (local) + response->error = error_code; else + onStorageInconsistency("Node to check is unexpectedly missing"); + }; + + auto & container = storage.container; + auto node_it = container.find(zk_request.path); + + if (zk_request.getOpNum() == Coordination::OpNum::CheckNotExists) + { + 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; + } + else + { + if (node_it == container.end()) + 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; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange 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()) { - typename Storage::AuthID new_auth{auth_request.scheme, auth_digest}; - if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return new_auth == auth_id; })) - new_deltas.emplace_back(zxid, typename Storage::AddAuthDelta{session_id, std::move(new_auth)}); + 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)}}}; + } } - return new_deltas; + 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)); } - Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override + digest = current_digest; + storage.uncommitted_state.addDeltas(std::move(new_deltas)); + return {}; +} + +KeeperStorageBase::DeltaRange extractSubdeltas(KeeperStorageBase::DeltaRange & deltas) +{ + std::list subdeltas; + auto it = deltas.begin(); + + for (; it != deltas.end(); ++it) { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); - - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) - auth_response.error = result; - - return response_ptr; + if (std::holds_alternative(it->operation)) + { + ++it; + break; + } } -}; + + KeeperStorageBase::DeltaRange result{.begin_it = deltas.begin(), .end_it = it}; + deltas.begin_it = it; + return result; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + 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; + + // 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.push_back(std::make_shared()); + response->responses[i]->error = failed_multi->error_codes[i]; + } + + response->error = failed_multi->global_error; + return response; + } + + for (const auto & multi_subrequest : subrequests) + { + auto subdeltas = extractSubdeltas(deltas); + response->responses.push_back(callOnConcreteRequestType( + *multi_subrequest, [&](const auto & subrequest) { return process(subrequest, storage, std::move(subdeltas)); })); + } + + response->error = Coordination::Error::ZOK; + return response; +} + +template +Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); + auto response = std::make_shared(); + response->responses.reserve(zk_request.requests.size()); + + for (const auto & multi_subrequest : zk_request.requests) + { + auto subdeltas = extractSubdeltas(deltas); + response->responses.push_back(callOnConcreteRequestType( + *multi_subrequest, [&](const auto & subrequest) { return processLocal(subrequest, storage, std::move(subdeltas)); })); + } + + response->error = Coordination::Error::ZOK; + return response; +} + +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperMultiRequest & zk_request, + KeeperStorageBase::DeltaRange deltas, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) +{ + KeeperStorageBase::ResponsesForSessions result; + + const auto & subrequests = zk_request.requests; + for (const auto & generic_request : subrequests) + { + auto subdeltas = extractSubdeltas(deltas); + auto responses = callOnConcreteRequestType( + *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; +} +/// MULTI Request /// + +/// 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*/) +{ + if (zk_request.scheme != "digest" || std::count(zk_request.data.begin(), zk_request.data.end(), ':') != 1) + return {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; + + 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)}); + } + + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperAuthRequest & /*zk_request*/, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + auto response = std::make_shared(); + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + response->error = result; + + return response; +} +/// AUTH Request /// + +/// CLOSE Request /// +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperCloseRequest & /* zk_request */, Storage &, KeeperStorageBase::DeltaRange /* 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) + { + 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}}; + } + + 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{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); + + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperSetACLRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + auto response = std::make_shared(); + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + { + response->error = result; + return response; + } + + auto node_it = storage.container.find(zk_request.path); + if (node_it == storage.container.end()) + onStorageInconsistency("Failed to set ACL because node is missing"); + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; + + return response; +} +/// 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, KeeperStorageBase::DeltaRange 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; + } + } + + 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("Failed to get ACL because node is missing"); + } + else + { + node_it->value.setResponseStat(response->stat); + response->acl = storage.acl_map.convertNumber(node_it->value.acl_id); + } + + return response; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, KeeperStorageBase::DeltaRange deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} +/// GETACL Request /// template void KeeperStorage::finalize() @@ -2651,11 +2819,12 @@ void KeeperStorage::finalize() finalized = true; - ephemerals.clear(); + committed_ephemerals.clear(); watches.clear(); list_watches.clear(); sessions_and_watchers.clear(); + session_expiry_queue.clear(); } @@ -2665,75 +2834,8 @@ 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); - registerKeeperRequestProcessor>(*this); -} - - template -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; @@ -2761,7 +2863,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, con updated_nodes.insert_or_assign(delta.path, nullptr); }, - [&](const UpdateNodeDelta & update_delta) + [&](const UpdateNodeStatDelta & update_delta) { std::shared_ptr node{nullptr}; @@ -2776,7 +2878,24 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, con else node = updated_node_it->second; - update_delta.update_fn(*node); + node->stats = update_delta.new_stats; + }, + [&](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); @@ -2821,106 +2940,169 @@ 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))) + std::lock_guard lock(transaction_mutex); + int64_t last_zxid = getNextZXIDLocked() - 1; + auto current_digest = getNodesDigest(false, /*lock_transaction_mutex=*/false); + + if (uncommitted_transactions.empty()) { - 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 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, current_digest)) + { + 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( + ErrorCodes::LOGICAL_ERROR, + "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", + new_last_zxid, last_zxid); } - 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); + new_digest = current_digest.value; + transaction = &uncommitted_transactions.emplace_back(TransactionInfo{.zxid = new_last_zxid, .nodes_digest = {}, .log_idx = log_idx}); } - std::vector new_deltas; - TransactionInfo transaction{.zxid = new_last_zxid, .nodes_digest = {}, .log_idx = log_idx}; - 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)); - }); - auto request_processor = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); + uncommitted_state.cleanup(getZXID()); + }); if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { - auto session_ephemerals = ephemerals.find(session_id); - if (session_ephemerals != ephemerals.end()) + std::unordered_map< + std::string, + UpdateNodeStatDelta, + StringHashForHeterogeneousLookup, + StringHashForHeterogeneousLookup::transparent_key_equal> + parent_updates; + + const auto process_ephemerals_for_session + = [&](const auto & current_ephemerals, auto & processed_ephemeral_nodes, bool check_processed_nodes) { - for (const auto & ephemeral_path : session_ephemerals->second) + auto session_ephemerals = current_ephemerals.find(session_id); + if (session_ephemerals != current_ephemerals.end()) { - new_deltas.emplace_back - ( - parentNodePath(ephemeral_path).toString(), - new_last_zxid, - UpdateNodeDelta + for (const auto & ephemeral_path : session_ephemerals->second) + { + if (check_processed_nodes) { - [ephemeral_path](Node & parent) - { - ++parent.cversion; - parent.decreaseNumChildren(); - } + if (processed_ephemeral_nodes.contains(ephemeral_path)) + continue; + } + else + { + processed_ephemeral_nodes.insert(ephemeral_path); } - ); - new_deltas.emplace_back(ephemeral_path, transaction.zxid, RemoveNodeDelta{.ephemeral_owner = session_id}); + 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()}}); + } } + }; - ephemerals.erase(session_ephemerals); + { + /// 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, processed_ephemeral_nodes, /*check_processed_nodes=*/true); } - new_deltas.emplace_back(transaction.zxid, CloseSessionDelta{session_id}); + 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; } - if (check_acl && !request_processor->checkAuth(*this, session_id, false)) + const auto preprocess_request = [&] T>(const T & concrete_zk_request) { - /// Multi requests handle failures using FailedMultiDelta - if (zk_request->getOpNum() == Coordination::OpNum::Multi || zk_request->getOpNum() == Coordination::OpNum::MultiRead) + if (check_acl && !checkAuth(concrete_zk_request, *this, session_id, false)) { - const auto & multi_request = dynamic_cast(*zk_request); - std::vector response_errors; - response_errors.resize(multi_request.requests.size(), Coordination::Error::ZOK); - uncommitted_state.deltas.emplace_back( - new_last_zxid, KeeperStorage::FailedMultiDelta{std::move(response_errors), Coordination::Error::ZNOAUTH}); + /// Multi requests handle failures using FailedMultiDelta + if (zk_request->getOpNum() == Coordination::OpNum::Multi || zk_request->getOpNum() == Coordination::OpNum::MultiRead) + { + const auto & multi_request = dynamic_cast(*zk_request); + std::vector response_errors; + response_errors.resize(multi_request.requests.size(), Coordination::Error::ZOK); + new_deltas.emplace_back( + new_last_zxid, KeeperStorage::FailedMultiDelta{std::move(response_errors), Coordination::Error::ZNOAUTH}); + } + else + { + new_deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); + } + return; } - else - { - uncommitted_state.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); } template @@ -2948,23 +3130,43 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process 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); - auto & front_transaction = uncommitted_transactions.front(); - if (front_transaction.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 (auto & front_transaction = uncommitted_transactions.front(); + front_transaction.zxid != *new_last_zxid) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Trying to commit a ZXID {} while the next ZXID to commit is {}", + *new_last_zxid, + front_transaction.zxid); - zxid = *new_last_zxid; - uncommitted_transactions.pop_front(); + commit_zxid = *new_last_zxid; + } + else + { + commit_zxid = zxid; + } } + std::list deltas; + if (!is_local) + { + 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); + } + + KeeperStorageBase::DeltaRange deltas_range{.begin_it = deltas.begin(), .end_it = deltas.end()}; + ResponsesForSessions results; /// ZooKeeper update sessions expirity for each request, not only for heartbeats @@ -2972,109 +3174,141 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process 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); + auto responses = processWatchesImpl(delta.path, watches, list_watches, sessions_and_watchers, Coordination::Event::DELETED); results.insert(results.end(), responses.begin(), responses.end()); } } + { + std::lock_guard lock(storage_mutex); + commit(deltas_range); + } + { + std::lock_guard lock(auth_mutex); + auto auth_it = committed_session_and_auth.find(session_id); + if (auth_it != committed_session_and_auth.end()) + committed_session_and_auth.erase(auth_it); + } + 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); /// 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}); } else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special { - auto storage_request = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); - auto response = storage_request->process(*this, zxid); + Coordination::ZooKeeperResponsePtr response = nullptr; + { + std::lock_guard lock(storage_mutex); + response = process(dynamic_cast(*zk_request), *this, deltas_range); + } response->xid = zk_request->xid; - response->zxid = getZXID(); + response->zxid = commit_zxid; results.push_back(ResponseForSession{session_id, response}); } 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, deltas_range); + } } else { - response = request_processor->processLocal(*this, zxid); + std::lock_guard lock(storage_mutex); + response = process(concrete_zk_request, *this, deltas_range); } - } - else - { - response = request_processor->process(*this, zxid); - } - /// 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 is_list_watch + = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end(); + + 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(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 [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(WatchInfo{.path = watch_it->first, .is_list_watch = false}); + } + } + } + + /// 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, deltas_range, watches, list_watches, sessions_and_watchers); + total_watches_count -= watch_responses.size(); + 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(*this, zxid, 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 = getZXID(); + results.push_back(ResponseForSession{session_id, response}); + }; - results.push_back(ResponseForSession{session_id, response}); + callOnConcreteRequestType(*zk_request, process_request); + } + + updateStats(); + + { + 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; } - uncommitted_state.commit(zxid); return results; } template -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 { if (allow_missing && (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid < rollback_zxid)) return; @@ -3101,13 +3335,28 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow } template -KeeperStorageBase::Digest KeeperStorage::getNodesDigest(bool committed) const +KeeperStorageBase::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::shared_lock 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::shared_lock storage_lock(storage_mutex); + return {CURRENT_DIGEST_VERSION, nodes_digest}; + } return uncommitted_transactions.back().nodes_digest; } @@ -3129,38 +3378,120 @@ void KeeperStorage::addDigest(const Node & node, const std::string_vi } } +/// Allocate new session id with the specified timeouts +template +int64_t KeeperStorage::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; +} + +/// Add session id. Used when restoring KeeperStorage from snapshot. +template +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); +} + +template +std::vector KeeperStorage::getDeadSessions() const +{ + return session_expiry_queue.getExpiredSessions(); +} + +template +SessionAndTimeout KeeperStorage::getActiveSessions() const +{ + return session_and_timeout; +} + +/// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. +template +void KeeperStorage::enableSnapshotMode(size_t up_to_version) +{ + container.enableSnapshotMode(up_to_version); +} + +/// Turn off snapshot mode. +template +void KeeperStorage::disableSnapshotMode() +{ + container.disableSnapshotMode(); +} + +template +KeeperStorage::Container::const_iterator KeeperStorage::getSnapshotIteratorBegin() const +{ + return container.begin(); +} + +/// Clear outdated data from internal container. +template +void KeeperStorage::clearGarbageAfterSnapshot() +{ + container.clearOutdatedNodes(); +} + +/// Introspection functions mostly used in 4-letter commands +template +uint64_t KeeperStorage::getNodesCount() const +{ + return container.size(); +} + +template +uint64_t KeeperStorage::getApproximateDataSize() const +{ + return container.getApproximateDataSize(); +} + +template +uint64_t KeeperStorage::getArenaDataSize() const +{ + return container.keyArenaSize(); +} + +template +uint64_t KeeperStorage::getWatchedPathsCount() const +{ + return watches.size() + list_watches.size(); +} + template 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()) + if (watches_it == sessions_and_watchers.end()) + return; + + for (const auto [watch_path, is_list_watch] : watches_it->second) { - for (const auto & watch_path : watches_it->second) + if (is_list_watch) { - /// 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()) - { - 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); - } + 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); } - - sessions_and_watchers.erase(watches_it); } + + total_watches_count -= watches_it->second.size(); + sessions_and_watchers.erase(watches_it); } template @@ -3169,7 +3500,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"; } } @@ -3217,21 +3548,42 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnStrin } buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCount() << "):\n"; - for (const auto & [session_id, ephemeral_paths] : ephemerals) + for (const auto & [session_id, ephemeral_paths] : committed_ephemerals) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; write_str_set(ephemeral_paths); } } +template +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); +} + +template +const KeeperStorageBase::Stats & KeeperStorage::getStorageStats() const +{ + return stats; +} + template uint64_t KeeperStorage::getTotalWatchesCount() const { - uint64_t ret = 0; - for (const auto & [session, paths] : sessions_and_watchers) - ret += paths.size(); + return total_watches_count; +} - return ret; +template +uint64_t KeeperStorage::getSessionWithEphemeralNodesCount() const +{ + return committed_ephemerals.size(); } template @@ -3243,11 +3595,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 ec502a38a72..7366782620a 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -1,10 +1,16 @@ #pragma once #include +#include #include #include #include #include +#include "Common/StringHashForHeterogeneousLookup.h" +#include +#include + +#include #include @@ -23,14 +29,11 @@ using ResponseCallback = std::function; using SessionAndTimeout = std::unordered_map; -/// KeeperRocksNodeInfo is used in RocksDB keeper. -/// It is serialized directly as POD to RocksDB. -struct KeeperRocksNodeInfo +struct NodeStats { int64_t czxid{0}; int64_t mzxid{0}; int64_t pzxid{0}; - uint64_t acl_id = 0; /// 0 -- no ACL by default int64_t mtime{0}; @@ -38,225 +41,9 @@ struct KeeperRocksNodeInfo int32_t cversion{0}; int32_t aversion{0}; - int32_t seq_num = 0; - mutable UInt64 digest = 0; /// we cached digest for this node. - - /// as ctime can't be negative because it stores the timestamp when the - /// node was created, we can use the MSB for a bool - struct - { - bool is_ephemeral : 1; - int64_t ctime : 63; - } is_ephemeral_and_ctime{false, 0}; - - /// ephemeral notes cannot have children so a node can set either - /// ephemeral_owner OR seq_num + num_children - union - { - int64_t ephemeral_owner; - struct - { - int32_t seq_num; - int32_t num_children; - } children_info; - } ephemeral_or_children_data{0}; - - bool isEphemeral() const - { - return is_ephemeral_and_ctime.is_ephemeral; - } - - int64_t ephemeralOwner() const - { - if (isEphemeral()) - return ephemeral_or_children_data.ephemeral_owner; - - return 0; - } - - void setEphemeralOwner(int64_t ephemeral_owner) - { - is_ephemeral_and_ctime.is_ephemeral = ephemeral_owner != 0; - ephemeral_or_children_data.ephemeral_owner = ephemeral_owner; - } - - int32_t numChildren() const - { - if (isEphemeral()) - return 0; - - return ephemeral_or_children_data.children_info.num_children; - } - - void setNumChildren(int32_t num_children) - { - ephemeral_or_children_data.children_info.num_children = num_children; - } - - /// dummy interface for test - void addChild(StringRef) {} - auto getChildren() const - { - return std::vector(numChildren()); - } - - void increaseNumChildren() - { - chassert(!isEphemeral()); - ++ephemeral_or_children_data.children_info.num_children; - } - - void decreaseNumChildren() - { - chassert(!isEphemeral()); - --ephemeral_or_children_data.children_info.num_children; - } - - int32_t seqNum() const - { - if (isEphemeral()) - return 0; - - return ephemeral_or_children_data.children_info.seq_num; - } - - void setSeqNum(int32_t seq_num_) - { - ephemeral_or_children_data.children_info.seq_num = seq_num_; - } - - void increaseSeqNum() - { - chassert(!isEphemeral()); - ++ephemeral_or_children_data.children_info.seq_num; - } - - int64_t ctime() const - { - return is_ephemeral_and_ctime.ctime; - } - - void setCtime(uint64_t ctime) - { - is_ephemeral_and_ctime.ctime = ctime; - } + uint32_t data_size{0}; void copyStats(const Coordination::Stat & stat); -}; - -/// KeeperRocksNode is the memory structure used by RocksDB -struct KeeperRocksNode : public KeeperRocksNodeInfo -{ -#if USE_ROCKSDB - friend struct RocksDBContainer; -#endif - using Meta = KeeperRocksNodeInfo; - - uint64_t size_bytes = 0; // only for compatible, should be deprecated - - uint64_t sizeInBytes() const { return data_size + sizeof(KeeperRocksNodeInfo); } - void setData(String new_data) - { - data_size = static_cast(new_data.size()); - if (data_size != 0) - { - data = std::unique_ptr(new char[new_data.size()]); - memcpy(data.get(), new_data.data(), data_size); - } - } - - void shallowCopy(const KeeperRocksNode & other) - { - czxid = other.czxid; - mzxid = other.mzxid; - pzxid = other.pzxid; - acl_id = other.acl_id; /// 0 -- no ACL by default - - mtime = other.mtime; - - is_ephemeral_and_ctime = other.is_ephemeral_and_ctime; - - ephemeral_or_children_data = other.ephemeral_or_children_data; - - data_size = other.data_size; - if (data_size != 0) - { - data = std::unique_ptr(new char[data_size]); - memcpy(data.get(), other.data.get(), data_size); - } - - version = other.version; - cversion = other.cversion; - aversion = other.aversion; - - /// cached_digest = other.cached_digest; - } - void invalidateDigestCache() const; - UInt64 getDigest(std::string_view path) const; - String getEncodedString(); - void decodeFromString(const String & buffer_str); - void recalculateSize() {} - std::string_view getData() const noexcept { return {data.get(), data_size}; } - - void setResponseStat(Coordination::Stat & response_stat) const - { - response_stat.czxid = czxid; - response_stat.mzxid = mzxid; - response_stat.ctime = ctime(); - response_stat.mtime = mtime; - response_stat.version = version; - response_stat.cversion = cversion; - response_stat.aversion = aversion; - response_stat.ephemeralOwner = ephemeralOwner(); - response_stat.dataLength = static_cast(data_size); - response_stat.numChildren = numChildren(); - response_stat.pzxid = pzxid; - } - - void reset() - { - serialized = false; - } - bool empty() const - { - return data_size == 0 && mzxid == 0; - } - std::unique_ptr data{nullptr}; - uint32_t data_size{0}; -private: - bool serialized = false; -}; - -/// KeeperMemNode should have as minimal size as possible to reduce memory footprint -/// of stored nodes -/// New fields should be added to the struct only if it's really necessary -struct KeeperMemNode -{ - int64_t czxid{0}; - int64_t mzxid{0}; - int64_t pzxid{0}; - uint64_t acl_id = 0; /// 0 -- no ACL by default - - int64_t mtime{0}; - - std::unique_ptr data{nullptr}; - uint32_t data_size{0}; - - int32_t version{0}; - int32_t cversion{0}; - int32_t aversion{0}; - - mutable uint64_t cached_digest = 0; - - KeeperMemNode() = default; - - KeeperMemNode & operator=(const KeeperMemNode & other); - KeeperMemNode(const KeeperMemNode & other); - - KeeperMemNode & operator=(KeeperMemNode && other) noexcept; - KeeperMemNode(KeeperMemNode && other) noexcept; - - bool empty() const; bool isEphemeral() const { @@ -287,6 +74,7 @@ struct KeeperMemNode void setNumChildren(int32_t num_children) { + is_ephemeral_and_ctime.is_ephemeral = false; ephemeral_or_children_data.children_info.num_children = num_children; } @@ -331,34 +119,6 @@ struct KeeperMemNode is_ephemeral_and_ctime.ctime = ctime; } - void copyStats(const Coordination::Stat & stat); - - void setResponseStat(Coordination::Stat & response_stat) const; - - /// Object memory size - uint64_t sizeInBytes() const; - - void setData(const String & new_data); - - std::string_view getData() const noexcept { return {data.get(), data_size}; } - - void addChild(StringRef child_path); - - void removeChild(StringRef child_path); - - const auto & getChildren() const noexcept { return children; } - auto & getChildren() { return children; } - - // Invalidate the calculated digest so it's recalculated again on the next - // getDigest call - void invalidateDigestCache() const; - - // get the calculated digest of the node - UInt64 getDigest(std::string_view path) const; - - // copy only necessary information for preprocessing and digest calculation - // (e.g. we don't need to copy list of children) - void shallowCopy(const KeeperMemNode & other); private: /// as ctime can't be negative because it stores the timestamp when the /// node was created, we can use the MSB for a bool @@ -379,7 +139,132 @@ private: int32_t num_children; } children_info; } ephemeral_or_children_data{0}; +}; +/// KeeperRocksNodeInfo is used in RocksDB keeper. +/// It is serialized directly as POD to RocksDB. +struct KeeperRocksNodeInfo +{ + NodeStats stats; + uint64_t acl_id = 0; /// 0 -- no ACL by default + + /// dummy interface for test + void addChild(StringRef) {} + auto getChildren() const + { + return std::vector(stats.numChildren()); + } + + void copyStats(const Coordination::Stat & stat); +}; + +/// KeeperRocksNode is the memory structure used by RocksDB +struct KeeperRocksNode : public KeeperRocksNodeInfo +{ +#if USE_ROCKSDB + friend struct RocksDBContainer; +#endif + using Meta = KeeperRocksNodeInfo; + + uint64_t size_bytes = 0; // only for compatible, should be deprecated + + uint64_t sizeInBytes() const { return stats.data_size + sizeof(KeeperRocksNodeInfo); } + + void setData(String new_data) + { + stats.data_size = static_cast(new_data.size()); + if (stats.data_size != 0) + { + data = std::unique_ptr(new char[new_data.size()]); + memcpy(data.get(), new_data.data(), stats.data_size); + } + } + + 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]); + memcpy(data.get(), other.data.get(), stats.data_size); + } + + /// cached_digest = other.cached_digest; + } + void invalidateDigestCache() const; + UInt64 getDigest(std::string_view path) const; + String getEncodedString(); + void decodeFromString(const String & buffer_str); + void recalculateSize() {} + std::string_view getData() const noexcept { return {data.get(), stats.data_size}; } + + void setResponseStat(Coordination::Stat & response_stat) const; + + void reset() + { + serialized = false; + } + bool empty() const + { + return stats.data_size == 0 && stats.mzxid == 0; + } + std::unique_ptr data{nullptr}; + mutable UInt64 cached_digest = 0; /// we cached digest for this node. +private: + bool serialized = false; +}; + +/// KeeperMemNode should have as minimal size as possible to reduce memory footprint +/// of stored nodes +/// New fields should be added to the struct only if it's really necessary +struct KeeperMemNode +{ + NodeStats stats; + 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); + KeeperMemNode(const KeeperMemNode & other); + + KeeperMemNode & operator=(KeeperMemNode && other) noexcept; + KeeperMemNode(KeeperMemNode && other) noexcept; + + bool empty() const; + + void copyStats(const Coordination::Stat & stat); + + void setResponseStat(Coordination::Stat & response_stat) const; + + /// Object memory size + uint64_t sizeInBytes() const; + + void setData(const String & new_data); + + std::string_view getData() const noexcept { return {data.get(), stats.data_size}; } + + void addChild(StringRef child_path); + + void removeChild(StringRef child_path); + + const auto & getChildren() const noexcept { return children; } + auto & getChildren() { return children; } + + // Invalidate the calculated digest so it's recalculated again on the next + // getDigest call + void invalidateDigestCache() const; + + // get the calculated digest of the node + UInt64 getDigest(std::string_view path) const; + + // copy only necessary information for preprocessing and digest calculation + // (e.g. we don't need to copy list of children) + void shallowCopy(const KeeperMemNode & other); +private: ChildrenSet children{}; }; @@ -430,18 +315,187 @@ 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, + // generates deltas with those changes, denoted with the request ZXID + // - processing which applies deltas with the correct ZXID to the storage + // + // Delta objects allow us two things: + // - fetch the latest, uncommitted state of an object by getting the committed + // state of that same object from the storage and applying the deltas + // in the same order as they are defined + // - quickly commit the changes to the storage + struct CreateNodeDelta + { + Coordination::Stat stat; + Coordination::ACLs acls; + String data; + }; + + struct RemoveNodeDelta + { + int32_t version{-1}; + NodeStats stat; + Coordination::ACLs acls; + String data; + }; + + struct UpdateNodeStatDelta + { + template Node> + explicit UpdateNodeStatDelta(const Node & node) + : old_stats(node.stats) + , new_stats(node.stats) + {} + + NodeStats old_stats; + NodeStats new_stats; + int32_t version{-1}; + }; + + struct UpdateNodeDataDelta + { + + std::string old_data; + std::string new_data; + int32_t version{-1}; + }; + + struct SetACLDelta + { + Coordination::ACLs old_acls; + Coordination::ACLs new_acls; + int32_t version{-1}; + }; + + struct ErrorDelta + { + Coordination::Error error; + }; + + struct FailedMultiDelta + { + std::vector error_codes; + Coordination::Error global_error{Coordination::Error::ZOK}; + }; + + // Denotes end of a subrequest in multi request + struct SubDeltaEnd + { + }; + + struct AddAuthDelta + { + int64_t session_id; + std::shared_ptr auth_id; + }; + + struct CloseSessionDelta + { + int64_t session_id; + }; + + using Operation = std::variant< + CreateNodeDelta, + RemoveNodeDelta, + UpdateNodeStatDelta, + UpdateNodeDataDelta, + SetACLDelta, + AddAuthDelta, + ErrorDelta, + SubDeltaEnd, + FailedMultiDelta, + CloseSessionDelta>; + + struct Delta + { + Delta(String path_, int64_t zxid_, Operation operation_) : path(std::move(path_)), zxid(zxid_), operation(std::move(operation_)) { } + + Delta(int64_t zxid_, Coordination::Error error) : Delta("", zxid_, ErrorDelta{error}) { } + + Delta(int64_t zxid_, Operation subdelta) : Delta("", zxid_, subdelta) { } + + String path; + int64_t zxid; + Operation operation; + }; + + using DeltaIterator = std::list::const_iterator; + struct DeltaRange + { + DeltaIterator begin_it; + DeltaIterator end_it; + + auto begin() const + { + return begin_it; + } + + auto end() const + { + return end_it; + } + + bool empty() const + { + return begin_it == end_it; + } + + const auto & front() const + { + return *begin_it; + } + }; + + 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; static bool checkDigest(const Digest & first, const Digest & second); - }; + /// Keeper state machine almost equal to the ZooKeeper's state machine. /// Implements all logic of operations, data changes, sessions allocation. /// In-memory and not thread safe. @@ -472,143 +526,49 @@ public: int64_t session_id_counter{1}; - SessionAndAuth session_and_auth; + mutable SharedMutex auth_mutex; + SessionAndAuth committed_session_and_auth; + 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. Container container; - // Applying ZooKeeper request to storage consists of two steps: - // - preprocessing which, instead of applying the changes directly to storage, - // generates deltas with those changes, denoted with the request ZXID - // - processing which applies deltas with the correct ZXID to the storage - // - // Delta objects allow us two things: - // - fetch the latest, uncommitted state of an object by getting the committed - // state of that same object from the storage and applying the deltas - // in the same order as they are defined - // - quickly commit the changes to the storage - struct CreateNodeDelta - { - Coordination::Stat stat; - Coordination::ACLs acls; - String data; - }; - - struct RemoveNodeDelta - { - int32_t version{-1}; - int64_t ephemeral_owner{0}; - }; - - struct UpdateNodeDelta - { - std::function update_fn; - int32_t version{-1}; - }; - - struct SetACLDelta - { - Coordination::ACLs acls; - int32_t version{-1}; - }; - - struct ErrorDelta - { - Coordination::Error error; - }; - - struct FailedMultiDelta - { - std::vector error_codes; - Coordination::Error global_error{Coordination::Error::ZOK}; - }; - - // Denotes end of a subrequest in multi request - struct SubDeltaEnd - { - }; - - struct AddAuthDelta - { - int64_t session_id; - AuthID auth_id; - }; - - struct CloseSessionDelta - { - int64_t session_id; - }; - - using Operation = std:: - variant; - - struct Delta - { - Delta(String path_, int64_t zxid_, Operation operation_) : path(std::move(path_)), zxid(zxid_), operation(std::move(operation_)) { } - - Delta(int64_t zxid_, Coordination::Error error) : Delta("", zxid_, ErrorDelta{error}) { } - - Delta(int64_t zxid_, Operation subdelta) : Delta("", zxid_, subdelta) { } - - String path; - int64_t zxid; - Operation operation; - }; - struct UncommittedState { explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { } - void addDelta(Delta new_delta); - void addDeltas(std::vector new_deltas); - void commit(int64_t commit_zxid); + 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; + std::shared_ptr getNode(StringRef path, bool should_lock_storage = true) const; const Node * getActualNodeView(StringRef path, const Node & storage_node) 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) const; 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; + using ZxidToNodes = std::map>; struct UncommittedNode { std::shared_ptr node{nullptr}; - Coordination::ACLs acls{}; - int64_t zxid{0}; - }; + std::optional acls{}; + std::unordered_set applied_zxids{}; - struct Hash - { - auto operator()(const std::string_view view) const - { - SipHash hash; - hash.update(view); - return hash.get64(); - } - - using is_transparent = void; // required to make find() work with different type than key_type - }; - - struct Equal - { - auto operator()(const std::string_view a, - const std::string_view b) const - { - return a == b; - } - - using is_transparent = void; // required to make find() work with different type than key_type + void materializeACL(const ACLMap & current_acl_map); }; struct PathCmp @@ -624,10 +584,15 @@ public: using is_transparent = void; // required to make find() work with different type than key_type }; - mutable std::map nodes; - std::unordered_map, Hash, Equal> deltas_for_path; + Ephemerals ephemerals; - std::list deltas; + std::unordered_map>>> session_and_auth; + + mutable std::map nodes; + mutable ZxidToNodes zxid_to_nodes; + + mutable std::mutex deltas_mutex; + std::list deltas TSA_GUARDED_BY(deltas_mutex); KeeperStorage & storage; }; @@ -637,7 +602,7 @@ public: // with zxid > last_zxid void applyUncommittedState(KeeperStorage & other, int64_t last_log_idx); - Coordination::Error commit(int64_t zxid); + Coordination::Error commit(DeltaRange deltas); // Create node in the storage // Returns false if it failed to create the node, true otherwise @@ -655,12 +620,11 @@ 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); - + std::mutex ephemeral_mutex; /// Mapping session_id -> set of ephemeral nodes paths - Ephemerals ephemerals; - /// Mapping session_id -> set of watched nodes paths - SessionAndWatcher sessions_and_watchers; + 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; /// All active sessions with timeout @@ -669,8 +633,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 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 @@ -687,11 +653,16 @@ public: int64_t log_idx = 0; }; - std::deque uncommitted_transactions; + std::list uncommitted_transactions TSA_GUARDED_BY(transaction_mutex); - uint64_t nodes_digest{0}; + uint64_t nodes_digest = 0; - bool finalized{false}; + std::atomic finalized{false}; + + + /// 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; @@ -700,45 +671,30 @@ 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; + int64_t getNextZXID() const; + int64_t getNextZXIDLocked() const TSA_REQUIRES(transaction_mutex); - return uncommitted_transactions.back().zxid + 1; - } - - Digest getNodesDigest(bool committed) const; + Digest getNodesDigest(bool committed, bool lock_transaction_mutex) const; KeeperContextPtr keeper_context; 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); - 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::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. @@ -765,42 +721,39 @@ 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; } + SessionAndTimeout getActiveSessions() const; /// Get all dead sessions - std::vector getDeadSessions() const { return session_expiry_queue.getExpiredSessions(); } + std::vector getDeadSessions() const; + + void updateStats(); + const Stats & getStorageStats() 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; diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index 12b40bbb87e..193e6d0b270 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -155,11 +155,11 @@ public: ReadBufferFromOwnString buffer(iter->value().ToStringView()); typename Node::Meta & meta = new_pair->value; readPODBinary(meta, buffer); - readVarUInt(new_pair->value.data_size, buffer); - if (new_pair->value.data_size) + readVarUInt(new_pair->value.stats.data_size, buffer); + if (new_pair->value.stats.data_size) { - new_pair->value.data = std::unique_ptr(new char[new_pair->value.data_size]); - buffer.readStrict(new_pair->value.data.get(), new_pair->value.data_size); + new_pair->value.data = std::unique_ptr(new char[new_pair->value.stats.data_size]); + buffer.readStrict(new_pair->value.data.get(), new_pair->value.stats.data_size); } pair = new_pair; } @@ -211,7 +211,7 @@ public: } } - std::vector> getChildren(const std::string & key_) + std::vector> getChildren(const std::string & key_, bool read_data = false) { rocksdb::ReadOptions read_options; read_options.total_order_seek = true; @@ -232,6 +232,15 @@ public: typename Node::Meta & meta = node; /// We do not read data here readPODBinary(meta, buffer); + if (read_data) + { + readVarUInt(meta.stats.data_size, buffer); + if (meta.stats.data_size) + { + node.data = std::unique_ptr(new char[meta.stats.data_size]); + buffer.readStrict(node.data.get(), meta.stats.data_size); + } + } std::string real_key(iter->key().data() + len, iter->key().size() - len); // std::cout << "real key: " << real_key << std::endl; result.emplace_back(std::move(real_key), std::move(node)); @@ -268,11 +277,11 @@ public: typename Node::Meta & meta = kv->value; readPODBinary(meta, buffer); /// TODO: Sometimes we don't need to load data. - readVarUInt(kv->value.data_size, buffer); - if (kv->value.data_size) + readVarUInt(kv->value.stats.data_size, buffer); + if (kv->value.stats.data_size) { - kv->value.data = std::unique_ptr(new char[kv->value.data_size]); - buffer.readStrict(kv->value.data.get(), kv->value.data_size); + kv->value.data = std::unique_ptr(new char[kv->value.stats.data_size]); + buffer.readStrict(kv->value.data.get(), kv->value.stats.data_size); } return const_iterator(kv); } @@ -281,7 +290,7 @@ public: { auto it = find(key); chassert(it != end()); - return MockNode(it->value.numChildren(), it->value.getData()); + return MockNode(it->value.stats.numChildren(), it->value.getData()); } const_iterator updateValue(StringRef key_, ValueUpdater updater) diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 99d71b85e78..5783a803aa8 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -93,7 +93,7 @@ void deserializeACLMap(Storage & storage, ReadBuffer & in) } template -int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log) +int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log) TSA_NO_THREAD_SAFETY_ANALYSIS { int64_t max_zxid = 0; std::string path; @@ -108,33 +108,33 @@ int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log Coordination::read(node.acl_id, in); /// Deserialize stat - Coordination::read(node.czxid, in); - Coordination::read(node.mzxid, in); + Coordination::read(node.stats.czxid, in); + Coordination::read(node.stats.mzxid, in); /// For some reason ZXID specified in filename can be smaller /// then actual zxid from nodes. In this case we will use zxid from nodes. - max_zxid = std::max(max_zxid, node.mzxid); + max_zxid = std::max(max_zxid, node.stats.mzxid); int64_t ctime; Coordination::read(ctime, in); - node.setCtime(ctime); - Coordination::read(node.mtime, in); - Coordination::read(node.version, in); - Coordination::read(node.cversion, in); - Coordination::read(node.aversion, in); + node.stats.setCtime(ctime); + Coordination::read(node.stats.mtime, in); + Coordination::read(node.stats.version, in); + Coordination::read(node.stats.cversion, in); + Coordination::read(node.stats.aversion, in); int64_t ephemeral_owner; Coordination::read(ephemeral_owner, in); if (ephemeral_owner != 0) - node.setEphemeralOwner(ephemeral_owner); - Coordination::read(node.pzxid, in); + node.stats.setEphemeralOwner(ephemeral_owner); + Coordination::read(node.stats.pzxid, in); if (!path.empty()) { if (ephemeral_owner == 0) - node.setSeqNum(node.cversion); + node.stats.setSeqNum(node.stats.cversion); 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); } @@ -149,7 +149,13 @@ int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log if (itr.key != "/") { auto parent_path = parentNodePath(itr.key); - storage.container.updateValue(parent_path, [my_path = itr.key] (typename Storage::Node & value) { value.addChild(getBaseNodeName(my_path)); value.increaseNumChildren(); }); + storage.container.updateValue( + parent_path, + [my_path = itr.key](typename Storage::Node & value) + { + value.addChild(getBaseNodeName(my_path)); + value.stats.increaseNumChildren(); + }); } } @@ -157,7 +163,7 @@ int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log } template -void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, LoggerPtr log) +void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, LoggerPtr log) TSA_NO_THREAD_SAFETY_ANALYSIS { LOG_INFO(log, "Deserializing storage snapshot {}", snapshot_path); int64_t zxid = getZxidFromName(snapshot_path); @@ -487,7 +493,7 @@ bool hasErrorsInMultiRequest(Coordination::ZooKeeperRequestPtr request) } template -bool deserializeTxn(Storage & storage, ReadBuffer & in, LoggerPtr /*log*/) +bool deserializeTxn(Storage & storage, ReadBuffer & in, LoggerPtr /*log*/) TSA_NO_THREAD_SAFETY_ANALYSIS { int64_t checksum; Coordination::read(checksum, in); @@ -568,7 +574,7 @@ void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_ } template -void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, LoggerPtr log) +void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, LoggerPtr log) TSA_NO_THREAD_SAFETY_ANALYSIS { std::map existing_logs; for (const auto & p : fs::directory_iterator(path)) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 4272e504da1..69990d1661c 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,9 +1571,9 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) addNode(storage, "/hello1", "world", 1); addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; - storage.zxid = 2; - storage.ephemerals[3] = {"/hello2"}; - storage.ephemerals[1] = {"/hello1"}; + TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; + storage.committed_ephemerals[3] = {"/hello2"}; + storage.committed_ephemerals[1] = {"/hello1"}; storage.getSessionID(130); storage.getSessionID(130); @@ -1601,10 +1602,10 @@ 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->ephemerals.size(), 2); - EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); - EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); + EXPECT_EQ(restored_storage->getZXID(), 2); + 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); } @@ -2027,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 @@ -2035,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); } @@ -2590,9 +2591,9 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) addNode(storage, "/hello1", "world", 1); addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; - storage.zxid = 2; - storage.ephemerals[3] = {"/hello2"}; - storage.ephemerals[1] = {"/hello1"}; + TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; + storage.committed_ephemerals[3] = {"/hello2"}; + storage.committed_ephemerals[1] = {"/hello1"}; storage.getSessionID(130); storage.getSessionID(130); @@ -2617,10 +2618,10 @@ 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->ephemerals.size(), 2); - EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); - EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); + EXPECT_EQ(restored_storage->getZXID(), 2); + 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); } @@ -2805,13 +2806,13 @@ 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); - DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); + DB::KeeperStorageSnapshot snapshot(&storage, storage.getZXID()); auto buf = manager.serializeSnapshotToBuffer(snapshot); @@ -3315,7 +3316,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"); @@ -3566,12 +3567,12 @@ TYPED_TEST(CoordinationTest, TestRemoveRecursiveRequest) { SCOPED_TRACE("Recursive Remove Ephemeral"); create("/T7", zkutil::CreateMode::Ephemeral); - ASSERT_EQ(storage.ephemerals.size(), 1); + ASSERT_EQ(storage.committed_ephemerals.size(), 1); auto responses = remove_recursive("/T7", 100); ASSERT_EQ(responses.size(), 1); ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK); - ASSERT_EQ(storage.ephemerals.size(), 0); + ASSERT_EQ(storage.committed_ephemerals.size(), 0); ASSERT_FALSE(exists("/T7")); } @@ -3581,12 +3582,12 @@ TYPED_TEST(CoordinationTest, TestRemoveRecursiveRequest) create("/T8/A", zkutil::CreateMode::Persistent); create("/T8/B", zkutil::CreateMode::Ephemeral); create("/T8/A/C", zkutil::CreateMode::Ephemeral); - ASSERT_EQ(storage.ephemerals.size(), 1); + ASSERT_EQ(storage.committed_ephemerals.size(), 1); auto responses = remove_recursive("/T8", 4); ASSERT_EQ(responses.size(), 1); ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK); - ASSERT_EQ(storage.ephemerals.size(), 0); + ASSERT_EQ(storage.committed_ephemerals.size(), 0); ASSERT_FALSE(exists("/T8")); ASSERT_FALSE(exists("/T8/A")); ASSERT_FALSE(exists("/T8/B")); @@ -3889,14 +3890,26 @@ TYPED_TEST(CoordinationTest, TestRemoveRecursiveWatches) auto responses = storage.processRequest(remove_request, 1, new_zxid); ASSERT_EQ(responses.size(), 7); + /// request response is last + ASSERT_EQ(dynamic_cast(responses.back().response.get()), nullptr); - for (size_t i = 0; i < 7; ++i) + std::unordered_map> expected_watch_responses + { + {"/A/B/D", {Coordination::Event::DELETED}}, + {"/A/B", {Coordination::Event::CHILD, Coordination::Event::DELETED}}, + {"/A/C", {Coordination::Event::DELETED}}, + {"/A", {Coordination::Event::CHILD, Coordination::Event::DELETED}}, + }; + + std::unordered_map> actual_watch_responses; + for (size_t i = 0; i < 6; ++i) { ASSERT_EQ(responses[i].response->error, Coordination::Error::ZOK); - if (const auto * watch_response = dynamic_cast(responses[i].response.get())) - ASSERT_EQ(watch_response->type, Coordination::Event::DELETED); + const auto & watch_response = dynamic_cast(*responses[i].response); + actual_watch_responses[watch_response.path].push_back(static_cast(watch_response.type)); } + ASSERT_EQ(expected_watch_responses, actual_watch_responses); ASSERT_EQ(storage.watches.size(), 0); ASSERT_EQ(storage.list_watches.size(), 0); 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)); } 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;