From b43c3d75a2e21af9630152c335d40ff1f852310a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 4 Sep 2023 14:49:49 +0000 Subject: [PATCH 001/128] Initial implementation --- src/Coordination/KeeperSnapshotManager.cpp | 2 +- src/Coordination/KeeperStateMachine.cpp | 3 - src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStorage.cpp | 983 ++++++++++++--------- src/Coordination/KeeperStorage.h | 101 ++- src/Coordination/ZooKeeperDataReader.cpp | 6 +- 6 files changed, 638 insertions(+), 459 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a216272a9e1..c0c59b4e242 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -277,7 +277,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr } } -void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context) +void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context) TSA_NO_THREAD_SAFETY_ANALYSIS { uint8_t version; readBinary(version, in); diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5a8cf88ea73..9d96a76e06a 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -271,8 +271,6 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; - std::lock_guard lock(storage_and_responses_lock); - if (storage->isFinalized()) return false; @@ -422,7 +420,6 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n parsed_request_cache.erase(request_for_session->session_id); } - std::lock_guard lock(storage_and_responses_lock); KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); for (auto & response_for_session : responses_for_sessions) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index aad5d3aafd4..670b15394ce 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -142,7 +142,7 @@ private: CoordinationSettingsPtr coordination_settings; /// Main state machine logic - KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock); + KeeperStoragePtr storage; /// Save/Load and Serialize/Deserialize logic for snapshots. KeeperSnapshotManager snapshot_manager; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 95983dbcf3c..0978f5f965e 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -229,6 +229,7 @@ void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) { stat = other.stat; seq_num = other.seq_num; + acl_id = other.acl_id; setData(other.getData()); cached_digest = other.cached_digest; } @@ -316,6 +317,7 @@ Overloaded(Ts...) -> Overloaded; std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const { + std::lock_guard lock(storage.container_mutex); if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; @@ -330,18 +332,34 @@ std::shared_ptr KeeperStorage::UncommittedState::tryGetNode void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) { assert(!delta.path.empty()); - if (!nodes.contains(delta.path)) + UncommittedNode * uncommitted_node = nullptr; + + if (auto it = nodes.find(delta.path); it != nodes.end()) + { + uncommitted_node = &it->second; + } + else { if (auto storage_node = tryGetNodeFromStorage(delta.path)) - nodes.emplace(delta.path, UncommittedNode{.node = std::move(storage_node)}); + { + auto [emplaced_it, _] = nodes.emplace(delta.path, UncommittedNode{.node = std::move(storage_node)}); + uncommitted_node = &emplaced_it->second; + } else - nodes.emplace(delta.path, UncommittedNode{.node = nullptr}); + { + auto [emplaced_it, _] = nodes.emplace(delta.path, UncommittedNode{.node = nullptr}); + uncommitted_node = &emplaced_it->second; + } } std::visit( [&](const DeltaType & operation) { - auto & [node, acls, last_applied_zxid] = nodes.at(delta.path); + auto & [node, acls, applied_zxids] = *uncommitted_node; + + int64_t last_applied_zxid = 0; + if (!applied_zxids.empty()) + last_applied_zxid = applied_zxids.back(); if constexpr (std::same_as) { @@ -350,132 +368,193 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) node->stat = operation.stat; node->setData(operation.data); acls = operation.acls; - last_applied_zxid = delta.zxid; } else if constexpr (std::same_as) { assert(node); node = nullptr; - last_applied_zxid = delta.zxid; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { assert(node); node->invalidateDigestCache(); - operation.update_fn(*node); - last_applied_zxid = delta.zxid; + node->stat = operation.new_stats; + node->seq_num = operation.new_seq_num; + } + else if constexpr (std::same_as) + { + assert(node); + node->invalidateDigestCache(); + node->setData(operation.new_data); } else if constexpr (std::same_as) { - acls = operation.acls; - last_applied_zxid = delta.zxid; + acls = operation.new_acls; } + + if (last_applied_zxid != delta.zxid) + last_applied_zxid = applied_zxids.emplace_back(delta.zxid); }, delta.operation); } -void KeeperStorage::UncommittedState::addDelta(Delta new_delta) +void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) { - const auto & added_delta = deltas.emplace_back(std::move(new_delta)); + assert(!delta.path.empty()); - if (!added_delta.path.empty()) - { - deltas_for_path[added_delta.path].push_back(&added_delta); - applyDelta(added_delta); - } - else if (const auto * auth_delta = std::get_if(&added_delta.operation)) - { - auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; - uncommitted_auth.emplace_back(&auth_delta->auth_id); - } -} - -void KeeperStorage::UncommittedState::addDeltas(std::vector new_deltas) -{ - for (auto & delta : new_deltas) - addDelta(std::move(delta)); -} - -void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) -{ - assert(deltas.empty() || deltas.front().zxid >= commit_zxid); - - // collect nodes that have no further modification in the current transaction - std::unordered_set modified_nodes; - - while (!deltas.empty() && deltas.front().zxid == commit_zxid) - { - if (std::holds_alternative(deltas.front().operation)) + std::visit( + [&](const DeltaType & operation) { - deltas.pop_front(); - break; - } + auto & [node, acls, applied_zxids] = nodes.at(delta.path); - auto & front_delta = deltas.front(); - - if (!front_delta.path.empty()) - { - auto & path_deltas = deltas_for_path.at(front_delta.path); - assert(path_deltas.front() == &front_delta); - path_deltas.pop_front(); - if (path_deltas.empty()) + if constexpr (std::same_as) { - deltas_for_path.erase(front_delta.path); - - // no more deltas for path -> no modification - modified_nodes.insert(std::move(front_delta.path)); + assert(node); + node = nullptr; } - else if (path_deltas.front()->zxid > commit_zxid) + else if constexpr (std::same_as) { - // next delta has a zxid from a different transaction -> no modification in this transaction - modified_nodes.insert(std::move(front_delta.path)); + assert(!node); + node = std::make_shared(); + node->stat = operation.stat; + node->setData(operation.data); + acls = operation.acls; + } + else if constexpr (std::same_as) + { + assert(node); + node->invalidateDigestCache(); + node->stat = operation.old_stats; + node->seq_num = operation.old_seq_num; + } + else if constexpr (std::same_as) + { + assert(node); + node->invalidateDigestCache(); + node->setData(operation.old_data); + } + else if constexpr (std::same_as) + { + acls = operation.old_acls; } - } - else if (auto * add_auth = std::get_if(&front_delta.operation)) - { - auto & uncommitted_auth = session_and_auth[add_auth->session_id]; - assert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id); - uncommitted_auth.pop_front(); - if (uncommitted_auth.empty()) - session_and_auth.erase(add_auth->session_id); - } + if (applied_zxids.back() != delta.zxid) + applied_zxids.pop_back(); + }, + delta.operation); +} - deltas.pop_front(); - } - - // delete all cached nodes that were not modified after the commit_zxid - // we only need to check the nodes that were modified in this transaction - for (const auto & node : modified_nodes) +void KeeperStorage::UncommittedState::applyDeltas(const std::list & new_deltas) +{ + for (const auto & delta : new_deltas) { - if (nodes[node].zxid == commit_zxid) - nodes.erase(node); + if (!delta.path.empty()) + { + applyDelta(delta); + } + else if (const auto * auth_delta = std::get_if(&delta.operation)) + { + auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; + uncommitted_auth.emplace_back(auth_delta->auth_id); + } } } +void KeeperStorage::UncommittedState::addDeltas(std::list new_deltas) +{ + std::lock_guard lock(deltas_mutex); + deltas.splice(deltas.end(), std::move(new_deltas)); +} + +//void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) +//{ +// assert(deltas.empty() || deltas.front().zxid >= commit_zxid); +// +// // collect nodes that have no further modification in the current transaction +// std::unordered_set modified_nodes; +// +// while (!deltas.empty() && deltas.front().zxid == commit_zxid) +// { +// if (std::holds_alternative(deltas.front().operation)) +// { +// deltas.pop_front(); +// break; +// } +// +// auto & front_delta = deltas.front(); +// +// if (!front_delta.path.empty()) +// { +// auto & path_deltas = deltas_for_path.at(front_delta.path); +// assert(path_deltas.front() == &front_delta); +// path_deltas.pop_front(); +// if (path_deltas.empty()) +// { +// deltas_for_path.erase(front_delta.path); +// +// // no more deltas for path -> no modification +// modified_nodes.insert(std::move(front_delta.path)); +// } +// else if (path_deltas.front()->zxid > commit_zxid) +// { +// // next delta has a zxid from a different transaction -> no modification in this transaction +// modified_nodes.insert(std::move(front_delta.path)); +// } +// } +// else if (auto * add_auth = std::get_if(&front_delta.operation)) +// { +// auto & uncommitted_auth = session_and_auth[add_auth->session_id]; +// assert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id); +// uncommitted_auth.pop_front(); +// if (uncommitted_auth.empty()) +// session_and_auth.erase(add_auth->session_id); +// +// } +// +// deltas.pop_front(); +// } +// +// // delete all cached nodes that were not modified after the commit_zxid +// // we only need to check the nodes that were modified in this transaction +// for (const auto & node : modified_nodes) +// { +// if (nodes[node].zxid == commit_zxid) +// nodes.erase(node); +// } +//} + void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) { - // we can only rollback the last zxid (if there is any) - // if there is a delta with a larger zxid, we have invalid state - if (!deltas.empty() && deltas.back().zxid > rollback_zxid) - throw DB::Exception( - DB::ErrorCodes::LOGICAL_ERROR, - "Invalid state of deltas found while trying to rollback request. Last ZXID ({}) is larger than the requested ZXID ({})", - deltas.back().zxid, - rollback_zxid); + std::list rollback_deltas; + { + std::lock_guard lock(deltas_mutex); + if (!deltas.empty() && deltas.back().zxid > rollback_zxid) + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Invalid state of deltas found while trying to rollback request. Last ZXID ({}) is larger than the requested ZXID ({})", + deltas.back().zxid, + rollback_zxid); - auto delta_it = deltas.rbegin(); + auto delta_it = deltas.rbegin(); + for (; delta_it != deltas.rend(); ++delta_it) + { + if (delta_it->zxid != rollback_zxid) + break; + } + + if (delta_it == deltas.rend()) + rollback_deltas = std::move(deltas); + else + rollback_deltas.splice(rollback_deltas.end(), deltas, delta_it.base(), deltas.end()); + } // we need to undo ephemeral mapping modifications // CreateNodeDelta added ephemeral for session id -> we need to remove it // RemoveNodeDelta removed ephemeral for session id -> we need to add it back - for (; delta_it != deltas.rend(); ++delta_it) + for (auto & delta : rollback_deltas) { - if (delta_it->zxid < rollback_zxid) - break; - - assert(delta_it->zxid == rollback_zxid); - if (!delta_it->path.empty()) + assert(delta.zxid == rollback_zxid); + if (!delta.path.empty()) { std::visit( [&](const DeltaType & operation) @@ -483,65 +562,28 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) if constexpr (std::same_as) { if (operation.stat.ephemeralOwner != 0) - storage.unregisterEphemeralPath(operation.stat.ephemeralOwner, delta_it->path); + storage.unregisterEphemeralPath(operation.stat.ephemeralOwner, delta.path); } else if constexpr (std::same_as) { - if (operation.ephemeral_owner != 0) - storage.ephemerals[operation.ephemeral_owner].emplace(delta_it->path); + if (operation.stat.ephemeralOwner != 0) + storage.ephemerals[operation.stat.ephemeralOwner].emplace(delta.path); } }, - delta_it->operation); - - auto & path_deltas = deltas_for_path.at(delta_it->path); - if (path_deltas.back() == &*delta_it) - { - path_deltas.pop_back(); - if (path_deltas.empty()) - deltas_for_path.erase(delta_it->path); - } + delta.operation); } - else if (auto * add_auth = std::get_if(&delta_it->operation)) + else if (auto * add_auth = std::get_if(&delta.operation)) { auto & uncommitted_auth = session_and_auth[add_auth->session_id]; - if (uncommitted_auth.back() == &add_auth->auth_id) + if (uncommitted_auth.back() == add_auth->auth_id) { uncommitted_auth.pop_back(); if (uncommitted_auth.empty()) session_and_auth.erase(add_auth->session_id); } } - } - if (delta_it == deltas.rend()) - deltas.clear(); - else - deltas.erase(delta_it.base(), deltas.end()); - - absl::flat_hash_set deleted_nodes; - std::erase_if( - nodes, - [&, rollback_zxid](const auto & node) - { - if (node.second.zxid == rollback_zxid) - { - deleted_nodes.emplace(std::move(node.first)); - return true; - } - return false; - }); - - // recalculate all the uncommitted deleted nodes - for (const auto & deleted_node : deleted_nodes) - { - auto path_delta_it = deltas_for_path.find(deleted_node); - if (path_delta_it != deltas_for_path.end()) - { - for (const auto & delta : path_delta_it->second) - { - applyDelta(*delta); - } - } + rollbackDelta(delta); } } @@ -550,19 +592,34 @@ std::shared_ptr KeeperStorage::UncommittedState::getNode(St if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.node; - return tryGetNodeFromStorage(path); + std::shared_ptr node = tryGetNodeFromStorage(path); + + if (node) + nodes.emplace(std::string{path}, UncommittedNode{node}); + + return node; } Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const { if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) - return node_it->second.acls; + { + if (!node_it->second.acls.has_value()) + node_it->second.acls.emplace(storage.acl_map.convertNumber(node_it->second.node->acl_id)); - auto node_it = storage.container.find(path); - if (node_it == storage.container.end()) - return {}; + return *node_it->second.acls; + } - return storage.acl_map.convertNumber(node_it->value.acl_id); + std::shared_ptr node = tryGetNodeFromStorage(path); + + if (node) + { + auto [it, inserted] = nodes.emplace(std::string{path}, UncommittedNode{node}); + it->second.acls = storage.acl_map.convertNumber(node->acl_id); + return *it->second.acls; + } + + return {}; } void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, std::function func) const @@ -571,21 +628,25 @@ void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, s { for (const auto & auth : auth_ids) { - using TAuth = std::remove_reference_t; + using TAuth = std::remove_cvref_t; const AuthID * auth_ptr = nullptr; - if constexpr (std::is_pointer_v) - auth_ptr = auth; - else + if constexpr (std::same_as) auth_ptr = &auth; + else + auth_ptr = auth.get(); func(*auth_ptr); } }; - // for committed - if (storage.session_and_auth.contains(session_id)) - call_for_each_auth(storage.session_and_auth.at(session_id)); + { + std::lock_guard lock(storage.auth_mutex); + // for committed + if (storage.session_and_auth.contains(session_id)) + call_for_each_auth(storage.session_and_auth.at(session_id)); + } + // for uncommitted if (session_and_auth.contains(session_id)) call_for_each_auth(session_and_auth.at(session_id)); @@ -604,37 +665,55 @@ namespace } -void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_zxid) -{ - for (const auto & transaction : uncommitted_transactions) - { - if (transaction.zxid <= last_zxid) - continue; - other.uncommitted_transactions.push_back(transaction); - } - - auto it = uncommitted_state.deltas.begin(); - - for (; it != uncommitted_state.deltas.end(); ++it) - { - if (it->zxid <= last_zxid) - continue; - - other.uncommitted_state.addDelta(*it); - } +/// Get current committed zxid +int64_t KeeperStorage::getZXID() const +{ + std::lock_guard lock(transaction_mutex); + return zxid; } -Coordination::Error KeeperStorage::commit(int64_t commit_zxid) +int64_t KeeperStorage::getNextZXIDLocked(std::lock_guard &) const { + if (uncommitted_transactions.empty()) + return zxid + 1; + + return uncommitted_transactions.back().zxid + 1; +} + +int64_t KeeperStorage::getNextZXID() const +{ + std::lock_guard lock(transaction_mutex); + return getNextZXIDLocked(lock); +} + +void KeeperStorage::applyUncommittedState(KeeperStorage & /*other*/, int64_t /*last_zxid*/) +{ + //for (const auto & transaction : uncommitted_transactions) + //{ + // if (transaction.zxid <= last_zxid) + // continue; + // other.uncommitted_transactions.push_back(transaction); + //} + + //auto it = uncommitted_state.deltas.begin(); + + //for (; it != uncommitted_state.deltas.end(); ++it) + //{ + // if (it->zxid <= last_zxid) + // continue; + + // other.uncommitted_state.addDelta(*it); + //} +} + +Coordination::Error KeeperStorage::commit(std::list deltas) +{ + std::lock_guard lock(container_mutex); // Deltas are added with increasing ZXIDs // If there are no deltas for the commit_zxid (e.g. read requests), we instantly return // on first delta - for (auto & delta : uncommitted_state.deltas) + for (auto & delta : deltas) { - if (delta.zxid > commit_zxid) - break; - - bool finish_subdelta = false; auto result = std::visit( [&, &path = delta.path](DeltaType & operation) -> Coordination::Error { @@ -650,7 +729,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as || std::same_as) { auto node_it = container.find(path); if (node_it == container.end()) @@ -660,7 +739,17 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) onStorageInconsistency(); removeDigest(node_it->value, path); - auto updated_node = container.updateValue(path, operation.update_fn); + auto updated_node = container.updateValue(path, [&](auto & node) { + if constexpr (std::same_as) + { + node.stat = operation.new_stats; + node.seq_num = operation.new_seq_num; + } + else + { + node.setData(std::move(operation.new_data)); + } + }); addDigest(updated_node->value, path); return Coordination::Error::ZOK; @@ -683,7 +772,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) acl_map.removeUsage(node_it->value.acl_id); - uint64_t acl_id = acl_map.convertACLs(operation.acls); + uint64_t acl_id = acl_map.convertACLs(operation.new_acls); acl_map.addUsage(acl_id); container.updateValue(path, [acl_id](KeeperStorage::Node & node) { node.acl_id = acl_id; }); @@ -694,12 +783,12 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) return operation.error; else if constexpr (std::same_as) { - finish_subdelta = true; return Coordination::Error::ZOK; } else if constexpr (std::same_as) { - session_and_auth[operation.session_id].emplace_back(std::move(operation.auth_id)); + std::lock_guard auth_lock(auth_mutex); + session_and_auth[operation.session_id].emplace_back(std::move(*operation.auth_id)); return Coordination::Error::ZOK; } else @@ -712,9 +801,6 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) if (result != Coordination::Error::ZOK) return result; - - if (finish_subdelta) - return Coordination::Error::ZOK; } return Coordination::Error::ZOK; @@ -799,8 +885,8 @@ struct KeeperStorageRequestProcessor Coordination::ZooKeeperRequestPtr zk_request; explicit KeeperStorageRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) : zk_request(zk_request_) { } - virtual Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const = 0; - virtual std::vector + virtual Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const = 0; + virtual std::list preprocess(KeeperStorage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const { return {}; @@ -808,7 +894,7 @@ struct KeeperStorageRequestProcessor // process the request using locally committed data virtual Coordination::ZooKeeperResponsePtr - processLocal(KeeperStorage & /*storage*/, int64_t /*zxid*/) const + processLocal(KeeperStorage & /*storage*/, std::list /* deltas */) const { throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Cannot process the request locally"}; } @@ -827,7 +913,7 @@ struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageReques { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; Coordination::ZooKeeperResponsePtr - process(KeeperStorage & /* storage */, int64_t /* zxid */) const override + process(KeeperStorage & /* storage */, std::list /* deltas */) const override { return zk_request->makeResponse(); } @@ -837,7 +923,7 @@ struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProc { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; Coordination::ZooKeeperResponsePtr - process(KeeperStorage & /* storage */, int64_t /* zxid */) const override + process(KeeperStorage & /* storage */, std::list /* deltas */) const override { auto response = zk_request->makeResponse(); dynamic_cast(*response).path @@ -853,6 +939,7 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ { if (is_local) { + std::lock_guard lock(storage.container_mutex); auto node_it = storage.container.find(path); if (node_it == storage.container.end()) return {}; @@ -877,6 +964,13 @@ void handleSystemNodeModification(const KeeperContext & keeper_context, std::str } +KeeperStorage::UpdateNodeStatDelta::UpdateNodeStatDelta(const KeeperStorage::Node & node) + : old_stats(node.stat) + , new_stats(node.stat) + , old_seq_num(node.seq_num) + , new_seq_num(node.seq_num) +{} + bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session_id, bool is_local) { const auto node_acls = getNodeACLs(*this, path, is_local); @@ -931,13 +1025,13 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); } - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperCreateRequest); Coordination::ZooKeeperCreateRequest & request = dynamic_cast(*zk_request); - std::vector new_deltas; + std::list new_deltas; auto parent_path = parentNodePath(request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); @@ -982,21 +1076,21 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr int32_t parent_cversion = request.parent_cversion; - auto parent_update = [parent_cversion, zxid](KeeperStorage::Node & node) - { - /// Increment sequential number even if node is not sequential - ++node.seq_num; - if (parent_cversion == -1) - ++node.stat.cversion; - else if (parent_cversion > node.stat.cversion) - node.stat.cversion = parent_cversion; - if (zxid > node.stat.pzxid) - node.stat.pzxid = zxid; - ++node.stat.numChildren; - }; + KeeperStorage::UpdateNodeStatDelta update_parent_delta(*parent_node); + ++update_parent_delta.new_seq_num; - new_deltas.emplace_back(std::string{parent_path}, zxid, KeeperStorage::UpdateNodeDelta{std::move(parent_update)}); + if (parent_cversion == -1) + ++update_parent_delta.new_stats.cversion; + else if (parent_cversion > update_parent_delta.old_stats.cversion) + update_parent_delta.new_stats.cversion = parent_cversion; + + if (zxid > update_parent_delta.old_stats.pzxid) + update_parent_delta.new_stats.pzxid = zxid; + + ++update_parent_delta.new_stats.numChildren; + + new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(update_parent_delta)); Coordination::Stat stat; stat.czxid = zxid; @@ -1020,25 +1114,28 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + std::string created_path; + auto create_delta_it = std::find_if( + deltas.begin(), + deltas.end(), + [](const auto & delta) + { return std::holds_alternative(delta.operation); }); + + if (create_delta_it != deltas.end()) + created_path = create_delta_it->path; + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; } - const auto & deltas = storage.uncommitted_state.deltas; - auto create_delta_it = std::find_if( - deltas.begin(), - deltas.end(), - [zxid](const auto & delta) - { return delta.zxid == zxid && std::holds_alternative(delta.operation); }); - - response.path_created = create_delta_it->path; + response.path_created = std::move(created_path); response.error = Coordination::Error::ZOK; return response_ptr; } @@ -1053,7 +1150,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); @@ -1070,7 +1167,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); @@ -1078,7 +1175,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1113,15 +1210,15 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } }; @@ -1133,13 +1230,13 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); Coordination::ZooKeeperRemoveRequest & request = dynamic_cast(*zk_request); - std::vector new_deltas; + std::list new_deltas; if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { @@ -1149,32 +1246,38 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } - const auto update_parent_pzxid = [&]() - { - auto parent_path = parentNodePath(request.path); - if (!storage.uncommitted_state.getNode(parent_path)) - return; + auto parent_path = parentNodePath(request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + KeeperStorage::UpdateNodeStatDelta update_parent_delta(*parent_node); + + const auto add_parent_update_delta = [&] + { new_deltas.emplace_back( std::string{parent_path}, zxid, - KeeperStorage::UpdateNodeDelta - { - [zxid](KeeperStorage::Node & parent) - { - if (parent.stat.pzxid < zxid) - parent.stat.pzxid = zxid; - } - } + std::move(update_parent_delta) ); }; + const auto update_parent_pzxid = [&]() + { + if (!parent_node) + return; + + if (update_parent_delta.old_stats.pzxid < zxid) + update_parent_delta.new_stats.pzxid = zxid; + }; + auto node = storage.uncommitted_state.getNode(request.path); if (!node) { if (request.restored_from_zookeeper_log) + { update_parent_pzxid(); + add_parent_update_delta(); + } return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; } else if (request.version != -1 && request.version != node->stat.version) @@ -1185,16 +1288,14 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr if (request.restored_from_zookeeper_log) update_parent_pzxid(); - new_deltas.emplace_back( - std::string{parentNodePath(request.path)}, - zxid, - KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent) - { - ++parent.stat.cversion; - --parent.stat.numChildren; - }}); + ++update_parent_delta.new_stats.cversion; + --update_parent_delta.new_stats.numChildren; + add_parent_update_delta(); - new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->stat.ephemeralOwner}); + new_deltas.emplace_back( + request.path, + zxid, + KeeperStorage::RemoveNodeDelta{request.version, node->stat, storage.uncommitted_state.getACLs(request.path), node->getData()}); if (node->stat.ephemeralOwner != 0) storage.unregisterEphemeralPath(node->stat.ephemeralOwner, request.path); @@ -1204,12 +1305,12 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); - response.error = storage.commit(zxid); + response.error = storage.commit(std::move(deltas)); return response_ptr; } @@ -1224,7 +1325,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); @@ -1237,7 +1338,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); @@ -1245,7 +1346,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1270,15 +1371,15 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } }; @@ -1290,13 +1391,13 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperSetRequest); Coordination::ZooKeeperSetRequest & request = dynamic_cast(*zk_request); - std::vector new_deltas; + std::list new_deltas; if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { @@ -1314,37 +1415,31 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce if (request.version != -1 && request.version != node->stat.version) return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + KeeperStorage::UpdateNodeStatDelta node_delta(*node); + node_delta.version = request.version; + auto & new_stats = node_delta.new_stats; + new_stats.version++; + new_stats.mzxid = zxid; + new_stats.mtime = time; + new_stats.dataLength = static_cast(request.data.length()); + + new_deltas.emplace_back(request.path, zxid, std::move(node_delta)); new_deltas.emplace_back( request.path, zxid, - KeeperStorage::UpdateNodeDelta{ - [zxid, data = request.data, time](KeeperStorage::Node & value) - { - value.stat.version++; - value.stat.mzxid = zxid; - value.stat.mtime = time; - value.stat.dataLength = static_cast(data.length()); - value.setData(data); - }, - request.version}); + KeeperStorage::UpdateNodeDataDelta{.old_data = node->getData(), .new_data = request.data, .version = request.version}); - new_deltas.emplace_back( - parentNodePath(request.path).toString(), - zxid, - KeeperStorage::UpdateNodeDelta - { - [](KeeperStorage::Node & parent) - { - parent.stat.cversion++; - } - } - ); + auto parent_path = parentNodePath(request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + KeeperStorage::UpdateNodeStatDelta parent_delta(*parent_node); + ++parent_delta.new_stats.cversion; + new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(parent_delta)); digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { auto & container = storage.container; @@ -1352,7 +1447,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); Coordination::ZooKeeperSetRequest & request = dynamic_cast(*zk_request); - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1383,7 +1478,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); @@ -1397,7 +1492,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); @@ -1405,7 +1500,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1466,15 +1561,15 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } }; @@ -1492,7 +1587,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return storage.checkACL(check_not_exists ? parentNodePath(path) : path, Coordination::ACL::Read, session_id, is_local); } - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); @@ -1518,7 +1613,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); @@ -1526,7 +1621,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1564,15 +1659,15 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } private: @@ -1589,7 +1684,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override { Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*zk_request); @@ -1616,35 +1711,26 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr if (!fixupACL(request.acls, session_id, uncommitted_state, node_acls)) return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; - std::vector new_deltas - { - { - request.path, - zxid, - KeeperStorage::SetACLDelta{std::move(node_acls), request.version} - }, - { - request.path, - zxid, - KeeperStorage::UpdateNodeDelta - { - [](KeeperStorage::Node & n) { ++n.stat.aversion; } - } - } - }; + KeeperStorage::UpdateNodeStatDelta update_stat_delta(*node); + ++update_stat_delta.new_stats.aversion; + std::list new_deltas{ + {request.path, + zxid, + KeeperStorage::SetACLDelta{std::move(node_acls), uncommitted_state.getACLs(request.path), request.version}}, + {request.path, zxid, std::move(update_stat_delta)}}; digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*zk_request); - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1669,7 +1755,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*zk_request); @@ -1681,7 +1767,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); @@ -1689,7 +1775,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr if constexpr (!local) { - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; @@ -1714,14 +1800,14 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { - return processImpl(storage, zxid); + return processImpl(storage, std::move(deltas)); } }; @@ -1799,18 +1885,19 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro assert(request.requests.empty() || operation_type.has_value()); } - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperMultiRequest); std::vector response_errors; response_errors.reserve(concrete_requests.size()); uint64_t current_digest = digest; + std::list new_deltas; for (size_t i = 0; i < concrete_requests.size(); ++i) { - auto new_deltas = concrete_requests[i]->preprocess(storage, zxid, session_id, time, current_digest, keeper_context); + auto new_subdeltas = concrete_requests[i]->preprocess(storage, zxid, session_id, time, current_digest, keeper_context); - if (!new_deltas.empty()) + if (!new_subdeltas.empty()) { if (auto * error = std::get_if(&new_deltas.back().operation); error && *operation_type == OperationType::Write) @@ -1826,24 +1913,49 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return {KeeperStorage::Delta{zxid, KeeperStorage::FailedMultiDelta{std::move(response_errors)}}}; } } - new_deltas.emplace_back(zxid, KeeperStorage::SubDeltaEnd{}); + + new_subdeltas.emplace_back(zxid, KeeperStorage::SubDeltaEnd{}); response_errors.push_back(Coordination::Error::ZOK); // manually add deltas so that the result of previous request in the transaction is used in the next request - storage.uncommitted_state.addDeltas(std::move(new_deltas)); + storage.uncommitted_state.applyDeltas(new_subdeltas); + new_deltas.splice(new_deltas.end(), std::move(new_subdeltas)); } digest = current_digest; + storage.uncommitted_state.addDeltas(std::move(new_deltas)); + return {}; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + std::list getSubdeltas(std::list & deltas) const + { + std::list subdeltas; + auto it = deltas.begin(); + + for (; it != deltas.end(); ++it) + { + if (std::holds_alternative(it->operation)) + { + ++it; + break; + } + } + + if (it == deltas.end()) + subdeltas = std::move(deltas); + else + subdeltas.splice(subdeltas.end(), deltas, deltas.begin(), it); + + return subdeltas; + } + + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); - auto & deltas = storage.uncommitted_state.deltas; // the deltas will have at least SubDeltaEnd or FailedMultiDelta assert(!deltas.empty()); if (auto * failed_multi = std::get_if(&deltas.front().operation)) @@ -1854,21 +1966,20 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro response.responses[i]->error = failed_multi->error_codes[i]; } - storage.uncommitted_state.commit(zxid); return response_ptr; } for (size_t i = 0; i < concrete_requests.size(); ++i) { - response.responses[i] = concrete_requests[i]->process(storage, zxid); - storage.uncommitted_state.commit(zxid); + std::list subdeltas = getSubdeltas(deltas); + response.responses[i] = concrete_requests[i]->process(storage, std::move(subdeltas)); } response.error = Coordination::Error::ZOK; return response_ptr; } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); @@ -1876,7 +1987,8 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro for (size_t i = 0; i < concrete_requests.size(); ++i) { - response.responses[i] = concrete_requests[i]->processLocal(storage, zxid); + std::list subdeltas = getSubdeltas(deltas); + response.responses[i] = concrete_requests[i]->processLocal(storage, std::move(subdeltas)); } response.error = Coordination::Error::ZOK; @@ -1899,7 +2011,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - Coordination::ZooKeeperResponsePtr process(KeeperStorage &, int64_t) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage &, std::list /* deltas */) const override { throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Called process on close request"); } @@ -1908,7 +2020,7 @@ struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestPro struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor { using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector + std::list preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast(*zk_request); @@ -1917,29 +2029,32 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc if (auth_request.scheme != "digest" || std::count(auth_request.data.begin(), auth_request.data.end(), ':') != 1) return {KeeperStorage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; - std::vector new_deltas; + std::list new_deltas; auto auth_digest = KeeperStorage::generateDigest(auth_request.data); if (auth_digest == storage.superdigest) { - KeeperStorage::AuthID auth{"super", ""}; + auto auth = std::make_shared(); + auth->scheme = "super"; new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(auth)}); } else { - KeeperStorage::AuthID new_auth{auth_request.scheme, auth_digest}; - if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return new_auth == auth_id; })) + auto new_auth = std::make_shared(); + new_auth->scheme = auth_request.scheme; + new_auth->id = std::move(auth_digest); + if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return *new_auth == auth_id; })) new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(new_auth)}); } return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); - if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) auth_response.error = result; return response_ptr; @@ -2030,7 +2145,7 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() } -UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const +UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::list & new_deltas) const { if (!keeper_context->digestEnabled()) return current_digest; @@ -2058,7 +2173,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec updated_nodes.insert_or_assign(delta.path, nullptr); }, - [&](const UpdateNodeDelta & update_delta) + [&](const UpdateNodeStatDelta & update_delta) { std::shared_ptr node{nullptr}; @@ -2073,7 +2188,25 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec else node = updated_node_it->second; - update_delta.update_fn(*node); + node->stat = update_delta.new_stats; + node->seq_num = update_delta.new_seq_num; + }, + [&](const UpdateNodeDataDelta & update_delta) + { + std::shared_ptr node{nullptr}; + + auto updated_node_it = updated_nodes.find(delta.path); + if (updated_node_it == updated_nodes.end()) + { + node = std::make_shared(); + node->shallowCopy(*uncommitted_state.getNode(delta.path)); + current_digest -= node->getDigest(delta.path); + updated_nodes.emplace(delta.path, node); + } + else + node = updated_node_it->second; + + node->setData(update_delta.new_data); }, [](auto && /* delta */) {}}, delta.operation); @@ -2102,43 +2235,50 @@ void KeeperStorage::preprocessRequest( if (!initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage system nodes are not initialized"); - int64_t last_zxid = getNextZXID() - 1; + TransactionInfo * transaction; + uint64_t new_digest = 0; - if (uncommitted_transactions.empty()) { - // if we have no uncommitted transactions it means the last zxid is possibly loaded from snapshot - if (last_zxid != old_snapshot_zxid && new_last_zxid <= last_zxid) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", - new_last_zxid, last_zxid); - } - else - { - if (last_zxid == new_last_zxid && digest && checkDigest(*digest, getNodesDigest(false))) - // we found the preprocessed request with the same ZXID, we can skip it - return; + std::lock_guard lock(transaction_mutex); + int64_t last_zxid = getNextZXIDLocked(lock) - 1; - if (new_last_zxid <= last_zxid) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", - new_last_zxid, last_zxid); + if (uncommitted_transactions.empty()) + { + // if we have no uncommitted transactions it means the last zxid is possibly loaded from snapshot + if (last_zxid != old_snapshot_zxid && new_last_zxid <= last_zxid) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", + new_last_zxid, last_zxid); + } + else + { + if (last_zxid == new_last_zxid && digest && checkDigest(*digest, getNodesDigest(false))) + // we found the preprocessed request with the same ZXID, we can skip it + return; + + if (new_last_zxid <= last_zxid) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Got new ZXID ({}) smaller or equal to current ZXID ({}). It's a bug", + new_last_zxid, last_zxid); + } + + transaction = &uncommitted_transactions.emplace_back(TransactionInfo{.zxid = new_last_zxid, .nodes_digest = {}}); + new_digest = getNodesDigest(false).value; } - std::vector new_deltas; - TransactionInfo transaction{.zxid = new_last_zxid, .nodes_digest = {}}; - uint64_t new_digest = getNodesDigest(false).value; + std::list new_deltas; SCOPE_EXIT({ if (keeper_context->digestEnabled()) // if the version of digest we got from the leader is the same as the one this instances has, we can simply copy the value // and just check the digest on the commit // a mistake can happen while applying the changes to the uncommitted_state so for now let's just recalculate the digest here also - transaction.nodes_digest = Digest{CURRENT_DIGEST_VERSION, new_digest}; + transaction->nodes_digest = Digest{CURRENT_DIGEST_VERSION, new_digest}; else - transaction.nodes_digest = Digest{DigestVersion::NO_DIGEST}; + transaction->nodes_digest = Digest{DigestVersion::NO_DIGEST}; - uncommitted_transactions.emplace_back(transaction); + uncommitted_state.applyDeltas(new_deltas); uncommitted_state.addDeltas(std::move(new_deltas)); }); @@ -2146,30 +2286,30 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { - auto session_ephemerals = ephemerals.find(session_id); - if (session_ephemerals != ephemerals.end()) - { - for (const auto & ephemeral_path : session_ephemerals->second) - { - new_deltas.emplace_back - ( - parentNodePath(ephemeral_path).toString(), - new_last_zxid, - UpdateNodeDelta - { - [ephemeral_path](Node & parent) - { - ++parent.stat.cversion; - --parent.stat.numChildren; - } - } - ); + //auto session_ephemerals = ephemerals.find(session_id); + //if (session_ephemerals != ephemerals.end()) + //{ + // for (const auto & ephemeral_path : session_ephemerals->second) + // { + // new_deltas.emplace_back + // ( + // parentNodePath(ephemeral_path).toString(), + // new_last_zxid, + // UpdateNodeDelta + // { + // [ephemeral_path](Node & parent) + // { + // ++parent.stat.cversion; + // --parent.stat.numChildren; + // } + // } + // ); - new_deltas.emplace_back(ephemeral_path, transaction.zxid, RemoveNodeDelta{.ephemeral_owner = session_id}); - } + // new_deltas.emplace_back(ephemeral_path, transaction->zxid, RemoveNodeDelta{.ephemeral_owner = session_id}); + // } - ephemerals.erase(session_ephemerals); - } + // ephemerals.erase(session_ephemerals); + //} new_digest = calculateNodesDigest(new_digest, new_deltas); return; @@ -2177,11 +2317,11 @@ void KeeperStorage::preprocessRequest( if (check_acl && !request_processor->checkAuth(*this, session_id, false)) { - uncommitted_state.deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); + new_deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); return; } - new_deltas = request_processor->preprocess(*this, transaction.zxid, session_id, time, new_digest, *keeper_context); + new_deltas = request_processor->preprocess(*this, transaction->zxid, session_id, time, new_digest, *keeper_context); } KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( @@ -2194,20 +2334,26 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( if (!initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage system nodes are not initialized"); - if (new_last_zxid) + int64_t commit_zxid = 0; { - if (uncommitted_transactions.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to commit a ZXID ({}) which was not preprocessed", *new_last_zxid); + std::lock_guard lock(transaction_mutex); + if (new_last_zxid) + { + if (uncommitted_transactions.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to commit a ZXID ({}) which was not preprocessed", *new_last_zxid); - if (uncommitted_transactions.front().zxid != *new_last_zxid) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Trying to commit a ZXID {} while the next ZXID to commit is {}", - *new_last_zxid, - uncommitted_transactions.front().zxid); + if (uncommitted_transactions.front().zxid != *new_last_zxid) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Trying to commit a ZXID {} while the next ZXID to commit is {}", + *new_last_zxid, + uncommitted_transactions.front().zxid); - zxid = *new_last_zxid; - uncommitted_transactions.pop_front(); + zxid = *new_last_zxid; + uncommitted_transactions.pop_front(); + } + + commit_zxid = zxid; } KeeperStorage::ResponsesForSessions results; @@ -2215,15 +2361,20 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// ZooKeeper update sessions expirity for each request, not only for heartbeats session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); + std::list deltas; + { + std::lock_guard lock(uncommitted_state.deltas_mutex); + auto it = uncommitted_state.deltas.begin(); + for (; it != uncommitted_state.deltas.end() && it->zxid == commit_zxid; ++it) + ; + + deltas.splice(deltas.end(), uncommitted_state.deltas, uncommitted_state.deltas.begin(), it); + } + if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { - commit(zxid); - - for (const auto & delta : uncommitted_state.deltas) + for (const auto & delta : deltas) { - if (delta.zxid > zxid) - break; - if (std::holds_alternative(delta.operation)) { auto responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); @@ -2231,17 +2382,20 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } } - uncommitted_state.commit(zxid); + commit(std::move(deltas)); clearDeadWatches(session_id); - auto auth_it = session_and_auth.find(session_id); - if (auth_it != session_and_auth.end()) - session_and_auth.erase(auth_it); + { + std::lock_guard lock(auth_mutex); + auto auth_it = session_and_auth.find(session_id); + if (auth_it != session_and_auth.end()) + session_and_auth.erase(auth_it); + } /// Finish connection auto response = std::make_shared(); response->xid = zk_request->xid; - response->zxid = getZXID(); + response->zxid = commit_zxid; session_expiry_queue.remove(session_id); session_and_timeout.erase(session_id); results.push_back(ResponseForSession{session_id, response}); @@ -2249,9 +2403,9 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special { KeeperStorageRequestProcessorPtr storage_request = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); - auto response = storage_request->process(*this, zxid); + auto response = storage_request->process(*this, std::move(deltas)); response->xid = zk_request->xid; - response->zxid = getZXID(); + response->zxid = commit_zxid; results.push_back(ResponseForSession{session_id, response}); } @@ -2271,13 +2425,12 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } else { - response = request_processor->processLocal(*this, zxid); + response = request_processor->processLocal(*this, std::move(deltas)); } } else { - response = request_processor->process(*this, zxid); - uncommitted_state.commit(zxid); + response = request_processor->process(*this, std::move(deltas)); } /// Watches for this requests are added to the watches lists @@ -2310,7 +2463,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } response->xid = zk_request->xid; - response->zxid = getZXID(); + response->zxid = commit_zxid; results.push_back(ResponseForSession{session_id, response}); } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 844cbf85c1e..ab1ef201a4e 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -11,6 +11,8 @@ #include #include +#include + #include namespace DB @@ -138,11 +140,13 @@ public: int64_t session_id_counter{1}; - SessionAndAuth session_and_auth; + mutable std::mutex auth_mutex; + SessionAndAuth session_and_auth TSA_GUARDED_BY(auth_mutex); /// Main hashtable with nodes. Contain all information about data. /// All other structures expect session_and_timeout can be restored from /// container. + mutable std::mutex container_mutex; Container container; // Applying ZooKeeper request to storage consists of two steps: @@ -166,18 +170,33 @@ public: struct RemoveNodeDelta { int32_t version{-1}; - int64_t ephemeral_owner{0}; + Coordination::Stat stat; + Coordination::ACLs acls; + String data; }; - struct UpdateNodeDelta + struct UpdateNodeStatDelta { - std::function update_fn; + explicit UpdateNodeStatDelta(const KeeperStorage::Node & node); + + Coordination::Stat old_stats; + Coordination::Stat new_stats; + int32_t old_seq_num; + int32_t new_seq_num; + int32_t version{-1}; + }; + + struct UpdateNodeDataDelta + { + std::string old_data; + std::string new_data; int32_t version{-1}; }; struct SetACLDelta { - Coordination::ACLs acls; + Coordination::ACLs old_acls; + Coordination::ACLs new_acls; int32_t version{-1}; }; @@ -199,11 +218,19 @@ public: struct AddAuthDelta { int64_t session_id; - AuthID auth_id; + std::shared_ptr auth_id; }; - using Operation = std:: - variant; + using Operation = std::variant< + CreateNodeDelta, + RemoveNodeDelta, + UpdateNodeStatDelta, + UpdateNodeDataDelta, + SetACLDelta, + AddAuthDelta, + ErrorDelta, + SubDeltaEnd, + FailedMultiDelta>; struct Delta { @@ -222,15 +249,16 @@ public: { explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { } - void addDelta(Delta new_delta); - void addDeltas(std::vector new_deltas); + void addDeltas(std::list new_deltas); void commit(int64_t commit_zxid); void rollback(int64_t rollback_zxid); std::shared_ptr getNode(StringRef path) const; Coordination::ACLs getACLs(StringRef path) const; + void applyDeltas(const std::list & new_deltas); void applyDelta(const Delta & delta); + void rollbackDelta(const Delta & delta); bool hasACL(int64_t session_id, bool is_local, std::function predicate) { @@ -238,13 +266,13 @@ public: { for (const auto & auth : auth_ids) { - using TAuth = std::remove_reference_t; + using TAuth = std::remove_cvref_t; const AuthID * auth_ptr = nullptr; - if constexpr (std::is_pointer_v) - auth_ptr = auth; - else + if constexpr (std::same_as) auth_ptr = &auth; + else + auth_ptr = auth.get(); if (predicate(*auth_ptr)) return true; @@ -253,30 +281,34 @@ public: }; if (is_local) + { + std::lock_guard lock(storage.auth_mutex); return check_auth(storage.session_and_auth[session_id]); - - if (check_auth(storage.session_and_auth[session_id])) - return true; + } // check if there are uncommitted const auto auth_it = session_and_auth.find(session_id); if (auth_it == session_and_auth.end()) return false; - return check_auth(auth_it->second); + if (check_auth(auth_it->second)) + return true; + + std::lock_guard lock(storage.auth_mutex); + return check_auth(storage.session_and_auth[session_id]); } void forEachAuthInSession(int64_t session_id, std::function func) const; std::shared_ptr tryGetNodeFromStorage(StringRef path) const; - std::unordered_map> session_and_auth; + std::unordered_map>> session_and_auth; struct UncommittedNode { std::shared_ptr node{nullptr}; - Coordination::ACLs acls{}; - int64_t zxid{0}; + std::optional acls{}; + std::vector applied_zxids{}; }; struct Hash @@ -303,9 +335,9 @@ public: }; mutable std::unordered_map nodes; - std::unordered_map, Hash, Equal> deltas_for_path; - std::list deltas; + mutable std::mutex deltas_mutex; + std::list deltas TSA_GUARDED_BY(deltas_mutex); KeeperStorage & storage; }; @@ -315,7 +347,7 @@ public: // with zxid > last_zxid void applyUncommittedState(KeeperStorage & other, int64_t last_zxid); - Coordination::Error commit(int64_t zxid); + Coordination::Error commit(std::list deltas); // Create node in the storage // Returns false if it failed to create the node, true otherwise @@ -348,8 +380,10 @@ public: /// ACLMap for more compact ACLs storage inside nodes. ACLMap acl_map; + mutable std::mutex transaction_mutex; + /// Global id of all requests applied to storage - int64_t zxid{0}; + int64_t zxid = 0; // older Keeper node (pre V5 snapshots) can create snapshots and receive logs from newer Keeper nodes // this can lead to some inconsistencies, e.g. from snapshot it will use log_idx as zxid @@ -364,7 +398,7 @@ public: Digest nodes_digest; }; - std::deque uncommitted_transactions; + std::list uncommitted_transactions; uint64_t nodes_digest{0}; @@ -377,15 +411,10 @@ public: void clearDeadWatches(int64_t session_id); /// Get current committed zxid - int64_t getZXID() const { return zxid; } + int64_t getZXID() const; - int64_t getNextZXID() const - { - if (uncommitted_transactions.empty()) - return zxid + 1; - - return uncommitted_transactions.back().zxid + 1; - } + int64_t getNextZXID() const; + int64_t getNextZXIDLocked(std::lock_guard & lock) const; Digest getNodesDigest(bool committed) const; @@ -393,7 +422,7 @@ public: const String superdigest; - bool initialized{false}; + std::atomic initialized{false}; KeeperStorage(int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, bool initialize_system_nodes = true); @@ -415,7 +444,7 @@ public: session_expiry_queue.addNewSessionOrUpdate(session_id, session_timeout_ms); } - UInt64 calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const; + UInt64 calculateNodesDigest(UInt64 current_digest, const std::list & new_deltas) const; /// Process user request and return response. /// check_acl = false only when converting data from ZooKeeper. diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 79929c4e66e..d7ce097b795 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -147,7 +147,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L return max_zxid; } -void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log) +void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log) TSA_NO_THREAD_SAFETY_ANALYSIS { LOG_INFO(log, "Deserializing storage snapshot {}", snapshot_path); int64_t zxid = getZxidFromName(snapshot_path); @@ -474,7 +474,7 @@ bool hasErrorsInMultiRequest(Coordination::ZooKeeperRequestPtr request) } -bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*log*/) +bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*log*/) TSA_NO_THREAD_SAFETY_ANALYSIS { int64_t checksum; Coordination::read(checksum, in); @@ -553,7 +553,7 @@ void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string LOG_INFO(log, "Finished {} deserialization, totally read {} records", log_path, counter); } -void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log) +void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log) TSA_NO_THREAD_SAFETY_ANALYSIS { namespace fs = std::filesystem; std::map existing_logs; From dd1bb579df57480f5df75e240e4c46f688c5204a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 5 Sep 2023 10:49:19 +0000 Subject: [PATCH 002/128] Better --- src/Coordination/KeeperStateMachine.cpp | 2 - src/Coordination/KeeperStorage.cpp | 223 +++++++++++------------- src/Coordination/KeeperStorage.h | 22 +-- 3 files changed, 116 insertions(+), 131 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 9d96a76e06a..c0f66f246d6 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -509,7 +509,6 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; - std::lock_guard lock(storage_and_responses_lock); storage->rollbackRequest(request_for_session.zxid, allow_missing); } @@ -756,7 +755,6 @@ std::vector KeeperStateMachine::getDeadSessions() int64_t KeeperStateMachine::getNextZxid() const { - std::lock_guard lock(storage_and_responses_lock); return storage->getNextZXID(); } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 0978f5f965e..f881b75a808 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -317,7 +317,7 @@ Overloaded(Ts...) -> Overloaded; std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const { - std::lock_guard lock(storage.container_mutex); + std::lock_guard lock(storage.storage_mutex); if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; @@ -455,7 +455,7 @@ void KeeperStorage::UncommittedState::applyDeltas(const std::list & new_d else if (const auto * auth_delta = std::get_if(&delta.operation)) { auto & uncommitted_auth = session_and_auth[auth_delta->session_id]; - uncommitted_auth.emplace_back(auth_delta->auth_id); + uncommitted_auth.push_back(std::pair{delta.zxid, auth_delta->auth_id}); } } } @@ -466,62 +466,30 @@ void KeeperStorage::UncommittedState::addDeltas(std::list new_deltas) deltas.splice(deltas.end(), std::move(new_deltas)); } -//void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) -//{ -// assert(deltas.empty() || deltas.front().zxid >= commit_zxid); -// -// // collect nodes that have no further modification in the current transaction -// std::unordered_set modified_nodes; -// -// while (!deltas.empty() && deltas.front().zxid == commit_zxid) -// { -// if (std::holds_alternative(deltas.front().operation)) -// { -// deltas.pop_front(); -// break; -// } -// -// auto & front_delta = deltas.front(); -// -// if (!front_delta.path.empty()) -// { -// auto & path_deltas = deltas_for_path.at(front_delta.path); -// assert(path_deltas.front() == &front_delta); -// path_deltas.pop_front(); -// if (path_deltas.empty()) -// { -// deltas_for_path.erase(front_delta.path); -// -// // no more deltas for path -> no modification -// modified_nodes.insert(std::move(front_delta.path)); -// } -// else if (path_deltas.front()->zxid > commit_zxid) -// { -// // next delta has a zxid from a different transaction -> no modification in this transaction -// modified_nodes.insert(std::move(front_delta.path)); -// } -// } -// else if (auto * add_auth = std::get_if(&front_delta.operation)) -// { -// auto & uncommitted_auth = session_and_auth[add_auth->session_id]; -// assert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id); -// uncommitted_auth.pop_front(); -// if (uncommitted_auth.empty()) -// session_and_auth.erase(add_auth->session_id); -// -// } -// -// deltas.pop_front(); -// } -// -// // delete all cached nodes that were not modified after the commit_zxid -// // we only need to check the nodes that were modified in this transaction -// for (const auto & node : modified_nodes) -// { -// if (nodes[node].zxid == commit_zxid) -// nodes.erase(node); -// } -//} +void KeeperStorage::UncommittedState::cleanup(int64_t commit_zxid) +{ + for (auto it = nodes.begin(); it != nodes.end();) + { + auto & applied_zxids = it->second.applied_zxids; + std::erase_if(applied_zxids, [commit_zxid](auto current_zxid) { return current_zxid <= commit_zxid; }); + if (applied_zxids.empty()) + it = nodes.erase(it); + else + ++it; + } + + for (auto it = session_and_auth.begin(); it != session_and_auth.end();) + { + auto & auths = it->second; + std::erase_if(auths, [commit_zxid](auto auth_pair) { return auth_pair.first <= commit_zxid; }); + if (auths.empty()) + it = session_and_auth.erase(it); + else + ++it; + } + + +} void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) { @@ -575,7 +543,7 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) else if (auto * add_auth = std::get_if(&delta.operation)) { auto & uncommitted_auth = session_and_auth[add_auth->session_id]; - if (uncommitted_auth.back() == add_auth->auth_id) + if (uncommitted_auth.back().second == add_auth->auth_id) { uncommitted_auth.pop_back(); if (uncommitted_auth.empty()) @@ -634,22 +602,22 @@ void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, s if constexpr (std::same_as) auth_ptr = &auth; else - auth_ptr = auth.get(); + auth_ptr = auth.second.get(); func(*auth_ptr); } }; { - std::lock_guard lock(storage.auth_mutex); + std::lock_guard lock(storage.storage_mutex); // for committed - if (storage.session_and_auth.contains(session_id)) - call_for_each_auth(storage.session_and_auth.at(session_id)); + if (auto auth_it = storage.session_and_auth.find(session_id); auth_it != storage.session_and_auth.end()) + call_for_each_auth(auth_it->second); } // for uncommitted - if (session_and_auth.contains(session_id)) - call_for_each_auth(session_and_auth.at(session_id)); + if (auto auth_it = session_and_auth.find(session_id); auth_it != session_and_auth.end()) + call_for_each_auth(auth_it->second); } namespace @@ -686,29 +654,29 @@ int64_t KeeperStorage::getNextZXID() const return getNextZXIDLocked(lock); } -void KeeperStorage::applyUncommittedState(KeeperStorage & /*other*/, int64_t /*last_zxid*/) +void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_zxid) TSA_NO_THREAD_SAFETY_ANALYSIS { - //for (const auto & transaction : uncommitted_transactions) - //{ - // if (transaction.zxid <= last_zxid) - // continue; - // other.uncommitted_transactions.push_back(transaction); - //} + for (const auto & transaction : uncommitted_transactions) + { + if (transaction.zxid <= last_zxid) + continue; + other.uncommitted_transactions.push_back(transaction); + } - //auto it = uncommitted_state.deltas.begin(); + auto it = uncommitted_state.deltas.begin(); - //for (; it != uncommitted_state.deltas.end(); ++it) - //{ - // if (it->zxid <= last_zxid) - // continue; + for (; it != uncommitted_state.deltas.end(); ++it) + { + if (it->zxid <= last_zxid) + continue; - // other.uncommitted_state.addDelta(*it); - //} + other.uncommitted_state.applyDelta(*it); + other.uncommitted_state.deltas.push_back(*it); + } } Coordination::Error KeeperStorage::commit(std::list deltas) { - std::lock_guard lock(container_mutex); // Deltas are added with increasing ZXIDs // If there are no deltas for the commit_zxid (e.g. read requests), we instantly return // on first delta @@ -787,7 +755,6 @@ Coordination::Error KeeperStorage::commit(std::list deltas) } else if constexpr (std::same_as) { - std::lock_guard auth_lock(auth_mutex); session_and_auth[operation.session_id].emplace_back(std::move(*operation.auth_id)); return Coordination::Error::ZOK; } @@ -939,7 +906,6 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ { if (is_local) { - std::lock_guard lock(storage.container_mutex); auto node_it = storage.container.find(path); if (node_it == storage.container.end()) return {}; @@ -2286,30 +2252,33 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { - //auto session_ephemerals = ephemerals.find(session_id); - //if (session_ephemerals != ephemerals.end()) - //{ - // for (const auto & ephemeral_path : session_ephemerals->second) - // { - // new_deltas.emplace_back - // ( - // parentNodePath(ephemeral_path).toString(), - // new_last_zxid, - // UpdateNodeDelta - // { - // [ephemeral_path](Node & parent) - // { - // ++parent.stat.cversion; - // --parent.stat.numChildren; - // } - // } - // ); + auto session_ephemerals = ephemerals.find(session_id); + if (session_ephemerals != ephemerals.end()) + { + for (const auto & ephemeral_path : session_ephemerals->second) + { + auto parent_node_path = parentNodePath(ephemeral_path); + auto parent_node = uncommitted_state.getNode(parent_node_path); + UpdateNodeStatDelta parent_update_delta(*parent_node); + ++parent_update_delta.new_stats.cversion; + --parent_update_delta.new_stats.numChildren; - // new_deltas.emplace_back(ephemeral_path, transaction->zxid, RemoveNodeDelta{.ephemeral_owner = session_id}); - // } + new_deltas.emplace_back + ( + parent_node_path.toString(), + new_last_zxid, + std::move(parent_update_delta) + ); - // ephemerals.erase(session_ephemerals); - //} + auto node = uncommitted_state.getNode(ephemeral_path); + new_deltas.emplace_back( + ephemeral_path, + transaction->zxid, + RemoveNodeDelta{.stat = node->stat, .acls = uncommitted_state.getACLs(ephemeral_path), .data = node->getData()}); + } + + ephemerals.erase(session_ephemerals); + } new_digest = calculateNodesDigest(new_digest, new_deltas); return; @@ -2322,6 +2291,8 @@ void KeeperStorage::preprocessRequest( } new_deltas = request_processor->preprocess(*this, transaction->zxid, session_id, time, new_digest, *keeper_context); + + uncommitted_state.cleanup(getZXID()); } KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( @@ -2349,18 +2320,14 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( *new_last_zxid, uncommitted_transactions.front().zxid); - zxid = *new_last_zxid; - uncommitted_transactions.pop_front(); + commit_zxid = *new_last_zxid; + } + else + { + commit_zxid = zxid; } - - commit_zxid = zxid; } - KeeperStorage::ResponsesForSessions results; - - /// ZooKeeper update sessions expirity for each request, not only for heartbeats - session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); - std::list deltas; { std::lock_guard lock(uncommitted_state.deltas_mutex); @@ -2371,6 +2338,11 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( deltas.splice(deltas.end(), uncommitted_state.deltas, uncommitted_state.deltas.begin(), it); } + KeeperStorage::ResponsesForSessions results; + + /// ZooKeeper update sessions expirity for each request, not only for heartbeats + session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); + if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { for (const auto & delta : deltas) @@ -2382,16 +2354,16 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } } - commit(std::move(deltas)); - - clearDeadWatches(session_id); { - std::lock_guard lock(auth_mutex); + std::lock_guard lock(storage_mutex); + commit(std::move(deltas)); auto auth_it = session_and_auth.find(session_id); if (auth_it != session_and_auth.end()) session_and_auth.erase(auth_it); } + clearDeadWatches(session_id); + /// Finish connection auto response = std::make_shared(); response->xid = zk_request->xid; @@ -2403,7 +2375,11 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special { KeeperStorageRequestProcessorPtr storage_request = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); - auto response = storage_request->process(*this, std::move(deltas)); + Coordination::ZooKeeperResponsePtr response = nullptr; + { + std::lock_guard lock(storage_mutex); + response = storage_request->process(*this, std::move(deltas)); + } response->xid = zk_request->xid; response->zxid = commit_zxid; @@ -2416,6 +2392,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( if (is_local) { + std::lock_guard lock(storage_mutex); assert(zk_request->isReadRequest()); if (check_acl && !request_processor->checkAuth(*this, session_id, true)) { @@ -2430,6 +2407,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } else { + std::lock_guard lock(storage_mutex); response = request_processor->process(*this, std::move(deltas)); } @@ -2468,11 +2446,19 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( results.push_back(ResponseForSession{session_id, response}); } + { + std::lock_guard lock(transaction_mutex); + if (new_last_zxid) + uncommitted_transactions.pop_front(); + zxid = commit_zxid; + } + return results; } void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) { + std::unique_lock transaction_lock(transaction_mutex); if (allow_missing && (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid < rollback_zxid)) return; @@ -2488,6 +2474,7 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) try { uncommitted_transactions.pop_back(); + transaction_lock.unlock(); uncommitted_state.rollback(rollback_zxid); } catch (...) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index ab1ef201a4e..84bf7d297b4 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -138,15 +139,17 @@ public: using SessionAndAuth = std::unordered_map; using Watches = std::map; + mutable SharedMutex main_mutex; + + mutable std::mutex storage_mutex; + int64_t session_id_counter{1}; - mutable std::mutex auth_mutex; - SessionAndAuth session_and_auth TSA_GUARDED_BY(auth_mutex); + SessionAndAuth session_and_auth; /// Main hashtable with nodes. Contain all information about data. /// All other structures expect session_and_timeout can be restored from /// container. - mutable std::mutex container_mutex; Container container; // Applying ZooKeeper request to storage consists of two steps: @@ -250,7 +253,7 @@ public: explicit UncommittedState(KeeperStorage & storage_) : storage(storage_) { } void addDeltas(std::list new_deltas); - void commit(int64_t commit_zxid); + void cleanup(int64_t commit_zxid); void rollback(int64_t rollback_zxid); std::shared_ptr getNode(StringRef path) const; @@ -272,7 +275,7 @@ public: if constexpr (std::same_as) auth_ptr = &auth; else - auth_ptr = auth.get(); + auth_ptr = auth.second.get(); if (predicate(*auth_ptr)) return true; @@ -281,10 +284,7 @@ public: }; if (is_local) - { - std::lock_guard lock(storage.auth_mutex); return check_auth(storage.session_and_auth[session_id]); - } // check if there are uncommitted const auto auth_it = session_and_auth.find(session_id); @@ -294,7 +294,7 @@ public: if (check_auth(auth_it->second)) return true; - std::lock_guard lock(storage.auth_mutex); + std::lock_guard lock(storage.storage_mutex); return check_auth(storage.session_and_auth[session_id]); } @@ -302,7 +302,7 @@ public: std::shared_ptr tryGetNodeFromStorage(StringRef path) const; - std::unordered_map>> session_and_auth; + std::unordered_map>>> session_and_auth; struct UncommittedNode { @@ -402,7 +402,7 @@ public: uint64_t nodes_digest{0}; - bool finalized{false}; + std::atomic finalized{false}; /// Currently active watches (node_path -> subscribed sessions) Watches watches; From 6be1d0724a2edc71fc2af7510cd504289946c19f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 6 Sep 2023 13:04:08 +0000 Subject: [PATCH 003/128] More mutex --- src/Coordination/ACLMap.cpp | 5 + src/Coordination/ACLMap.h | 2 + src/Coordination/KeeperStateMachine.cpp | 66 +++--- src/Coordination/KeeperStateMachine.h | 7 +- src/Coordination/KeeperStorage.cpp | 277 +++++++++++++++++------ src/Coordination/KeeperStorage.h | 92 ++++---- src/Coordination/ZooKeeperDataReader.cpp | 2 +- 7 files changed, 299 insertions(+), 152 deletions(-) diff --git a/src/Coordination/ACLMap.cpp b/src/Coordination/ACLMap.cpp index 5c4a8f397ca..f92e497119e 100644 --- a/src/Coordination/ACLMap.cpp +++ b/src/Coordination/ACLMap.cpp @@ -45,6 +45,7 @@ uint64_t ACLMap::convertACLs(const Coordination::ACLs & acls) if (acls.empty()) return 0; + std::lock_guard lock(map_mutex); if (acl_to_num.contains(acls)) return acl_to_num[acls]; @@ -62,6 +63,7 @@ Coordination::ACLs ACLMap::convertNumber(uint64_t acls_id) const if (acls_id == 0) return Coordination::ACLs{}; + std::lock_guard lock(map_mutex); if (!num_to_acl.contains(acls_id)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown ACL id {}. It's a bug", acls_id); @@ -70,6 +72,7 @@ Coordination::ACLs ACLMap::convertNumber(uint64_t acls_id) const void ACLMap::addMapping(uint64_t acls_id, const Coordination::ACLs & acls) { + std::lock_guard lock(map_mutex); num_to_acl[acls_id] = acls; acl_to_num[acls] = acls_id; max_acl_id = std::max(acls_id + 1, max_acl_id); /// max_acl_id pointer next slot @@ -77,11 +80,13 @@ void ACLMap::addMapping(uint64_t acls_id, const Coordination::ACLs & acls) void ACLMap::addUsage(uint64_t acl_id) { + std::lock_guard lock(map_mutex); usage_counter[acl_id]++; } void ACLMap::removeUsage(uint64_t acl_id) { + std::lock_guard lock(map_mutex); if (!usage_counter.contains(acl_id)) return; diff --git a/src/Coordination/ACLMap.h b/src/Coordination/ACLMap.h index e1b2ce1eff6..7a635951346 100644 --- a/src/Coordination/ACLMap.h +++ b/src/Coordination/ACLMap.h @@ -32,6 +32,8 @@ private: NumToACLMap num_to_acl; UsageCounter usage_counter; uint64_t max_acl_id{1}; + + mutable std::mutex map_mutex; public: /// Convert ACL to number. If it's new ACL than adds it to map diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index c0f66f246d6..2d1d3a7658b 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -271,6 +271,7 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; + std::shared_lock storage_lock(storage_mutex); if (storage->isFinalized()) return false; @@ -291,14 +292,14 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req } if (keeper_context->digestEnabled() && request_for_session.digest) - assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false); + assertDigest(*request_for_session.digest, storage->getNodesDigest(false, /*lock_transaction_mutex=*/true), *request_for_session.request, false); return true; } void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session) { - std::lock_guard _(storage_and_responses_lock); + std::lock_guard lock(process_and_responses_lock); KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session); if (!responses_queue.push(response)) { @@ -393,6 +394,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n } }; + std::shared_lock storage_lock(storage_mutex); const auto op_num = request_for_session->request->getOpNum(); if (op_num == Coordination::OpNum::SessionID) { @@ -406,7 +408,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n response_for_session.session_id = -1; response_for_session.response = response; - std::lock_guard lock(storage_and_responses_lock); + std::lock_guard lock(process_and_responses_lock); session_id = storage->getSessionID(session_id_request.session_timeout_ms); LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); response->session_id = session_id; @@ -420,13 +422,14 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n parsed_request_cache.erase(request_for_session->session_id); } + std::lock_guard lock(process_and_responses_lock); KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); for (auto & response_for_session : responses_for_sessions) try_push(response_for_session); if (keeper_context->digestEnabled() && request_for_session->digest) - assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, true); + assertDigest(*request_for_session->digest, storage->getNodesDigest(true, /*lock_transaction_mutex=*/true), *request_for_session->request, true); } ProfileEvents::increment(ProfileEvents::KeeperCommits); @@ -462,8 +465,6 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) } { /// deserialize and apply snapshot to storage - std::lock_guard lock(storage_and_responses_lock); - SnapshotDeserializationResult snapshot_deserialization_result; if (latest_snapshot_ptr) snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr); @@ -471,6 +472,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx())); + std::unique_lock storage_lock(storage_mutex); /// maybe some logs were preprocessed with log idx larger than the snapshot idx /// we have to apply them to the new storage storage->applyUncommittedState(*snapshot_deserialization_result.storage, snapshot_deserialization_result.storage->getZXID()); @@ -509,14 +511,7 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; - storage->rollbackRequest(request_for_session.zxid, allow_missing); -} - -void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing) -{ - if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) - return; - + std::shared_lock lock(storage_mutex); storage->rollbackRequest(request_for_session.zxid, allow_missing); } @@ -535,7 +530,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res auto snapshot_meta_copy = nuraft::snapshot::deserialize(*snp_buf); CreateSnapshotTask snapshot_task; { /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking. - std::lock_guard lock(storage_and_responses_lock); + std::unique_lock lock(storage_mutex); snapshot_task.snapshot = std::make_shared(storage.get(), snapshot_meta_copy, getClusterConfig()); } @@ -582,8 +577,6 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res } { - /// Destroy snapshot with lock - std::lock_guard lock(storage_and_responses_lock); LOG_TRACE(log, "Clearing garbage after snapshot"); /// Turn off "snapshot mode" and clear outdate part of storage state storage->clearGarbageAfterSnapshot(); @@ -732,8 +725,10 @@ int KeeperStateMachine::read_logical_snp_obj( void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session) { + std::shared_lock storage_lock(storage_mutex); + /// Pure local request, just process it with storage - std::lock_guard lock(storage_and_responses_lock); + std::lock_guard lock(process_and_responses_lock); auto responses = storage->processRequest( request_for_session.request, request_for_session.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); for (const auto & response : responses) @@ -743,96 +738,97 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi void KeeperStateMachine::shutdownStorage() { - std::lock_guard lock(storage_and_responses_lock); + std::unique_lock storage_lock(storage_mutex); storage->finalize(); } std::vector KeeperStateMachine::getDeadSessions() { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getDeadSessions(); } int64_t KeeperStateMachine::getNextZxid() const { + std::shared_lock storage_lock(storage_mutex); return storage->getNextZXID(); } KeeperStorage::Digest KeeperStateMachine::getNodesDigest() const { - std::lock_guard lock(storage_and_responses_lock); - return storage->getNodesDigest(false); + std::shared_lock storage_lock(storage_mutex); + return storage->getNodesDigest(false, /*lock_transaction_mutex=*/true); } uint64_t KeeperStateMachine::getLastProcessedZxid() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getZXID(); } uint64_t KeeperStateMachine::getNodesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getNodesCount(); } uint64_t KeeperStateMachine::getTotalWatchesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getTotalWatchesCount(); } uint64_t KeeperStateMachine::getWatchedPathsCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getWatchedPathsCount(); } uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getSessionsWithWatchesCount(); } uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getTotalEphemeralNodesCount(); } uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getSessionWithEphemeralNodesCount(); } void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); storage->dumpWatches(buf); } void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); storage->dumpWatchesByPath(buf); } void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); storage->dumpSessionsAndEphemerals(buf); } uint64_t KeeperStateMachine::getApproximateDataSize() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getApproximateDataSize(); } uint64_t KeeperStateMachine::getKeyArenaSize() const { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); return storage->getArenaDataSize(); } @@ -858,7 +854,7 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const void KeeperStateMachine::recalculateStorageStats() { - std::lock_guard lock(storage_and_responses_lock); + std::shared_lock storage_lock(storage_mutex); LOG_INFO(log, "Recalculating storage stats"); storage->recalculateStats(); LOG_INFO(log, "Done recalculating storage stats"); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 670b15394ce..39410044e20 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -132,6 +132,8 @@ public: void reconfigure(const KeeperStorage::RequestForSession& request_for_session); private: + mutable SharedMutex storage_mutex; + CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored /// in memory in compressed (serialized) format. @@ -161,7 +163,7 @@ private: /// we can get strange cases when, for example client send read request with /// watch and after that receive watch response and only receive response /// for request. - mutable std::mutex storage_and_responses_lock; + mutable std::mutex process_and_responses_lock; std::unordered_map>> parsed_request_cache; uint64_t min_request_size_to_cache{0}; @@ -189,7 +191,6 @@ private: KeeperSnapshotManagerS3 * snapshot_manager_s3; KeeperStorage::ResponseForSession processReconfiguration( - const KeeperStorage::RequestForSession& request_for_session) - TSA_REQUIRES(storage_and_responses_lock); + const KeeperStorage::RequestForSession& request_for_session); }; } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index f881b75a808..c44588680e2 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -535,7 +535,10 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) else if constexpr (std::same_as) { if (operation.stat.ephemeralOwner != 0) + { + std::lock_guard lock(storage.ephemerals_mutex); storage.ephemerals[operation.stat.ephemeralOwner].emplace(delta.path); + } } }, delta.operation); @@ -635,12 +638,12 @@ namespace /// Get current committed zxid int64_t KeeperStorage::getZXID() const -{ +{ std::lock_guard lock(transaction_mutex); return zxid; } -int64_t KeeperStorage::getNextZXIDLocked(std::lock_guard &) const +int64_t KeeperStorage::getNextZXIDLocked() const { if (uncommitted_transactions.empty()) return zxid + 1; @@ -651,7 +654,7 @@ int64_t KeeperStorage::getNextZXIDLocked(std::lock_guard &) const int64_t KeeperStorage::getNextZXID() const { std::lock_guard lock(transaction_mutex); - return getNextZXIDLocked(lock); + return getNextZXIDLocked(); } void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_zxid) TSA_NO_THREAD_SAFETY_ANALYSIS @@ -683,7 +686,7 @@ Coordination::Error KeeperStorage::commit(std::list deltas) for (auto & delta : deltas) { auto result = std::visit( - [&, &path = delta.path](DeltaType & operation) -> Coordination::Error + [&, &path = delta.path](DeltaType & operation) TSA_REQUIRES(storage_mutex) -> Coordination::Error { if constexpr (std::same_as) { @@ -906,6 +909,7 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ { if (is_local) { + std::lock_guard lock(storage.storage_mutex); auto node_it = storage.container.find(path); if (node_it == storage.container.end()) return {}; @@ -966,6 +970,7 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) { + std::lock_guard ephemerals_lock(ephemerals_mutex); auto ephemerals_it = ephemerals.find(session_id); if (ephemerals_it == ephemerals.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path"); @@ -1038,7 +1043,10 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; if (request.is_ephemeral) + { + std::lock_guard lock(storage.ephemerals_mutex); storage.ephemerals[session_id].emplace(path_created); + } int32_t parent_cversion = request.parent_cversion; @@ -1080,7 +1088,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); @@ -1133,7 +1141,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); @@ -1176,12 +1184,12 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); return processImpl(storage, std::move(deltas)); @@ -1271,7 +1279,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); @@ -1304,7 +1312,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); @@ -1337,12 +1345,12 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); return processImpl(storage, std::move(deltas)); @@ -1405,7 +1413,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { auto & container = storage.container; @@ -1458,7 +1466,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); @@ -1492,7 +1500,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc const auto & children = node_it->value.getChildren(); response.names.reserve(children.size()); - const auto add_child = [&](const auto child) + const auto add_child = [&](const auto child) TSA_REQUIRES(storage.storage_mutex) { using enum Coordination::ListRequestType; @@ -1527,12 +1535,12 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) TSA_REQUIRES(storage.storage_mutex) { ProfileEvents::increment(ProfileEvents::KeeperListRequest); return processImpl(storage, std::move(deltas)); @@ -1579,7 +1587,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); @@ -1625,12 +1633,12 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); return processImpl(storage, std::move(deltas)); @@ -1690,7 +1698,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); @@ -1733,7 +1741,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, std::list deltas) const TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); @@ -1766,12 +1774,12 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { return processImpl(storage, std::move(deltas)); } @@ -2015,7 +2023,7 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, std::list deltas) const override TSA_REQUIRES(storage.storage_mutex) { Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); @@ -2034,11 +2042,19 @@ void KeeperStorage::finalize() finalized = true; - ephemerals.clear(); + { + std::lock_guard lock(ephemerals_mutex); + ephemerals.clear(); + } - watches.clear(); - list_watches.clear(); - sessions_and_watchers.clear(); + { + std::lock_guard lock(watches_mutex); + watches.clear(); + list_watches.clear(); + sessions_and_watchers.clear(); + } + + std::lock_guard lock(session_mutex); session_expiry_queue.clear(); } @@ -2206,7 +2222,8 @@ void KeeperStorage::preprocessRequest( { std::lock_guard lock(transaction_mutex); - int64_t last_zxid = getNextZXIDLocked(lock) - 1; + int64_t last_zxid = getNextZXIDLocked() - 1; + auto current_digest = getNodesDigest(false, /*lock_transaction_mutex=*/false); if (uncommitted_transactions.empty()) { @@ -2219,7 +2236,7 @@ void KeeperStorage::preprocessRequest( } else { - if (last_zxid == new_last_zxid && digest && checkDigest(*digest, getNodesDigest(false))) + if (last_zxid == new_last_zxid && digest && checkDigest(*digest, current_digest)) // we found the preprocessed request with the same ZXID, we can skip it return; @@ -2230,8 +2247,8 @@ void KeeperStorage::preprocessRequest( new_last_zxid, last_zxid); } + new_digest = current_digest.value; transaction = &uncommitted_transactions.emplace_back(TransactionInfo{.zxid = new_last_zxid, .nodes_digest = {}}); - new_digest = getNodesDigest(false).value; } std::list new_deltas; @@ -2252,32 +2269,35 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { - auto session_ephemerals = ephemerals.find(session_id); - if (session_ephemerals != ephemerals.end()) { - for (const auto & ephemeral_path : session_ephemerals->second) + std::lock_guard lock(ephemerals_mutex); + auto session_ephemerals = ephemerals.find(session_id); + if (session_ephemerals != ephemerals.end()) { - auto parent_node_path = parentNodePath(ephemeral_path); - auto parent_node = uncommitted_state.getNode(parent_node_path); - UpdateNodeStatDelta parent_update_delta(*parent_node); - ++parent_update_delta.new_stats.cversion; - --parent_update_delta.new_stats.numChildren; + for (const auto & ephemeral_path : session_ephemerals->second) + { + auto parent_node_path = parentNodePath(ephemeral_path); + auto parent_node = uncommitted_state.getNode(parent_node_path); + UpdateNodeStatDelta parent_update_delta(*parent_node); + ++parent_update_delta.new_stats.cversion; + --parent_update_delta.new_stats.numChildren; - new_deltas.emplace_back - ( - parent_node_path.toString(), - new_last_zxid, - std::move(parent_update_delta) - ); + new_deltas.emplace_back + ( + parent_node_path.toString(), + new_last_zxid, + std::move(parent_update_delta) + ); - auto node = uncommitted_state.getNode(ephemeral_path); - new_deltas.emplace_back( - ephemeral_path, - transaction->zxid, - RemoveNodeDelta{.stat = node->stat, .acls = uncommitted_state.getACLs(ephemeral_path), .data = node->getData()}); + auto node = uncommitted_state.getNode(ephemeral_path); + new_deltas.emplace_back( + ephemeral_path, + transaction->zxid, + RemoveNodeDelta{.stat = node->stat, .acls = uncommitted_state.getACLs(ephemeral_path), .data = node->getData()}); + } + + ephemerals.erase(session_ephemerals); } - - ephemerals.erase(session_ephemerals); } new_digest = calculateNodesDigest(new_digest, new_deltas); @@ -2331,7 +2351,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( std::list deltas; { std::lock_guard lock(uncommitted_state.deltas_mutex); - auto it = uncommitted_state.deltas.begin(); + auto it = uncommitted_state.deltas.begin(); for (; it != uncommitted_state.deltas.end() && it->zxid == commit_zxid; ++it) ; @@ -2341,7 +2361,10 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( KeeperStorage::ResponsesForSessions results; /// ZooKeeper update sessions expirity for each request, not only for heartbeats - session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); + { + std::lock_guard lock(session_mutex); + session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); + } if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { @@ -2349,6 +2372,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( { if (std::holds_alternative(delta.operation)) { + std::lock_guard lock(watches_mutex); auto responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); results.insert(results.end(), responses.begin(), responses.end()); } @@ -2368,8 +2392,11 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( auto response = std::make_shared(); response->xid = zk_request->xid; response->zxid = commit_zxid; - session_expiry_queue.remove(session_id); - session_and_timeout.erase(session_id); + { + std::lock_guard lock(session_mutex); + session_expiry_queue.remove(session_id); + session_and_timeout.erase(session_id); + } results.push_back(ResponseForSession{session_id, response}); } else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special @@ -2392,7 +2419,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( if (is_local) { - std::lock_guard lock(storage_mutex); assert(zk_request->isReadRequest()); if (check_acl && !request_processor->checkAuth(*this, session_id, true)) { @@ -2414,6 +2440,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// Watches for this requests are added to the watches lists if (zk_request->has_watch) { + std::lock_guard lock(watches_mutex); if (response->error == Coordination::Error::ZOK) { static constexpr std::array list_requests{ @@ -2436,6 +2463,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { + std::lock_guard lock(watches_mutex); auto watch_responses = request_processor->processWatches(watches, list_watches); results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } @@ -2456,7 +2484,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( return results; } -void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) +void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS { std::unique_lock transaction_lock(transaction_mutex); if (allow_missing && (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid < rollback_zxid)) @@ -2484,13 +2512,28 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) } } -KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const +KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed, bool lock_transaction_mutex) const TSA_NO_THREAD_SAFETY_ANALYSIS { if (!keeper_context->digestEnabled()) return {.version = DigestVersion::NO_DIGEST}; - if (committed || uncommitted_transactions.empty()) + if (committed) + { + std::lock_guard storage_lock(storage_mutex); return {CURRENT_DIGEST_VERSION, nodes_digest}; + } + + std::unique_lock transaction_lock(transaction_mutex, std::defer_lock); + if (lock_transaction_mutex) + transaction_lock.lock(); + + if (uncommitted_transactions.empty()) + { + if (lock_transaction_mutex) + transaction_lock.unlock(); + std::lock_guard storage_lock(storage_mutex); + return {CURRENT_DIGEST_VERSION, nodes_digest}; + } return uncommitted_transactions.back().nodes_digest; } @@ -2510,8 +2553,89 @@ void KeeperStorage::addDigest(const Node & node, const std::string_view path) } } +/// Allocate new session id with the specified timeouts +int64_t KeeperStorage::getSessionID(int64_t session_timeout_ms) +{ + std::lock_guard lock(session_mutex); + auto result = session_id_counter++; + session_and_timeout.emplace(result, session_timeout_ms); + session_expiry_queue.addNewSessionOrUpdate(result, session_timeout_ms); + return result; +} + +/// Add session id. Used when restoring KeeperStorage from snapshot. +void KeeperStorage::addSessionID(int64_t session_id, int64_t session_timeout_ms) +{ + session_and_timeout.emplace(session_id, session_timeout_ms); + session_expiry_queue.addNewSessionOrUpdate(session_id, session_timeout_ms); +} + +std::vector KeeperStorage::getDeadSessions() const +{ + std::lock_guard lock(session_mutex); + return session_expiry_queue.getExpiredSessions(); +} + +const SessionAndTimeout & KeeperStorage::getActiveSessions() const +{ + return session_and_timeout; +} + +/// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. +void KeeperStorage::enableSnapshotMode(size_t up_to_version) +{ + std::lock_guard lock(storage_mutex); + container.enableSnapshotMode(up_to_version); +} + +/// Turn off snapshot mode. +void KeeperStorage::disableSnapshotMode() +{ + std::lock_guard lock(storage_mutex); + container.disableSnapshotMode(); +} + +KeeperStorage::Container::const_iterator KeeperStorage::getSnapshotIteratorBegin() const +{ + std::lock_guard lock(storage_mutex); + return container.begin(); +} + +/// Clear outdated data from internal container. +void KeeperStorage::clearGarbageAfterSnapshot() +{ + std::lock_guard lock(storage_mutex); + container.clearOutdatedNodes(); +} + +/// Introspection functions mostly used in 4-letter commands +uint64_t KeeperStorage::getNodesCount() const +{ + std::lock_guard lock(storage_mutex); + return container.size(); +} + +uint64_t KeeperStorage::getApproximateDataSize() const +{ + std::lock_guard lock(storage_mutex); + return container.getApproximateDataSize(); +} + +uint64_t KeeperStorage::getArenaDataSize() const +{ + std::lock_guard lock(storage_mutex); + return container.keyArenaSize(); +} + +uint64_t KeeperStorage::getWatchedPathsCount() const +{ + std::lock_guard lock(watches_mutex); + return watches.size() + list_watches.size(); +} + void KeeperStorage::clearDeadWatches(int64_t session_id) { + std::lock_guard lock(watches_mutex); /// Clear all watches for this session auto watches_it = sessions_and_watchers.find(session_id); if (watches_it != sessions_and_watchers.end()) @@ -2545,6 +2669,7 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const { + std::lock_guard lock(watches_mutex); for (const auto & [session_id, watches_paths] : sessions_and_watchers) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; @@ -2563,6 +2688,7 @@ void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const } }; + std::lock_guard lock(watches_mutex); for (const auto & [watch_path, sessions] : watches) { buf << watch_path << "\n"; @@ -2586,14 +2712,18 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co } }; - buf << "Sessions dump (" << session_and_timeout.size() << "):\n"; - - for (const auto & [session_id, _] : session_and_timeout) { - buf << "0x" << getHexUIntLowercase(session_id) << "\n"; + std::lock_guard lock(session_mutex); + buf << "Sessions dump (" << session_and_timeout.size() << "):\n"; + + for (const auto & [session_id, _] : session_and_timeout) + { + buf << "0x" << getHexUIntLowercase(session_id) << "\n"; + } } - buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCount() << "):\n"; + std::lock_guard ephemerals_lock(ephemerals_mutex); + buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCountLocked() << "):\n"; for (const auto & [session_id, ephemeral_paths] : ephemerals) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; @@ -2603,6 +2733,7 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co uint64_t KeeperStorage::getTotalWatchesCount() const { + std::lock_guard lock(watches_mutex); uint64_t ret = 0; for (const auto & [path, subscribed_sessions] : watches) ret += subscribed_sessions.size(); @@ -2613,9 +2744,21 @@ uint64_t KeeperStorage::getTotalWatchesCount() const return ret; } +uint64_t KeeperStorage::getSessionWithEphemeralNodesCount() const +{ + std::lock_guard ephemerals_lock(ephemerals_mutex); + return getSessionWithEphemeralNodesCountLocked(); +} + +uint64_t KeeperStorage::getSessionWithEphemeralNodesCountLocked() const +{ + return ephemerals.size(); +} + uint64_t KeeperStorage::getSessionsWithWatchesCount() const { std::unordered_set counter; + std::lock_guard lock(watches_mutex); for (const auto & [path, subscribed_sessions] : watches) counter.insert(subscribed_sessions.begin(), subscribed_sessions.end()); @@ -2627,6 +2770,7 @@ uint64_t KeeperStorage::getSessionsWithWatchesCount() const uint64_t KeeperStorage::getTotalEphemeralNodesCount() const { + std::lock_guard ephemerals_lock(ephemerals_mutex); uint64_t ret = 0; for (const auto & [session_id, nodes] : ephemerals) ret += nodes.size(); @@ -2636,6 +2780,7 @@ uint64_t KeeperStorage::getTotalEphemeralNodesCount() const void KeeperStorage::recalculateStats() { + std::lock_guard lock(storage_mutex); container.recalculateDataSize(); } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 84bf7d297b4..3ac5165e38b 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -139,18 +139,14 @@ public: using SessionAndAuth = std::unordered_map; using Watches = std::map; - mutable SharedMutex main_mutex; - mutable std::mutex storage_mutex; - int64_t session_id_counter{1}; - - SessionAndAuth session_and_auth; + SessionAndAuth session_and_auth TSA_GUARDED_BY(storage_mutex); /// Main hashtable with nodes. Contain all information about data. /// All other structures expect session_and_timeout can be restored from /// container. - Container container; + Container container TSA_GUARDED_BY(storage_mutex); // Applying ZooKeeper request to storage consists of two steps: // - preprocessing which, instead of applying the changes directly to storage, @@ -284,7 +280,10 @@ public: }; if (is_local) + { + std::lock_guard lock(storage.storage_mutex); return check_auth(storage.session_and_auth[session_id]); + } // check if there are uncommitted const auto auth_it = session_and_auth.find(session_id); @@ -347,7 +346,7 @@ public: // with zxid > last_zxid void applyUncommittedState(KeeperStorage & other, int64_t last_zxid); - Coordination::Error commit(std::list deltas); + Coordination::Error commit(std::list deltas) TSA_REQUIRES(storage_mutex); // Create node in the storage // Returns false if it failed to create the node, true otherwise @@ -357,25 +356,27 @@ public: String data, const Coordination::Stat & stat, bool is_sequental, - Coordination::ACLs node_acls); + Coordination::ACLs node_acls) TSA_REQUIRES(storage_mutex); // Remove node in the storage // Returns false if it failed to remove the node, true otherwise // We don't care about the exact failure because we should've caught it during preprocessing - bool removeNode(const std::string & path, int32_t version); + bool removeNode(const std::string & path, int32_t version) TSA_REQUIRES(storage_mutex); bool checkACL(StringRef path, int32_t permissions, int64_t session_id, bool is_local); void unregisterEphemeralPath(int64_t session_id, const std::string & path); + mutable std::mutex ephemerals_mutex; /// Mapping session_id -> set of ephemeral nodes paths - Ephemerals ephemerals; - /// Mapping session_id -> set of watched nodes paths - SessionAndWatcher sessions_and_watchers; + Ephemerals ephemerals TSA_GUARDED_BY(ephemerals_mutex); + + mutable std::mutex session_mutex; + int64_t session_id_counter TSA_GUARDED_BY(session_mutex) = 1; /// Expiration queue for session, allows to get dead sessions at some point of time - SessionExpiryQueue session_expiry_queue; + SessionExpiryQueue session_expiry_queue TSA_GUARDED_BY(session_mutex); /// All active sessions with timeout - SessionAndTimeout session_and_timeout; + SessionAndTimeout session_and_timeout TSA_GUARDED_BY(session_mutex); /// ACLMap for more compact ACLs storage inside nodes. ACLMap acl_map; @@ -383,7 +384,7 @@ public: mutable std::mutex transaction_mutex; /// Global id of all requests applied to storage - int64_t zxid = 0; + int64_t zxid TSA_GUARDED_BY(transaction_mutex) = 0; // older Keeper node (pre V5 snapshots) can create snapshots and receive logs from newer Keeper nodes // this can lead to some inconsistencies, e.g. from snapshot it will use log_idx as zxid @@ -398,15 +399,20 @@ public: Digest nodes_digest; }; - std::list uncommitted_transactions; + std::list uncommitted_transactions TSA_GUARDED_BY(transaction_mutex); - uint64_t nodes_digest{0}; + uint64_t nodes_digest TSA_GUARDED_BY(storage_mutex) = 0; std::atomic finalized{false}; + + mutable std::mutex watches_mutex; + /// Mapping session_id -> set of watched nodes paths + SessionAndWatcher sessions_and_watchers TSA_GUARDED_BY(watches_mutex); + /// Currently active watches (node_path -> subscribed sessions) - Watches watches; - Watches list_watches; /// Watches for 'list' request (watches on children). + Watches watches TSA_GUARDED_BY(watches_mutex); + Watches list_watches TSA_GUARDED_BY(watches_mutex); /// Watches for 'list' request (watches on children). void clearDeadWatches(int64_t session_id); @@ -414,9 +420,9 @@ public: int64_t getZXID() const; int64_t getNextZXID() const; - int64_t getNextZXIDLocked(std::lock_guard & lock) const; + int64_t getNextZXIDLocked() const TSA_REQUIRES(transaction_mutex); - Digest getNodesDigest(bool committed) const; + Digest getNodesDigest(bool committed, bool lock_transaction_mutex) const; KeeperContextPtr keeper_context; @@ -426,23 +432,13 @@ public: KeeperStorage(int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, bool initialize_system_nodes = true); - void initializeSystemNodes(); + void initializeSystemNodes() TSA_NO_THREAD_SAFETY_ANALYSIS; /// Allocate new session id with the specified timeouts - int64_t getSessionID(int64_t session_timeout_ms) - { - auto result = session_id_counter++; - session_and_timeout.emplace(result, session_timeout_ms); - session_expiry_queue.addNewSessionOrUpdate(result, session_timeout_ms); - return result; - } + int64_t getSessionID(int64_t session_timeout_ms); /// Add session id. Used when restoring KeeperStorage from snapshot. - void addSessionID(int64_t session_id, int64_t session_timeout_ms) - { - session_and_timeout.emplace(session_id, session_timeout_ms); - session_expiry_queue.addNewSessionOrUpdate(session_id, session_timeout_ms); - } + void addSessionID(int64_t session_id, int64_t session_timeout_ms) TSA_NO_THREAD_SAFETY_ANALYSIS; UInt64 calculateNodesDigest(UInt64 current_digest, const std::list & new_deltas) const; @@ -470,36 +466,36 @@ public: /// Set of methods for creating snapshots /// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. - void enableSnapshotMode(size_t up_to_version) { container.enableSnapshotMode(up_to_version); } + void enableSnapshotMode(size_t up_to_version); /// Turn off snapshot mode. - void disableSnapshotMode() { container.disableSnapshotMode(); } + void disableSnapshotMode(); - Container::const_iterator getSnapshotIteratorBegin() const { return container.begin(); } + Container::const_iterator getSnapshotIteratorBegin() const; /// Clear outdated data from internal container. - void clearGarbageAfterSnapshot() { container.clearOutdatedNodes(); } + void clearGarbageAfterSnapshot(); /// Get all active sessions - const SessionAndTimeout & getActiveSessions() const { return session_and_timeout; } + const SessionAndTimeout & getActiveSessions() const; /// Get all dead sessions - std::vector getDeadSessions() const { return session_expiry_queue.getExpiredSessions(); } + std::vector getDeadSessions() const; /// Introspection functions mostly used in 4-letter commands - uint64_t getNodesCount() const { return container.size(); } + uint64_t getNodesCount() const; - uint64_t getApproximateDataSize() const { return container.getApproximateDataSize(); } + uint64_t getApproximateDataSize() const; - uint64_t getArenaDataSize() const { return container.keyArenaSize(); } + uint64_t getArenaDataSize() const; uint64_t getTotalWatchesCount() const; - uint64_t getWatchedPathsCount() const { return watches.size() + list_watches.size(); } + uint64_t getWatchedPathsCount() const; uint64_t getSessionsWithWatchesCount() const; - uint64_t getSessionWithEphemeralNodesCount() const { return ephemerals.size(); } + uint64_t getSessionWithEphemeralNodesCount() const; uint64_t getTotalEphemeralNodesCount() const; void dumpWatches(WriteBufferFromOwnString & buf) const; @@ -508,8 +504,10 @@ public: void recalculateStats(); private: - void removeDigest(const Node & node, std::string_view path); - void addDigest(const Node & node, std::string_view path); + uint64_t getSessionWithEphemeralNodesCountLocked() const TSA_REQUIRES(ephemerals_mutex); + + void removeDigest(const Node & node, std::string_view path) TSA_REQUIRES(storage_mutex); + void addDigest(const Node & node, std::string_view path) TSA_REQUIRES(storage_mutex); }; using KeeperStoragePtr = std::unique_ptr; diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index d7ce097b795..54a917e03a5 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -90,7 +90,7 @@ void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in) } } -int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * log) +int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * log) TSA_NO_THREAD_SAFETY_ANALYSIS { int64_t max_zxid = 0; std::string path; From 5a6fe87b7c8902bcef6be355f157bc07a1fa0182 Mon Sep 17 00:00:00 2001 From: gao chuan Date: Wed, 17 Apr 2024 23:43:36 +0800 Subject: [PATCH 004/128] [bugfix]alter postgresql subscription error --- .../PostgreSQLReplicationHandler.cpp | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 2bb1e2dde0d..f6c84699313 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -157,7 +157,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( checkReplicationSlot(replication_slot); - LOG_INFO(log, "Using replication slot {} and publication {}", replication_slot, publication_name); + LOG_INFO(log, "Using replication slot {} and publication {}", replication_slot, doubleQuoteString(publication_name)); startup_task = getContext()->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ checkConnectionAndStart(); }); consumer_task = getContext()->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); @@ -537,7 +537,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::nontransactio /// This is a case for single Materialized storage. In case of database engine this check is done in advance. LOG_WARNING(log, "Publication {} already exists, but it is a CREATE query, not ATTACH. Publication will be dropped", - publication_name); + doubleQuoteString(publication_name)); dropPublication(tx); } @@ -567,7 +567,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::nontransactio try { tx.exec(query_str); - LOG_DEBUG(log, "Created publication {} with tables list: {}", publication_name, tables_list); + LOG_DEBUG(log, "Created publication {} with tables list: {}", doubleQuoteString(publication_name), tables_list); } catch (Exception & e) { @@ -577,7 +577,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::nontransactio } else { - LOG_DEBUG(log, "Using existing publication ({}) version", publication_name); + LOG_DEBUG(log, "Using existing publication ({}) version", doubleQuoteString(publication_name)); } } @@ -653,15 +653,15 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(pqxx::nontransaction & tx void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) { - std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name); + std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", doubleQuoteString(publication_name)); tx.exec(query_str); - LOG_DEBUG(log, "Dropped publication: {}", publication_name); + LOG_DEBUG(log, "Dropped publication: {}", doubleQuoteString(publication_name)); } void PostgreSQLReplicationHandler::addTableToPublication(pqxx::nontransaction & ntx, const String & table_name) { - std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", publication_name, doubleQuoteWithSchema(table_name)); + std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", doubleQuoteString(publication_name), doubleQuoteWithSchema(table_name)); ntx.exec(query_str); LOG_TRACE(log, "Added table {} to publication `{}`", doubleQuoteWithSchema(table_name), publication_name); } @@ -671,7 +671,7 @@ void PostgreSQLReplicationHandler::removeTableFromPublication(pqxx::nontransacti { try { - std::string query_str = fmt::format("ALTER PUBLICATION {} DROP TABLE ONLY {}", publication_name, doubleQuoteWithSchema(table_name)); + std::string query_str = fmt::format("ALTER PUBLICATION {} DROP TABLE ONLY {}", doubleQuoteString(publication_name), doubleQuoteWithSchema(table_name)); ntx.exec(query_str); LOG_TRACE(log, "Removed table `{}` from publication `{}`", doubleQuoteWithSchema(table_name), publication_name); } @@ -758,7 +758,7 @@ std::set PostgreSQLReplicationHandler::fetchRequiredTables() { LOG_WARNING(log, "Publication {} already exists, but it is a CREATE query, not ATTACH. Publication will be dropped", - publication_name); + doubleQuoteString(publication_name)); connection.execWithRetry([&](pqxx::nontransaction & tx_){ dropPublication(tx_); }); } @@ -768,7 +768,7 @@ std::set PostgreSQLReplicationHandler::fetchRequiredTables() { LOG_WARNING(log, "Publication {} already exists and tables list is empty. Assuming publication is correct.", - publication_name); + doubleQuoteString(publication_name)); { pqxx::nontransaction tx(connection.getRef()); @@ -819,7 +819,7 @@ std::set PostgreSQLReplicationHandler::fetchRequiredTables() "To avoid redundant work, you can try ALTER PUBLICATION query to remove redundant tables. " "Or you can you ALTER SETTING. " "\nPublication tables: {}.\nTables list: {}", - publication_name, diff_tables, publication_tables, listed_tables); + doubleQuoteString(publication_name), diff_tables, publication_tables, listed_tables); return std::set(expected_tables.begin(), expected_tables.end()); } From 5ffa2c9ca1e5c6030cab2c49ebdd397b6917ec13 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 25 Apr 2024 13:37:24 +0200 Subject: [PATCH 005/128] Add a test --- tests/integration/helpers/postgres_utility.py | 70 ++++++++----- .../test.py | 97 +++++++++++++++++++ 2 files changed, 141 insertions(+), 26 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 468c3b3bb63..2f5686d96d6 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -82,24 +82,24 @@ def drop_postgres_schema(cursor, schema_name): def create_postgres_table( cursor, table_name, - database_name="", replica_identity_full=False, template=postgres_table_template, ): - if database_name == "": - name = table_name - else: - name = f"{database_name}.{table_name}" - drop_postgres_table(cursor, name) - query = template.format(name) - cursor.execute(query) + drop_postgres_table(cursor, table_name) + query = template.format(table_name) + print(f"Query: {query}") + cursor.execute(query) + if replica_identity_full: - cursor.execute(f"ALTER TABLE {name} REPLICA IDENTITY FULL;") + cursor.execute(f"""ALTER TABLE "{table_name}" REPLICA IDENTITY FULL;""") -def drop_postgres_table(cursor, name): - cursor.execute(f"""DROP TABLE IF EXISTS "{name}" """) +def drop_postgres_table(cursor, name, database_name=""): + if database_name != "": + cursor.execute(f"""DROP TABLE IF EXISTS "{database_name}"."{name}" """) + else: + cursor.execute(f"""DROP TABLE IF EXISTS "{name}" """) def create_postgres_table_with_schema(cursor, schema_name, table_name): @@ -245,9 +245,9 @@ class PostgresManager: ): postgres_database = self.database_or_default(postgres_database) self.created_materialized_postgres_db_list.add(materialized_database) - self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database}") + self.instance.query(f"DROP DATABASE IF EXISTS `{materialized_database}`") - create_query = f"CREATE DATABASE {materialized_database} ENGINE = MaterializedPostgreSQL('{ip}:{port}', '{postgres_database}', '{user}', '{password}')" + create_query = f"CREATE DATABASE `{materialized_database}` ENGINE = MaterializedPostgreSQL('{ip}:{port}', '{postgres_database}', '{user}', '{password}')" if len(settings) > 0: create_query += " SETTINGS " for i in range(len(settings)): @@ -259,7 +259,7 @@ class PostgresManager: assert materialized_database in self.instance.query("SHOW DATABASES") def drop_materialized_db(self, materialized_database="test_database"): - self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database} SYNC") + self.instance.query(f"DROP DATABASE IF EXISTS `{materialized_database}` SYNC") if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) @@ -269,15 +269,28 @@ class PostgresManager: def create_postgres_table( self, table_name, database_name="", template=postgres_table_template ): - create_postgres_table( - self.cursor, table_name, database_name=database_name, template=template - ) + database_name = self.database_or_default(database_name) + cursor = self.cursor + if database_name != self.get_default_database: + try: + self.create_postgres_db(database_name) + except: + # postgres does not support create database if not exists + pass + conn = get_postgres_conn( + ip=self.ip, + port=self.port, + database=True, + database_name=database_name, + ) + cursor = conn.cursor() + create_postgres_table(cursor, table_name, template=template) def create_and_fill_postgres_table(self, table_name, database_name=""): - create_postgres_table(self.cursor, table_name, database_name) database_name = self.database_or_default(database_name) + self.create_postgres_table(table_name, database_name) self.instance.query( - f"INSERT INTO {database_name}.{table_name} SELECT number, number from numbers(50)" + f"INSERT INTO `{database_name}`.`{table_name}` SELECT number, number from numbers(50)" ) def create_and_fill_postgres_tables( @@ -289,11 +302,11 @@ class PostgresManager: ): for i in range(tables_num): table_name = f"{table_name_base}_{i}" - create_postgres_table(self.cursor, table_name, database_name) + self.create_postgres_table(table_name, database_name) if numbers > 0: db = self.database_or_default(database_name) self.instance.query( - f"INSERT INTO {db}.{table_name} SELECT number, number from numbers({numbers})" + f"INSERT INTO `{db}`.{table_name} SELECT number, number from numbers({numbers})" ) @@ -329,11 +342,11 @@ def assert_nested_table_is_created( table = schema_name + "." + table_name print(f"Checking table {table} exists in {materialized_database}") - database_tables = instance.query(f"SHOW TABLES FROM {materialized_database}") + database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}`") while table not in database_tables: time.sleep(0.2) - database_tables = instance.query(f"SHOW TABLES FROM {materialized_database}") + database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}`") assert table in database_tables @@ -366,9 +379,9 @@ def check_tables_are_synchronized( table_path = "" if len(schema_name) == 0: - table_path = f"{materialized_database}.{table_name}" + table_path = f"`{materialized_database}`.`{table_name}`" else: - table_path = f"{materialized_database}.`{schema_name}.{table_name}`" + table_path = f"`{materialized_database}`.`{schema_name}.{table_name}`" print(f"Checking table is synchronized: {table_path}") result_query = f"select * from {table_path} order by {order_by};" @@ -403,4 +416,9 @@ def check_several_tables_are_synchronized( schema_name="", ): for i in range(tables_num): - check_tables_are_synchronized(instance, f"postgresql_replica_{i}") + check_tables_are_synchronized( + instance, + f"postgresql_replica_{i}", + postgres_database=postgres_database, + materialized_database=materialized_database, + ) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 5e04c9e4d12..4c03f731152 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -1097,6 +1097,103 @@ def test_dependent_loading(started_cluster): instance.query(f"DROP TABLE {table} SYNC") +def test_quoting_publication(started_cluster): + NUM_TABLES = 5 + postgres_database = "postgres-postgres" + materialized_database = "test-database" + + pg_manager3.create_and_fill_postgres_tables(NUM_TABLES, 10000) + + check_table_name_1 = "postgresql-replica-5" + pg_manager3.create_and_fill_postgres_table(check_table_name_1) + + pg_manager3.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + materialized_database=materialized_database, + ) + check_several_tables_are_synchronized( + instance, + NUM_TABLES, + materialized_database=materialized_database, + postgres_database=postgres_database, + ) + + result = instance.query(f"SHOW TABLES FROM `{materialized_database}`") + assert ( + result + == "postgresql-replica-5\npostgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n" + ) + + check_tables_are_synchronized( + instance, + check_table_name_1, + materialized_database=materialized_database, + postgres_database=postgres_database, + ) + instance.query( + f"INSERT INTO `{postgres_database}`.`{check_table_name_1}` SELECT number, number from numbers(10000, 10000)" + ) + check_tables_are_synchronized( + instance, + check_table_name_1, + materialized_database=materialized_database, + postgres_database=postgres_database, + ) + + check_table_name_2 = "postgresql-replica-6" + pg_manager3.create_and_fill_postgres_table(check_table_name_2) + + instance.query(f"ATTACH TABLE `{materialized_database}`.`{check_table_name_2}`") + + result = instance.query(f"SHOW TABLES FROM `{materialized_database}`") + assert ( + result + == "postgresql-replica-5\npostgresql-replica-6\npostgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n" + ) + + check_tables_are_synchronized( + instance, + check_table_name_2, + materialized_database=materialized_database, + postgres_database=postgres_database, + ) + instance.query( + f"INSERT INTO `{postgres_database}`.`{check_table_name_2}` SELECT number, number from numbers(10000, 10000)" + ) + check_tables_are_synchronized( + instance, + check_table_name_2, + materialized_database=materialized_database, + postgres_database=postgres_database, + ) + + instance.restart_clickhouse() + check_tables_are_synchronized( + instance, + check_table_name_1, + materialized_database=materialized_database, + postgres_database=postgres_database, + ) + check_tables_are_synchronized( + instance, + check_table_name_2, + materialized_database=materialized_database, + postgres_database=postgres_database, + ) + + instance.query( + f"DETACH TABLE `{materialized_database}`.`{check_table_name_2}` PERMANENTLY" + ) + time.sleep(5) + + result = instance.query(f"SHOW TABLES FROM `{materialized_database}`") + assert ( + result + == "postgresql-replica-5\npostgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n" + ) + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From acc08c65d951175123d62e3e55ea8db87d385e5e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 22 May 2024 11:56:45 +0200 Subject: [PATCH 006/128] Add stopwatch --- src/Coordination/KeeperDispatcher.cpp | 8 ++++++++ src/Coordination/KeeperStateMachine.cpp | 5 +++-- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index c9eba87590e..94ce159f055 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -448,11 +448,19 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) { + Stopwatch watch; if (server->isLeaderAlive()) server->putLocalReadRequest(request_queue_it->second); else addErrorResponses(request_queue_it->second, Coordination::Error::ZCONNECTIONLOSS); + if (request_queue_it->second.size() > 500) + LOG_INFO( + getLogger("Speed"), + "It took {}ms for {} requests", + watch.elapsedMilliseconds(), + request_queue_it->second.size()); + xid_to_request_queue.erase(request_queue_it); } } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 8a94ef7347e..67ff316a6f3 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -768,14 +768,15 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestsForSess /// Pure local request, just process it with storage std::lock_guard lock(process_and_responses_lock); std::vector all_responses; - if (request_for_session.size() > 50) + if (request_for_session.size() > 100) { all_responses.resize(request_for_session.size()); //LOG_INFO(getLogger("Keeper"), "Has read requests {}", request_queue_it->second.size()); for (size_t i = 0; i < request_for_session.size(); ++i) { - read_pool.scheduleOrThrowOnError([&, i, read_request = request_for_session[i]] + read_pool.scheduleOrThrowOnError([&, i] { + const auto & read_request = request_for_session[i]; all_responses[i] = storage->processRequest( read_request.request, read_request.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); }); From f0e970338454e2a5e09968974ea0850f357ec43e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 6 Jun 2024 09:45:07 +0200 Subject: [PATCH 007/128] Some small improvements --- src/Coordination/FourLetterCommand.cpp | 22 +++++++++++-------- src/Coordination/Keeper4LWInfo.h | 5 ----- .../KeeperAsynchronousMetrics.cpp | 17 +++++++------- src/Coordination/KeeperServer.cpp | 2 -- src/Coordination/KeeperStateMachine.cpp | 5 ++--- src/Coordination/KeeperStateMachine.h | 1 + src/Coordination/KeeperStorage.cpp | 18 +++++++++++++++ src/Coordination/KeeperStorage.h | 17 ++++++++++++++ 8 files changed, 60 insertions(+), 27 deletions(-) diff --git a/src/Coordination/FourLetterCommand.cpp b/src/Coordination/FourLetterCommand.cpp index 8de9f8dfa1c..18b25ae8e77 100644 --- a/src/Coordination/FourLetterCommand.cpp +++ b/src/Coordination/FourLetterCommand.cpp @@ -300,11 +300,13 @@ String MonitorCommand::run() print(ret, "server_state", keeper_info.getRole()); - print(ret, "znode_count", state_machine.getNodesCount()); - print(ret, "watch_count", state_machine.getTotalWatchesCount()); - print(ret, "ephemerals_count", state_machine.getTotalEphemeralNodesCount()); - print(ret, "approximate_data_size", state_machine.getApproximateDataSize()); - print(ret, "key_arena_size", state_machine.getKeyArenaSize()); + const auto & storage_stats = state_machine.getStorageStats(); + + print(ret, "znode_count", storage_stats.nodes_count.load(std::memory_order_relaxed)); + print(ret, "watch_count", storage_stats.total_watches_count.load(std::memory_order_relaxed)); + print(ret, "ephemerals_count", storage_stats.total_emphemeral_nodes_count.load(std::memory_order_relaxed)); + print(ret, "approximate_data_size", storage_stats.approximate_data_size.load(std::memory_order_relaxed)); + print(ret, "key_arena_size", 0); print(ret, "latest_snapshot_size", state_machine.getLatestSnapshotSize()); #if defined(OS_LINUX) || defined(OS_DARWIN) @@ -386,6 +388,7 @@ String ServerStatCommand::run() auto & stats = keeper_dispatcher.getKeeperConnectionStats(); Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); + const auto & storage_stats = keeper_dispatcher.getStateMachine().getStorageStats(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); @@ -397,9 +400,9 @@ String ServerStatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", formatZxid(keeper_info.last_zxid)); + write("Zxid", formatZxid(storage_stats.last_zxid.load(std::memory_order_relaxed))); write("Mode", keeper_info.getRole()); - write("Node count", toString(keeper_info.total_nodes_count)); + write("Node count", toString(storage_stats.nodes_count.load(std::memory_order_relaxed))); return buf.str(); } @@ -415,6 +418,7 @@ String StatCommand::run() auto & stats = keeper_dispatcher.getKeeperConnectionStats(); Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo(); + const auto & storage_stats = keeper_dispatcher.getStateMachine().getStorageStats(); write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH); @@ -430,9 +434,9 @@ String StatCommand::run() write("Sent", toString(stats.getPacketsSent())); write("Connections", toString(keeper_info.alive_connections_count)); write("Outstanding", toString(keeper_info.outstanding_requests_count)); - write("Zxid", formatZxid(keeper_info.last_zxid)); + write("Zxid", formatZxid(storage_stats.last_zxid.load(std::memory_order_relaxed))); write("Mode", keeper_info.getRole()); - write("Node count", toString(keeper_info.total_nodes_count)); + write("Node count", toString(storage_stats.nodes_count.load(std::memory_order_relaxed))); return buf.str(); } diff --git a/src/Coordination/Keeper4LWInfo.h b/src/Coordination/Keeper4LWInfo.h index 80b00b3f36e..d02c9a8f91c 100644 --- a/src/Coordination/Keeper4LWInfo.h +++ b/src/Coordination/Keeper4LWInfo.h @@ -1,7 +1,5 @@ #pragma once -#include - #include #include @@ -30,9 +28,6 @@ struct Keeper4LWInfo uint64_t follower_count; uint64_t synced_follower_count; - uint64_t total_nodes_count; - int64_t last_zxid; - String getRole() const { if (is_standalone) diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 86166ffe31b..1f317c4b303 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -38,15 +38,16 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM is_follower = static_cast(keeper_info.is_follower); is_exceeding_mem_soft_limit = static_cast(keeper_info.is_exceeding_mem_soft_limit); - zxid = keeper_info.last_zxid; const auto & state_machine = keeper_dispatcher.getStateMachine(); - znode_count = state_machine.getNodesCount(); - watch_count = state_machine.getTotalWatchesCount(); - ephemerals_count = state_machine.getTotalEphemeralNodesCount(); - approximate_data_size = state_machine.getApproximateDataSize(); - key_arena_size = state_machine.getKeyArenaSize(); - session_with_watches = state_machine.getSessionsWithWatchesCount(); - paths_watched = state_machine.getWatchedPathsCount(); + const auto & storage_stats = state_machine.getStorageStats(); + zxid = storage_stats.last_zxid.load(std::memory_order_relaxed); + znode_count = storage_stats.nodes_count.load(std::memory_order_relaxed); + watch_count = storage_stats.total_watches_count.load(std::memory_order_relaxed); + ephemerals_count = storage_stats.total_emphemeral_nodes_count.load(std::memory_order_relaxed); + approximate_data_size = storage_stats.approximate_data_size.load(std::memory_order_relaxed); + key_arena_size = 0; + session_with_watches = storage_stats.sessions_with_watches_count.load(std::memory_order_relaxed); + paths_watched = storage_stats.watched_paths_count.load(std::memory_order_relaxed); # if defined(__linux__) || defined(__APPLE__) open_file_descriptor_count = getCurrentProcessFDCount(); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 736a01443ce..02b05c6346c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -1126,8 +1126,6 @@ Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const result.synced_follower_count = getSyncedFollowerCount(); } result.is_exceeding_mem_soft_limit = isExceedingMemorySoftLimit(); - result.total_nodes_count = getKeeperStateMachine()->getNodesCount(); - result.last_zxid = getKeeperStateMachine()->getLastProcessedZxid(); return result; } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index e4d661dfe17..cbb4c55de33 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -802,10 +802,9 @@ uint64_t KeeperStateMachine::getLastProcessedZxid() const return storage->getZXID(); } -uint64_t KeeperStateMachine::getNodesCount() const +const KeeperStorage::Stats & KeeperStateMachine::getStorageStats() const TSA_NO_THREAD_SAFETY_ANALYSIS { - std::lock_guard lock(storage_and_responses_lock); - return storage->getNodesCount(); + return storage->getStorageStats(); } uint64_t KeeperStateMachine::getTotalWatchesCount() const diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index ee6109f0a17..6a34ef7ddd8 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -111,6 +111,7 @@ public: /// Introspection functions for 4lw commands uint64_t getLastProcessedZxid() const; + const KeeperStorage::Stats & getStorageStats() const; uint64_t getNodesCount() const; uint64_t getTotalWatchesCount() const; uint64_t getWatchedPathsCount() const; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 9bcd0608bf7..c1dc38cec5d 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -2514,6 +2514,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( results.push_back(ResponseForSession{session_id, response}); } + updateStats(); return results; } @@ -2660,6 +2661,23 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co } } +void KeeperStorage::updateStats() +{ + stats.nodes_count.store(getNodesCount(), std::memory_order_relaxed); + stats.approximate_data_size.store(getApproximateDataSize(), std::memory_order_relaxed); + stats.total_watches_count.store(getTotalWatchesCount(), std::memory_order_relaxed); + stats.watched_paths_count.store(getWatchedPathsCount(), std::memory_order_relaxed); + stats.sessions_with_watches_count.store(getSessionsWithWatchesCount(), std::memory_order_relaxed); + stats.session_with_ephemeral_nodes_count.store(getSessionWithEphemeralNodesCount(), std::memory_order_relaxed); + stats.total_emphemeral_nodes_count.store(getTotalEphemeralNodesCount(), std::memory_order_relaxed); + stats.last_zxid.store(getZXID(), std::memory_order_relaxed); +} + +const KeeperStorage::Stats & KeeperStorage::getStorageStats() const +{ + return stats; +} + uint64_t KeeperStorage::getTotalWatchesCount() const { uint64_t ret = 0; diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index d9e67f799f8..2d20bb38532 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -539,6 +539,23 @@ public: /// Get all dead sessions std::vector getDeadSessions() const { return session_expiry_queue.getExpiredSessions(); } + struct Stats + { + std::atomic nodes_count = 0; + std::atomic approximate_data_size = 0; + std::atomic total_watches_count = 0; + std::atomic watched_paths_count = 0; + std::atomic sessions_with_watches_count = 0; + std::atomic session_with_ephemeral_nodes_count = 0; + std::atomic total_emphemeral_nodes_count = 0; + std::atomic last_zxid = 0; + }; + + Stats stats; + + void updateStats(); + const Stats & getStorageStats() const; + /// Introspection functions mostly used in 4-letter commands uint64_t getNodesCount() const { return container.size(); } From 1e97d73bd07e0d8ce398e02b47afdb6afe244aa6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Jun 2024 17:13:37 +0200 Subject: [PATCH 008/128] Squashed commit of the following: commit 27fe0439fad3987ca6769504c73713e24f3fb81f Merge: bfb1c4c7939 bb469e0d45a Author: Antonio Andelic Date: Thu Jun 6 14:36:02 2024 +0200 Merge branch 'master' into fix-global-trace-collector commit bfb1c4c79396bb234b63a0a963fc643af292956d Author: Antonio Andelic Date: Thu Jun 6 11:29:42 2024 +0200 better commit fcee260b259b7750f18389e26debc90de22017a5 Author: Antonio Andelic Date: Thu Jun 6 11:22:48 2024 +0200 Update src/Interpreters/TraceCollector.h Co-authored-by: alesapin commit 1d3cf170534b1ae8382397d8d2672868e237f33f Author: Antonio Andelic Date: Thu Jun 6 11:11:08 2024 +0200 Fix global trace collector --- programs/server/Server.cpp | 90 ++++++++++++++--------------- src/Common/QueryProfiler.cpp | 6 +- src/Interpreters/Context.cpp | 17 +++++- src/Interpreters/Context.h | 2 + src/Interpreters/TraceCollector.cpp | 30 ++++++++-- src/Interpreters/TraceCollector.h | 11 +++- 6 files changed, 96 insertions(+), 60 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 8fcb9d87a93..84ff0e94cef 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -773,7 +773,27 @@ try LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info); #endif - bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log"); + bool has_trace_collector = false; + /// Disable it if we collect test coverage information, because it will work extremely slow. +#if !WITH_COVERAGE + /// Profilers cannot work reliably with any other libunwind or without PHDR cache. + has_trace_collector = hasPHDRCache() && config().has("trace_log"); +#endif + + /// Describe multiple reasons when query profiler cannot work. + +#if WITH_COVERAGE + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); +#endif + +#if defined(SANITIZER) + LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" + " when two different stack unwinding methods will interfere with each other."); +#endif + + if (!hasPHDRCache()) + LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" + " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will @@ -782,8 +802,27 @@ try server_settings.max_thread_pool_size, server_settings.max_thread_pool_free_size, server_settings.thread_pool_queue_size, - will_have_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, - will_have_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + has_trace_collector ? server_settings.global_profiler_real_time_period_ns : 0, + has_trace_collector ? server_settings.global_profiler_cpu_time_period_ns : 0); + + if (has_trace_collector) + { + global_context->createTraceCollector(); + + /// Set up server-wide memory profiler (for total memory tracker). + if (server_settings.total_memory_profiler_step) + total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); + + if (server_settings.total_memory_tracker_sample_probability > 0.0) + total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); + + if (server_settings.total_memory_profiler_sample_min_allocation_size) + total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); + + if (server_settings.total_memory_profiler_sample_max_allocation_size) + total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); + } + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; @@ -1950,52 +1989,9 @@ try LOG_DEBUG(log, "Loaded metadata."); - /// Init trace collector only after trace_log system table was created - /// Disable it if we collect test coverage information, because it will work extremely slow. -#if !WITH_COVERAGE - /// Profilers cannot work reliably with any other libunwind or without PHDR cache. - if (hasPHDRCache()) - { + if (has_trace_collector) global_context->initializeTraceCollector(); - /// Set up server-wide memory profiler (for total memory tracker). - if (server_settings.total_memory_profiler_step) - { - total_memory_tracker.setProfilerStep(server_settings.total_memory_profiler_step); - } - - if (server_settings.total_memory_tracker_sample_probability > 0.0) - { - total_memory_tracker.setSampleProbability(server_settings.total_memory_tracker_sample_probability); - } - - if (server_settings.total_memory_profiler_sample_min_allocation_size) - { - total_memory_tracker.setSampleMinAllocationSize(server_settings.total_memory_profiler_sample_min_allocation_size); - } - - if (server_settings.total_memory_profiler_sample_max_allocation_size) - { - total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); - } - } -#endif - - /// Describe multiple reasons when query profiler cannot work. - -#if WITH_COVERAGE - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they work extremely slow with test coverage."); -#endif - -#if defined(SANITIZER) - LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" - " when two different stack unwinding methods will interfere with each other."); -#endif - - if (!hasPHDRCache()) - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" - " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); - #if defined(OS_LINUX) auto tasks_stats_provider = TasksStatsCounters::findBestAvailableProvider(); if (tasks_stats_provider == TasksStatsCounters::MetricsProvider::None) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index c3affbdd968..746010b5462 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -228,9 +228,9 @@ void Timer::cleanup() #endif template -QueryProfilerBase::QueryProfilerBase([[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) - : log(getLogger("QueryProfiler")) - , pause_signal(pause_signal_) +QueryProfilerBase::QueryProfilerBase( + [[maybe_unused]] UInt64 thread_id, [[maybe_unused]] int clock_type, [[maybe_unused]] UInt32 period, [[maybe_unused]] int pause_signal_) + : log(getLogger("QueryProfiler")), pause_signal(pause_signal_) { #if defined(SANITIZER) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "QueryProfiler disabled because they cannot work under sanitizers"); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 06b3adb328d..b1fb6a68618 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -740,12 +740,18 @@ struct ContextSharedPart : boost::noncopyable void initializeTraceCollector(std::shared_ptr trace_log) { - if (!trace_log) - return; + if (!trace_collector.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector needs to be first created before initialization"); + + trace_collector->initialize(trace_log); + } + + void createTraceCollector() + { if (hasTraceCollector()) return; - trace_collector.emplace(std::move(trace_log)); + trace_collector.emplace(); } void addWarningMessage(const String & message) TSA_REQUIRES(mutex) @@ -3891,6 +3897,11 @@ void Context::initializeSystemLogs() }); } +void Context::createTraceCollector() +{ + shared->createTraceCollector(); +} + void Context::initializeTraceCollector() { shared->initializeTraceCollector(getTraceLog()); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 87a7baa0469..68f37377926 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1077,6 +1077,8 @@ public: void initializeSystemLogs(); /// Call after initialization before using trace collector. + void createTraceCollector(); + void initializeTraceCollector(); /// Call after unexpected crash happen. diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index 8e9c397b7a1..77f70d754c8 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -1,5 +1,4 @@ -#include "TraceCollector.h" - +#include #include #include #include @@ -14,8 +13,12 @@ namespace DB { -TraceCollector::TraceCollector(std::shared_ptr trace_log_) - : trace_log(std::move(trace_log_)) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +TraceCollector::TraceCollector() { TraceSender::pipe.open(); @@ -28,6 +31,23 @@ TraceCollector::TraceCollector(std::shared_ptr trace_log_) thread = ThreadFromGlobalPool(&TraceCollector::run, this); } +void TraceCollector::initialize(std::shared_ptr trace_log_) +{ + if (is_trace_log_initialized) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "TraceCollector is already initialized"); + + trace_log_ptr = trace_log_; + is_trace_log_initialized.store(true, std::memory_order_release); +} + +std::shared_ptr TraceCollector::getTraceLog() +{ + if (!is_trace_log_initialized.load(std::memory_order_acquire)) + return nullptr; + + return trace_log_ptr; +} + void TraceCollector::tryClosePipe() { try @@ -120,7 +140,7 @@ void TraceCollector::run() ProfileEvents::Count increment; readPODBinary(increment, in); - if (trace_log) + if (auto trace_log = getTraceLog()) { // time and time_in_microseconds are both being constructed from the same timespec so that the // times will be equal up to the precision of a second. diff --git a/src/Interpreters/TraceCollector.h b/src/Interpreters/TraceCollector.h index 382e7511ac6..c2894394dd0 100644 --- a/src/Interpreters/TraceCollector.h +++ b/src/Interpreters/TraceCollector.h @@ -1,4 +1,5 @@ #pragma once +#include #include class StackTrace; @@ -16,11 +17,17 @@ class TraceLog; class TraceCollector { public: - explicit TraceCollector(std::shared_ptr trace_log_); + TraceCollector(); ~TraceCollector(); + void initialize(std::shared_ptr trace_log_); + private: - std::shared_ptr trace_log; + std::shared_ptr getTraceLog(); + + std::atomic is_trace_log_initialized = false; + std::shared_ptr trace_log_ptr; + ThreadFromGlobalPool thread; void tryClosePipe(); From 7ea3345e0da818a6f9ea3127c9cffc54dbe9fd8a Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 6 Jun 2024 17:08:23 +0200 Subject: [PATCH 009/128] Use ThreadFromGlobalPool in Poco::ThreadPool --- base/poco/Foundation/CMakeLists.txt | 3 + base/poco/Foundation/src/ThreadPool.cpp | 79 ++++++++++++++++++------- programs/server/Server.cpp | 10 ++-- src/Server/KeeperTCPHandler.cpp | 1 - src/Server/TCPHandler.cpp | 1 - 5 files changed, 64 insertions(+), 30 deletions(-) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index dfb41a33fb1..fe59e72569c 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -168,6 +168,9 @@ set (SRCS add_library (_poco_foundation ${SRCS}) add_library (Poco::Foundation ALIAS _poco_foundation) +target_link_libraries (_poco_foundation PRIVATE clickhouse_common_io) +target_compile_definitions(_poco_foundation PUBLIC USE_CLICKHOUSE_THREADS=1) + # TODO: remove these warning exclusions target_compile_options (_poco_foundation PRIVATE diff --git a/base/poco/Foundation/src/ThreadPool.cpp b/base/poco/Foundation/src/ThreadPool.cpp index 6335ee82b47..a13e91d8b76 100644 --- a/base/poco/Foundation/src/ThreadPool.cpp +++ b/base/poco/Foundation/src/ThreadPool.cpp @@ -20,12 +20,15 @@ #include "Poco/ErrorHandler.h" #include #include +#if USE_CLICKHOUSE_THREADS +#include +#endif namespace Poco { -class PooledThread: public Runnable +class PooledThread : public Runnable { public: PooledThread(const std::string& name, int stackSize = POCO_THREAD_STACK_SIZE); @@ -46,7 +49,11 @@ private: volatile std::time_t _idleTime; Runnable* _pTarget; std::string _name; +#if USE_CLICKHOUSE_THREADS + ThreadFromGlobalPool _thread; +#else Thread _thread; +#endif Event _targetReady; Event _targetCompleted; Event _started; @@ -54,16 +61,20 @@ private: }; -PooledThread::PooledThread(const std::string& name, int stackSize): - _idle(true), - _idleTime(0), - _pTarget(0), - _name(name), +PooledThread::PooledThread(const std::string& name, [[maybe_unused]] int stackSize): + _idle(true), + _idleTime(0), + _pTarget(0), + _name(name), +#if !USE_CLICKHOUSE_THREADS _thread(name), +#endif _targetCompleted(false) { poco_assert_dbg (stackSize >= 0); +#if !USE_CLICKHOUSE_THREADS _thread.setStackSize(stackSize); +#endif _idleTime = std::time(NULL); } @@ -75,24 +86,32 @@ PooledThread::~PooledThread() void PooledThread::start() { +#if USE_CLICKHOUSE_THREADS + _thread = ThreadFromGlobalPool([this]() { run(); }); +#else _thread.start(*this); +#endif _started.wait(); } -void PooledThread::start(Thread::Priority priority, Runnable& target) +void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& target) { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_pTarget == 0); _pTarget = ⌖ + +#if !USE_CLICKHOUSE_THREADS _thread.setPriority(priority); +#endif + _targetReady.set(); } -void PooledThread::start(Thread::Priority priority, Runnable& target, const std::string& name) +void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& target, const std::string& name) { FastMutex::ScopedLock lock(_mutex); @@ -107,9 +126,12 @@ void PooledThread::start(Thread::Priority priority, Runnable& target, const std: fullName.append(_name); fullName.append(")"); } + +#if !USE_CLICKHOUSE_THREADS _thread.setName(fullName); _thread.setPriority(priority); - +#endif + poco_assert (_pTarget == 0); _pTarget = ⌖ @@ -145,7 +167,7 @@ void PooledThread::join() void PooledThread::activate() { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_idle); _idle = false; _targetCompleted.reset(); @@ -154,21 +176,30 @@ void PooledThread::activate() void PooledThread::release() { - const long JOIN_TIMEOUT = 10000; - _mutex.lock(); _pTarget = 0; _mutex.unlock(); + // In case of a statically allocated thread pool (such // as the default thread pool), Windows may have already // terminated the thread before we got here. +#if USE_CLICKHOUSE_THREADS + if (_thread.joinable()) +#else if (_thread.isRunning()) +#endif _targetReady.set(); +#if USE_CLICKHOUSE_THREADS + if (_thread.joinable()) + _thread.join(); +#else + const long JOIN_TIMEOUT = 10000; if (_thread.tryJoin(JOIN_TIMEOUT)) { delete this; } +#endif } @@ -205,8 +236,10 @@ void PooledThread::run() _idle = true; _targetCompleted.set(); ThreadLocalStorage::clear(); +#if !USE_CLICKHOUSE_THREADS _thread.setName(_name); _thread.setPriority(Thread::PRIO_NORMAL); +#endif } else { @@ -220,9 +253,9 @@ void PooledThread::run() ThreadPool::ThreadPool(int minCapacity, int maxCapacity, int idleTime, - int stackSize): - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + int stackSize): + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), @@ -245,8 +278,8 @@ ThreadPool::ThreadPool(const std::string& name, int idleTime, int stackSize): _name(name), - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), @@ -393,15 +426,15 @@ void ThreadPool::housekeep() ThreadVec activeThreads; idleThreads.reserve(_threads.size()); activeThreads.reserve(_threads.size()); - + for (ThreadVec::iterator it = _threads.begin(); it != _threads.end(); ++it) { if ((*it)->idle()) { if ((*it)->idleTime() < _idleTime) idleThreads.push_back(*it); - else - expiredThreads.push_back(*it); + else + expiredThreads.push_back(*it); } else activeThreads.push_back(*it); } @@ -481,7 +514,7 @@ public: ThreadPool* pool() { FastMutex::ScopedLock lock(_mutex); - + if (!_pPool) { _pPool = new ThreadPool("default"); @@ -490,7 +523,7 @@ public: } return _pPool; } - + private: ThreadPool* _pPool; FastMutex _mutex; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 84ff0e94cef..9d4ff00c8a1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -721,11 +721,6 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); - Poco::ThreadPool server_pool(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -823,6 +818,11 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } + Poco::ThreadPool server_pool(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 6709cd298e5..4612e2e9fa8 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -309,7 +309,6 @@ Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length, bool void KeeperTCPHandler::runImpl() { setThreadName("KeeperHandler"); - ThreadStatus thread_status; socket().setReceiveTimeout(receive_timeout); socket().setSendTimeout(send_timeout); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index e3a820340ad..b60339e9fd8 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -246,7 +246,6 @@ TCPHandler::~TCPHandler() void TCPHandler::runImpl() { setThreadName("TCPHandler"); - ThreadStatus thread_status; extractConnectionSettingsFromContext(server.context()); From e30fa1da4d9a51c5626bde79629981fb693c401a Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 Jun 2024 15:03:13 +0200 Subject: [PATCH 010/128] Fix ThreadStatus --- src/Server/HTTPHandler.cpp | 1 - src/Server/InterserverIOHTTPHandler.cpp | 2 -- src/Server/MySQLHandler.cpp | 2 -- src/Server/PostgreSQLHandler.cpp | 2 -- 4 files changed, 7 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 02d0959ff50..d6afa571e71 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1060,7 +1060,6 @@ void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("HTTPHandler"); - ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::HTTP, request.isSecure()); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 0d79aaa227b..45c28babe3a 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include @@ -81,7 +80,6 @@ void InterserverIOHTTPHandler::processQuery(HTTPServerRequest & request, HTTPSer void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("IntersrvHandler"); - ThreadStatus thread_status; /// In order to work keep-alive. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 6456f6d24ff..9471509ad4b 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include @@ -199,7 +198,6 @@ MySQLHandler::~MySQLHandler() = default; void MySQLHandler::run() { setThreadName("MySQLHandler"); - ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::MYSQL); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 473d681ddb2..8ba8421e6f0 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -59,7 +58,6 @@ void PostgreSQLHandler::changeIO(Poco::Net::StreamSocket & socket) void PostgreSQLHandler::run() { setThreadName("PostgresHandler"); - ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::POSTGRESQL); SCOPE_EXIT({ session.reset(); }); From 6604d94271a491772f9fbd42f129bd28afb8c950 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 7 Jun 2024 17:11:49 +0200 Subject: [PATCH 011/128] Ping CI: skip fast test to see all stateless runs From a11ba3f437c2262f1a1263ebe012bd3a85dc6ea3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 Jun 2024 15:19:03 +0200 Subject: [PATCH 012/128] Fix shutdown --- programs/server/Server.cpp | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9d4ff00c8a1..802c994e872 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -818,7 +818,7 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } - Poco::ThreadPool server_pool(3, server_settings.max_connections); + auto server_pool = std::make_unique(3, server_settings.max_connections); std::mutex servers_lock; std::vector servers; std::vector servers_to_start_before_tables; @@ -898,7 +898,8 @@ try global_context->shutdownKeeperDispatcher(); /// Wait server pool to avoid use-after-free of destroyed context in the handlers - server_pool.joinAll(); + server_pool->joinAll(); + server_pool.reset(); /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. * At this moment, no one could own shared part of Context. @@ -1629,7 +1630,7 @@ try if (global_context->isServerCompletelyStarted()) { std::lock_guard lock(servers_lock); - updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); + updateServers(*config, *server_pool, async_metrics, servers, servers_to_start_before_tables); } } @@ -1726,7 +1727,7 @@ try config_getter, global_context->getKeeperDispatcher(), global_context->getSettingsRef().receive_timeout.totalSeconds(), global_context->getSettingsRef().send_timeout.totalSeconds(), - false), server_pool, socket)); + false), *server_pool, socket)); }); const char * secure_port_name = "keeper_server.tcp_port_secure"; @@ -1748,7 +1749,7 @@ try new KeeperTCPHandlerFactory( config_getter, global_context->getKeeperDispatcher(), global_context->getSettingsRef().receive_timeout.totalSeconds(), - global_context->getSettingsRef().send_timeout.totalSeconds(), true), server_pool, socket)); + global_context->getSettingsRef().send_timeout.totalSeconds(), true), *server_pool, socket)); #else UNUSED(port); throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); @@ -1780,7 +1781,7 @@ try createKeeperHTTPControlMainHandlerFactory( config_getter(), global_context->getKeeperDispatcher(), - "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params)); + "KeeperHTTPControlHandler-factory"), *server_pool, socket, http_params)); }); } #else @@ -1804,7 +1805,7 @@ try config(), interserver_listen_hosts, listen_try, - server_pool, + *server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ false); @@ -1855,7 +1856,7 @@ try config(), listen_hosts, listen_try, - server_pool, + *server_pool, async_metrics, servers, /* start_servers= */ true, @@ -2027,7 +2028,7 @@ try { std::lock_guard lock(servers_lock); - createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); + createServers(config(), listen_hosts, listen_try, *server_pool, async_metrics, servers); if (servers.empty()) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " From c3d4b429d9b98b0629435cacbba59cc61a3e8ebd Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 Jun 2024 15:39:54 +0200 Subject: [PATCH 013/128] Fix merge --- programs/server/Server.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 802c994e872..dfbcd36bcb3 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -721,6 +721,11 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); + auto server_pool = std::make_unique(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -818,11 +823,6 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } - auto server_pool = std::make_unique(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; From 6514d72fea8e723683a614d258fc24bdc00f78ba Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 10 Jun 2024 18:53:51 +0200 Subject: [PATCH 014/128] Move servers pool back --- programs/server/Server.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index dfbcd36bcb3..802c994e872 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -721,11 +721,6 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); - auto server_pool = std::make_unique(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -823,6 +818,11 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } + auto server_pool = std::make_unique(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; From c802d7d58aa6d8e369497cc769dd0860bf929124 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 11 Jun 2024 14:35:26 +0200 Subject: [PATCH 015/128] Writing improvements --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 216 +++++++++++++++++++++-- src/Common/ZooKeeper/ZooKeeperCommon.h | 40 ++++- src/Common/ZooKeeper/ZooKeeperIO.cpp | 26 +++ src/Common/ZooKeeper/ZooKeeperIO.h | 30 ++++ 4 files changed, 296 insertions(+), 16 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 48bb510e589..602b05274e4 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -9,7 +9,6 @@ #include #include #include -#include namespace Coordination @@ -19,14 +18,16 @@ using namespace DB; void ZooKeeperResponse::write(WriteBuffer & out) const { - /// Excessive copy to calculate length. - WriteBufferFromOwnString buf; - Coordination::write(xid, buf); - Coordination::write(zxid, buf); - Coordination::write(error, buf); + auto response_size = Coordination::size(xid) + Coordination::size(zxid) + Coordination::size(error); if (error == Error::ZOK) - writeImpl(buf); - Coordination::write(buf.str(), out); + response_size += sizeImpl(); + + Coordination::write(static_cast(response_size), out); + Coordination::write(xid, out); + Coordination::write(zxid, out); + Coordination::write(error, out); + if (error == Error::ZOK) + writeImpl(out); } std::string ZooKeeperRequest::toString() const @@ -42,12 +43,12 @@ std::string ZooKeeperRequest::toString() const void ZooKeeperRequest::write(WriteBuffer & out) const { - /// Excessive copy to calculate length. - WriteBufferFromOwnString buf; - Coordination::write(xid, buf); - Coordination::write(getOpNum(), buf); - writeImpl(buf); - Coordination::write(buf.str(), out); + auto request_size = Coordination::size(xid) + Coordination::size(getOpNum()) + sizeImpl(); + + Coordination::write(static_cast(request_size), out); + Coordination::write(xid, out); + Coordination::write(getOpNum(), out); + writeImpl(out); } void ZooKeeperSyncRequest::writeImpl(WriteBuffer & out) const @@ -55,6 +56,11 @@ void ZooKeeperSyncRequest::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +size_t ZooKeeperSyncRequest::sizeImpl() const +{ + return Coordination::size(path); +} + void ZooKeeperSyncRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -75,6 +81,11 @@ void ZooKeeperSyncResponse::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +size_t ZooKeeperSyncResponse::sizeImpl() const +{ + return Coordination::size(path); +} + void ZooKeeperReconfigRequest::writeImpl(WriteBuffer & out) const { Coordination::write(joining, out); @@ -83,6 +94,11 @@ void ZooKeeperReconfigRequest::writeImpl(WriteBuffer & out) const Coordination::write(version, out); } +size_t ZooKeeperReconfigRequest::sizeImpl() const +{ + return Coordination::size(joining) + Coordination::size(leaving) + Coordination::size(new_members) + Coordination::size(version); +} + void ZooKeeperReconfigRequest::readImpl(ReadBuffer & in) { Coordination::read(joining, in); @@ -110,6 +126,11 @@ void ZooKeeperReconfigResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperReconfigResponse::sizeImpl() const +{ + return Coordination::size(value) + Coordination::size(stat); +} + void ZooKeeperWatchResponse::readImpl(ReadBuffer & in) { Coordination::read(type, in); @@ -124,6 +145,11 @@ void ZooKeeperWatchResponse::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +size_t ZooKeeperWatchResponse::sizeImpl() const +{ + return Coordination::size(type) + Coordination::size(state) + Coordination::size(path); +} + void ZooKeeperWatchResponse::write(WriteBuffer & out) const { if (error == Error::ZOK) @@ -138,6 +164,11 @@ void ZooKeeperAuthRequest::writeImpl(WriteBuffer & out) const Coordination::write(data, out); } +size_t ZooKeeperAuthRequest::sizeImpl() const +{ + return Coordination::size(type) + Coordination::size(scheme) + Coordination::size(data); +} + void ZooKeeperAuthRequest::readImpl(ReadBuffer & in) { Coordination::read(type, in); @@ -176,6 +207,19 @@ void ZooKeeperCreateRequest::writeImpl(WriteBuffer & out) const Coordination::write(flags, out); } +size_t ZooKeeperCreateRequest::sizeImpl() const +{ + /// See https://github.com/ClickHouse/clickhouse-private/issues/3029 + if (path.starts_with("/clickhouse/tables/") && path.find("/parts/") != std::string::npos) + { + LOG_TRACE(getLogger(__PRETTY_FUNCTION__), "Creating part at path {}", path); + } + + int32_t flags = 0; + + return Coordination::size(path) + Coordination::size(data) + Coordination::size(acls) + Coordination::size(flags); +} + void ZooKeeperCreateRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -212,12 +256,22 @@ void ZooKeeperCreateResponse::writeImpl(WriteBuffer & out) const Coordination::write(path_created, out); } +size_t ZooKeeperCreateResponse::sizeImpl() const +{ + return Coordination::size(path_created); +} + void ZooKeeperRemoveRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); Coordination::write(version, out); } +size_t ZooKeeperRemoveRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(version); +} + std::string ZooKeeperRemoveRequest::toStringImpl() const { return fmt::format( @@ -239,6 +293,11 @@ void ZooKeeperExistsRequest::writeImpl(WriteBuffer & out) const Coordination::write(has_watch, out); } +size_t ZooKeeperExistsRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(has_watch); +} + void ZooKeeperExistsRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -260,12 +319,22 @@ void ZooKeeperExistsResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperExistsResponse::sizeImpl() const +{ + return Coordination::size(stat); +} + void ZooKeeperGetRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); Coordination::write(has_watch, out); } +size_t ZooKeeperGetRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(has_watch); +} + void ZooKeeperGetRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -289,6 +358,11 @@ void ZooKeeperGetResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperGetResponse::sizeImpl() const +{ + return Coordination::size(data) + Coordination::size(stat); +} + void ZooKeeperSetRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); @@ -296,6 +370,11 @@ void ZooKeeperSetRequest::writeImpl(WriteBuffer & out) const Coordination::write(version, out); } +size_t ZooKeeperSetRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(data) + Coordination::size(version); +} + void ZooKeeperSetRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -322,12 +401,22 @@ void ZooKeeperSetResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperSetResponse::sizeImpl() const +{ + return Coordination::size(stat); +} + void ZooKeeperListRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); Coordination::write(has_watch, out); } +size_t ZooKeeperListRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(has_watch); +} + void ZooKeeperListRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -346,6 +435,11 @@ void ZooKeeperFilteredListRequest::writeImpl(WriteBuffer & out) const Coordination::write(static_cast(list_request_type), out); } +size_t ZooKeeperFilteredListRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(has_watch) + Coordination::size(static_cast(list_request_type)); +} + void ZooKeeperFilteredListRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -377,6 +471,11 @@ void ZooKeeperListResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperListResponse::sizeImpl() const +{ + return Coordination::size(names) + Coordination::size(stat); +} + void ZooKeeperSimpleListResponse::readImpl(ReadBuffer & in) { Coordination::read(names, in); @@ -387,6 +486,11 @@ void ZooKeeperSimpleListResponse::writeImpl(WriteBuffer & out) const Coordination::write(names, out); } +size_t ZooKeeperSimpleListResponse::sizeImpl() const +{ + return Coordination::size(names); +} + void ZooKeeperSetACLRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); @@ -394,6 +498,11 @@ void ZooKeeperSetACLRequest::writeImpl(WriteBuffer & out) const Coordination::write(version, out); } +size_t ZooKeeperSetACLRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(acls) + Coordination::size(version); +} + void ZooKeeperSetACLRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -411,6 +520,11 @@ void ZooKeeperSetACLResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperSetACLResponse::sizeImpl() const +{ + return Coordination::size(stat); +} + void ZooKeeperSetACLResponse::readImpl(ReadBuffer & in) { Coordination::read(stat, in); @@ -426,6 +540,11 @@ void ZooKeeperGetACLRequest::writeImpl(WriteBuffer & out) const Coordination::write(path, out); } +size_t ZooKeeperGetACLRequest::sizeImpl() const +{ + return Coordination::size(path); +} + std::string ZooKeeperGetACLRequest::toStringImpl() const { return fmt::format("path = {}", path); @@ -437,6 +556,11 @@ void ZooKeeperGetACLResponse::writeImpl(WriteBuffer & out) const Coordination::write(stat, out); } +size_t ZooKeeperGetACLResponse::sizeImpl() const +{ + return Coordination::size(acl) + Coordination::size(stat); +} + void ZooKeeperGetACLResponse::readImpl(ReadBuffer & in) { Coordination::read(acl, in); @@ -449,6 +573,11 @@ void ZooKeeperCheckRequest::writeImpl(WriteBuffer & out) const Coordination::write(version, out); } +size_t ZooKeeperCheckRequest::sizeImpl() const +{ + return Coordination::size(path) + Coordination::size(version); +} + void ZooKeeperCheckRequest::readImpl(ReadBuffer & in) { Coordination::read(path, in); @@ -474,6 +603,11 @@ void ZooKeeperErrorResponse::writeImpl(WriteBuffer & out) const Coordination::write(error, out); } +size_t ZooKeeperErrorResponse::sizeImpl() const +{ + return Coordination::size(error); +} + void ZooKeeperMultiRequest::checkOperationType(OperationType type) { chassert(!operation_type.has_value() || *operation_type == type); @@ -571,6 +705,27 @@ void ZooKeeperMultiRequest::writeImpl(WriteBuffer & out) const Coordination::write(error, out); } +size_t ZooKeeperMultiRequest::sizeImpl() const +{ + size_t total_size = 0; + for (const auto & request : requests) + { + const auto & zk_request = dynamic_cast(*request); + + bool done = false; + int32_t error = -1; + + total_size + += Coordination::size(zk_request.getOpNum()) + Coordination::size(done) + Coordination::size(error) + zk_request.sizeImpl(); + } + + OpNum op_num = OpNum::Error; + bool done = true; + int32_t error = -1; + + return total_size + Coordination::size(op_num) + Coordination::size(done) + Coordination::size(error); +} + void ZooKeeperMultiRequest::readImpl(ReadBuffer & in) { while (true) @@ -701,6 +856,29 @@ void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const } } +size_t ZooKeeperMultiResponse::sizeImpl() const +{ + size_t total_size = 0; + for (const auto & response : responses) + { + const ZooKeeperResponse & zk_response = dynamic_cast(*response); + OpNum op_num = zk_response.getOpNum(); + bool done = false; + Error op_error = zk_response.error; + + total_size += Coordination::size(op_num) + Coordination::size(done) + Coordination::size(op_error); + if (op_error == Error::ZOK || op_num == OpNum::Error) + total_size += zk_response.sizeImpl(); + } + + /// Footer. + OpNum op_num = OpNum::Error; + bool done = true; + int32_t error_read = - 1; + + return total_size + Coordination::size(op_num) + Coordination::size(done) + Coordination::size(error_read); +} + ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return setTime(std::make_shared()); } @@ -748,6 +926,11 @@ void ZooKeeperSessionIDRequest::writeImpl(WriteBuffer & out) const Coordination::write(server_id, out); } +size_t ZooKeeperSessionIDRequest::sizeImpl() const +{ + return Coordination::size(internal_id) + Coordination::size(session_timeout_ms) + Coordination::size(server_id); +} + void ZooKeeperSessionIDRequest::readImpl(ReadBuffer & in) { Coordination::read(internal_id, in); @@ -774,6 +957,11 @@ void ZooKeeperSessionIDResponse::writeImpl(WriteBuffer & out) const Coordination::write(server_id, out); } +size_t ZooKeeperSessionIDResponse::sizeImpl() const +{ + return Coordination::size(internal_id) + Coordination::size(session_id) + Coordination::size(server_id); +} + void ZooKeeperRequest::createLogElements(LogElements & elems) const { diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 490c2dce4f8..7c318a97676 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -7,13 +7,11 @@ #include #include #include -#include #include #include #include #include #include -#include namespace Coordination @@ -32,6 +30,7 @@ struct ZooKeeperResponse : virtual Response ~ZooKeeperResponse() override; virtual void readImpl(ReadBuffer &) = 0; virtual void writeImpl(WriteBuffer &) const = 0; + virtual size_t sizeImpl() const = 0; virtual void write(WriteBuffer & out) const; virtual OpNum getOpNum() const = 0; virtual void fillLogElements(LogElements & elems, size_t idx) const; @@ -66,6 +65,7 @@ struct ZooKeeperRequest : virtual Request std::string toString() const; virtual void writeImpl(WriteBuffer &) const = 0; + virtual size_t sizeImpl() const = 0; virtual void readImpl(ReadBuffer &) = 0; virtual std::string toStringImpl() const { return ""; } @@ -86,6 +86,7 @@ struct ZooKeeperHeartbeatRequest final : ZooKeeperRequest String getPath() const override { return {}; } OpNum getOpNum() const override { return OpNum::Heartbeat; } void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } void readImpl(ReadBuffer &) override {} ZooKeeperResponsePtr makeResponse() const override; bool isReadRequest() const override { return false; } @@ -97,6 +98,7 @@ struct ZooKeeperSyncRequest final : ZooKeeperRequest String getPath() const override { return path; } OpNum getOpNum() const override { return OpNum::Sync; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -109,6 +111,7 @@ struct ZooKeeperSyncResponse final : SyncResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Sync; } }; @@ -122,6 +125,7 @@ struct ZooKeeperReconfigRequest final : ZooKeeperRequest String getPath() const override { return keeper_config_path; } OpNum getOpNum() const override { return OpNum::Reconfig; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -138,6 +142,7 @@ struct ZooKeeperReconfigResponse final : ReconfigResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Reconfig; } }; @@ -145,6 +150,7 @@ struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Heartbeat; } }; @@ -153,6 +159,7 @@ struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void write(WriteBuffer & out) const override; @@ -175,6 +182,7 @@ struct ZooKeeperAuthRequest final : ZooKeeperRequest String getPath() const override { return {}; } OpNum getOpNum() const override { return OpNum::Auth; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -189,6 +197,7 @@ struct ZooKeeperAuthResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Auth; } @@ -200,6 +209,7 @@ struct ZooKeeperCloseRequest final : ZooKeeperRequest String getPath() const override { return {}; } OpNum getOpNum() const override { return OpNum::Close; } void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } void readImpl(ReadBuffer &) override {} ZooKeeperResponsePtr makeResponse() const override; @@ -214,6 +224,7 @@ struct ZooKeeperCloseResponse final : ZooKeeperResponse } void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Close; } }; @@ -228,6 +239,7 @@ struct ZooKeeperCreateRequest final : public CreateRequest, ZooKeeperRequest OpNum getOpNum() const override { return not_exists ? OpNum::CreateIfNotExists : OpNum::Create; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -244,6 +256,7 @@ struct ZooKeeperCreateResponse : CreateResponse, ZooKeeperResponse void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Create; } @@ -265,6 +278,7 @@ struct ZooKeeperRemoveRequest final : RemoveRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::Remove; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -280,6 +294,7 @@ struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Remove; } size_t bytesSize() const override { return RemoveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -292,6 +307,7 @@ struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::Exists; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -305,6 +321,7 @@ struct ZooKeeperExistsResponse final : ExistsResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Exists; } size_t bytesSize() const override { return ExistsResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -319,6 +336,7 @@ struct ZooKeeperGetRequest final : GetRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::Get; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -332,6 +350,7 @@ struct ZooKeeperGetResponse final : GetResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Get; } size_t bytesSize() const override { return GetResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -346,6 +365,7 @@ struct ZooKeeperSetRequest final : SetRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::Set; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -360,6 +380,7 @@ struct ZooKeeperSetResponse final : SetResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Set; } size_t bytesSize() const override { return SetResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -374,6 +395,7 @@ struct ZooKeeperListRequest : ListRequest, ZooKeeperRequest OpNum getOpNum() const override { return OpNum::List; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -394,6 +416,7 @@ struct ZooKeeperFilteredListRequest final : ZooKeeperListRequest OpNum getOpNum() const override { return OpNum::FilteredList; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -404,6 +427,7 @@ struct ZooKeeperListResponse : ListResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::List; } size_t bytesSize() const override { return ListResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -415,6 +439,7 @@ struct ZooKeeperSimpleListResponse final : ZooKeeperListResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::SimpleList; } size_t bytesSize() const override { return ZooKeeperListResponse::bytesSize() - sizeof(stat); } @@ -427,6 +452,7 @@ struct ZooKeeperCheckRequest : CheckRequest, ZooKeeperRequest OpNum getOpNum() const override { return not_exists ? OpNum::CheckNotExists : OpNum::Check; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -442,6 +468,7 @@ struct ZooKeeperCheckResponse : CheckResponse, ZooKeeperResponse { void readImpl(ReadBuffer &) override {} void writeImpl(WriteBuffer &) const override {} + size_t sizeImpl() const override { return 0; } OpNum getOpNum() const override { return OpNum::Check; } size_t bytesSize() const override { return CheckResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -458,6 +485,7 @@ struct ZooKeeperErrorResponse final : ErrorResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::Error; } @@ -468,6 +496,7 @@ struct ZooKeeperSetACLRequest final : SetACLRequest, ZooKeeperRequest { OpNum getOpNum() const override { return OpNum::SetACL; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -480,6 +509,7 @@ struct ZooKeeperSetACLResponse final : SetACLResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::SetACL; } size_t bytesSize() const override { return SetACLResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -489,6 +519,7 @@ struct ZooKeeperGetACLRequest final : GetACLRequest, ZooKeeperRequest { OpNum getOpNum() const override { return OpNum::GetACL; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; ZooKeeperResponsePtr makeResponse() const override; @@ -501,6 +532,7 @@ struct ZooKeeperGetACLResponse final : GetACLResponse, ZooKeeperResponse { void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; OpNum getOpNum() const override { return OpNum::GetACL; } size_t bytesSize() const override { return GetACLResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -515,6 +547,7 @@ struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest ZooKeeperMultiRequest(std::span generic_requests, const ACLs & default_acls); void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; std::string toStringImpl() const override; @@ -554,6 +587,7 @@ struct ZooKeeperMultiResponse : MultiResponse, ZooKeeperResponse void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; size_t bytesSize() const override { return MultiResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } @@ -584,6 +618,7 @@ struct ZooKeeperSessionIDRequest final : ZooKeeperRequest Coordination::OpNum getOpNum() const override { return OpNum::SessionID; } String getPath() const override { return {}; } void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; void readImpl(ReadBuffer & in) override; Coordination::ZooKeeperResponsePtr makeResponse() const override; @@ -602,6 +637,7 @@ struct ZooKeeperSessionIDResponse final : ZooKeeperResponse void readImpl(ReadBuffer & in) override; void writeImpl(WriteBuffer & out) const override; + size_t sizeImpl() const override; Coordination::OpNum getOpNum() const override { return OpNum::SessionID; } }; diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp index 6a51ffb36fa..a1afe039a04 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.cpp +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -42,6 +42,32 @@ void write(const Error & x, WriteBuffer & out) write(static_cast(x), out); } +size_t size(OpNum x) +{ + return size(static_cast(x)); +} + +size_t size(const std::string & s) +{ + return size(static_cast(s.size())) + s.size(); +} + +size_t size(const ACL & acl) +{ + return size(acl.permissions) + size(acl.scheme) + size(acl.id); +} + +size_t size(const Stat & stat) +{ + return size(stat.czxid) + size(stat.mzxid) + size(stat.ctime) + size(stat.mtime) + size(stat.version) + size(stat.cversion) + + size(stat.aversion) + size(stat.ephemeralOwner) + size(stat.dataLength) + size(stat.numChildren) + size(stat.pzxid); +} + +size_t size(const Error & x) +{ + return size(static_cast(x)); +} + void read(OpNum & x, ReadBuffer & in) { int32_t raw_op_num; diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h index 83973c1ae22..242370d2d25 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.h +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -43,6 +43,36 @@ void write(const std::vector & arr, WriteBuffer & out) write(elem, out); } +template +requires is_arithmetic_v +size_t size(T x) +{ + return sizeof(x); +} + +size_t size(OpNum x); +size_t size(const std::string & s); +size_t size(const ACL & acl); +size_t size(const Stat & stat); +size_t size(const Error & x); + +template +size_t size(const std::array) +{ + return size(static_cast(N)) + N; +} + +template +size_t size(const std::vector & arr) +{ + size_t total_size = size(static_cast(arr.size())); + for (const auto & elem : arr) + total_size += size(elem); + + return total_size; +} + + template requires is_arithmetic_v void read(T & x, ReadBuffer & in) From 0fa45c39540129bd68b3e50c2c5084f6959d48b6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 11 Jun 2024 14:35:34 +0200 Subject: [PATCH 016/128] More parallel storage --- src/Coordination/KeeperDispatcher.cpp | 19 ++-- src/Coordination/KeeperServer.cpp | 5 +- src/Coordination/KeeperServer.h | 2 +- src/Coordination/KeeperStateMachine.cpp | 92 +++++++------------- src/Coordination/KeeperStateMachine.h | 5 +- src/Coordination/KeeperStorage.cpp | 111 +++++++----------------- src/Coordination/KeeperStorage.h | 19 ++-- 7 files changed, 87 insertions(+), 166 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 0fe0b8cce1f..b9518c51893 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -449,18 +449,13 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf if (auto request_queue_it = xid_to_request_queue.find(request_for_session.request->xid); request_queue_it != xid_to_request_queue.end()) { - Stopwatch watch; - if (server->isLeaderAlive()) - server->putLocalReadRequest(request_queue_it->second); - else - addErrorResponses(request_queue_it->second, Coordination::Error::ZCONNECTIONLOSS); - - if (request_queue_it->second.size() > 500) - LOG_INFO( - getLogger("Speed"), - "It took {}ms for {} requests", - watch.elapsedMilliseconds(), - request_queue_it->second.size()); + for (const auto & read_request : request_queue_it->second) + { + if (server->isLeaderAlive()) + server->putLocalReadRequest(read_request); + else + addErrorResponses({read_request}, Coordination::Error::ZCONNECTIONLOSS); + } xid_to_request_queue.erase(request_queue_it); } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index c14658ddccf..02b05c6346c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -533,10 +533,9 @@ nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestFor } -void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestsForSessions & request_for_session) +void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & request_for_session) { - if (std::any_of( - request_for_session.begin(), request_for_session.end(), [](const auto & request) { return !request.request->isReadRequest(); })) + if (!request_for_session.request->isReadRequest()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot process non-read request locally"); state_machine->processReadRequest(request_for_session); diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index c43fb35efe1..5e45a552cba 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -84,7 +84,7 @@ public: /// Put local read request and execute in state machine directly and response into /// responses queue - void putLocalReadRequest(const KeeperStorage::RequestsForSessions & request); + void putLocalReadRequest(const KeeperStorage::RequestForSession & request); bool isRecovering() const { return is_recovering; } bool reconfigEnabled() const { return enable_reconfiguration; } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 3b5b7e07f06..a5402944701 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -278,12 +278,12 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) return true; - std::shared_lock storage_lock(storage_mutex); if (storage->isFinalized()) return false; try { + std::shared_lock storage_lock(storage_mutex); storage->preprocessRequest( request_for_session.request, request_for_session.session_id, @@ -312,7 +312,7 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session) { - std::lock_guard lock(process_and_responses_lock); + std::lock_guard lock(storage_mutex); KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session); if (!responses_queue.push(response)) { @@ -414,7 +414,6 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n try { - std::shared_lock storage_lock(storage_mutex); const auto op_num = request_for_session->request->getOpNum(); if (op_num == Coordination::OpNum::SessionID) { @@ -428,7 +427,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n response_for_session.session_id = -1; response_for_session.response = response; - std::lock_guard lock(process_and_responses_lock); + std::unique_lock lock(storage_mutex); session_id = storage->getSessionID(session_id_request.session_timeout_ms); LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); response->session_id = session_id; @@ -438,15 +437,18 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n { if (op_num == Coordination::OpNum::Close) { - std::lock_guard lock(request_cache_mutex); + std::lock_guard cache_lock(request_cache_mutex); parsed_request_cache.erase(request_for_session->session_id); } - std::lock_guard lock(process_and_responses_lock); - KeeperStorage::ResponsesForSessions responses_for_sessions - = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); - for (auto & response_for_session : responses_for_sessions) - try_push(response_for_session); + { + std::shared_lock lock(storage_mutex); + std::lock_guard response_lock(process_and_responses_lock); + KeeperStorage::ResponsesForSessions responses_for_sessions + = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); + for (auto & response_for_session : responses_for_sessions) + try_push(response_for_session); + } if (keeper_context->digestEnabled() && request_for_session->digest) assertDigest( @@ -549,7 +551,7 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; - std::shared_lock lock(storage_mutex); + std::unique_lock lock(storage_mutex); storage->rollbackRequest(request_for_session.zxid, allow_missing); } @@ -767,42 +769,16 @@ int KeeperStateMachine::read_logical_snp_obj( return 1; } -void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestsForSessions & request_for_session) +void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session) { std::shared_lock storage_lock(storage_mutex); + std::lock_guard response_lock(process_and_responses_lock); + auto responses = storage->processRequest( + request_for_session.request, request_for_session.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); + for (const auto & response : responses) + if (!responses_queue.push(response)) + LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id); - /// Pure local request, just process it with storage - std::lock_guard lock(process_and_responses_lock); - std::vector all_responses; - if (request_for_session.size() > 100) - { - all_responses.resize(request_for_session.size()); - //LOG_INFO(getLogger("Keeper"), "Has read requests {}", request_queue_it->second.size()); - for (size_t i = 0; i < request_for_session.size(); ++i) - { - read_pool.scheduleOrThrowOnError([&, i] - { - const auto & read_request = request_for_session[i]; - all_responses[i] = storage->processRequest( - read_request.request, read_request.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/); - }); - } - read_pool.wait(); - } - else - { - all_responses.reserve(request_for_session.size()); - for (const auto & read_request : request_for_session) - { - all_responses.push_back(storage->processRequest( - read_request.request, read_request.session_id, std::nullopt, true /*check_acl*/, true /*is_local*/)); - } - } - - for (const auto & responses : all_responses) - for (const auto & response : responses) - if (!responses_queue.push(response)) - LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id); } void KeeperStateMachine::shutdownStorage() @@ -813,25 +789,23 @@ void KeeperStateMachine::shutdownStorage() std::vector KeeperStateMachine::getDeadSessions() { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getDeadSessions(); } int64_t KeeperStateMachine::getNextZxid() const { - std::shared_lock storage_lock(storage_mutex); return storage->getNextZXID(); } KeeperStorage::Digest KeeperStateMachine::getNodesDigest() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getNodesDigest(false, /*lock_transaction_mutex=*/true); } uint64_t KeeperStateMachine::getLastProcessedZxid() const { - std::shared_lock storage_lock(storage_mutex); return storage->getZXID(); } @@ -842,61 +816,61 @@ const KeeperStorage::Stats & KeeperStateMachine::getStorageStats() const TSA_NO_ uint64_t KeeperStateMachine::getTotalWatchesCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getTotalWatchesCount(); } uint64_t KeeperStateMachine::getWatchedPathsCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getWatchedPathsCount(); } uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getSessionsWithWatchesCount(); } uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getTotalEphemeralNodesCount(); } uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getSessionWithEphemeralNodesCount(); } void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); storage->dumpWatches(buf); } void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); storage->dumpWatchesByPath(buf); } void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); storage->dumpSessionsAndEphemerals(buf); } uint64_t KeeperStateMachine::getApproximateDataSize() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getApproximateDataSize(); } uint64_t KeeperStateMachine::getKeyArenaSize() const { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); return storage->getArenaDataSize(); } @@ -937,7 +911,7 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const void KeeperStateMachine::recalculateStorageStats() { - std::shared_lock storage_lock(storage_mutex); + std::unique_lock storage_lock(storage_mutex); LOG_INFO(log, "Recalculating storage stats"); storage->recalculateStats(); LOG_INFO(log, "Done recalculating storage stats"); diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 68b9366611d..c1702394754 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -100,7 +100,7 @@ public: ClusterConfigPtr getClusterConfig() const; /// Process local read request - void processReadRequest(const KeeperStorage::RequestsForSessions & request_for_session); + void processReadRequest(const KeeperStorage::RequestForSession & request_for_session); std::vector getDeadSessions(); @@ -132,8 +132,6 @@ public: void reconfigure(const KeeperStorage::RequestForSession& request_for_session); private: - mutable SharedMutex storage_mutex; - CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored /// in memory in compressed (serialized) format. @@ -156,6 +154,7 @@ private: /// Mutex for snapshots mutable std::mutex snapshots_lock; + mutable SharedMutex storage_mutex; /// Lock for storage and responses_queue. It's important to process requests /// and push them to the responses queue while holding this lock. Otherwise /// we can get strange cases when, for example client send read request with diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index e62be70ede5..238ab07a276 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -412,7 +412,7 @@ Overloaded(Ts...) -> Overloaded; std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const { - std::lock_guard lock(storage.storage_mutex); + std::shared_lock lock(storage.storage_mutex); if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; @@ -532,7 +532,7 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool is_local, void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) { - assert(!delta.path.empty()); + chassert(!delta.path.empty()); std::visit( [&](const DeltaType & operation) @@ -541,12 +541,12 @@ void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) if constexpr (std::same_as) { - assert(node); + chassert(node); node = nullptr; } else if constexpr (std::same_as) { - assert(!node); + chassert(!node); node = std::make_shared(); node->stats = operation.stat; node->setData(operation.data); @@ -554,13 +554,13 @@ void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) } else if constexpr (std::same_as) { - assert(node); + chassert(node); node->invalidateDigestCache(); node->stats = operation.old_stats; } else if constexpr (std::same_as) { - assert(node); + chassert(node); node->invalidateDigestCache(); node->setData(operation.old_data); } @@ -569,7 +569,7 @@ void KeeperStorage::UncommittedState::rollbackDelta(const Delta & delta) acls = operation.old_acls; } - if (applied_zxids.back() != delta.zxid) + if (applied_zxids.back() == delta.zxid) applied_zxids.pop_back(); }, delta.operation); @@ -645,15 +645,16 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) rollback_deltas.splice(rollback_deltas.end(), deltas, delta_it.base(), deltas.end()); } + rollback(std::move(rollback_deltas)); +} + +void KeeperStorage::UncommittedState::rollback(std::list rollback_deltas) +{ // we need to undo ephemeral mapping modifications // CreateNodeDelta added ephemeral for session id -> we need to remove it // RemoveNodeDelta removed ephemeral for session id -> we need to add it back for (auto & delta : rollback_deltas) { - if (delta.zxid < rollback_zxid) - break; - - chassert(delta.zxid == rollback_zxid); if (!delta.path.empty()) { std::visit( @@ -674,6 +675,8 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) } }, delta.operation); + + rollbackDelta(delta); } else if (auto * add_auth = std::get_if(&delta.operation)) { @@ -685,8 +688,6 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) session_and_auth.erase(add_auth->session_id); } } - - rollbackDelta(delta); } } @@ -849,13 +850,9 @@ Coordination::Error KeeperStorage::commit(std::list deltas) removeDigest(node_it->value, path); auto updated_node = container.updateValue(path, [&](auto & node) { if constexpr (std::same_as) - { node.stats = operation.new_stats; - } else - { node.setData(std::move(operation.new_data)); - } }); addDigest(updated_node->value, path); @@ -1044,7 +1041,7 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_ { if (is_local) { - std::lock_guard lock(storage.storage_mutex); + std::shared_lock lock(storage.storage_mutex); auto node_it = storage.container.find(path); if (node_it == storage.container.end()) return {}; @@ -1223,6 +1220,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr zxid, KeeperStorage::CreateNodeDelta{stat, std::move(node_acls), request.data}); + digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } @@ -2019,16 +2017,14 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro if (!new_subdeltas.empty()) { - if (auto * error = std::get_if(&new_deltas.back().operation); + if (auto * error = std::get_if(&new_subdeltas.back().operation); error && *operation_type == OperationType::Write) { - storage.uncommitted_state.rollback(zxid); + storage.uncommitted_state.rollback(std::move(new_deltas)); response_errors.push_back(error->error); for (size_t j = i + 1; j < concrete_requests.size(); ++j) - { response_errors.push_back(Coordination::Error::ZRUNTIMEINCONSISTENCY); - } return {KeeperStorage::Delta{zxid, KeeperStorage::FailedMultiDelta{std::move(response_errors)}}}; } @@ -2043,9 +2039,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro } digest = current_digest; - storage.uncommitted_state.addDeltas(std::move(new_deltas)); - return {}; } @@ -2188,19 +2182,12 @@ void KeeperStorage::finalize() finalized = true; - { - std::lock_guard lock(ephemerals_mutex); - ephemerals.clear(); - } + ephemerals.clear(); - { - std::lock_guard lock(watches_mutex); - watches.clear(); - list_watches.clear(); - sessions_and_watchers.clear(); - } + watches.clear(); + list_watches.clear(); + sessions_and_watchers.clear(); - std::lock_guard lock(session_mutex); session_expiry_queue.clear(); } @@ -2509,10 +2496,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( KeeperStorage::ResponsesForSessions results; /// ZooKeeper update sessions expirity for each request, not only for heartbeats - { - std::lock_guard lock(session_mutex); - session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); - } + session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { @@ -2520,7 +2504,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( { if (std::holds_alternative(delta.operation)) { - std::lock_guard lock(watches_mutex); auto responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); results.insert(results.end(), responses.begin(), responses.end()); } @@ -2540,11 +2523,8 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( auto response = std::make_shared(); response->xid = zk_request->xid; response->zxid = commit_zxid; - { - std::lock_guard lock(session_mutex); - session_expiry_queue.remove(session_id); - session_and_timeout.erase(session_id); - } + session_expiry_queue.remove(session_id); + session_and_timeout.erase(session_id); results.push_back(ResponseForSession{session_id, response}); } else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special @@ -2589,7 +2569,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// Watches for this requests are added to the watches lists if (zk_request->has_watch) { - std::lock_guard lock(watches_mutex); if (response->error == Coordination::Error::ZOK) { static constexpr std::array list_requests{ @@ -2614,7 +2593,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { - std::lock_guard lock(watches_mutex); auto watch_responses = request_processor->processWatches(watches, list_watches); results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } @@ -2631,6 +2609,8 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( std::lock_guard lock(transaction_mutex); if (new_last_zxid) uncommitted_transactions.pop_front(); + if (commit_zxid < zxid) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to commit smaller ZXID, commit ZXID: {}, current ZXID {}", commit_zxid, zxid); zxid = commit_zxid; } @@ -2639,7 +2619,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS { - std::unique_lock transaction_lock(transaction_mutex); if (allow_missing && (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid < rollback_zxid)) return; @@ -2655,7 +2634,6 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) T try { uncommitted_transactions.pop_back(); - transaction_lock.unlock(); uncommitted_state.rollback(rollback_zxid); } catch (...) @@ -2672,7 +2650,7 @@ KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed, bool lock_tr if (committed) { - std::lock_guard storage_lock(storage_mutex); + std::shared_lock storage_lock(storage_mutex); return {CURRENT_DIGEST_VERSION, nodes_digest}; } @@ -2684,7 +2662,7 @@ KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed, bool lock_tr { if (lock_transaction_mutex) transaction_lock.unlock(); - std::lock_guard storage_lock(storage_mutex); + std::shared_lock storage_lock(storage_mutex); return {CURRENT_DIGEST_VERSION, nodes_digest}; } @@ -2709,7 +2687,6 @@ void KeeperStorage::addDigest(const Node & node, const std::string_view path) /// Allocate new session id with the specified timeouts int64_t KeeperStorage::getSessionID(int64_t session_timeout_ms) { - std::lock_guard lock(session_mutex); auto result = session_id_counter++; session_and_timeout.emplace(result, session_timeout_ms); session_expiry_queue.addNewSessionOrUpdate(result, session_timeout_ms); @@ -2725,71 +2702,60 @@ void KeeperStorage::addSessionID(int64_t session_id, int64_t session_timeout_ms) std::vector KeeperStorage::getDeadSessions() const { - std::lock_guard lock(session_mutex); return session_expiry_queue.getExpiredSessions(); } SessionAndTimeout KeeperStorage::getActiveSessions() const { - std::lock_guard lock(session_mutex); return session_and_timeout; } /// Turn on snapshot mode, so data inside Container is not deleted, but replaced with new version. void KeeperStorage::enableSnapshotMode(size_t up_to_version) { - std::lock_guard lock(storage_mutex); container.enableSnapshotMode(up_to_version); } /// Turn off snapshot mode. void KeeperStorage::disableSnapshotMode() { - std::lock_guard lock(storage_mutex); container.disableSnapshotMode(); } KeeperStorage::Container::const_iterator KeeperStorage::getSnapshotIteratorBegin() const { - std::lock_guard lock(storage_mutex); return container.begin(); } /// Clear outdated data from internal container. void KeeperStorage::clearGarbageAfterSnapshot() { - std::lock_guard lock(storage_mutex); container.clearOutdatedNodes(); } /// Introspection functions mostly used in 4-letter commands uint64_t KeeperStorage::getNodesCount() const { - std::lock_guard lock(storage_mutex); return container.size(); } uint64_t KeeperStorage::getApproximateDataSize() const { - std::lock_guard lock(storage_mutex); return container.getApproximateDataSize(); } uint64_t KeeperStorage::getArenaDataSize() const { - std::lock_guard lock(storage_mutex); return container.keyArenaSize(); } uint64_t KeeperStorage::getWatchedPathsCount() const { - std::lock_guard lock(watches_mutex); return watches.size() + list_watches.size(); } void KeeperStorage::clearDeadWatches(int64_t session_id) { - std::lock_guard lock(watches_mutex); /// Clear all watches for this session auto watches_it = sessions_and_watchers.find(session_id); if (watches_it != sessions_and_watchers.end()) @@ -2823,7 +2789,6 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const { - std::lock_guard lock(watches_mutex); for (const auto & [session_id, watches_paths] : sessions_and_watchers) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; @@ -2842,7 +2807,6 @@ void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const } }; - std::lock_guard lock(watches_mutex); for (const auto & [watch_path, sessions] : watches) { buf << watch_path << "\n"; @@ -2866,17 +2830,13 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co } }; - { - std::lock_guard lock(session_mutex); - buf << "Sessions dump (" << session_and_timeout.size() << "):\n"; + buf << "Sessions dump (" << session_and_timeout.size() << "):\n"; - for (const auto & [session_id, _] : session_and_timeout) - { - buf << "0x" << getHexUIntLowercase(session_id) << "\n"; - } + for (const auto & [session_id, _] : session_and_timeout) + { + buf << "0x" << getHexUIntLowercase(session_id) << "\n"; } - std::lock_guard ephemerals_lock(ephemerals_mutex); buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCountLocked() << "):\n"; for (const auto & [session_id, ephemeral_paths] : ephemerals) { @@ -2904,7 +2864,6 @@ const KeeperStorage::Stats & KeeperStorage::getStorageStats() const uint64_t KeeperStorage::getTotalWatchesCount() const { - std::lock_guard lock(watches_mutex); uint64_t ret = 0; for (const auto & [session, paths] : sessions_and_watchers) ret += paths.size(); @@ -2914,7 +2873,6 @@ uint64_t KeeperStorage::getTotalWatchesCount() const uint64_t KeeperStorage::getSessionWithEphemeralNodesCount() const { - std::lock_guard ephemerals_lock(ephemerals_mutex); return getSessionWithEphemeralNodesCountLocked(); } @@ -2925,13 +2883,11 @@ uint64_t KeeperStorage::getSessionWithEphemeralNodesCountLocked() const uint64_t KeeperStorage::getSessionsWithWatchesCount() const { - std::lock_guard lock(watches_mutex); return sessions_and_watchers.size(); } uint64_t KeeperStorage::getTotalEphemeralNodesCount() const { - std::lock_guard ephemerals_lock(ephemerals_mutex); uint64_t ret = 0; for (const auto & [session_id, nodes] : ephemerals) ret += nodes.size(); @@ -2941,7 +2897,6 @@ uint64_t KeeperStorage::getTotalEphemeralNodesCount() const void KeeperStorage::recalculateStats() { - std::lock_guard lock(storage_mutex); container.recalculateDataSize(); } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 0e773fe68ef..d72ae46dee2 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -370,6 +370,7 @@ public: void addDeltas(std::list new_deltas); void cleanup(int64_t commit_zxid); void rollback(int64_t rollback_zxid); + void rollback(std::list rollback_deltas); std::shared_ptr getNode(StringRef path) const; Coordination::ACLs getACLs(StringRef path) const; @@ -451,14 +452,13 @@ public: mutable std::mutex ephemerals_mutex; /// Mapping session_id -> set of ephemeral nodes paths - Ephemerals ephemerals TSA_GUARDED_BY(ephemerals_mutex); + Ephemerals ephemerals; - mutable std::mutex session_mutex; - int64_t session_id_counter TSA_GUARDED_BY(session_mutex) = 1; + int64_t session_id_counter = 1; /// Expiration queue for session, allows to get dead sessions at some point of time - SessionExpiryQueue session_expiry_queue TSA_GUARDED_BY(session_mutex); + SessionExpiryQueue session_expiry_queue; /// All active sessions with timeout - SessionAndTimeout session_and_timeout TSA_GUARDED_BY(session_mutex); + SessionAndTimeout session_and_timeout; /// ACLMap for more compact ACLs storage inside nodes. ACLMap acl_map; @@ -490,13 +490,12 @@ public: std::atomic finalized{false}; - mutable std::mutex watches_mutex; /// Mapping session_id -> set of watched nodes paths - SessionAndWatcher sessions_and_watchers TSA_GUARDED_BY(watches_mutex); + SessionAndWatcher sessions_and_watchers; /// Currently active watches (node_path -> subscribed sessions) - Watches watches TSA_GUARDED_BY(watches_mutex); - Watches list_watches TSA_GUARDED_BY(watches_mutex); /// Watches for 'list' request (watches on children). + Watches watches; + Watches list_watches; /// Watches for 'list' request (watches on children). void clearDeadWatches(int64_t session_id); @@ -606,7 +605,7 @@ public: void recalculateStats(); private: - uint64_t getSessionWithEphemeralNodesCountLocked() const TSA_REQUIRES(ephemerals_mutex); + uint64_t getSessionWithEphemeralNodesCountLocked() const; void removeDigest(const Node & node, std::string_view path); void addDigest(const Node & node, std::string_view path); From 7dca59da5644a76b48324b841a4ce6d8a0b1f559 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 17 Jun 2024 09:03:49 +0200 Subject: [PATCH 017/128] Revert "Merge branch 'use-thread-from-global-pool-in-poco-threadpool' into keeper-some-improvement" This reverts commit 737d7484c5b3b737eb1ca1033c489c927bcba1cc, reversing changes made to b3a742304eb8981a52569fce56208b33efe94b11. --- base/poco/Foundation/CMakeLists.txt | 3 - base/poco/Foundation/src/ThreadPool.cpp | 79 +++++++------------------ programs/server/Server.cpp | 27 ++++----- src/Server/HTTPHandler.cpp | 1 + src/Server/InterserverIOHTTPHandler.cpp | 2 + src/Server/KeeperTCPHandler.cpp | 1 + src/Server/MySQLHandler.cpp | 2 + src/Server/PostgreSQLHandler.cpp | 2 + src/Server/TCPHandler.cpp | 1 + 9 files changed, 45 insertions(+), 73 deletions(-) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index fe59e72569c..dfb41a33fb1 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -168,9 +168,6 @@ set (SRCS add_library (_poco_foundation ${SRCS}) add_library (Poco::Foundation ALIAS _poco_foundation) -target_link_libraries (_poco_foundation PRIVATE clickhouse_common_io) -target_compile_definitions(_poco_foundation PUBLIC USE_CLICKHOUSE_THREADS=1) - # TODO: remove these warning exclusions target_compile_options (_poco_foundation PRIVATE diff --git a/base/poco/Foundation/src/ThreadPool.cpp b/base/poco/Foundation/src/ThreadPool.cpp index a13e91d8b76..6335ee82b47 100644 --- a/base/poco/Foundation/src/ThreadPool.cpp +++ b/base/poco/Foundation/src/ThreadPool.cpp @@ -20,15 +20,12 @@ #include "Poco/ErrorHandler.h" #include #include -#if USE_CLICKHOUSE_THREADS -#include -#endif namespace Poco { -class PooledThread : public Runnable +class PooledThread: public Runnable { public: PooledThread(const std::string& name, int stackSize = POCO_THREAD_STACK_SIZE); @@ -49,11 +46,7 @@ private: volatile std::time_t _idleTime; Runnable* _pTarget; std::string _name; -#if USE_CLICKHOUSE_THREADS - ThreadFromGlobalPool _thread; -#else Thread _thread; -#endif Event _targetReady; Event _targetCompleted; Event _started; @@ -61,20 +54,16 @@ private: }; -PooledThread::PooledThread(const std::string& name, [[maybe_unused]] int stackSize): - _idle(true), - _idleTime(0), - _pTarget(0), - _name(name), -#if !USE_CLICKHOUSE_THREADS +PooledThread::PooledThread(const std::string& name, int stackSize): + _idle(true), + _idleTime(0), + _pTarget(0), + _name(name), _thread(name), -#endif _targetCompleted(false) { poco_assert_dbg (stackSize >= 0); -#if !USE_CLICKHOUSE_THREADS _thread.setStackSize(stackSize); -#endif _idleTime = std::time(NULL); } @@ -86,32 +75,24 @@ PooledThread::~PooledThread() void PooledThread::start() { -#if USE_CLICKHOUSE_THREADS - _thread = ThreadFromGlobalPool([this]() { run(); }); -#else _thread.start(*this); -#endif _started.wait(); } -void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& target) +void PooledThread::start(Thread::Priority priority, Runnable& target) { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_pTarget == 0); _pTarget = ⌖ - -#if !USE_CLICKHOUSE_THREADS _thread.setPriority(priority); -#endif - _targetReady.set(); } -void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& target, const std::string& name) +void PooledThread::start(Thread::Priority priority, Runnable& target, const std::string& name) { FastMutex::ScopedLock lock(_mutex); @@ -126,12 +107,9 @@ void PooledThread::start([[maybe_unused]] Thread::Priority priority, Runnable& t fullName.append(_name); fullName.append(")"); } - -#if !USE_CLICKHOUSE_THREADS _thread.setName(fullName); _thread.setPriority(priority); -#endif - + poco_assert (_pTarget == 0); _pTarget = ⌖ @@ -167,7 +145,7 @@ void PooledThread::join() void PooledThread::activate() { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_idle); _idle = false; _targetCompleted.reset(); @@ -176,30 +154,21 @@ void PooledThread::activate() void PooledThread::release() { + const long JOIN_TIMEOUT = 10000; + _mutex.lock(); _pTarget = 0; _mutex.unlock(); - // In case of a statically allocated thread pool (such // as the default thread pool), Windows may have already // terminated the thread before we got here. -#if USE_CLICKHOUSE_THREADS - if (_thread.joinable()) -#else if (_thread.isRunning()) -#endif _targetReady.set(); -#if USE_CLICKHOUSE_THREADS - if (_thread.joinable()) - _thread.join(); -#else - const long JOIN_TIMEOUT = 10000; if (_thread.tryJoin(JOIN_TIMEOUT)) { delete this; } -#endif } @@ -236,10 +205,8 @@ void PooledThread::run() _idle = true; _targetCompleted.set(); ThreadLocalStorage::clear(); -#if !USE_CLICKHOUSE_THREADS _thread.setName(_name); _thread.setPriority(Thread::PRIO_NORMAL); -#endif } else { @@ -253,9 +220,9 @@ void PooledThread::run() ThreadPool::ThreadPool(int minCapacity, int maxCapacity, int idleTime, - int stackSize): - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + int stackSize): + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), @@ -278,8 +245,8 @@ ThreadPool::ThreadPool(const std::string& name, int idleTime, int stackSize): _name(name), - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), @@ -426,15 +393,15 @@ void ThreadPool::housekeep() ThreadVec activeThreads; idleThreads.reserve(_threads.size()); activeThreads.reserve(_threads.size()); - + for (ThreadVec::iterator it = _threads.begin(); it != _threads.end(); ++it) { if ((*it)->idle()) { if ((*it)->idleTime() < _idleTime) idleThreads.push_back(*it); - else - expiredThreads.push_back(*it); + else + expiredThreads.push_back(*it); } else activeThreads.push_back(*it); } @@ -514,7 +481,7 @@ public: ThreadPool* pool() { FastMutex::ScopedLock lock(_mutex); - + if (!_pPool) { _pPool = new ThreadPool("default"); @@ -523,7 +490,7 @@ public: } return _pPool; } - + private: ThreadPool* _pPool; FastMutex _mutex; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 802c994e872..84ff0e94cef 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -721,6 +721,11 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); + Poco::ThreadPool server_pool(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -818,11 +823,6 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } - auto server_pool = std::make_unique(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; @@ -898,8 +898,7 @@ try global_context->shutdownKeeperDispatcher(); /// Wait server pool to avoid use-after-free of destroyed context in the handlers - server_pool->joinAll(); - server_pool.reset(); + server_pool.joinAll(); /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. * At this moment, no one could own shared part of Context. @@ -1630,7 +1629,7 @@ try if (global_context->isServerCompletelyStarted()) { std::lock_guard lock(servers_lock); - updateServers(*config, *server_pool, async_metrics, servers, servers_to_start_before_tables); + updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); } } @@ -1727,7 +1726,7 @@ try config_getter, global_context->getKeeperDispatcher(), global_context->getSettingsRef().receive_timeout.totalSeconds(), global_context->getSettingsRef().send_timeout.totalSeconds(), - false), *server_pool, socket)); + false), server_pool, socket)); }); const char * secure_port_name = "keeper_server.tcp_port_secure"; @@ -1749,7 +1748,7 @@ try new KeeperTCPHandlerFactory( config_getter, global_context->getKeeperDispatcher(), global_context->getSettingsRef().receive_timeout.totalSeconds(), - global_context->getSettingsRef().send_timeout.totalSeconds(), true), *server_pool, socket)); + global_context->getSettingsRef().send_timeout.totalSeconds(), true), server_pool, socket)); #else UNUSED(port); throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); @@ -1781,7 +1780,7 @@ try createKeeperHTTPControlMainHandlerFactory( config_getter(), global_context->getKeeperDispatcher(), - "KeeperHTTPControlHandler-factory"), *server_pool, socket, http_params)); + "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params)); }); } #else @@ -1805,7 +1804,7 @@ try config(), interserver_listen_hosts, listen_try, - *server_pool, + server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ false); @@ -1856,7 +1855,7 @@ try config(), listen_hosts, listen_try, - *server_pool, + server_pool, async_metrics, servers, /* start_servers= */ true, @@ -2028,7 +2027,7 @@ try { std::lock_guard lock(servers_lock); - createServers(config(), listen_hosts, listen_try, *server_pool, async_metrics, servers); + createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); if (servers.empty()) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index d6afa571e71..02d0959ff50 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1060,6 +1060,7 @@ void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("HTTPHandler"); + ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::HTTP, request.isSecure()); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 45c28babe3a..0d79aaa227b 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -80,6 +81,7 @@ void InterserverIOHTTPHandler::processQuery(HTTPServerRequest & request, HTTPSer void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("IntersrvHandler"); + ThreadStatus thread_status; /// In order to work keep-alive. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 4612e2e9fa8..6709cd298e5 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -309,6 +309,7 @@ Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length, bool void KeeperTCPHandler::runImpl() { setThreadName("KeeperHandler"); + ThreadStatus thread_status; socket().setReceiveTimeout(receive_timeout); socket().setSendTimeout(send_timeout); diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 9471509ad4b..6456f6d24ff 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -198,6 +199,7 @@ MySQLHandler::~MySQLHandler() = default; void MySQLHandler::run() { setThreadName("MySQLHandler"); + ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::MYSQL); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 8ba8421e6f0..473d681ddb2 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -58,6 +59,7 @@ void PostgreSQLHandler::changeIO(Poco::Net::StreamSocket & socket) void PostgreSQLHandler::run() { setThreadName("PostgresHandler"); + ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::POSTGRESQL); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b60339e9fd8..e3a820340ad 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -246,6 +246,7 @@ TCPHandler::~TCPHandler() void TCPHandler::runImpl() { setThreadName("TCPHandler"); + ThreadStatus thread_status; extractConnectionSettingsFromContext(server.context()); From b6c361954319957533ee5171a51742001b803ec8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Aug 2024 14:51:06 +0200 Subject: [PATCH 018/128] Whitespace --- src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStorage.cpp | 45 ++++++++++++++------------- src/Coordination/KeeperStorage.h | 2 +- 3 files changed, 25 insertions(+), 24 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index dd6e19a6118..8de14ba301b 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -223,7 +223,7 @@ public: uint64_t getLastProcessedZxid() const override; const KeeperStorageBase::Stats & getStorageStats() const override; - + uint64_t getNodesCount() const override; uint64_t getTotalWatchesCount() const override; uint64_t getWatchedPathsCount() const override; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 579bcd877b4..9361ce9a4fc 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -972,7 +972,8 @@ Coordination::Error KeeperStorage::commit(std::list deltas) if constexpr (!use_rocksdb) removeDigest(node_it->value, path); - auto updated_node = container.updateValue(path, [&](auto & node) { + auto updated_node = container.updateValue(path, [&](auto & node) + { if constexpr (std::same_as) node.stats = operation.new_stats; else @@ -1376,7 +1377,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); @@ -1438,7 +1439,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); @@ -1482,12 +1483,12 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); return processImpl(storage, std::move(deltas)); @@ -1578,7 +1579,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); @@ -1612,7 +1613,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); @@ -1645,12 +1646,12 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); return processImpl(storage, std::move(deltas)); @@ -1666,7 +1667,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce } using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - + std::list preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { @@ -1714,7 +1715,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { auto & container = storage.container; @@ -1768,7 +1769,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); @@ -1857,12 +1858,12 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); return processImpl(storage, std::move(deltas)); @@ -1910,7 +1911,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); @@ -1956,12 +1957,12 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); return processImpl(storage, std::move(deltas)); @@ -2022,7 +2023,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); @@ -2066,7 +2067,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr } template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); @@ -2099,12 +2100,12 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override { return processImpl(storage, std::move(deltas)); } @@ -2348,7 +2349,7 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override { Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 97f790abe3d..6326e1c76a9 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -360,7 +360,7 @@ public: struct UpdateNodeDataDelta { - + std::string old_data; std::string new_data; int32_t version{-1}; From c61fc591c42c80882e09e432dc51349686698f7d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 13 Aug 2024 11:33:17 +0200 Subject: [PATCH 019/128] Use functions instead of classes --- src/Coordination/KeeperStorage.cpp | 2300 ++++++++++++++-------------- 1 file changed, 1149 insertions(+), 1151 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 9361ce9a4fc..80b917deb2f 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1127,65 +1127,54 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers return true; } -template -struct KeeperStorageRequestProcessor +template +auto callOnConcreteRequestType(const Coordination::ZooKeeperRequest & zk_request, F function) { - Coordination::ZooKeeperRequestPtr zk_request; - - explicit KeeperStorageRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) : zk_request(zk_request_) { } - virtual Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const = 0; - virtual std::list - preprocess(Storage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const + switch (zk_request.getOpNum()) { - return {}; + case Coordination::OpNum::Heartbeat: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Sync: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Get: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Create: + case Coordination::OpNum::CreateIfNotExists: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Remove: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Exists: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Set: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::List: + case Coordination::OpNum::FilteredList: + case Coordination::OpNum::SimpleList: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Check: + case Coordination::OpNum::CheckNotExists: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Multi: + case Coordination::OpNum::MultiRead: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Auth: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::Close: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::SetACL: + return function(dynamic_cast(zk_request)); + case Coordination::OpNum::GetACL: + return function(dynamic_cast(zk_request)); + default: + throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Unexpected request type: {}", zk_request.getOpNum()}; } +} - // process the request using locally committed data - virtual Coordination::ZooKeeperResponsePtr - processLocal(Storage & /*storage*/, std::list /* deltas */) const - { - throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Cannot process the request locally"}; - } - - virtual KeeperStorageBase::ResponsesForSessions - processWatches(KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) const - { - return {}; - } - - virtual bool checkAuth(Storage & /*storage*/, int64_t /*session_id*/, bool /*is_local*/) const { return true; } - - virtual ~KeeperStorageRequestProcessor() = default; -}; - -template -struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageRequestProcessor +template +auto callOnConcreteRequestType(const Coordination::Request & request, F function) { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - Coordination::ZooKeeperResponsePtr - process(Storage & storage, std::list deltas) const override - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - response_ptr->error = storage.commit(std::move(deltas)); - return response_ptr; - } -}; - -template -struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - Coordination::ZooKeeperResponsePtr - process(Storage & /* storage */, std::list /* deltas */) const override - { - auto response = this->zk_request->makeResponse(); - dynamic_cast(*response).path - = dynamic_cast(*this->zk_request).path; - return response; - } -}; + return callOnConcreteRequestType(dynamic_cast(request), function); +} namespace { @@ -1261,1105 +1250,1174 @@ void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const ephemerals.erase(ephemerals_it); } -template -struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor +/// Default implementations /// +template T, typename Storage> +Coordination::ZooKeeperResponsePtr +processLocal(const T & zk_request, Storage & /*storage*/, std::list /*deltas*/) { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Local processing not supported for request with type {}", zk_request.getOpNum()); +} - KeeperStorageBase::ResponsesForSessions - processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override +template T, typename Storage> +std::list preprocess( + const T & /*zk_request*/, + Storage & /*storage*/, + int64_t /*zxid*/, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + return {}; +} + +template T> +KeeperStorageBase::ResponsesForSessions +processWatches(const T & /*zk_request*/, KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) +{ + return {}; +} + +template T, typename Storage> +bool checkAuth(const T & /*zk_request*/, Storage & /*storage*/, int64_t /*session_id*/, bool /*is_local*/) +{ + return true; +} +/// Default implementations /// + +/// HEARTBEAT Request /// + +template +Coordination::ZooKeeperResponsePtr process( + const Coordination::ZooKeeperHeartbeatRequest & zk_request, + Storage & storage, + std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + response_ptr->error = storage.commit(std::move(deltas)); + return response_ptr; +} + +/// HEARTBEAT Request /// + +/// SYNC Request /// + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* storage */, std::list /* deltas */) +{ + auto response = zk_request.makeResponse(); + dynamic_cast(*response).path = zk_request.path; + return response; +} + +/// SYNC Request /// + +/// CREATE Request /// +template +bool checkAuth(const Coordination::ZooKeeperCreateRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + auto path = zk_request.getPath(); + return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); +} + +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperCreateRequest & zk_request, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches) +{ + return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CREATED); +} + +template +std::list preprocess( + const Coordination::ZooKeeperCreateRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t session_id, + int64_t time, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + ProfileEvents::increment(ProfileEvents::KeeperCreateRequest); + + std::list new_deltas; + + auto parent_path = parentNodePath(zk_request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + if (parent_node == nullptr) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + else if (parent_node->stats.isEphemeral()) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}}; + + std::string path_created = zk_request.path; + if (zk_request.is_sequential) { - return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED); - } - - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override - { - auto path = this->zk_request->getPath(); - return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); - } - - std::list - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override - { - ProfileEvents::increment(ProfileEvents::KeeperCreateRequest); - Coordination::ZooKeeperCreateRequest & request = dynamic_cast(*this->zk_request); - - std::list new_deltas; - - auto parent_path = parentNodePath(request.path); - auto parent_node = storage.uncommitted_state.getNode(parent_path); - if (parent_node == nullptr) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - else if (parent_node->stats.isEphemeral()) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}}; - - std::string path_created = request.path; - if (request.is_sequential) - { - if (request.not_exists) - return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - - auto seq_num = parent_node->stats.seqNum(); - - std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - seq_num_str.exceptions(std::ios::failbit); - seq_num_str << std::setw(10) << std::setfill('0') << seq_num; - - path_created += seq_num_str.str(); - } - - if (Coordination::matchPath(path_created, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) - { - auto error_msg = fmt::format("Trying to create a node inside the internal Keeper path ({}) which is not allowed. Path: {}", keeper_system_path, path_created); - - handleSystemNodeModification(keeper_context, error_msg); - return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - } - - if (storage.uncommitted_state.getNode(path_created)) - { - if (this->zk_request->getOpNum() == Coordination::OpNum::CreateIfNotExists) - return new_deltas; - - return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; - } - - if (getBaseNodeName(path_created).size == 0) + if (zk_request.not_exists) return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - Coordination::ACLs node_acls; - if (!fixupACL(request.acls, session_id, storage.uncommitted_state, node_acls)) - return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + auto seq_num = parent_node->stats.seqNum(); - if (request.is_ephemeral) - { - std::lock_guard lock(storage.ephemerals_mutex); - storage.ephemerals[session_id].emplace(path_created); - } + std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + seq_num_str.exceptions(std::ios::failbit); + seq_num_str << std::setw(10) << std::setfill('0') << seq_num; - int32_t parent_cversion = request.parent_cversion; - - KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); - update_parent_delta.new_stats.increaseSeqNum(); - - if (parent_cversion == -1) - ++update_parent_delta.new_stats.cversion; - else if (parent_cversion > update_parent_delta.old_stats.cversion) - update_parent_delta.new_stats.cversion = parent_cversion; - - if (zxid > update_parent_delta.old_stats.pzxid) - update_parent_delta.new_stats.pzxid = zxid; - - update_parent_delta.new_stats.increaseNumChildren(); - - new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(update_parent_delta)); - - Coordination::Stat stat; - stat.czxid = zxid; - stat.mzxid = zxid; - stat.pzxid = zxid; - stat.ctime = time; - stat.mtime = time; - stat.numChildren = 0; - stat.version = 0; - stat.aversion = 0; - stat.cversion = 0; - stat.ephemeralOwner = request.is_ephemeral ? session_id : 0; - - new_deltas.emplace_back( - std::move(path_created), - zxid, - typename Storage::CreateNodeDelta{stat, std::move(node_acls), request.data}); - - - digest = storage.calculateNodesDigest(digest, new_deltas); - return new_deltas; + path_created += seq_num_str.str(); } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + if (Coordination::matchPath(path_created, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); + auto error_msg = fmt::format("Trying to create a node inside the internal Keeper path ({}) which is not allowed. Path: {}", keeper_system_path, path_created); - if (deltas.empty()) - { - response.path_created = this->zk_request->getPath(); - response.error = Coordination::Error::ZOK; - return response_ptr; - } + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + } - std::string created_path; - auto create_delta_it = std::find_if( - deltas.begin(), - deltas.end(), - [](const auto & delta) - { return std::holds_alternative(delta.operation); }); + if (storage.uncommitted_state.getNode(path_created)) + { + if (zk_request.getOpNum() == Coordination::OpNum::CreateIfNotExists) + return new_deltas; - if (create_delta_it != deltas.end()) - created_path = create_delta_it->path; + return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; + } - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } + if (getBaseNodeName(path_created).size == 0) + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - response.path_created = std::move(created_path); + Coordination::ACLs node_acls; + if (!fixupACL(zk_request.acls, session_id, storage.uncommitted_state, node_acls)) + return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + + if (zk_request.is_ephemeral) + { + std::lock_guard lock(storage.ephemerals_mutex); + storage.ephemerals[session_id].emplace(path_created); + } + + int32_t parent_cversion = zk_request.parent_cversion; + + KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); + update_parent_delta.new_stats.increaseSeqNum(); + + if (parent_cversion == -1) + ++update_parent_delta.new_stats.cversion; + else if (parent_cversion > update_parent_delta.old_stats.cversion) + update_parent_delta.new_stats.cversion = parent_cversion; + + if (zxid > update_parent_delta.old_stats.pzxid) + update_parent_delta.new_stats.pzxid = zxid; + + update_parent_delta.new_stats.increaseNumChildren(); + + new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(update_parent_delta)); + + Coordination::Stat stat; + stat.czxid = zxid; + stat.mzxid = zxid; + stat.pzxid = zxid; + stat.ctime = time; + stat.mtime = time; + stat.numChildren = 0; + stat.version = 0; + stat.aversion = 0; + stat.cversion = 0; + stat.ephemeralOwner = zk_request.is_ephemeral ? session_id : 0; + + new_deltas.emplace_back( + std::move(path_created), + zxid, + typename Storage::CreateNodeDelta{stat, std::move(node_acls), zk_request.data}); + + + digest = storage.calculateNodesDigest(digest, new_deltas); + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperCreateRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); + + if (deltas.empty()) + { + response.path_created = zk_request.getPath(); response.error = Coordination::Error::ZOK; return response_ptr; } -}; -template -struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor -{ - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + std::string created_path; + auto create_delta_it = std::find_if( + deltas.begin(), + deltas.end(), + [](const auto & delta) + { return std::holds_alternative(delta.operation); }); + + if (create_delta_it != deltas.end()) + created_path = create_delta_it->path; + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); + response.error = result; + return response_ptr; } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + response.path_created = std::move(created_path); + response.error = Coordination::Error::ZOK; + return response_ptr; +} +/// CREATE Request /// - std::list - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override - { - ProfileEvents::increment(ProfileEvents::KeeperGetRequest); - Coordination::ZooKeeperGetRequest & request = dynamic_cast(*this->zk_request); +/// GET Request /// +template +bool checkAuth(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Read, session_id, is_local); +} - if (request.path == Coordination::keeper_api_feature_flags_path - || request.path == Coordination::keeper_config_path - || request.path == Coordination::keeper_availability_zone_path) - return {}; - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; +template +std::list preprocess( + const Coordination::ZooKeeperGetRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + ProfileEvents::increment(ProfileEvents::KeeperGetRequest); + if (zk_request.path == Coordination::keeper_api_feature_flags_path + || zk_request.path == Coordination::keeper_config_path + || zk_request.path == Coordination::keeper_availability_zone_path) return {}; - } - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + return {}; +} + +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperGetRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - } - - if (request.path == Coordination::keeper_config_path) - { - response.data = serializeClusterConfig( - storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); - response.error = Coordination::Error::ZOK; - return response_ptr; - } - - auto & container = storage.container; - auto node_it = container.find(request.path); - if (node_it == container.end()) - { - if constexpr (local) - response.error = Coordination::Error::ZNONODE; - else - onStorageInconsistency(); - } - else - { - node_it->value.setResponseStat(response.stat); - auto data = node_it->value.getData(); - response.data = std::string(data); - response.error = Coordination::Error::ZOK; - } - - return response_ptr; - } - - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - ProfileEvents::increment(ProfileEvents::KeeperGetRequest); - return processImpl(storage, std::move(deltas)); - } -}; - -template -struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor -{ - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override - { - return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); - } - - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::list - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override - { - ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); - Coordination::ZooKeeperRemoveRequest & request = dynamic_cast(*this->zk_request); - - std::list new_deltas; - - if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) - { - auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path); - - handleSystemNodeModification(keeper_context, error_msg); - return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - } - - auto parent_path = parentNodePath(request.path); - auto parent_node = storage.uncommitted_state.getNode(parent_path); - - KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); - - const auto add_parent_update_delta = [&] - { - new_deltas.emplace_back( - std::string{parent_path}, - zxid, - std::move(update_parent_delta) - ); - }; - - const auto update_parent_pzxid = [&]() - { - if (!parent_node) - return; - - if (update_parent_delta.old_stats.pzxid < zxid) - update_parent_delta.new_stats.pzxid = zxid; - }; - - auto node = storage.uncommitted_state.getNode(request.path); - - if (!node) - { - if (request.restored_from_zookeeper_log) - { - update_parent_pzxid(); - add_parent_update_delta(); - } - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNONODE}}; - } - else if (request.version != -1 && request.version != node->stats.version) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; - else if (node->stats.numChildren() != 0) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; - - if (request.restored_from_zookeeper_log) - update_parent_pzxid(); - - ++update_parent_delta.new_stats.cversion; - update_parent_delta.new_stats.decreaseNumChildren(); - add_parent_update_delta(); - - new_deltas.emplace_back( - request.path, - zxid, - KeeperStorageBase::RemoveNodeDelta{request.version, node->stats, storage.uncommitted_state.getACLs(request.path), std::string{node->getData()}}); - - if (node->stats.isEphemeral()) - storage.unregisterEphemeralPath(node->stats.ephemeralOwner(), request.path); - - digest = storage.calculateNodesDigest(digest, new_deltas); - - return new_deltas; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); - - response.error = storage.commit(std::move(deltas)); - return response_ptr; - } - - KeeperStorageBase::ResponsesForSessions - processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override - { - return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); - } -}; - -template -struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::list - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override - { - ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); - Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*this->zk_request); - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - } - - auto & container = storage.container; - auto node_it = container.find(request.path); - if (node_it == container.end()) - { - if constexpr (local) - response.error = Coordination::Error::ZNONODE; - else - onStorageInconsistency(); - } - else - { - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; - } - - return response_ptr; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); - return processImpl(storage, std::move(deltas)); - } -}; - -template -struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProcessor -{ - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override - { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Write, session_id, is_local); - } - - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::list - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override - { - ProfileEvents::increment(ProfileEvents::KeeperSetRequest); - Coordination::ZooKeeperSetRequest & request = dynamic_cast(*this->zk_request); - - std::list new_deltas; - - if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) - { - auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path); - - handleSystemNodeModification(keeper_context, error_msg); - return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - } - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - auto node = storage.uncommitted_state.getNode(request.path); - - if (request.version != -1 && request.version != node->stats.version) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; - - KeeperStorageBase::UpdateNodeStatDelta node_delta(*node); - node_delta.version = request.version; - auto & new_stats = node_delta.new_stats; - new_stats.version++; - new_stats.mzxid = zxid; - new_stats.mtime = time; - - new_deltas.emplace_back(request.path, zxid, std::move(node_delta)); - new_deltas.emplace_back( - request.path, - zxid, - KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = request.data, .version = request.version}); - - auto parent_path = parentNodePath(request.path); - auto parent_node = storage.uncommitted_state.getNode(parent_path); - KeeperStorageBase::UpdateNodeStatDelta parent_delta(*parent_node); - ++parent_delta.new_stats.cversion; - new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(parent_delta)); - - digest = storage.calculateNodesDigest(digest, new_deltas); - return new_deltas; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - auto & container = storage.container; - - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperSetRequest & request = dynamic_cast(*this->zk_request); - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; } + } - auto node_it = container.find(request.path); - if (node_it == container.end()) + if (zk_request.path == Coordination::keeper_config_path) + { + response.data = serializeClusterConfig( + storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); + response.error = Coordination::Error::ZOK; + return response_ptr; + } + + auto & container = storage.container; + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) + { + if constexpr (local) + response.error = Coordination::Error::ZNONODE; + else onStorageInconsistency(); + } + else + { + node_it->value.setResponseStat(response.stat); + auto data = node_it->value.getData(); + response.data = std::string(data); + response.error = Coordination::Error::ZOK; + } + return response_ptr; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) +{ + ProfileEvents::increment(ProfileEvents::KeeperGetRequest); + return processImpl(zk_request, storage, std::move(deltas)); +} +/// GET Request /// + +/// REMOVE Request /// +template +bool checkAuth(const Coordination::ZooKeeperRemoveRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(parentNodePath(zk_request.getPath()), Coordination::ACL::Delete, session_id, is_local); +} + +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperRemoveRequest & zk_request, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches) +{ + return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CREATED); +} + +template +std::list preprocess( + const Coordination::ZooKeeperRemoveRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /* session_id */, + int64_t /* time */, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); + + std::list new_deltas; + + if (Coordination::matchPath(zk_request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) + { + auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", zk_request.path); + + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + } + + auto parent_path = parentNodePath(zk_request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + + KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); + + const auto add_parent_update_delta = [&] + { + new_deltas.emplace_back( + std::string{parent_path}, + zxid, + std::move(update_parent_delta) + ); + }; + + const auto update_parent_pzxid = [&]() + { + if (!parent_node) + return; + + if (update_parent_delta.old_stats.pzxid < zxid) + update_parent_delta.new_stats.pzxid = zxid; + }; + + auto node = storage.uncommitted_state.getNode(zk_request.path); + + if (!node) + { + if (zk_request.restored_from_zookeeper_log) + { + update_parent_pzxid(); + add_parent_update_delta(); + } + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNONODE}}; + } + else if (zk_request.version != -1 && zk_request.version != node->stats.version) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; + else if (node->stats.numChildren() != 0) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; + + if (zk_request.restored_from_zookeeper_log) + update_parent_pzxid(); + + ++update_parent_delta.new_stats.cversion; + update_parent_delta.new_stats.decreaseNumChildren(); + add_parent_update_delta(); + + new_deltas.emplace_back( + zk_request.path, + zxid, + KeeperStorageBase::RemoveNodeDelta{ + zk_request.version, node->stats, storage.uncommitted_state.getACLs(zk_request.path), std::string{node->getData()}}); + + if (node->stats.isEphemeral()) + storage.unregisterEphemeralPath(node->stats.ephemeralOwner(), zk_request.path); + + digest = storage.calculateNodesDigest(digest, new_deltas); + + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperRemoveRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); + + response.error = storage.commit(std::move(deltas)); + return response_ptr; +} + +/// REMOVE Request /// + +/// EXISTS Request /// +template +bool checkAuth(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Read, session_id, is_local); +} + +template +std::list preprocess( + const Coordination::ZooKeeperExistsRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); + + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + return {}; +} + +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) { + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) + { + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + { + response.error = result; + return response_ptr; + } + } + + auto & container = storage.container; + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) + { + if constexpr (local) + response.error = Coordination::Error::ZNONODE; + else + onStorageInconsistency(); + } + else + { node_it->value.setResponseStat(response.stat); response.error = Coordination::Error::ZOK; + } + return response_ptr; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) +{ + ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); + return processImpl(zk_request, storage, std::move(deltas)); +} +/// EXISTS Request /// + +/// SET Request /// +template +bool checkAuth(const Coordination::ZooKeeperSetRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Write, session_id, is_local); +} + +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperSetRequest & zk_request, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) +{ + return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CHANGED); +} + +template +std::list preprocess( + const Coordination::ZooKeeperSetRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /* session_id */, + int64_t time, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + ProfileEvents::increment(ProfileEvents::KeeperSetRequest); + + std::list new_deltas; + + if (Coordination::matchPath(zk_request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) + { + auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", zk_request.path); + + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + } + + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + auto node = storage.uncommitted_state.getNode(zk_request.path); + + if (zk_request.version != -1 && zk_request.version != node->stats.version) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; + + KeeperStorageBase::UpdateNodeStatDelta node_delta(*node); + node_delta.version = zk_request.version; + auto & new_stats = node_delta.new_stats; + new_stats.version++; + new_stats.mzxid = zxid; + new_stats.mtime = time; + + new_deltas.emplace_back(zk_request.path, zxid, std::move(node_delta)); + new_deltas.emplace_back( + zk_request.path, + zxid, + KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = zk_request.data, .version = zk_request.version}); + + auto parent_path = parentNodePath(zk_request.path); + auto parent_node = storage.uncommitted_state.getNode(parent_path); + KeeperStorageBase::UpdateNodeStatDelta parent_delta(*parent_node); + ++parent_delta.new_stats.cversion; + new_deltas.emplace_back(std::string{parent_path}, zxid, std::move(parent_delta)); + + digest = storage.calculateNodesDigest(digest, new_deltas); + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSetRequest & zk_request, Storage & storage, std::list deltas) +{ + auto & container = storage.container; + + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + { + response.error = result; return response_ptr; } - KeeperStorageBase::ResponsesForSessions - processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override - { - return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); - } -}; + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) + onStorageInconsistency(); -template -struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProcessor + node_it->value.setResponseStat(response.stat); + response.error = Coordination::Error::ZOK; + + return response_ptr; +} +/// SET Request /// + +/// LIST Request /// +template +bool checkAuth(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) { - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Read, session_id, is_local); +} + +template +std::list preprocess( + const Coordination::ZooKeeperListRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + ProfileEvents::increment(ProfileEvents::KeeperListRequest); + + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + return {}; +} + +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + { + response.error = result; + return response_ptr; + } } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + auto & container = storage.container; - std::list - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) { - ProfileEvents::increment(ProfileEvents::KeeperListRequest); - Coordination::ZooKeeperListRequest & request = dynamic_cast(*this->zk_request); - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperListRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - } - - auto & container = storage.container; - - auto node_it = container.find(request.path); - if (node_it == container.end()) - { - if constexpr (local) - response.error = Coordination::Error::ZNONODE; - else - onStorageInconsistency(); - } + if constexpr (local) + response.error = Coordination::Error::ZNONODE; else - { - auto path_prefix = request.path; - if (path_prefix.empty()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty"); + onStorageInconsistency(); + } + else + { + auto path_prefix = zk_request.path; + if (path_prefix.empty()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty"); - const auto & get_children = [&]() + const auto & get_children = [&]() + { + if constexpr (Storage::use_rocksdb) + return container.getChildren(zk_request.path); + else + return node_it->value.getChildren(); + }; + const auto & children = get_children(); + response.names.reserve(children.size()); + + const auto add_child = [&](const auto & child) + { + using enum Coordination::ListRequestType; + + auto list_request_type = ALL; + if (const auto * filtered_list = dynamic_cast(&zk_request)) + { + list_request_type = filtered_list->list_request_type; + } + + if (list_request_type == ALL) + return true; + + bool is_ephemeral; + if constexpr (!Storage::use_rocksdb) + { + auto child_path = (std::filesystem::path(zk_request.path) / child.toView()).generic_string(); + auto child_it = container.find(child_path); + if (child_it == container.end()) + onStorageInconsistency(); + is_ephemeral = child_it->value.stats.isEphemeral(); + } + else + { + is_ephemeral = child.second.stats.isEphemeral(); + } + + return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); + }; + + for (const auto & child : children) + { + if (add_child(child)) { if constexpr (Storage::use_rocksdb) - return container.getChildren(request.path); + response.names.push_back(child.first); else - return node_it->value.getChildren(); - }; - const auto & children = get_children(); - response.names.reserve(children.size()); - - const auto add_child = [&](const auto & child) - { - using enum Coordination::ListRequestType; - - auto list_request_type = ALL; - if (auto * filtered_list = dynamic_cast(&request)) - { - list_request_type = filtered_list->list_request_type; - } - - if (list_request_type == ALL) - return true; - - bool is_ephemeral; - if constexpr (!Storage::use_rocksdb) - { - auto child_path = (std::filesystem::path(request.path) / child.toView()).generic_string(); - auto child_it = container.find(child_path); - if (child_it == container.end()) - onStorageInconsistency(); - is_ephemeral = child_it->value.stats.isEphemeral(); - } - else - { - is_ephemeral = child.second.stats.isEphemeral(); - } - - return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); - }; - - for (const auto & child : children) - { - if (add_child(child)) - { - if constexpr (Storage::use_rocksdb) - response.names.push_back(child.first); - else - response.names.push_back(child.toString()); - } - } - - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; - } - - return response_ptr; - } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } - - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - ProfileEvents::increment(ProfileEvents::KeeperListRequest); - return processImpl(storage, std::move(deltas)); - } -}; - -template -struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor -{ - explicit KeeperStorageCheckRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequestProcessor(zk_request_) - { - check_not_exists = this->zk_request->getOpNum() == Coordination::OpNum::CheckNotExists; - } - - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override - { - auto path = this->zk_request->getPath(); - return storage.checkACL(check_not_exists ? parentNodePath(path) : path, Coordination::ACL::Read, session_id, is_local); - } - - std::list - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override - { - ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); - - Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*this->zk_request); - - auto node = storage.uncommitted_state.getNode(request.path); - if (check_not_exists) - { - if (node && (request.version == -1 || request.version == node->stats.version)) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; - } - else - { - if (!node) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - if (request.version != -1 && request.version != node->stats.version) - return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; - } - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; + response.names.push_back(child.toString()); } } - const auto on_error = [&]([[maybe_unused]] const auto error_code) - { - if constexpr (local) - response.error = error_code; - else - onStorageInconsistency(); - }; - - auto & container = storage.container; - auto node_it = container.find(request.path); - - if (check_not_exists) - { - if (node_it != container.end() && (request.version == -1 || request.version == node_it->value.stats.version)) - on_error(Coordination::Error::ZNODEEXISTS); - else - response.error = Coordination::Error::ZOK; - } - else - { - if (node_it == container.end()) - on_error(Coordination::Error::ZNONODE); - else if (request.version != -1 && request.version != node_it->value.stats.version) - on_error(Coordination::Error::ZBADVERSION); - else - response.error = Coordination::Error::ZOK; - } - - return response_ptr; + node_it->value.setResponseStat(response.stat); + response.error = Coordination::Error::ZOK; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } + return response_ptr; +} - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); - return processImpl(storage, std::move(deltas)); - } - -private: - bool check_not_exists; -}; - - -template -struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) { - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) +{ + ProfileEvents::increment(ProfileEvents::KeeperListRequest); + return processImpl(zk_request, storage, std::move(deltas)); +} +/// LIST Request /// + +/// CHECK Request /// +template +bool checkAuth(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + auto path = zk_request.getPath(); + return storage.checkACL( + zk_request.getOpNum() == Coordination::OpNum::CheckNotExists ? parentNodePath(path) : path, + Coordination::ACL::Read, + session_id, + is_local); +} + +template +std::list preprocess( + const Coordination::ZooKeeperCheckRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); + + auto node = storage.uncommitted_state.getNode(zk_request.path); + if (zk_request.getOpNum() == Coordination::OpNum::CheckNotExists) { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin, session_id, is_local); + if (node && (zk_request.version == -1 || zk_request.version == node->stats.version)) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; } - - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::list - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override + else { - Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*this->zk_request); - - if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) - { - auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path); - - handleSystemNodeModification(keeper_context, error_msg); - return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - } - - auto & uncommitted_state = storage.uncommitted_state; - if (!uncommitted_state.getNode(request.path)) + if (!node) return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - auto node = uncommitted_state.getNode(request.path); - - if (request.version != -1 && request.version != node->stats.aversion) + if (zk_request.version != -1 && zk_request.version != node->stats.version) return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; - - - Coordination::ACLs node_acls; - if (!fixupACL(request.acls, session_id, uncommitted_state, node_acls)) - return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; - - KeeperStorageBase::UpdateNodeStatDelta update_stat_delta(*node); - ++update_stat_delta.new_stats.aversion; - std::list new_deltas{ - {request.path, - zxid, - KeeperStorageBase::SetACLDelta{std::move(node_acls), uncommitted_state.getACLs(request.path), request.version}}, - {request.path, zxid, std::move(update_stat_delta)}}; - - digest = storage.calculateNodesDigest(digest, new_deltas); - - return new_deltas; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*this->zk_request); + return {}; +} +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) + { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { response.error = result; return response_ptr; } + } - auto node_it = storage.container.find(request.path); - if (node_it == storage.container.end()) + const auto on_error = [&]([[maybe_unused]] const auto error_code) + { + if constexpr (local) + response.error = error_code; + else onStorageInconsistency(); - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + }; - return response_ptr; - } -}; + auto & container = storage.container; + auto node_it = container.find(zk_request.path); -template -struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor -{ - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + if (zk_request.getOpNum() == Coordination::OpNum::CheckNotExists) { - return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local); + if (node_it != container.end() && (zk_request.version == -1 || zk_request.version == node_it->value.stats.version)) + on_error(Coordination::Error::ZNODEEXISTS); + else + response.error = Coordination::Error::ZOK; } - - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - - std::list - preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + else { - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); - - if (!storage.uncommitted_state.getNode(request.path)) - return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; - - return {}; - } - - template - Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, std::list deltas) const - { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); - - if constexpr (!local) - { - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - { - response.error = result; - return response_ptr; - } - } - - auto & container = storage.container; - auto node_it = container.find(request.path); if (node_it == container.end()) - { - if constexpr (local) - response.error = Coordination::Error::ZNONODE; - else - onStorageInconsistency(); - } + on_error(Coordination::Error::ZNONODE); + else if (zk_request.version != -1 && zk_request.version != node_it->value.stats.version) + on_error(Coordination::Error::ZBADVERSION); else + response.error = Coordination::Error::ZOK; + } + + return response_ptr; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr +processLocal(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, std::list deltas) +{ + ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); + return processImpl(zk_request, storage, std::move(deltas)); +} +/// CHECK Request /// + +/// MULTI Request /// +using OperationType = Coordination::ZooKeeperMultiRequest::OperationType; +template +bool checkAuth(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + for (const auto & concrete_request : zk_request.requests) + { + if (!callOnConcreteRequestType( + *concrete_request, [&](const auto & subrequest) { return checkAuth(subrequest, storage, session_id, is_local); })) + return false; + } + return true; +} + +template +std::list preprocess( + const Coordination::ZooKeeperMultiRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t session_id, + int64_t time, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + ProfileEvents::increment(ProfileEvents::KeeperMultiRequest); + std::vector response_errors; + const auto & subrequests = zk_request.requests; + response_errors.reserve(subrequests.size()); + uint64_t current_digest = digest; + std::list new_deltas; + for (size_t i = 0; i < subrequests.size(); ++i) + { + auto new_subdeltas = callOnConcreteRequestType( + *subrequests[i], + [&](const auto & subrequest) + { return preprocess(subrequest, storage, zxid, session_id, time, current_digest, keeper_context); }); + + if (!new_subdeltas.empty()) { - node_it->value.setResponseStat(response.stat); - response.acl = storage.acl_map.convertNumber(node_it->value.stats.acl_id); + if (auto * error = std::get_if(&new_subdeltas.back().operation); + error && zk_request.getOpNum() == Coordination::OpNum::Multi) + { + storage.uncommitted_state.rollback(std::move(new_deltas)); + response_errors.push_back(error->error); + + for (size_t j = i + 1; j < subrequests.size(); ++j) + response_errors.push_back(Coordination::Error::ZRUNTIMEINCONSISTENCY); + + return {typename Storage::Delta{zxid, typename Storage::FailedMultiDelta{std::move(response_errors)}}}; + } + } + + new_subdeltas.emplace_back(zxid, KeeperStorageBase::SubDeltaEnd{}); + response_errors.push_back(Coordination::Error::ZOK); + + // manually add deltas so that the result of previous request in the transaction is used in the next request + storage.uncommitted_state.applyDeltas(new_subdeltas); + new_deltas.splice(new_deltas.end(), std::move(new_subdeltas)); + } + + digest = current_digest; + storage.uncommitted_state.addDeltas(std::move(new_deltas)); + return {}; +} + +std::list getSubdeltas(std::list & deltas) +{ + std::list subdeltas; + auto it = deltas.begin(); + + for (; it != deltas.end(); ++it) + { + if (std::holds_alternative(it->operation)) + { + ++it; + break; + } + } + + if (it == deltas.end()) + subdeltas = std::move(deltas); + else + subdeltas.splice(subdeltas.end(), deltas, deltas.begin(), it); + + return subdeltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); + + const auto & subrequests = zk_request.requests; + + // the deltas will have at least SubDeltaEnd or FailedMultiDelta + chassert(!deltas.empty()); + if (auto * failed_multi = std::get_if(&deltas.front().operation)) + { + for (size_t i = 0; i < subrequests.size(); ++i) + { + response.responses[i] = std::make_shared(); + response.responses[i]->error = failed_multi->error_codes[i]; } return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + for (size_t i = 0; i < subrequests.size(); ++i) { - return processImpl(storage, std::move(deltas)); + std::list subdeltas = getSubdeltas(deltas); + response.responses[i] = callOnConcreteRequestType( + *subrequests[i], [&](const auto & subrequest) { return process(subrequest, storage, std::move(subdeltas)); }); } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override - { - return processImpl(storage, std::move(deltas)); - } -}; + response.error = Coordination::Error::ZOK; + return response_ptr; +} -template -struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor +template +Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, std::list deltas) { - using OperationType = Coordination::ZooKeeperMultiRequest::OperationType; - std::optional operation_type; + ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); - bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + const auto & subrequests = zk_request.requests; + for (size_t i = 0; i < subrequests.size(); ++i) { - for (const auto & concrete_request : concrete_requests) - if (!concrete_request->checkAuth(storage, session_id, is_local)) - return false; - return true; + std::list subdeltas = getSubdeltas(deltas); + response.responses[i] = callOnConcreteRequestType( + *subrequests[i], [&](const auto & subrequest) { return processLocal(subrequest, storage, std::move(subdeltas)); }); } - std::vector>> concrete_requests; - explicit KeeperStorageMultiRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequestProcessor(zk_request_) + response.error = Coordination::Error::ZOK; + return response_ptr; +} + +template +KeeperStorageBase::ResponsesForSessions processWatches( + const Coordination::ZooKeeperMultiRequest & zk_request, typename Storage::Watches & watches, typename Storage::Watches & list_watches) +{ + typename Storage::ResponsesForSessions result; + + const auto & subrequests = zk_request.requests; + for (const auto & generic_request : subrequests) { - Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*this->zk_request); - concrete_requests.reserve(request.requests.size()); + auto responses = callOnConcreteRequestType( + generic_request, [&](const auto & subrequest) { return processWatches(subrequest, watches, list_watches); }); + result.insert(result.end(), responses.begin(), responses.end()); + } + return result; +} +/// MULTI Request /// - const auto check_operation_type = [&](OperationType type) - { - if (operation_type.has_value() && *operation_type != type) - throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal mixing of read and write operations in multi request"); - operation_type = type; - }; +/// AUTH Request /// +template +std::list preprocess( + const Coordination::ZooKeeperAuthRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t session_id, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - for (const auto & sub_request : request.requests) - { - auto sub_zk_request = std::dynamic_pointer_cast(sub_request); - switch (sub_zk_request->getOpNum()) - { - case Coordination::OpNum::Create: - case Coordination::OpNum::CreateIfNotExists: - check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared>(sub_zk_request)); - break; - case Coordination::OpNum::Remove: - check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared>(sub_zk_request)); - break; - case Coordination::OpNum::Set: - check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared>(sub_zk_request)); - break; - case Coordination::OpNum::Check: - case Coordination::OpNum::CheckNotExists: - check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared>(sub_zk_request)); - break; - case Coordination::OpNum::Get: - check_operation_type(OperationType::Read); - concrete_requests.push_back(std::make_shared>(sub_zk_request)); - break; - case Coordination::OpNum::Exists: - check_operation_type(OperationType::Read); - concrete_requests.push_back(std::make_shared>(sub_zk_request)); - break; - case Coordination::OpNum::List: - case Coordination::OpNum::FilteredList: - case Coordination::OpNum::SimpleList: - check_operation_type(OperationType::Read); - concrete_requests.push_back(std::make_shared>(sub_zk_request)); - break; - default: - throw DB::Exception( - ErrorCodes::BAD_ARGUMENTS, - "Illegal command as part of multi ZooKeeper request {}", - sub_zk_request->getOpNum()); - } - } + if (zk_request.scheme != "digest" || std::count(zk_request.data.begin(), zk_request.data.end(), ':') != 1) + return {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; - chassert(request.requests.empty() || operation_type.has_value()); + std::list new_deltas; + auto auth_digest = Storage::generateDigest(zk_request.data); + if (auth_digest == storage.superdigest) + { + auto auth = std::make_shared(); + auth->scheme = "super"; + new_deltas.emplace_back(zxid, KeeperStorageBase::AddAuthDelta{session_id, std::move(auth)}); + } + else + { + auto new_auth = std::make_shared(); + new_auth->scheme = zk_request.scheme; + new_auth->id = std::move(auth_digest); + if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return *new_auth == auth_id; })) + new_deltas.emplace_back(zxid, KeeperStorageBase::AddAuthDelta{session_id, std::move(new_auth)}); } - std::list - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperAuthRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) + auth_response.error = result; + + return response_ptr; +} +/// AUTH Request /// + +/// CLOSE Request /// +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperCloseRequest & /* zk_request */, Storage &, std::list /* deltas */) +{ + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Called process on close request"); +} +/// CLOSE Request /// + +/// SETACL Request /// +template +bool checkAuth(const Coordination::ZooKeeperSetACLRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Admin, session_id, is_local); +} + +template +std::list preprocess( + const Coordination::ZooKeeperSetACLRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t session_id, + int64_t /*time*/, + uint64_t & digest, + const KeeperContext & keeper_context) +{ + if (Coordination::matchPath(zk_request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { - ProfileEvents::increment(ProfileEvents::KeeperMultiRequest); - std::vector response_errors; - response_errors.reserve(concrete_requests.size()); - uint64_t current_digest = digest; - std::list new_deltas; - for (size_t i = 0; i < concrete_requests.size(); ++i) - { - auto new_subdeltas = concrete_requests[i]->preprocess(storage, zxid, session_id, time, current_digest, keeper_context); + auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", zk_request.path); - if (!new_subdeltas.empty()) - { - if (auto * error = std::get_if(&new_subdeltas.back().operation); - error && *operation_type == OperationType::Write) - { - storage.uncommitted_state.rollback(std::move(new_deltas)); - response_errors.push_back(error->error); - - for (size_t j = i + 1; j < concrete_requests.size(); ++j) - response_errors.push_back(Coordination::Error::ZRUNTIMEINCONSISTENCY); - - return {typename Storage::Delta{zxid, typename Storage::FailedMultiDelta{std::move(response_errors)}}}; - } - } - - new_subdeltas.emplace_back(zxid, KeeperStorageBase::SubDeltaEnd{}); - response_errors.push_back(Coordination::Error::ZOK); - - // manually add deltas so that the result of previous request in the transaction is used in the next request - storage.uncommitted_state.applyDeltas(new_subdeltas); - new_deltas.splice(new_deltas.end(), std::move(new_subdeltas)); - } - - digest = current_digest; - storage.uncommitted_state.addDeltas(std::move(new_deltas)); - return {}; + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } - std::list getSubdeltas(std::list & deltas) const + auto & uncommitted_state = storage.uncommitted_state; + if (!uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + auto node = uncommitted_state.getNode(zk_request.path); + + if (zk_request.version != -1 && zk_request.version != node->stats.aversion) + return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; + + + Coordination::ACLs node_acls; + if (!fixupACL(zk_request.acls, session_id, uncommitted_state, node_acls)) + return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + + KeeperStorageBase::UpdateNodeStatDelta update_stat_delta(*node); + ++update_stat_delta.new_stats.aversion; + std::list new_deltas{ + {zk_request.path, + zxid, + KeeperStorageBase::SetACLDelta{std::move(node_acls), uncommitted_state.getACLs(zk_request.path), zk_request.version}}, + {zk_request.path, zxid, std::move(update_stat_delta)}}; + + digest = storage.calculateNodesDigest(digest, new_deltas); + + return new_deltas; +} + +template +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperSetACLRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); + + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - std::list subdeltas; - auto it = deltas.begin(); - - for (; it != deltas.end(); ++it) - { - if (std::holds_alternative(it->operation)) - { - ++it; - break; - } - } - - if (it == deltas.end()) - subdeltas = std::move(deltas); - else - subdeltas.splice(subdeltas.end(), deltas, deltas.begin(), it); - - return subdeltas; + response.error = result; + return response_ptr; } - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + auto node_it = storage.container.find(zk_request.path); + if (node_it == storage.container.end()) + onStorageInconsistency(); + node_it->value.setResponseStat(response.stat); + response.error = Coordination::Error::ZOK; + + return response_ptr; +} +/// SETACL Request /// + +/// GETACL Request /// +template +bool checkAuth(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) +{ + return storage.checkACL(zk_request.getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local); +} + +template +std::list preprocess( + const Coordination::ZooKeeperGetACLRequest & zk_request, + Storage & storage, + int64_t zxid, + int64_t /*session_id*/, + int64_t /*time*/, + uint64_t & /*digest*/, + const KeeperContext & /*keeper_context*/) +{ + if (!storage.uncommitted_state.getNode(zk_request.path)) + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + + return {}; +} + +template +Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, std::list deltas) +{ + Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); + Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); + + if constexpr (!local) { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); - - // the deltas will have at least SubDeltaEnd or FailedMultiDelta - chassert(!deltas.empty()); - if (auto * failed_multi = std::get_if(&deltas.front().operation)) + if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - for (size_t i = 0; i < concrete_requests.size(); ++i) - { - response.responses[i] = std::make_shared(); - response.responses[i]->error = failed_multi->error_codes[i]; - } - + response.error = result; return response_ptr; } - - for (size_t i = 0; i < concrete_requests.size(); ++i) - { - std::list subdeltas = getSubdeltas(deltas); - response.responses[i] = concrete_requests[i]->process(storage, std::move(subdeltas)); - } - - response.error = Coordination::Error::ZOK; - return response_ptr; } - Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, std::list deltas) const override + auto & container = storage.container; + auto node_it = container.find(zk_request.path); + if (node_it == container.end()) { - ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); - - for (size_t i = 0; i < concrete_requests.size(); ++i) - { - std::list subdeltas = getSubdeltas(deltas); - response.responses[i] = concrete_requests[i]->processLocal(storage, std::move(subdeltas)); - } - - response.error = Coordination::Error::ZOK; - return response_ptr; - } - - KeeperStorageBase::ResponsesForSessions - processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override - { - typename Storage::ResponsesForSessions result; - for (const auto & generic_request : concrete_requests) - { - auto responses = generic_request->processWatches(watches, list_watches); - result.insert(result.end(), responses.begin(), responses.end()); - } - return result; - } -}; - -template -struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - Coordination::ZooKeeperResponsePtr process(Storage &, std::list /* deltas */) const override - { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Called process on close request"); - } -}; - -template -struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor -{ - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::list - preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override - { - Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast(*this->zk_request); - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - - if (auth_request.scheme != "digest" || std::count(auth_request.data.begin(), auth_request.data.end(), ':') != 1) - return {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; - - std::list new_deltas; - auto auth_digest = Storage::generateDigest(auth_request.data); - if (auth_digest == storage.superdigest) - { - auto auth = std::make_shared(); - auth->scheme = "super"; - new_deltas.emplace_back(zxid, KeeperStorageBase::AddAuthDelta{session_id, std::move(auth)}); - } + if constexpr (local) + response.error = Coordination::Error::ZNONODE; else - { - auto new_auth = std::make_shared(); - new_auth->scheme = auth_request.scheme; - new_auth->id = std::move(auth_digest); - if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return *new_auth == auth_id; })) - new_deltas.emplace_back(zxid, KeeperStorageBase::AddAuthDelta{session_id, std::move(new_auth)}); - } - - return new_deltas; + onStorageInconsistency(); } - - Coordination::ZooKeeperResponsePtr process(Storage & storage, std::list deltas) const override + else { - Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); - Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); - - if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - auth_response.error = result; - - return response_ptr; + node_it->value.setResponseStat(response.stat); + response.acl = storage.acl_map.convertNumber(node_it->value.stats.acl_id); } -}; + + return response_ptr; +} + +template +Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} + +template +Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, std::list deltas) +{ + return processImpl(zk_request, storage, std::move(deltas)); +} +/// GETACL Request /// template void KeeperStorage::finalize() @@ -2384,72 +2442,6 @@ bool KeeperStorage::isFinalized() const return finalized; } -template -class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable -{ -public: - using Creator = std::function>(const Coordination::ZooKeeperRequestPtr &)>; - using OpNumToRequest = std::unordered_map; - - static KeeperStorageRequestProcessorsFactory & instance() - { - static KeeperStorageRequestProcessorsFactory factory; - return factory; - } - - std::shared_ptr> get(const Coordination::ZooKeeperRequestPtr & zk_request) const - { - auto request_it = op_num_to_request.find(zk_request->getOpNum()); - if (request_it == op_num_to_request.end()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", zk_request->getOpNum()); - - return request_it->second(zk_request); - } - - void registerRequest(Coordination::OpNum op_num, Creator creator) - { - if (!op_num_to_request.try_emplace(op_num, creator).second) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Request with op num {} already registered", op_num); - } - -private: - OpNumToRequest op_num_to_request; - KeeperStorageRequestProcessorsFactory(); -}; - -template -void registerKeeperRequestProcessor(Factory & factory) -{ - factory.registerRequest( - num, [](const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared(zk_request); }); -} - - -template -KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() -{ - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); - registerKeeperRequestProcessor>(*this); -} - - template UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::list & new_deltas) const { @@ -2604,8 +2596,6 @@ void KeeperStorage::preprocessRequest( uncommitted_state.addDeltas(std::move(new_deltas)); }); - auto request_processor = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); - if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { { @@ -2643,13 +2633,18 @@ void KeeperStorage::preprocessRequest( return; } - if (check_acl && !request_processor->checkAuth(*this, session_id, false)) + const auto preprocess_request = [&] T>(const T & concrete_zk_request) { - new_deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); - return; - } + if (check_acl && !checkAuth(concrete_zk_request, *this, session_id, false)) + { + new_deltas.emplace_back(new_last_zxid, Coordination::Error::ZNOAUTH); + return; + } - new_deltas = request_processor->preprocess(*this, transaction->zxid, session_id, time, new_digest, *keeper_context); + new_deltas = preprocess(concrete_zk_request, *this, transaction->zxid, session_id, time, new_digest, *keeper_context); + }; + + callOnConcreteRequestType(*zk_request, preprocess_request); uncommitted_state.cleanup(getZXID()); } @@ -2750,11 +2745,10 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process } else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special { - auto storage_request = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); Coordination::ZooKeeperResponsePtr response = nullptr; { std::lock_guard lock(storage_mutex); - response = storage_request->process(*this, std::move(deltas)); + response = process(dynamic_cast(*zk_request), *this, std::move(deltas)); } response->xid = zk_request->xid; response->zxid = commit_zxid; @@ -2763,65 +2757,69 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process } else /// normal requests proccession { - auto request_processor = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); - Coordination::ZooKeeperResponsePtr response; - - if (is_local) + const auto process_request = [&] T>(const T & concrete_zk_request) { - chassert(zk_request->isReadRequest()); - if (check_acl && !request_processor->checkAuth(*this, session_id, true)) + Coordination::ZooKeeperResponsePtr response; + + if (is_local) { - response = zk_request->makeResponse(); - /// Original ZooKeeper always throws no auth, even when user provided some credentials - response->error = Coordination::Error::ZNOAUTH; + chassert(zk_request->isReadRequest()); + if (check_acl && !checkAuth(concrete_zk_request, *this, session_id, true)) + { + response = zk_request->makeResponse(); + /// Original ZooKeeper always throws no auth, even when user provided some credentials + response->error = Coordination::Error::ZNOAUTH; + } + else + { + std::shared_lock lock(storage_mutex); + response = processLocal(concrete_zk_request, *this, std::move(deltas)); + } } else { - std::shared_lock lock(storage_mutex); - response = request_processor->processLocal(*this, std::move(deltas)); + std::lock_guard lock(storage_mutex); + response = process(concrete_zk_request, *this, std::move(deltas)); } - } - else - { - std::lock_guard lock(storage_mutex); - response = request_processor->process(*this, std::move(deltas)); - } - /// Watches for this requests are added to the watches lists - if (zk_request->has_watch) - { + /// Watches for this requests are added to the watches lists + if (zk_request->has_watch) + { + if (response->error == Coordination::Error::ZOK) + { + static constexpr std::array list_requests{ + Coordination::OpNum::List, Coordination::OpNum::SimpleList, Coordination::OpNum::FilteredList}; + + auto & watches_type = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end() + ? list_watches + : watches; + + auto add_watch_result = watches_type[zk_request->getPath()].emplace(session_id); + if (add_watch_result.second) + sessions_and_watchers[session_id].emplace(zk_request->getPath()); + } + else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists) + { + auto add_watch_result = watches[zk_request->getPath()].emplace(session_id); + if (add_watch_result.second) + sessions_and_watchers[session_id].emplace(zk_request->getPath()); + } + } + + /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { - static constexpr std::array list_requests{ - Coordination::OpNum::List, Coordination::OpNum::SimpleList, Coordination::OpNum::FilteredList}; - - auto & watches_type = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end() - ? list_watches - : watches; - - auto add_watch_result = watches_type[zk_request->getPath()].emplace(session_id); - if (add_watch_result.second) - sessions_and_watchers[session_id].emplace(zk_request->getPath()); + auto watch_responses = processWatches(concrete_zk_request, watches, list_watches); + results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } - else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists) - { - auto add_watch_result = watches[zk_request->getPath()].emplace(session_id); - if (add_watch_result.second) - sessions_and_watchers[session_id].emplace(zk_request->getPath()); - } - } - /// If this requests processed successfully we need to check watches - if (response->error == Coordination::Error::ZOK) - { - auto watch_responses = request_processor->processWatches(watches, list_watches); - results.insert(results.end(), watch_responses.begin(), watch_responses.end()); - } + response->xid = zk_request->xid; + response->zxid = commit_zxid; - response->xid = zk_request->xid; - response->zxid = commit_zxid; + results.push_back(ResponseForSession{session_id, response}); + }; - results.push_back(ResponseForSession{session_id, response}); + callOnConcreteRequestType(*zk_request, process_request); } updateStats(); From 5004e4d2cc48c652393df60fa67b93e32df0c21e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 03:27:05 +0200 Subject: [PATCH 020/128] Miscellaneous --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 004da5ed341..fa977e40226 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -3679,10 +3679,10 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode( auto hints = IdentifierResolver::collectIdentifierTypoHints(unresolved_identifier, valid_identifiers); - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {}{} identifier '{}' in scope {}{}", + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {}{} identifier {} in scope {}{}", toStringLowercase(IdentifierLookupContext::EXPRESSION), message_clarification, - unresolved_identifier.getFullName(), + backQuote(unresolved_identifier.getFullName()), scope.scope_node->formatASTForErrorMessage(), getHintsErrorMessageSuffix(hints)); } From 95edca513c3f1cc8677b637facdaaf916f3a71c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 05:43:01 +0200 Subject: [PATCH 021/128] Fix tests --- .../0_stateless/03096_text_log_format_string_args_not_empty.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql index 0e60ee77fe5..12fd26e2141 100644 --- a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql @@ -8,6 +8,6 @@ system flush logs; select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB'; -select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier \'{}\' in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count'; +select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier `{}` in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count'; select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name \'{}\' does not exist. In scope {}{}' and value1 = 'conut' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%'; From 4bb2f7b3f61ee1abb2c493dc44e5101485eb6675 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 09:09:58 +0200 Subject: [PATCH 022/128] Miscellaneous --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 12 ++++++------ tests/integration/test_access_for_functions/test.py | 2 +- .../test.py | 2 +- .../test_user_defined_object_persistence/test.py | 6 +++--- .../03096_text_log_format_string_args_not_empty.sql | 4 ++-- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index fa977e40226..427b583861b 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -2564,8 +2564,8 @@ void checkFunctionNodeHasEmptyNullsAction(FunctionNode const & node) if (node.getNullsAction() != NullsAction::EMPTY) throw Exception( ErrorCodes::SYNTAX_ERROR, - "Function with name '{}' cannot use {} NULLS", - node.getFunctionName(), + "Function with name {} cannot use {} NULLS", + backQuote(node.getFunctionName()), node.getNullsAction() == NullsAction::IGNORE_NULLS ? "IGNORE" : "RESPECT"); } } @@ -3228,16 +3228,16 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto hints = NamePrompter<2>::getHints(function_name, possible_function_names); throw Exception(ErrorCodes::UNKNOWN_FUNCTION, - "Function with name '{}' does not exist. In scope {}{}", - function_name, + "Function with name {} does not exist. In scope {}{}", + backQuote(function_name), scope.scope_node->formatASTForErrorMessage(), getHintsErrorMessageSuffix(hints)); } if (!function_lambda_arguments_indexes.empty()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Aggregate function '{}' does not support lambda arguments", - function_name); + "Aggregate function {} does not support lambda arguments", + backQuote(function_name)); auto action = function_node_ptr->getNullsAction(); std::string aggregate_function_name = rewriteAggregateFunctionNameIfNeeded(function_name, action, scope.context); diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index 82125b35b49..138152904a8 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -42,7 +42,7 @@ def test_access_rights_for_function(): function_resolution_error = instance.query_and_get_error("SELECT MySum(1, 2)") assert ( "Unknown function MySum" in function_resolution_error - or "Function with name 'MySum' does not exist." in function_resolution_error + or "Function with name `MySum` does not exist." in function_resolution_error ) instance.query("REVOKE CREATE FUNCTION ON *.* FROM A") diff --git a/tests/integration/test_sql_user_defined_functions_on_cluster/test.py b/tests/integration/test_sql_user_defined_functions_on_cluster/test.py index dc36a7b32b8..76dc254c84e 100644 --- a/tests/integration/test_sql_user_defined_functions_on_cluster/test.py +++ b/tests/integration/test_sql_user_defined_functions_on_cluster/test.py @@ -28,7 +28,7 @@ def test_sql_user_defined_functions_on_cluster(): error_message = node.query_and_get_error("SELECT test_function(1);") assert ( "Unknown function test_function" in error_message - or "Function with name 'test_function' does not exist. In scope SELECT test_function(1)" + or "Function with name `test_function` does not exist. In scope SELECT test_function(1)" in error_message ) diff --git a/tests/integration/test_user_defined_object_persistence/test.py b/tests/integration/test_user_defined_object_persistence/test.py index bd491dfa195..eb6b3621a54 100644 --- a/tests/integration/test_user_defined_object_persistence/test.py +++ b/tests/integration/test_user_defined_object_persistence/test.py @@ -43,20 +43,20 @@ def test_persistence(): error_message = instance.query_and_get_error("SELECT MySum1(1, 2)") assert ( "Unknown function MySum1" in error_message - or "Function with name 'MySum1' does not exist. In scope SELECT MySum1(1, 2)" + or "Function with name `MySum1` does not exist. In scope SELECT MySum1(1, 2)" in error_message ) error_message = instance.query_and_get_error("SELECT MySum2(1, 2)") assert ( "Unknown function MySum2" in error_message - or "Function with name 'MySum2' does not exist. In scope SELECT MySum2(1, 2)" + or "Function with name `MySum2` does not exist. In scope SELECT MySum2(1, 2)" in error_message ) error_message = instance.query_and_get_error("SELECT MyUnion()") assert ( "Unknown function MyUnion" in error_message - or "Function with name 'MyUnion' does not exist. In scope SELECT MyUnion" + or "Function with name `MyUnion` does not exist. In scope SELECT MyUnion" in error_message ) diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql index 12fd26e2141..6cd71ac3270 100644 --- a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql @@ -8,6 +8,6 @@ system flush logs; select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB'; -select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier `{}` in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count'; +select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier {} in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count'; -select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name \'{}\' does not exist. In scope {}{}' and value1 = 'conut' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%'; +select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name {} does not exist. In scope {}{}' and value1 = 'conut' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%'; From f88b5988c1439dc40a164bf98d3b3f0d5bcd4f44 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 09:44:39 +0200 Subject: [PATCH 023/128] Update test --- .../03096_text_log_format_string_args_not_empty.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql index 6cd71ac3270..bb036a6c133 100644 --- a/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql +++ b/tests/queries/0_stateless/03096_text_log_format_string_args_not_empty.sql @@ -8,6 +8,6 @@ system flush logs; select count() > 0 from system.text_log where message_format_string = 'Peak memory usage{}: {}.' and value1 is not null and value2 like '% MiB'; -select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier {} in scope {}{}' and value1 = 'expression' and value3 = 'count' and value4 = 'SELECT count'; +select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Unknown {}{} identifier {} in scope {}{}' and value1 = 'expression' and value3 = '`count`' and value4 = 'SELECT count'; -select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name {} does not exist. In scope {}{}' and value1 = 'conut' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%'; +select count() > 0 from system.text_log where level = 'Error' and message_format_string = 'Function with name {} does not exist. In scope {}{}' and value1 = '`conut`' and value2 = 'SELECT conut()' and value3 ilike '%\'count\'%'; From f97551e2ad8622cf87c1b4f9f6231f256a48742a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Aug 2024 22:17:16 +0200 Subject: [PATCH 024/128] Fix tests --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- tests/integration/test_storage_hdfs/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index fbdc7f29f98..8e18a6a8d02 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1608,7 +1608,7 @@ def test_hive_partitioning_without_setting(cluster): f"blob_path='{path}', format='CSV', structure='{table_format}') WHERE column1=_column1;" ) pattern = re.compile( - r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + r"DB::Exception: Unknown expression identifier `.*` in scope.*", re.DOTALL ) with pytest.raises(Exception, match=pattern): diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index fdbf7c5bacb..79b59191e93 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -1296,7 +1296,7 @@ def test_hive_partitioning_without_setting(started_cluster): == f"Elizabeth\tGordon\n" ) pattern = re.compile( - r"DB::Exception: Unknown expression identifier '.*' in scope.*", re.DOTALL + r"DB::Exception: Unknown expression identifier `.*` in scope.*", re.DOTALL ) with pytest.raises(QueryRuntimeException, match=pattern): From 189cbe25fe4321f020a9f7b72c901d7358c1f1af Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 27 Aug 2024 16:28:18 +0200 Subject: [PATCH 025/128] init --- .../StorageObjectStorageSource.cpp | 12 +++--- .../StorageObjectStorageSource.h | 1 + src/Storages/StorageFile.cpp | 4 +- src/Storages/StorageURL.cpp | 7 +++- src/Storages/VirtualColumnUtils.cpp | 37 +++++++++++++++--- src/Storages/VirtualColumnUtils.h | 8 ++-- ...3231_hive_partitioning_filtering.reference | 6 +++ .../03231_hive_partitioning_filtering.sh | 29 ++++++++++++++ .../partitioning/array=[1,2,3]/sample.parquet | Bin 0 -> 1308 bytes .../partitioning/array=[1,2,4]/sample.parquet | Bin 0 -> 1308 bytes .../partitioning/column0=Stacy/sample.parquet | Bin 0 -> 1308 bytes .../partitioning/identifier=2071/email.csv | 5 +++ 12 files changed, 91 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/03231_hive_partitioning_filtering.reference create mode 100644 tests/queries/0_stateless/03231_hive_partitioning_filtering.sh create mode 100644 tests/queries/0_stateless/data_hive/partitioning/array=[1,2,3]/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/array=[1,2,4]/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/column0=Stacy/sample.parquet create mode 100644 tests/queries/0_stateless/data_hive/partitioning/identifier=2071/email.csv diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 04e319cd0b8..0d4471e3bda 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -131,10 +131,11 @@ std::shared_ptr StorageObjectStorageSourc else { ConfigurationPtr copy_configuration = configuration->clone(); - auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + auto keys = configuration->getPaths(); + String partitioning_path = fs::path(configuration->getNamespace()) / keys[0]; + auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, partitioning_path, local_context); if (filter_dag) { - auto keys = configuration->getPaths(); std::vector paths; paths.reserve(keys.size()); for (const auto & key : keys) @@ -142,7 +143,7 @@ std::shared_ptr StorageObjectStorageSourc VirtualColumnUtils::buildSetsForDAG(*filter_dag, local_context); auto actions = std::make_shared(std::move(*filter_dag)); - VirtualColumnUtils::filterByPathOrFile(keys, paths, actions, virtual_columns); + VirtualColumnUtils::filterByPathOrFile(keys, paths, actions, virtual_columns, local_context); copy_configuration->setPaths(keys); } @@ -492,6 +493,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( , virtual_columns(virtual_columns_) , throw_on_zero_files_match(throw_on_zero_files_match_) , read_keys(read_keys_) + , local_context(context_) , file_progress_callback(file_progress_callback_) { if (configuration->isNamespaceWithGlobs()) @@ -513,7 +515,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } recursive = key_with_globs == "/**"; - if (auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns)) + if (auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, key_with_globs, local_context)) { VirtualColumnUtils::buildSetsForDAG(*filter_dag, getContext()); filter_expr = std::make_shared(std::move(*filter_dag)); @@ -588,7 +590,7 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne for (const auto & object_info : new_batch) paths.push_back(getUniqueStoragePathIdentifier(*configuration, *object_info, false)); - VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_expr, virtual_columns); + VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_expr, virtual_columns, local_context); LOG_TEST(logger, "Filtered files: {} -> {}", paths.size(), new_batch.size()); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 7ae7a2358e9..8ee3b023638 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -220,6 +220,7 @@ private: bool is_finished = false; bool first_iteration = true; std::mutex next_mutex; + const ContextPtr local_context; std::function file_progress_callback; }; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 50294df32a4..639af41c1cd 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1140,13 +1140,13 @@ StorageFileSource::FilesIterator::FilesIterator( { std::optional filter_dag; if (!distributed_processing && !archive_info && !files.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, files[0], context_); if (filter_dag) { VirtualColumnUtils::buildSetsForDAG(*filter_dag, context_); auto actions = std::make_shared(std::move(*filter_dag)); - VirtualColumnUtils::filterByPathOrFile(files, files, actions, virtual_columns); + VirtualColumnUtils::filterByPathOrFile(files, files, actions, virtual_columns, context_); } } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index fc1354b780a..572c4f20fa3 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -214,7 +214,10 @@ public: std::optional filter_dag; if (!uris.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + { + String partitioning_path = Poco::URI(uris[0]).getPath(); + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, partitioning_path, context); + } if (filter_dag) { @@ -225,7 +228,7 @@ public: VirtualColumnUtils::buildSetsForDAG(*filter_dag, context); auto actions = std::make_shared(std::move(*filter_dag)); - VirtualColumnUtils::filterByPathOrFile(uris, paths, actions, virtual_columns); + VirtualColumnUtils::filterByPathOrFile(uris, paths, actions, virtual_columns, context); } } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index f0d276e4e56..788aeb66657 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -46,6 +46,7 @@ #include "Functions/IFunction.h" #include "Functions/IFunctionAdaptors.h" #include "Functions/indexHint.h" +#include #include #include #include @@ -197,7 +198,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto return desc; } -static void addPathAndFileToVirtualColumns(Block & block, const String & path, size_t idx) +static void addFilterDataToVirtualColumns(Block & block, const String & path, size_t idx, ColumnsWithTypeAndName partitioning_keys, const ContextPtr & context) { if (block.has("_path")) block.getByName("_path").column->assumeMutableRef().insert(path); @@ -214,18 +215,31 @@ static void addPathAndFileToVirtualColumns(Block & block, const String & path, s block.getByName("_file").column->assumeMutableRef().insert(file); } + for (const auto & item : partitioning_keys) + { + if (block.has(item.name)) + { + auto column = block.getByName(item.name).column; + ReadBufferFromString buf(item.column->getDataAt(0).toView()); + item.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); + } + } + block.getByName("_idx").column->assumeMutableRef().insert(idx); } -std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns) +std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const String & path, const ContextPtr & context) { if (!predicate || virtual_columns.empty()) return {}; Block block; + std::unordered_map keys; + if (context->getSettingsRef().use_hive_partitioning) + keys = parseHivePartitioningKeysAndValues(path); for (const auto & column : virtual_columns) { - if (column.name == "_file" || column.name == "_path") + if (column.name == "_file" || column.name == "_path" || keys.contains(column.name)) block.insert({column.type->createColumn(), column.type, column.name}); } @@ -233,18 +247,31 @@ std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * pr return splitFilterDagForAllowedInputs(predicate, &block); } -ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns) +ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { Block block; + std::unordered_map keys; + ColumnsWithTypeAndName partitioning_columns; + if (context->getSettingsRef().use_hive_partitioning) + keys = parseHivePartitioningKeysAndValues(paths[0]); for (const auto & column : virtual_columns) { if (column.name == "_file" || column.name == "_path") block.insert({column.type->createColumn(), column.type, column.name}); + + auto it = keys.find(column.name); + if (it != keys.end()) + { + auto c = std::make_shared()->createColumn(); + c->insert(it->second); + block.insert({column.type->createColumn(), column.type, column.name}); + partitioning_columns.push_back({c->getPtr(), column.type, column.name}); + } } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); for (size_t i = 0; i != paths.size(); ++i) - addPathAndFileToVirtualColumns(block, paths[i], i); + addFilterDataToVirtualColumns(block, paths[i], i, partitioning_columns, context); filterBlockWithExpression(actions, block); diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 6aa08b2aef2..ecfe44a1956 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -75,14 +75,14 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage( const std::string & sample_path = "", std::optional format_settings_ = std::nullopt); -std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns); +std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const String & path, const ContextPtr & context); -ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns); +ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns, const ContextPtr & context); template -void filterByPathOrFile(std::vector & sources, const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns) +void filterByPathOrFile(std::vector & sources, const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { - auto indexes_column = getFilterByPathAndFileIndexes(paths, actions, virtual_columns); + auto indexes_column = getFilterByPathAndFileIndexes(paths, actions, virtual_columns, context); const auto & indexes = typeid_cast(*indexes_column).getData(); if (indexes.size() == sources.size()) return; diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.reference b/tests/queries/0_stateless/03231_hive_partitioning_filtering.reference new file mode 100644 index 00000000000..a9e2f17562a --- /dev/null +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.reference @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh new file mode 100644 index 00000000000..719fed5bdaa --- /dev/null +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query_id="test_03231_1" --query " + SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; +" + +${CLICKHOUSE_CLIENT} --query " + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1'; +" + +${CLICKHOUSE_CLIENT} --query_id="test_03231_2" --query " + SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070; +" + +${CLICKHOUSE_CLIENT} --query " + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2'; +" + +${CLICKHOUSE_CLIENT} --query_id="test_03231_3" --query " + SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3]; +" + +${CLICKHOUSE_CLIENT} --query " + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3'; +" diff --git a/tests/queries/0_stateless/data_hive/partitioning/array=[1,2,3]/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/array=[1,2,3]/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/array=[1,2,4]/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/array=[1,2,4]/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/column0=Stacy/sample.parquet b/tests/queries/0_stateless/data_hive/partitioning/column0=Stacy/sample.parquet new file mode 100644 index 0000000000000000000000000000000000000000..9b6a78cf8cc7cd3ece15e13c9b2f222c8f09b81e GIT binary patch literal 1308 zcmWG=3^EjD5Z%Hr`iosh^b{kI%_hpmz#!kv!2kyTLxU6Z9~tnZHa*@opEc(Au?K1g zOD4aYo#~scS*oJ`_R8gD$~^!1^Jl8v?6d#uZT@&1Z&h*OEsK+iS@vM( z^NvMD|`UJH2qG^xTWJ-dT6$7G6DVTky7Woy5#*nvWVEJpR{CJ{Fy0- zE8ux@_5^8x!?dEIRau&2MyW=j!5h*xtj<|H$T%nI_ zrsjz?W}YW@dt8{DRBI|`*(jU(m2ZmM@u#NQ!s{)z%{yLgtZF$)cAddC?xOT5D^_mz z-x7J9sr1v$v$K{(^`5h;Sz-1gc2*AGUh7}8F0R?}-B&E(IrH;G`GUhY z?q@1K*wQW0otd;iYI&}N?~AIE{%tkCroWN7t$#4bGw~KP0|PJ-eBc+|z=1tM#0JF{ zU3TEfTh6OHr)jl`=8?k_CV5&tiR=x1?{{sI`|Af*?oUEIqS_tiuleY8e||}EY3bMB zzp9qaKhIf|e>9xYs^&t{(WWC|y8X+=Uc{}=?T>Xh_5JxVk(1Vsywf&)T&i$tu2}yJ zsTDW>>9!Q_yZT7oEaCof4t43QdkFv1JFG`q9?h6g zxTpBgk6%&qwlli6{)!hkc#l_C=)}P;-Ys+NvjP>bYG~cCGCw}YQ1x-0z@w1)u@}^n zTV#|>Z7-{GtbTT=rr=<)~?``+iTxh4l+3|MS-tdVRHm+9w`h0!z=3knV zrSnX_{WmK}KJ?@4(a#30zmF(AmC{eNN7s8Lx}H>x1pMHFk2oys;%$ zvXN_R)m$dd8M|y^7q?Bh-x;&%icdYm3!CL}KR{`PNz%rYL4r4>G&wsZDZV&4BQ-Zs zl!ZZ*N0mu}Jvl$8G&j!xn4o|vkwidc4g-VODMm>dNgXu?8BrcdQ3gqbdKRFR7=zd% z4mA!N3D&gCqT&(>R>!2I%v3Q34HQ1GkiyV!C<@hogF|f<&;XY3{QMLNR)w6z;u4^K eWG+xU(4JF_Y8(t2Y%V}QxHvIf1_}lM%S8a*|2_@? literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_hive/partitioning/identifier=2071/email.csv b/tests/queries/0_stateless/data_hive/partitioning/identifier=2071/email.csv new file mode 100644 index 00000000000..936d995cc64 --- /dev/null +++ b/tests/queries/0_stateless/data_hive/partitioning/identifier=2071/email.csv @@ -0,0 +1,5 @@ +_login_email,_identifier,_first_name,_last_name +laura@example.com,2070,Laura,Grey +craig@example.com,4081,Craig,Johnson +mary@example.com,9346,Mary,Jenkins +jamie@example.com,5079,Jamie,Smith From c6804122cb8785e744b9fa472c67ed95b7525bdf Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 27 Aug 2024 16:52:29 +0200 Subject: [PATCH 026/128] fix shell --- .../queries/0_stateless/03231_hive_partitioning_filtering.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 719fed5bdaa..435f6fe4c4e 100644 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -1,8 +1,8 @@ #!/usr/bin/env bash -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh +. "$CURDIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} --query_id="test_03231_1" --query " SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; From 4eca00a66635b167e35ad58be0beb176710c5b9b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 27 Aug 2024 18:10:41 +0200 Subject: [PATCH 027/128] fix style --- .../0_stateless/03231_hive_partitioning_filtering.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 435f6fe4c4e..396cfc9da26 100644 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -9,7 +9,7 @@ ${CLICKHOUSE_CLIENT} --query_id="test_03231_1" --query " " ${CLICKHOUSE_CLIENT} --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1'; + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' WHERE current_database = currentDatabase(); " ${CLICKHOUSE_CLIENT} --query_id="test_03231_2" --query " @@ -17,7 +17,7 @@ ${CLICKHOUSE_CLIENT} --query_id="test_03231_2" --query " " ${CLICKHOUSE_CLIENT} --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2'; + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' WHERE current_database = currentDatabase(); " ${CLICKHOUSE_CLIENT} --query_id="test_03231_3" --query " @@ -25,5 +25,5 @@ ${CLICKHOUSE_CLIENT} --query_id="test_03231_3" --query " " ${CLICKHOUSE_CLIENT} --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3'; + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3' WHERE current_database = currentDatabase(); " From 2741bf00e4c19d1e548cdda62452b078c7a65a05 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 27 Aug 2024 16:53:14 +0000 Subject: [PATCH 028/128] chmod +x --- tests/queries/0_stateless/03231_hive_partitioning_filtering.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/03231_hive_partitioning_filtering.sh diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh old mode 100644 new mode 100755 From 9133505952ead54df50aeca643e947823560371b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 27 Aug 2024 19:16:05 +0200 Subject: [PATCH 029/128] fix the test --- .../03231_hive_partitioning_filtering.sh | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 396cfc9da26..3a1f51bf0cc 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -4,26 +4,30 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query_id="test_03231_1" --query " +$CLICKHOUSE_LOCAL --query_id="test_03231_1" --query " + SET use_hive_partitioning = 1; +" + +$CLICKHOUSE_LOCAL --query_id="test_03231_1" --query " SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; " -${CLICKHOUSE_CLIENT} --query " +$CLICKHOUSE_LOCAL --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' WHERE current_database = currentDatabase(); " -${CLICKHOUSE_CLIENT} --query_id="test_03231_2" --query " +$CLICKHOUSE_LOCAL --query_id="test_03231_2" --query " SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070; " -${CLICKHOUSE_CLIENT} --query " +$CLICKHOUSE_LOCAL --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' WHERE current_database = currentDatabase(); " -${CLICKHOUSE_CLIENT} --query_id="test_03231_3" --query " +$CLICKHOUSE_LOCAL --query_id="test_03231_3" --query " SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3]; " -${CLICKHOUSE_CLIENT} --query " +$CLICKHOUSE_LOCAL --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3' WHERE current_database = currentDatabase(); " From 60c6eb26100d61b0a1f642a0bf00293725d91408 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 27 Aug 2024 19:42:47 +0200 Subject: [PATCH 030/128] trying to fix the test --- .../03231_hive_partitioning_filtering.sh | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 3a1f51bf0cc..e545f53f257 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -5,29 +5,25 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_LOCAL --query_id="test_03231_1" --query " - SET use_hive_partitioning = 1; -" - -$CLICKHOUSE_LOCAL --query_id="test_03231_1" --query " - SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth'; + SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; " $CLICKHOUSE_LOCAL --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' WHERE current_database = currentDatabase(); + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' AND current_database = currentDatabase(); " $CLICKHOUSE_LOCAL --query_id="test_03231_2" --query " - SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070; + SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; " $CLICKHOUSE_LOCAL --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' WHERE current_database = currentDatabase(); + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' AND current_database = currentDatabase(); " $CLICKHOUSE_LOCAL --query_id="test_03231_3" --query " - SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3]; + SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; " $CLICKHOUSE_LOCAL --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3' WHERE current_database = currentDatabase(); + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3' AND current_database = currentDatabase(); " From dc97bd6b9280b8640058c759357b3c12f520dfed Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 28 Aug 2024 17:22:47 +0000 Subject: [PATCH 031/128] review + testing the code --- src/Core/NamesAndTypes.cpp | 9 ++++ src/Core/NamesAndTypes.h | 1 + .../StorageObjectStorageSource.cpp | 7 ++-- src/Storages/StorageFile.cpp | 9 +++- src/Storages/StorageURL.cpp | 5 +-- src/Storages/VirtualColumnUtils.cpp | 41 +++++++++++-------- src/Storages/VirtualColumnUtils.h | 2 +- 7 files changed, 48 insertions(+), 26 deletions(-) diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 49ab822c738..23c174f9e7c 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -151,6 +151,15 @@ Names NamesAndTypesList::getNames() const return res; } +NameSet NamesAndTypesList::getNameSet() const +{ + NameSet res; + res.reserve(size()); + for (const NameAndTypePair & column : *this) + res.insert(column.name); + return res; +} + DataTypes NamesAndTypesList::getTypes() const { DataTypes res; diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 29f40c45938..7f874172df3 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -100,6 +100,7 @@ public: void getDifference(const NamesAndTypesList & rhs, NamesAndTypesList & deleted, NamesAndTypesList & added) const; Names getNames() const; + NameSet getNameSet() const; DataTypes getTypes() const; /// Remove columns which names are not in the `names`. diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 0d4471e3bda..00155aee4c3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -131,11 +131,10 @@ std::shared_ptr StorageObjectStorageSourc else { ConfigurationPtr copy_configuration = configuration->clone(); - auto keys = configuration->getPaths(); - String partitioning_path = fs::path(configuration->getNamespace()) / keys[0]; - auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, partitioning_path, local_context); + auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, local_context); if (filter_dag) { + auto keys = configuration->getPaths(); std::vector paths; paths.reserve(keys.size()); for (const auto & key : keys) @@ -515,7 +514,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } recursive = key_with_globs == "/**"; - if (auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, key_with_globs, local_context)) + if (auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, local_context)) { VirtualColumnUtils::buildSetsForDAG(*filter_dag, getContext()); filter_expr = std::make_shared(std::move(*filter_dag)); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 639af41c1cd..265a03242b5 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -355,14 +355,18 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user { fs::path user_files_absolute_path = fs::weakly_canonical(user_files_path); fs::path fs_table_path(table_path); + LOG_TRACE(getLogger("testing the paths"), "{} , {}", user_files_absolute_path, fs_table_path); if (fs_table_path.is_relative()) fs_table_path = user_files_absolute_path / fs_table_path; + LOG_TRACE(getLogger("testing the paths"), "fs_table_path = {}", fs_table_path); + Strings paths; /// Do not use fs::canonical or fs::weakly_canonical. /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. String path = fs::absolute(fs_table_path).lexically_normal(); /// Normalize path. + LOG_TRACE(getLogger("testing the paths"), "path = {}", path); bool can_be_directory = true; if (path.find(PartitionedSink::PARTITION_ID_WILDCARD) != std::string::npos) @@ -395,7 +399,10 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user } for (const auto & cur_path : paths) + { checkCreationIsAllowed(context, user_files_absolute_path, cur_path, can_be_directory); + LOG_TRACE(getLogger("checking all paths"), "{}", cur_path); + } return paths; } @@ -1140,7 +1147,7 @@ StorageFileSource::FilesIterator::FilesIterator( { std::optional filter_dag; if (!distributed_processing && !archive_info && !files.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, files[0], context_); + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, context_); if (filter_dag) { diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 572c4f20fa3..ab72d6a3a5a 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -214,10 +214,7 @@ public: std::optional filter_dag; if (!uris.empty()) - { - String partitioning_path = Poco::URI(uris[0]).getPath(); - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, partitioning_path, context); - } + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns, context); if (filter_dag) { diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 788aeb66657..2bd7325a789 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -125,9 +126,18 @@ void filterBlockWithExpression(const ExpressionActionsPtr & actions, Block & blo } } +NamesAndTypesList getCommonVirtualsForFileLikeStorage() +{ + return {{"_path", std::make_shared(std::make_shared())}, + {"_file", std::make_shared(std::make_shared())}, + {"_size", makeNullable(std::make_shared())}, + {"_time", makeNullable(std::make_shared())}, + {"_etag", std::make_shared(std::make_shared())}}; +} + NameSet getVirtualNamesForFileLikeStorage() { - return {"_path", "_file", "_size", "_time", "_etag"}; + return getCommonVirtualsForFileLikeStorage().getNameSet(); } std::unordered_map parseHivePartitioningKeysAndValues(const String & path) @@ -155,8 +165,10 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto { VirtualColumnsDescription desc; - auto add_virtual = [&](const auto & name, const auto & type) + auto add_virtual = [&](const NameAndTypePair & pair) { + const auto & name = pair.getNameInStorage(); + const auto & type = pair.getTypeInStorage(); if (storage_columns.has(name)) { if (!context->getSettingsRef().use_hive_partitioning) @@ -173,11 +185,8 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto desc.addEphemeral(name, type, ""); }; - add_virtual("_path", std::make_shared(std::make_shared())); - add_virtual("_file", std::make_shared(std::make_shared())); - add_virtual("_size", makeNullable(std::make_shared())); - add_virtual("_time", makeNullable(std::make_shared())); - add_virtual("_etag", std::make_shared(std::make_shared())); + for (const auto & item : getCommonVirtualsForFileLikeStorage()) + add_virtual(item); if (context->getSettingsRef().use_hive_partitioning) { @@ -189,9 +198,9 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto if (type == nullptr) type = std::make_shared(); if (type->canBeInsideLowCardinality()) - add_virtual(item.first, std::make_shared(type)); + add_virtual({item.first, std::make_shared(type)}); else - add_virtual(item.first, type); + add_virtual({item.first, type}); } } @@ -228,18 +237,18 @@ static void addFilterDataToVirtualColumns(Block & block, const String & path, si block.getByName("_idx").column->assumeMutableRef().insert(idx); } -std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const String & path, const ContextPtr & context) +std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { if (!predicate || virtual_columns.empty()) return {}; Block block; - std::unordered_map keys; + NameSet common_virtuals; if (context->getSettingsRef().use_hive_partitioning) - keys = parseHivePartitioningKeysAndValues(path); + common_virtuals = getVirtualNamesForFileLikeStorage(); for (const auto & column : virtual_columns) { - if (column.name == "_file" || column.name == "_path" || keys.contains(column.name)) + if (column.name == "_file" || column.name == "_path" || !common_virtuals.contains(column.name)) block.insert({column.type->createColumn(), column.type, column.name}); } @@ -262,10 +271,10 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const auto it = keys.find(column.name); if (it != keys.end()) { - auto c = std::make_shared()->createColumn(); - c->insert(it->second); + auto string_column = std::make_shared()->createColumn(); + string_column->insert(it->second); block.insert({column.type->createColumn(), column.type, column.name}); - partitioning_columns.push_back({c->getPtr(), column.type, column.name}); + partitioning_columns.push_back({string_column->getPtr(), column.type, column.name}); } } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index ecfe44a1956..a9c46569a83 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -75,7 +75,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage( const std::string & sample_path = "", std::optional format_settings_ = std::nullopt); -std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const String & path, const ContextPtr & context); +std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns, const ContextPtr & context); From d6b2a9d5343f77738f4f4f911acf2397a3841f8c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 28 Aug 2024 22:32:44 +0000 Subject: [PATCH 032/128] CLICKHOUSE_LOCAL -> CLIENT --- .../03231_hive_partitioning_filtering.sh | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index e545f53f257..d32d5596110 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -4,26 +4,31 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --query_id="test_03231_1" --query " - SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; +DATA_DIR=$USER_FILES_PATH/$CLICKHOUSE_TEST_UNIQUE_NAME +mkdir -p $DATA_DIR +cp -r $CURDIR/data_hive/ $DATA_DIR + +$CLICKHOUSE_CLIENT --query_id="test_03231_1" --query " + SELECT countDistinct(_path) FROM file('$DATA_DIR/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; " -$CLICKHOUSE_LOCAL --query " +$CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' AND current_database = currentDatabase(); " -$CLICKHOUSE_LOCAL --query_id="test_03231_2" --query " - SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; +$CLICKHOUSE_CLIENT --query_id="test_03231_2" --query " + SELECT countDistinct(_path) FROM file('$DATA_DIR/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; " -$CLICKHOUSE_LOCAL --query " +$CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' AND current_database = currentDatabase(); " -$CLICKHOUSE_LOCAL --query_id="test_03231_3" --query " - SELECT countDistinct(_path) FROM file('$CURDIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; +$CLICKHOUSE_CLIENT --query_id="test_03231_3" --query " + SELECT countDistinct(_path) FROM file('$DATA_DIR/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; " -$CLICKHOUSE_LOCAL --query " +$CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3' AND current_database = currentDatabase(); " +rm -rf $DATA_DIR From edc5d8dd92f18ec9636a910676e7b8f0334ca805 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 28 Aug 2024 23:15:01 +0000 Subject: [PATCH 033/128] fix path --- .../0_stateless/03231_hive_partitioning_filtering.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index d32d5596110..52067299b0c 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -9,7 +9,7 @@ mkdir -p $DATA_DIR cp -r $CURDIR/data_hive/ $DATA_DIR $CLICKHOUSE_CLIENT --query_id="test_03231_1" --query " - SELECT countDistinct(_path) FROM file('$DATA_DIR/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; + SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; " $CLICKHOUSE_CLIENT --query " @@ -17,7 +17,7 @@ $CLICKHOUSE_CLIENT --query " " $CLICKHOUSE_CLIENT --query_id="test_03231_2" --query " - SELECT countDistinct(_path) FROM file('$DATA_DIR/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; + SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; " $CLICKHOUSE_CLIENT --query " @@ -25,7 +25,7 @@ $CLICKHOUSE_CLIENT --query " " $CLICKHOUSE_CLIENT --query_id="test_03231_3" --query " - SELECT countDistinct(_path) FROM file('$DATA_DIR/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; + SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; " $CLICKHOUSE_CLIENT --query " From afc4d08aadd8a5a1ad3b66bf2faada143100b8b6 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:31:05 +0200 Subject: [PATCH 034/128] add no-fasttest tag --- tests/queries/0_stateless/03231_hive_partitioning_filtering.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 52067299b0c..8bd5f10bc4e 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 2adc61c21503e3c55e09c1ecd42833f6508d8584 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 16:39:22 +0200 Subject: [PATCH 035/128] add flush logs --- .../0_stateless/03231_hive_partitioning_filtering.sh | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 8bd5f10bc4e..00763faafb8 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -9,24 +9,27 @@ DATA_DIR=$USER_FILES_PATH/$CLICKHOUSE_TEST_UNIQUE_NAME mkdir -p $DATA_DIR cp -r $CURDIR/data_hive/ $DATA_DIR -$CLICKHOUSE_CLIENT --query_id="test_03231_1" --query " +$CLICKHOUSE_CLIENT --query_id="test_03231_1" --query -nm " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; + SYSTEM FLUSH LOGS; " $CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' AND current_database = currentDatabase(); " -$CLICKHOUSE_CLIENT --query_id="test_03231_2" --query " +$CLICKHOUSE_CLIENT --query_id="test_03231_2" --query -nm " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; + SYSTEM FLUSH LOGS; " $CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' AND current_database = currentDatabase(); " -$CLICKHOUSE_CLIENT --query_id="test_03231_3" --query " +$CLICKHOUSE_CLIENT --query_id="test_03231_3" --query -nm " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; + SYSTEM FLUSH LOGS; " $CLICKHOUSE_CLIENT --query " From 7a879980d8e2deace80b084e8281488995ca23c4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 18:25:11 +0200 Subject: [PATCH 036/128] try to fix tests --- .../0_stateless/03231_hive_partitioning_filtering.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 00763faafb8..1bfb5101ef3 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -9,7 +9,7 @@ DATA_DIR=$USER_FILES_PATH/$CLICKHOUSE_TEST_UNIQUE_NAME mkdir -p $DATA_DIR cp -r $CURDIR/data_hive/ $DATA_DIR -$CLICKHOUSE_CLIENT --query_id="test_03231_1" --query -nm " +$CLICKHOUSE_CLIENT --query_id="test_03231_1" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " @@ -18,7 +18,7 @@ $CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' AND current_database = currentDatabase(); " -$CLICKHOUSE_CLIENT --query_id="test_03231_2" --query -nm " +$CLICKHOUSE_CLIENT --query_id="test_03231_2" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " @@ -27,7 +27,7 @@ $CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' AND current_database = currentDatabase(); " -$CLICKHOUSE_CLIENT --query_id="test_03231_3" --query -nm " +$CLICKHOUSE_CLIENT --query_id="test_03231_3" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " From ec469a117d9cd808a0a4b89d8e5ce22c38b8fe2a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 00:56:35 +0200 Subject: [PATCH 037/128] testing --- .../0_stateless/03203_hive_style_partitioning.reference | 4 ++-- .../0_stateless/03231_hive_partitioning_filtering.sh | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03203_hive_style_partitioning.reference b/tests/queries/0_stateless/03203_hive_style_partitioning.reference index 0fbc1fb556e..bb6a345c6ec 100644 --- a/tests/queries/0_stateless/03203_hive_style_partitioning.reference +++ b/tests/queries/0_stateless/03203_hive_style_partitioning.reference @@ -33,8 +33,8 @@ Cross Elizabeth [1,2,3] 42.42 Array(Int64) LowCardinality(Float64) 101 -2070 -2070 +2071 +2071 b 1 1 diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 1bfb5101ef3..b6a62d3bc33 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -10,7 +10,7 @@ mkdir -p $DATA_DIR cp -r $CURDIR/data_hive/ $DATA_DIR $CLICKHOUSE_CLIENT --query_id="test_03231_1" --query " - SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; + SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT --query " " $CLICKHOUSE_CLIENT --query_id="test_03231_2" --query " - SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; + SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " @@ -28,7 +28,7 @@ $CLICKHOUSE_CLIENT --query " " $CLICKHOUSE_CLIENT --query_id="test_03231_3" --query " - SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; + SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " From 620640a0423155800a6b3a80f15e67eba7614ecb Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 12:58:21 +0200 Subject: [PATCH 038/128] just to test --- .../0_stateless/03231_hive_partitioning_filtering.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index b6a62d3bc33..d24c4e94c08 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -10,7 +10,7 @@ mkdir -p $DATA_DIR cp -r $CURDIR/data_hive/ $DATA_DIR $CLICKHOUSE_CLIENT --query_id="test_03231_1" --query " - SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; + SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 1 SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT --query " " $CLICKHOUSE_CLIENT --query_id="test_03231_2" --query " - SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; + SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 LIMIT 1 SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " @@ -28,7 +28,7 @@ $CLICKHOUSE_CLIENT --query " " $CLICKHOUSE_CLIENT --query_id="test_03231_3" --query " - SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; + SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] LIMIT 1 SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " From e968984d170817a0d9922be862a82411ded21e00 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 14 Aug 2024 16:38:59 +0200 Subject: [PATCH 039/128] More changes --- src/Common/ZooKeeper/IKeeper.cpp | 6 - src/Common/ZooKeeper/IKeeper.h | 10 +- src/Common/ZooKeeper/TestKeeper.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 72 ++--- src/Common/ZooKeeper/ZooKeeperCommon.h | 14 +- src/Coordination/KeeperSnapshotManager.cpp | 21 +- src/Coordination/KeeperStorage.cpp | 291 +++++++++--------- src/Coordination/KeeperStorage.h | 12 +- src/Coordination/ZooKeeperDataReader.cpp | 4 +- src/Coordination/tests/gtest_coordination.cpp | 17 +- 10 files changed, 217 insertions(+), 232 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 7cca262baca..7bf0951817d 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -180,12 +180,6 @@ void SetACLRequest::addRootPath(const String & root_path) { Coordination::addRoo void GetACLRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void SyncRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } -void MultiRequest::addRootPath(const String & root_path) -{ - for (auto & request : requests) - request->addRootPath(root_path); -} - void CreateResponse::removeRootPath(const String & root_path) { Coordination::removeRootPath(path_created, root_path); } void WatchResponse::removeRootPath(const String & root_path) { Coordination::removeRootPath(path, root_path); } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index ce7489a33e5..c95dbd7d7bd 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -391,11 +391,17 @@ struct ReconfigResponse : virtual Response size_t bytesSize() const override { return value.size() + sizeof(stat); } }; +template struct MultiRequest : virtual Request { - Requests requests; + std::vector requests; + + void addRootPath(const String & root_path) override + { + for (auto & request : requests) + request->addRootPath(root_path); + } - void addRootPath(const String & root_path) override; String getPath() const override { return {}; } size_t bytesSize() const override diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 16ea412eb77..af1a3a54593 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -154,7 +154,7 @@ struct TestKeeperReconfigRequest final : ReconfigRequest, TestKeeperRequest std::pair process(TestKeeper::Container & container, int64_t zxid) const override; }; -struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest +struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest { explicit TestKeeperMultiRequest(const Requests & generic_requests) : TestKeeperMultiRequest(std::span(generic_requests)) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 6948e6a3830..8214390a10f 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -882,30 +882,30 @@ size_t ZooKeeperMultiResponse::sizeImpl() const return total_size + Coordination::size(op_num) + Coordination::size(done) + Coordination::size(error_read); } -ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperReconfigRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperSimpleListRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperReconfigRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSimpleListRequest::makeResponse() const { return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperCreateRequest::makeResponse() const { if (not_exists) - return setTime(std::make_shared()); - return setTime(std::make_shared()); + return std::make_shared(); + return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperCheckRequest::makeResponse() const { if (not_exists) - return setTime(std::make_shared()); + return std::make_shared(); - return setTime(std::make_shared()); + return std::make_shared(); } ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const @@ -916,11 +916,12 @@ ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const else response = std::make_shared(requests); - return setTime(std::move(response)); + return std::move(response); } -ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperSetACLRequest::makeResponse() const { return setTime(std::make_shared()); } -ZooKeeperResponsePtr ZooKeeperGetACLRequest::makeResponse() const { return setTime(std::make_shared()); } + +ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperSetACLRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperGetACLRequest::makeResponse() const { return std::make_shared(); } void ZooKeeperSessionIDRequest::writeImpl(WriteBuffer & out) const { @@ -1122,40 +1123,6 @@ std::shared_ptr ZooKeeperRequest::read(ReadBuffer & in) return request; } -ZooKeeperRequest::~ZooKeeperRequest() -{ - if (!request_created_time_ns) - return; - UInt64 elapsed_ns = clock_gettime_ns() - request_created_time_ns; - constexpr UInt64 max_request_time_ns = 1000000000ULL; /// 1 sec - if (max_request_time_ns < elapsed_ns) - { - LOG_TEST(getLogger(__PRETTY_FUNCTION__), "Processing of request xid={} took {} ms", xid, elapsed_ns / 1000000UL); - } -} - -ZooKeeperResponsePtr ZooKeeperRequest::setTime(ZooKeeperResponsePtr response) const -{ - if (request_created_time_ns) - { - response->response_created_time_ns = clock_gettime_ns(); - } - return response; -} - -ZooKeeperResponse::~ZooKeeperResponse() -{ - if (!response_created_time_ns) - return; - UInt64 elapsed_ns = clock_gettime_ns() - response_created_time_ns; - constexpr UInt64 max_request_time_ns = 1000000000ULL; /// 1 sec - if (max_request_time_ns < elapsed_ns) - { - LOG_TEST(getLogger(__PRETTY_FUNCTION__), "Processing of response xid={} took {} ms", xid, elapsed_ns / 1000000UL); - } -} - - ZooKeeperRequestPtr ZooKeeperRequestFactory::get(OpNum op_num) const { auto it = op_num_to_request.find(op_num); @@ -1177,7 +1144,6 @@ void registerZooKeeperRequest(ZooKeeperRequestFactory & factory) factory.registerRequest(num, [] { auto res = std::make_shared(); - res->request_created_time_ns = clock_gettime_ns(); if constexpr (num == OpNum::MultiRead) res->operation_type = ZooKeeperMultiRequest::OperationType::Read; diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index c5096e1e641..75b2ddb0c04 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -23,11 +23,8 @@ struct ZooKeeperResponse : virtual Response { XID xid = 0; - UInt64 response_created_time_ns = 0; - ZooKeeperResponse() = default; ZooKeeperResponse(const ZooKeeperResponse &) = default; - ~ZooKeeperResponse() override; virtual void readImpl(ReadBuffer &) = 0; virtual void writeImpl(WriteBuffer &) const = 0; virtual size_t sizeImpl() const = 0; @@ -50,13 +47,11 @@ struct ZooKeeperRequest : virtual Request bool restored_from_zookeeper_log = false; - UInt64 request_created_time_ns = 0; UInt64 thread_id = 0; String query_id; ZooKeeperRequest() = default; ZooKeeperRequest(const ZooKeeperRequest &) = default; - ~ZooKeeperRequest() override; virtual OpNum getOpNum() const = 0; @@ -73,7 +68,6 @@ struct ZooKeeperRequest : virtual Request static std::shared_ptr read(ReadBuffer & in); virtual ZooKeeperResponsePtr makeResponse() const = 0; - ZooKeeperResponsePtr setTime(ZooKeeperResponsePtr response) const; virtual bool isReadRequest() const = 0; virtual void createLogElements(LogElements & elems) const; @@ -538,7 +532,7 @@ struct ZooKeeperGetACLResponse final : GetACLResponse, ZooKeeperResponse size_t bytesSize() const override { return GetACLResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } }; -struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest +struct ZooKeeperMultiRequest final : MultiRequest, ZooKeeperRequest { OpNum getOpNum() const override; ZooKeeperMultiRequest() = default; @@ -571,12 +565,14 @@ private: struct ZooKeeperMultiResponse : MultiResponse, ZooKeeperResponse { - explicit ZooKeeperMultiResponse(const Requests & requests) + ZooKeeperMultiResponse() = default; + + explicit ZooKeeperMultiResponse(const std::vector & requests) { responses.reserve(requests.size()); for (const auto & request : requests) - responses.emplace_back(dynamic_cast(*request).makeResponse()); + responses.emplace_back(request->makeResponse()); } explicit ZooKeeperMultiResponse(const Responses & responses_) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 54ef0a53cef..69ebaeff9ca 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -72,7 +72,7 @@ namespace writeBinary(node.getData(), out); /// Serialize ACL - writeBinary(node.stats.acl_id, out); + writeBinary(node.acl_id, out); /// Write is_sequential for backwards compatibility if (version < SnapshotVersion::V6) writeBinary(false, out); @@ -109,7 +109,7 @@ namespace if (version >= SnapshotVersion::V1) { - readBinary(node.stats.acl_id, in); + readBinary(node.acl_id, in); } else if (version == SnapshotVersion::V0) { @@ -125,14 +125,14 @@ namespace readBinary(acl.id, in); acls.push_back(acl); } - node.stats.acl_id = acl_map.convertACLs(acls); + node.acl_id = acl_map.convertACLs(acls); } /// Some strange ACLID during deserialization from ZooKeeper - if (node.stats.acl_id == std::numeric_limits::max()) - node.stats.acl_id = 0; + if (node.acl_id == std::numeric_limits::max()) + node.acl_id = 0; - acl_map.addUsage(node.stats.acl_id); + acl_map.addUsage(node.acl_id); if (version < SnapshotVersion::V6) { @@ -455,9 +455,12 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult(new char[stats.data_size]); @@ -522,20 +524,26 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) chassert(!delta.path.empty()); UncommittedNode * uncommitted_node = nullptr; + std::string_view node_path; if (auto it = nodes.find(delta.path); it != nodes.end()) { uncommitted_node = &it->second; + node_path = it->first; } else { if (auto storage_node = tryGetNodeFromStorage(delta.path)) { auto [emplaced_it, _] = nodes.emplace(delta.path, UncommittedNode{.node = std::move(storage_node)}); + node_path = emplaced_it->first; + zxid_to_nodes[0].insert(node_path); uncommitted_node = &emplaced_it->second; } else { auto [emplaced_it, _] = nodes.emplace(delta.path, UncommittedNode{.node = nullptr}); + node_path = emplaced_it->first; + zxid_to_nodes[0].insert(node_path); uncommitted_node = &emplaced_it->second; } } @@ -545,10 +553,6 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) { auto & [node, acls, applied_zxids] = *uncommitted_node; - int64_t last_applied_zxid = 0; - if (!applied_zxids.empty()) - last_applied_zxid = applied_zxids.back(); - if constexpr (std::same_as) { chassert(!node); @@ -579,8 +583,8 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) acls = operation.new_acls; } - if (last_applied_zxid != delta.zxid) - last_applied_zxid = applied_zxids.emplace_back(delta.zxid); + applied_zxids.insert(delta.zxid); + zxid_to_nodes[delta.zxid].insert(node_path); }, delta.operation); } @@ -671,8 +675,8 @@ void KeeperStorage::UncommittedState::rollbackDelta(const Delta & del acls = operation.old_acls; } - if (applied_zxids.back() == delta.zxid) - applied_zxids.pop_back(); + applied_zxids.erase(delta.zxid); + zxid_to_nodes.erase(delta.zxid); }, delta.operation); } @@ -708,14 +712,21 @@ void KeeperStorage::UncommittedState::addDeltas(std::list new_ template void KeeperStorage::UncommittedState::cleanup(int64_t commit_zxid) { - for (auto it = nodes.begin(); it != nodes.end();) + for (const auto & [transaction_zxid, transaction_nodes] : zxid_to_nodes) { - auto & applied_zxids = it->second.applied_zxids; - std::erase_if(applied_zxids, [commit_zxid](auto current_zxid) { return current_zxid <= commit_zxid; }); - if (applied_zxids.empty()) - it = nodes.erase(it); - else - ++it; + if (transaction_zxid > commit_zxid) + break; + + std::cout << transaction_zxid << std::endl; + for (const auto node : transaction_nodes) + { + std::cout << node << std::endl; + auto node_it = nodes.find(node); + chassert(node_it != nodes.end()); + node_it->second.applied_zxids.erase(transaction_zxid); + if (node_it->second.applied_zxids.empty()) + nodes.erase(node_it); + } } for (auto it = session_and_auth.begin(); it != session_and_auth.end();) @@ -815,7 +826,10 @@ std::shared_ptr KeeperStorage::UncommittedS std::shared_ptr node = tryGetNodeFromStorage(path); if (node) - nodes.emplace(std::string{path}, UncommittedNode{node}); + { + auto [node_it, _] = nodes.emplace(std::string{path}, UncommittedNode{.node = node}); + zxid_to_nodes[0].insert(node_it->first); + } return node; } @@ -826,7 +840,7 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) { if (!node_it->second.acls.has_value()) - node_it->second.acls.emplace(storage.acl_map.convertNumber(node_it->second.node->stats.acl_id)); + node_it->second.acls.emplace(storage.acl_map.convertNumber(node_it->second.node->acl_id)); return *node_it->second.acls; } @@ -835,8 +849,9 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef if (node) { - auto [it, inserted] = nodes.emplace(std::string{path}, UncommittedNode{node}); - it->second.acls = storage.acl_map.convertNumber(node->stats.acl_id); + auto [it, inserted] = nodes.emplace(std::string{path}, UncommittedNode{.node = node}); + zxid_to_nodes[0].insert(it->first); + it->second.acls = storage.acl_map.convertNumber(node->acl_id); return *it->second.acls; } @@ -1000,12 +1015,12 @@ Coordination::Error KeeperStorage::commit(std::list deltas) if (operation.version != -1 && operation.version != node_it->value.stats.aversion) onStorageInconsistency(); - acl_map.removeUsage(node_it->value.stats.acl_id); + acl_map.removeUsage(node_it->value.acl_id); uint64_t acl_id = acl_map.convertACLs(operation.new_acls); acl_map.addUsage(acl_id); - container.updateValue(path, [acl_id](Node & node) { node.stats.acl_id = acl_id; }); + container.updateValue(path, [acl_id](Node & node) { node.acl_id = acl_id; }); return Coordination::Error::ZOK; } @@ -1063,7 +1078,7 @@ bool KeeperStorage::createNode( uint64_t acl_id = acl_map.convertACLs(node_acls); acl_map.addUsage(acl_id); - created_node.stats.acl_id = acl_id; + created_node.acl_id = acl_id; created_node.copyStats(stat); created_node.setData(data); @@ -1105,7 +1120,7 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers KeeperStorage::Node prev_node; prev_node.shallowCopy(node_it->value); - acl_map.removeUsage(node_it->value.stats.acl_id); + acl_map.removeUsage(node_it->value.acl_id); if constexpr (use_rocksdb) container.erase(path); @@ -1170,12 +1185,6 @@ auto callOnConcreteRequestType(const Coordination::ZooKeeperRequest & zk_request } } -template -auto callOnConcreteRequestType(const Coordination::Request & request, F function) -{ - return callOnConcreteRequestType(dynamic_cast(request), function); -} - namespace { @@ -1189,7 +1198,7 @@ Coordination::ACLs getNodeACLs(Storage & storage, StringRef path, bool is_local) if (node_it == storage.container.end()) return {}; - return storage.acl_map.convertNumber(node_it->value.stats.acl_id); + return storage.acl_map.convertNumber(node_it->value.acl_id); } return storage.uncommitted_state.getACLs(path); @@ -1306,8 +1315,8 @@ template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* storage */, std::list /* deltas */) { - auto response = zk_request.makeResponse(); - dynamic_cast(*response).path = zk_request.path; + auto response = std::make_shared(); + response->path = zk_request.path; return response; } @@ -1437,14 +1446,16 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperCreateRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); + std::shared_ptr response = zk_request.not_exists + ? std::make_shared() + : std::make_shared(); if (deltas.empty()) { - response.path_created = zk_request.getPath(); - response.error = Coordination::Error::ZOK; - return response_ptr; + chassert(zk_request.not_exists); + response->path_created = zk_request.getPath(); + response->error = Coordination::Error::ZOK; + return response; } std::string created_path; @@ -1459,13 +1470,13 @@ Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperCreateRe if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } - response.path_created = std::move(created_path); - response.error = Coordination::Error::ZOK; - return response_ptr; + response->path_created = std::move(created_path); + response->error = Coordination::Error::ZOK; + return response; } /// CREATE Request /// @@ -1500,25 +1511,26 @@ std::list preprocess( } template -Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); +Coordination::ZooKeeperResponsePtr +processImpl(const Coordination::ZooKeeperGetRequest & zk_request, Storage & storage, std::list deltas) +{ + auto response = std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } if (zk_request.path == Coordination::keeper_config_path) { - response.data = serializeClusterConfig( + response->data = serializeClusterConfig( storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig()); - response.error = Coordination::Error::ZOK; - return response_ptr; + response->error = Coordination::Error::ZOK; + return response; } auto & container = storage.container; @@ -1526,19 +1538,19 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetR if (node_it == container.end()) { if constexpr (local) - response.error = Coordination::Error::ZNONODE; + response->error = Coordination::Error::ZNONODE; else onStorageInconsistency(); } else { - node_it->value.setResponseStat(response.stat); + node_it->value.setResponseStat(response->stat); auto data = node_it->value.getData(); - response.data = std::string(data); - response.error = Coordination::Error::ZOK; + response->data = std::string(data); + response->error = Coordination::Error::ZOK; } - return response_ptr; + return response; } template @@ -1654,13 +1666,13 @@ std::list preprocess( } template -Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperRemoveRequest & zk_request, Storage & storage, std::list deltas) +Coordination::ZooKeeperResponsePtr +process(const Coordination::ZooKeeperRemoveRequest & /*zk_request*/, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); - response.error = storage.commit(std::move(deltas)); - return response_ptr; + response->error = storage.commit(std::move(deltas)); + return response; } /// REMOVE Request /// @@ -1691,16 +1703,17 @@ std::list preprocess( } template -Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); +Coordination::ZooKeeperResponsePtr +processImpl(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, std::list deltas) +{ + auto response = std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } @@ -1709,17 +1722,17 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperExis if (node_it == container.end()) { if constexpr (local) - response.error = Coordination::Error::ZNONODE; + response->error = Coordination::Error::ZNONODE; else onStorageInconsistency(); } else { - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; } - return response_ptr; + return response; } template @@ -1808,23 +1821,22 @@ Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSetReque { auto & container = storage.container; - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } auto node_it = container.find(zk_request.path); if (node_it == container.end()) onStorageInconsistency(); - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; - return response_ptr; + return response; } /// SET Request /// @@ -1856,15 +1868,14 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } @@ -1874,7 +1885,7 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList if (node_it == container.end()) { if constexpr (local) - response.error = Coordination::Error::ZNONODE; + response->error = Coordination::Error::ZNONODE; else onStorageInconsistency(); } @@ -1892,7 +1903,7 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList return node_it->value.getChildren(); }; const auto & children = get_children(); - response.names.reserve(children.size()); + response->names.reserve(children.size()); const auto add_child = [&](const auto & child) { @@ -1929,17 +1940,17 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList if (add_child(child)) { if constexpr (Storage::use_rocksdb) - response.names.push_back(child.first); + response->names.push_back(child.first); else - response.names.push_back(child.toString()); + response->names.push_back(child.toString()); } } - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; } - return response_ptr; + return response; } template @@ -2002,22 +2013,23 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperCheckRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); + std::shared_ptr response = zk_request.not_exists + ? std::make_shared() + : std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } const auto on_error = [&]([[maybe_unused]] const auto error_code) { if constexpr (local) - response.error = error_code; + response->error = error_code; else onStorageInconsistency(); }; @@ -2030,7 +2042,7 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperChec if (node_it != container.end() && (zk_request.version == -1 || zk_request.version == node_it->value.stats.version)) on_error(Coordination::Error::ZNODEEXISTS); else - response.error = Coordination::Error::ZOK; + response->error = Coordination::Error::ZOK; } else { @@ -2039,10 +2051,10 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperChec else if (zk_request.version != -1 && zk_request.version != node_it->value.stats.version) on_error(Coordination::Error::ZBADVERSION); else - response.error = Coordination::Error::ZOK; + response->error = Coordination::Error::ZOK; } - return response_ptr; + return response; } template @@ -2151,8 +2163,13 @@ template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); + std::shared_ptr response; + if (zk_request.getOpNum() == Coordination::OpNum::Multi) + response = std::make_shared(); + else + response = std::make_shared(); + + response->responses.reserve(zk_request.requests.size()); const auto & subrequests = zk_request.requests; @@ -2162,41 +2179,40 @@ process(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storag { for (size_t i = 0; i < subrequests.size(); ++i) { - response.responses[i] = std::make_shared(); - response.responses[i]->error = failed_multi->error_codes[i]; + response->responses.push_back(std::make_shared()); + response->responses[i]->error = failed_multi->error_codes[i]; } - return response_ptr; + return response; } - for (size_t i = 0; i < subrequests.size(); ++i) + for (const auto & multi_subrequest : subrequests) { std::list subdeltas = getSubdeltas(deltas); - response.responses[i] = callOnConcreteRequestType( - *subrequests[i], [&](const auto & subrequest) { return process(subrequest, storage, std::move(subdeltas)); }); + response->responses.push_back(callOnConcreteRequestType( + *multi_subrequest, [&](const auto & subrequest) { return process(subrequest, storage, std::move(subdeltas)); })); } - response.error = Coordination::Error::ZOK; - return response_ptr; + response->error = Coordination::Error::ZOK; + return response; } template Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperMultiRequest & zk_request, Storage & storage, std::list deltas) { ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); + response->responses.reserve(zk_request.requests.size()); - const auto & subrequests = zk_request.requests; - for (size_t i = 0; i < subrequests.size(); ++i) + for (const auto & multi_subrequest : zk_request.requests) { std::list subdeltas = getSubdeltas(deltas); - response.responses[i] = callOnConcreteRequestType( - *subrequests[i], [&](const auto & subrequest) { return processLocal(subrequest, storage, std::move(subdeltas)); }); + response->responses.push_back(callOnConcreteRequestType( + *multi_subrequest, [&](const auto & subrequest) { return processLocal(subrequest, storage, std::move(subdeltas)); })); } - response.error = Coordination::Error::ZOK; - return response_ptr; + response->error = Coordination::Error::ZOK; + return response; } template @@ -2227,8 +2243,6 @@ std::list preprocess( uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - if (zk_request.scheme != "digest" || std::count(zk_request.data.begin(), zk_request.data.end(), ':') != 1) return {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; @@ -2254,15 +2268,14 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr -process(const Coordination::ZooKeeperAuthRequest & zk_request, Storage & storage, std::list deltas) +process(const Coordination::ZooKeeperAuthRequest & /*zk_request*/, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) - auth_response.error = result; + response->error = result; - return response_ptr; + return response; } /// AUTH Request /// @@ -2319,7 +2332,7 @@ std::list preprocess( std::list new_deltas{ {zk_request.path, zxid, - KeeperStorageBase::SetACLDelta{std::move(node_acls), uncommitted_state.getACLs(zk_request.path), zk_request.version}}, + KeeperStorageBase::SetACLDelta{uncommitted_state.getACLs(zk_request.path), std::move(node_acls), zk_request.version}}, {zk_request.path, zxid, std::move(update_stat_delta)}}; digest = storage.calculateNodesDigest(digest, new_deltas); @@ -2331,22 +2344,21 @@ template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSetACLRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } auto node_it = storage.container.find(zk_request.path); if (node_it == storage.container.end()) onStorageInconsistency(); - node_it->value.setResponseStat(response.stat); - response.error = Coordination::Error::ZOK; + node_it->value.setResponseStat(response->stat); + response->error = Coordination::Error::ZOK; - return response_ptr; + return response; } /// SETACL Request /// @@ -2376,15 +2388,14 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetACLRequest & zk_request, Storage & storage, std::list deltas) { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request.makeResponse(); - Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); + auto response = std::make_shared(); if constexpr (!local) { if (const auto result = storage.commit(std::move(deltas)); result != Coordination::Error::ZOK) { - response.error = result; - return response_ptr; + response->error = result; + return response; } } @@ -2393,17 +2404,17 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperGetA if (node_it == container.end()) { if constexpr (local) - response.error = Coordination::Error::ZNONODE; + response->error = Coordination::Error::ZNONODE; else onStorageInconsistency(); } else { - node_it->value.setResponseStat(response.stat); - response.acl = storage.acl_map.convertNumber(node_it->value.stats.acl_id); + node_it->value.setResponseStat(response->stat); + response->acl = storage.acl_map.convertNumber(node_it->value.acl_id); } - return response_ptr; + return response; } template @@ -2594,6 +2605,8 @@ void KeeperStorage::preprocessRequest( uncommitted_state.applyDeltas(new_deltas); uncommitted_state.addDeltas(std::move(new_deltas)); + + uncommitted_state.cleanup(getZXID()); }); if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special @@ -2645,8 +2658,6 @@ void KeeperStorage::preprocessRequest( }; callOnConcreteRequestType(*zk_request, preprocess_request); - - uncommitted_state.cleanup(getZXID()); } template @@ -2761,6 +2772,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process { Coordination::ZooKeeperResponsePtr response; + uncommitted_state.tryGetNodeFromStorage("/node_with_acl"); if (is_local) { chassert(zk_request->isReadRequest()); @@ -2781,6 +2793,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process std::lock_guard lock(storage_mutex); response = process(concrete_zk_request, *this, std::move(deltas)); } + uncommitted_state.tryGetNodeFromStorage("/node_with_acl"); /// Watches for this requests are added to the watches lists if (zk_request->has_watch) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 6326e1c76a9..1c3c77e140b 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -41,8 +42,6 @@ struct NodeStats uint32_t data_size{0}; - uint64_t acl_id = 0; /// 0 -- no ACL by default - /// void copyStats(const Coordination::Stat & stat); bool isEphemeral() const @@ -146,6 +145,8 @@ private: struct KeeperRocksNodeInfo { NodeStats stats; + uint64_t acl_id = 0; /// 0 -- no ACL by default + /// dummy interface for test void addChild(StringRef) {} auto getChildren() const @@ -181,6 +182,7 @@ struct KeeperRocksNode : public KeeperRocksNodeInfo void shallowCopy(const KeeperRocksNode & other) { stats = other.stats; + acl_id = other.acl_id; if (stats.data_size != 0) { data = std::unique_ptr(new char[stats.data_size]); @@ -221,6 +223,8 @@ struct KeeperMemNode std::unique_ptr data{nullptr}; mutable uint64_t cached_digest = 0; + uint64_t acl_id = 0; /// 0 -- no ACL by default + KeeperMemNode() = default; KeeperMemNode & operator=(const KeeperMemNode & other); @@ -505,11 +509,12 @@ public: std::unordered_map>>> session_and_auth; std::unordered_set closed_sessions; + using ZxidToNodes = std::map>; struct UncommittedNode { std::shared_ptr node{nullptr}; std::optional acls{}; - std::vector applied_zxids{}; + std::unordered_set applied_zxids{}; }; struct Hash @@ -536,6 +541,7 @@ public: }; mutable std::unordered_map nodes; + mutable ZxidToNodes zxid_to_nodes; mutable std::mutex deltas_mutex; std::list deltas TSA_GUARDED_BY(deltas_mutex); diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 08f8cc67aac..2ac0009ace1 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -105,7 +105,7 @@ int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log String data; Coordination::read(data, in); node.setData(data); - Coordination::read(node.stats.acl_id, in); + Coordination::read(node.acl_id, in); /// Deserialize stat Coordination::read(node.stats.czxid, in); @@ -136,7 +136,7 @@ int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log if (ephemeral_owner != 0) storage.ephemerals[ephemeral_owner].insert(path); - storage.acl_map.addUsage(node.stats.acl_id); + storage.acl_map.addUsage(node.acl_id); } Coordination::read(path, in); count++; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index d39031773cd..52cd68131e6 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1,6 +1,7 @@ #include #include +#include "base/defines.h" #include "config.h" #if USE_NURAFT @@ -1540,7 +1541,7 @@ void addNode(Storage & storage, const std::string & path, const std::string & da using Node = typename Storage::Node; Node node{}; node.setData(data); - node.setEphemeralOwner(ephemeral_owner); + node.stats.setEphemeralOwner(ephemeral_owner); storage.container.insertOrReplace(path, node); auto child_it = storage.container.find(path); auto child_path = DB::getBaseNodeName(child_it->key); @@ -1549,7 +1550,7 @@ void addNode(Storage & storage, const std::string & path, const std::string & da [&](auto & parent) { parent.addChild(child_path); - parent.increaseNumChildren(); + parent.stats.increaseNumChildren(); }); } @@ -1570,7 +1571,7 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) addNode(storage, "/hello1", "world", 1); addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; - storage.zxid = 2; + TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; storage.ephemerals[3] = {"/hello2"}; storage.ephemerals[1] = {"/hello1"}; storage.getSessionID(130); @@ -1601,7 +1602,7 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) EXPECT_EQ(restored_storage->container.getValue("/hello1").getData(), "world"); EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); - EXPECT_EQ(restored_storage->zxid, 2); + EXPECT_EQ(restored_storage->getZXID(), 2); EXPECT_EQ(restored_storage->ephemerals.size(), 2); EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); @@ -2534,7 +2535,7 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) addNode(storage, "/hello1", "world", 1); addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; - storage.zxid = 2; + TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; storage.ephemerals[3] = {"/hello2"}; storage.ephemerals[1] = {"/hello1"}; storage.getSessionID(130); @@ -2561,7 +2562,7 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) EXPECT_EQ(restored_storage->container.getValue("/hello1").getData(), "world"); EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); - EXPECT_EQ(restored_storage->zxid, 2); + EXPECT_EQ(restored_storage->getZXID(), 2); EXPECT_EQ(restored_storage->ephemerals.size(), 2); EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); @@ -2755,7 +2756,7 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotEqual) for (size_t j = 0; j < 3333; ++j) storage.getSessionID(130 * j); - DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); + DB::KeeperStorageSnapshot snapshot(&storage, storage.getZXID()); auto buf = manager.serializeSnapshotToBuffer(snapshot); @@ -3257,7 +3258,7 @@ TYPED_TEST(CoordinationTest, TestCheckNotExistsRequest) create_path("/test_node"); auto node_it = storage.container.find("/test_node"); ASSERT_NE(node_it, storage.container.end()); - auto node_version = node_it->value.version; + auto node_version = node_it->value.stats.version; { SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS"); From 79fc8d67ad9665d011cee06855d9bd4ce1065597 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 2 Sep 2024 11:44:21 +0200 Subject: [PATCH 040/128] More fixes --- src/Coordination/KeeperSnapshotManager.cpp | 19 +++++++--- src/Coordination/KeeperStorage.cpp | 42 +++++++++++----------- src/Coordination/KeeperStorage.h | 2 -- utils/keeper-bench/Runner.cpp | 1 + 4 files changed, 35 insertions(+), 29 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 81c72c1beb0..a5cb8218f1f 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -471,12 +471,21 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult::UncommittedState::addDeltas(std::list new_ template void KeeperStorage::UncommittedState::cleanup(int64_t commit_zxid) { - for (const auto & [transaction_zxid, transaction_nodes] : zxid_to_nodes) + for (auto it = zxid_to_nodes.begin(); it != zxid_to_nodes.end(); it = zxid_to_nodes.erase(it)) { + const auto & [transaction_zxid, transaction_nodes] = *it; + if (transaction_zxid > commit_zxid) break; - std::cout << transaction_zxid << std::endl; for (const auto node : transaction_nodes) { - std::cout << node << std::endl; auto node_it = nodes.find(node); - chassert(node_it != nodes.end()); + if (node_it == nodes.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing expected uncommitted node '{}'", node); + node_it->second.applied_zxids.erase(transaction_zxid); if (node_it->second.applied_zxids.empty()) nodes.erase(node_it); @@ -1295,7 +1297,6 @@ bool checkAuth(const T & /*zk_request*/, Storage & /*storage*/, int64_t /*sessio /// Default implementations /// /// HEARTBEAT Request /// - template Coordination::ZooKeeperResponsePtr process( const Coordination::ZooKeeperHeartbeatRequest & zk_request, @@ -1306,11 +1307,9 @@ Coordination::ZooKeeperResponsePtr process( response_ptr->error = storage.commit(std::move(deltas)); return response_ptr; } - /// HEARTBEAT Request /// /// SYNC Request /// - template Coordination::ZooKeeperResponsePtr process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* storage */, std::list /* deltas */) @@ -1319,7 +1318,6 @@ process(const Coordination::ZooKeeperSyncRequest & zk_request, Storage & /* stor response->path = zk_request.path; return response; } - /// SYNC Request /// /// CREATE Request /// @@ -1793,18 +1791,19 @@ std::list preprocess( if (zk_request.version != -1 && zk_request.version != node->stats.version) return {KeeperStorageBase::Delta{zxid, Coordination::Error::ZBADVERSION}}; + new_deltas.emplace_back( + zk_request.path, + zxid, + KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = zk_request.data, .version = zk_request.version}); + KeeperStorageBase::UpdateNodeStatDelta node_delta(*node); node_delta.version = zk_request.version; auto & new_stats = node_delta.new_stats; new_stats.version++; new_stats.mzxid = zxid; new_stats.mtime = time; - + new_stats.data_size = static_cast(zk_request.data.size()); new_deltas.emplace_back(zk_request.path, zxid, std::move(node_delta)); - new_deltas.emplace_back( - zk_request.path, - zxid, - KeeperStorageBase::UpdateNodeDataDelta{.old_data = std::string{node->getData()}, .new_data = zk_request.data, .version = zk_request.version}); auto parent_path = parentNodePath(zk_request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); @@ -1868,7 +1867,9 @@ std::list preprocess( template Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperListRequest & zk_request, Storage & storage, std::list deltas) { - auto response = std::make_shared(); + std::shared_ptr response = zk_request.getOpNum() == Coordination::OpNum::SimpleList + ? std::make_shared() + : std::make_shared(); if constexpr (!local) { @@ -2215,17 +2216,16 @@ Coordination::ZooKeeperResponsePtr processLocal(const Coordination::ZooKeeperMul return response; } -template KeeperStorageBase::ResponsesForSessions processWatches( - const Coordination::ZooKeeperMultiRequest & zk_request, typename Storage::Watches & watches, typename Storage::Watches & list_watches) + const Coordination::ZooKeeperMultiRequest & zk_request, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) { - typename Storage::ResponsesForSessions result; + KeeperStorageBase::ResponsesForSessions result; const auto & subrequests = zk_request.requests; for (const auto & generic_request : subrequests) { auto responses = callOnConcreteRequestType( - generic_request, [&](const auto & subrequest) { return processWatches(subrequest, watches, list_watches); }); + *generic_request, [&](const auto & subrequest) { return processWatches(subrequest, watches, list_watches); }); result.insert(result.end(), responses.begin(), responses.end()); } return result; @@ -2506,7 +2506,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, con auto updated_node_it = updated_nodes.find(delta.path); if (updated_node_it == updated_nodes.end()) { - node = std::make_shared(); + node = std::make_shared(); node->shallowCopy(*uncommitted_state.getNode(delta.path)); current_digest -= node->getDigest(delta.path); updated_nodes.emplace(delta.path, node); @@ -2772,7 +2772,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process { Coordination::ZooKeeperResponsePtr response; - uncommitted_state.tryGetNodeFromStorage("/node_with_acl"); if (is_local) { chassert(zk_request->isReadRequest()); @@ -2793,7 +2792,6 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process std::lock_guard lock(storage_mutex); response = process(concrete_zk_request, *this, std::move(deltas)); } - uncommitted_state.tryGetNodeFromStorage("/node_with_acl"); /// Watches for this requests are added to the watches lists if (zk_request->has_watch) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 1c3c77e140b..684f75c4bdf 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -357,8 +357,6 @@ public: NodeStats old_stats; NodeStats new_stats; - int32_t old_seq_num; - int32_t new_seq_num; int32_t version{-1}; }; diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 59761d827e1..0584e9e34c8 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -1114,6 +1114,7 @@ void Runner::runBenchmarkFromLog() else { request_from_log->connection = get_zookeeper_connection(request_from_log->session_id); + request_from_log->executor_id %= concurrency; push_request(std::move(*request_from_log)); } From 24f4e87f8bf6fbd424895ab75ca71583c18ce320 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 3 Sep 2024 15:20:22 +0200 Subject: [PATCH 041/128] revert debugging in tests --- .../0_stateless/03231_hive_partitioning_filtering.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index d24c4e94c08..5c63370f38e 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -10,7 +10,7 @@ mkdir -p $DATA_DIR cp -r $CURDIR/data_hive/ $DATA_DIR $CLICKHOUSE_CLIENT --query_id="test_03231_1" --query " - SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' LIMIT 1 SETTINGS use_hive_partitioning=1; + SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " @@ -19,7 +19,7 @@ $CLICKHOUSE_CLIENT --query " " $CLICKHOUSE_CLIENT --query_id="test_03231_2" --query " - SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 LIMIT 1 SETTINGS use_hive_partitioning=1; + SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " @@ -28,7 +28,7 @@ $CLICKHOUSE_CLIENT --query " " $CLICKHOUSE_CLIENT --query_id="test_03231_3" --query " - SELECT _path FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] LIMIT 1 SETTINGS use_hive_partitioning=1; + SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " From 21f9669836fc39052c98ea8aa54f5d82888af100 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 3 Sep 2024 15:41:43 +0200 Subject: [PATCH 042/128] empty commit From f688b903dbc83c36a1d1526df4d50519790f0eb9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 3 Sep 2024 15:58:22 +0200 Subject: [PATCH 043/128] empty commit From f1377b0b4abde46ef4c1cc162e33ad539692bfac Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 3 Sep 2024 14:10:28 +0000 Subject: [PATCH 044/128] Fix uniq and GROUP BY for JSON/Dynamic types --- src/Columns/ColumnDynamic.cpp | 16 ++ src/Columns/ColumnObject.cpp | 244 +++++++++++------- src/Columns/ColumnObject.h | 5 + .../03231_dynamic_uniq_group_by.reference | 5 + .../03231_dynamic_uniq_group_by.sql | 15 ++ .../03232_json_uniq_group_by.reference | 12 + .../0_stateless/03232_json_uniq_group_by.sql | 39 +++ 7 files changed, 236 insertions(+), 100 deletions(-) create mode 100644 tests/queries/0_stateless/03231_dynamic_uniq_group_by.reference create mode 100644 tests/queries/0_stateless/03231_dynamic_uniq_group_by.sql create mode 100644 tests/queries/0_stateless/03232_json_uniq_group_by.reference create mode 100644 tests/queries/0_stateless/03232_json_uniq_group_by.sql diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 9b55879a4f0..0b1dc3c363a 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -816,6 +816,22 @@ void ColumnDynamic::updateHashWithValue(size_t n, SipHash & hash) const return; } + /// If it's not null we update hash with the type name and the actual value. + + /// If value in this row is in shared variant, deserialize type and value and + /// update hash with it. + if (discr == getSharedVariantDiscriminator()) + { + auto value = getSharedVariant().getDataAt(variant_col.offsetAt(n)); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + hash.update(type->getName()); + auto tmp_column = type->createColumn(); + type->getDefaultSerialization()->deserializeBinary(*tmp_column, buf, getFormatSettings()); + tmp_column->updateHashWithValue(0, hash); + return; + } + hash.update(variant_info.variant_names[discr]); variant_col.getVariantByGlobalDiscriminator(discr).updateHashWithValue(variant_col.offsetAt(n), hash); } diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index e397b03b69e..920b0384448 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -47,15 +47,20 @@ ColumnObject::ColumnObject( , statistics(statistics_) { typed_paths.reserve(typed_paths_.size()); + sorted_typed_paths.reserve(typed_paths_.size()); for (auto & [path, column] : typed_paths_) - typed_paths[path] = std::move(column); + { + auto it = typed_paths.emplace(path, std::move(column)).first; + sorted_typed_paths.push_back(it->first); + } dynamic_paths.reserve(dynamic_paths_.size()); dynamic_paths_ptrs.reserve(dynamic_paths_.size()); for (auto & [path, column] : dynamic_paths_) { - dynamic_paths[path] = std::move(column); - dynamic_paths_ptrs[path] = assert_cast(dynamic_paths[path].get()); + auto it = dynamic_paths.emplace(path, std::move(column)).first; + dynamic_paths_ptrs[path] = assert_cast(it->second.get()); + sorted_dynamic_paths.insert(it->first); } } @@ -68,7 +73,8 @@ ColumnObject::ColumnObject( { if (!column->empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected non-empty typed path column in ColumnObject constructor"); - typed_paths[path] = std::move(column); + auto it = typed_paths.emplace(path, std::move(column)).first; + sorted_typed_paths.push_back(it->first); } MutableColumns paths_and_values; @@ -129,13 +135,8 @@ std::string ColumnObject::getName() const ss << "Object("; ss << "max_dynamic_paths=" << global_max_dynamic_paths; ss << ", max_dynamic_types=" << max_dynamic_types; - std::vector sorted_typed_paths; - sorted_typed_paths.reserve(typed_paths.size()); - for (const auto & [path, column] : typed_paths) - sorted_typed_paths.push_back(path); - std::sort(sorted_typed_paths.begin(), sorted_typed_paths.end()); for (const auto & path : sorted_typed_paths) - ss << ", " << path << " " << typed_paths.at(path)->getName(); + ss << ", " << path << " " << typed_paths.find(path)->second->getName(); ss << ")"; return ss.str(); } @@ -260,6 +261,7 @@ ColumnDynamic * ColumnObject::tryToAddNewDynamicPath(std::string_view path) new_dynamic_column->insertManyDefaults(size()); auto it = dynamic_paths.emplace(path, std::move(new_dynamic_column)).first; auto it_ptr = dynamic_paths_ptrs.emplace(path, assert_cast(it->second.get())).first; + sorted_dynamic_paths.insert(it->first); return it_ptr->second; } @@ -288,8 +290,9 @@ void ColumnObject::setDynamicPaths(const std::vector & paths) auto new_dynamic_column = ColumnDynamic::create(max_dynamic_types); if (size) new_dynamic_column->insertManyDefaults(size); - dynamic_paths[path] = std::move(new_dynamic_column); - dynamic_paths_ptrs[path] = assert_cast(dynamic_paths[path].get()); + auto it = dynamic_paths.emplace(path, std::move(new_dynamic_column)).first; + dynamic_paths_ptrs[path] = assert_cast(it->second.get()); + sorted_dynamic_paths.insert(it->first); } } @@ -658,39 +661,61 @@ void ColumnObject::popBack(size_t n) StringRef ColumnObject::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const { StringRef res(begin, 0); - // Serialize all paths and values in binary format. + /// First serialize values from typed paths in sorted order. They are the same for all instances of this column. + for (auto path : sorted_typed_paths) + { + auto data_ref = typed_paths.find(path)->second->serializeValueIntoArena(n, arena, begin); + res.data = data_ref.data - res.size; + res.size += data_ref.size; + } + + /// Second, serialize paths and values in bunary format from dynamic paths and shared data in sorted by path order. + /// Calculate total number of paths to serialize and write it. const auto & shared_data_offsets = getSharedDataOffsets(); size_t offset = shared_data_offsets[static_cast(n) - 1]; size_t end = shared_data_offsets[static_cast(n)]; - size_t num_paths = typed_paths.size() + dynamic_paths.size() + (end - offset); + size_t num_paths = (end - offset); + /// Don't serialize Nulls from dynamic paths. + for (const auto & [_, column] : dynamic_paths) + num_paths += !column->isNullAt(n); + char * pos = arena.allocContinue(sizeof(size_t), begin); memcpy(pos, &num_paths, sizeof(size_t)); res.data = pos - res.size; res.size += sizeof(size_t); - /// Serialize paths and values from typed paths. - for (const auto & [path, column] : typed_paths) - { - size_t path_size = path.size(); - pos = arena.allocContinue(sizeof(size_t) + path_size, begin); - memcpy(pos, &path_size, sizeof(size_t)); - memcpy(pos + sizeof(size_t), path.data(), path_size); - auto data_ref = column->serializeValueIntoArena(n, arena, begin); - res.data = data_ref.data - res.size - sizeof(size_t) - path_size; - res.size += data_ref.size + sizeof(size_t) + path_size; - } - /// Serialize paths and values from dynamic paths. - for (const auto & [path, column] : dynamic_paths) - { - WriteBufferFromOwnString buf; - getDynamicSerialization()->serializeBinary(*column, n, buf, getFormatSettings()); - serializePathAndValueIntoArena(arena, begin, path, buf.str(), res); - } - - /// Serialize paths and values from shared data. + auto dynamic_paths_it = sorted_dynamic_paths.begin(); auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); for (size_t i = offset; i != end; ++i) - serializePathAndValueIntoArena(arena, begin, shared_data_paths->getDataAt(i), shared_data_values->getDataAt(i), res); + { + auto path = shared_data_paths->getDataAt(i).toView(); + /// Paths in shared data are sorted. Serialize all paths from dynamic paths that go before this path in sorted order. + while (dynamic_paths_it != sorted_dynamic_paths.end() && *dynamic_paths_it < path) + { + const auto * dynamic_column = dynamic_paths_ptrs.find(*dynamic_paths_it)->second; + /// Don't serialize Nulls. + if (!dynamic_column->isNullAt(n)) + { + WriteBufferFromOwnString buf; + getDynamicSerialization()->serializeBinary(*dynamic_column, n, buf, getFormatSettings()); + serializePathAndValueIntoArena(arena, begin, StringRef(*dynamic_paths_it), buf.str(), res); + } + ++dynamic_paths_it; + } + serializePathAndValueIntoArena(arena, begin, StringRef(path), shared_data_values->getDataAt(i), res); + } + + /// Serialize all remaining paths in dynamic paths. + for (; dynamic_paths_it != sorted_dynamic_paths.end(); ++dynamic_paths_it) + { + const auto * dynamic_column = dynamic_paths_ptrs.find(*dynamic_paths_it)->second; + if (!dynamic_column->isNullAt(n)) + { + WriteBufferFromOwnString buf; + getDynamicSerialization()->serializeBinary(*dynamic_column, n, buf, getFormatSettings()); + serializePathAndValueIntoArena(arena, begin, StringRef(*dynamic_paths_it), buf.str(), res); + } + } return res; } @@ -711,70 +736,49 @@ void ColumnObject::serializePathAndValueIntoArena(DB::Arena & arena, const char const char * ColumnObject::deserializeAndInsertFromArena(const char * pos) { size_t current_size = size(); - /// Deserialize paths and values and insert them into typed paths, dynamic paths or shared data. - /// Serialized paths could be unsorted, so we will have to sort all paths that will be inserted into shared data. - std::vector> paths_and_values_for_shared_data; + /// First deserialize typed paths. They come first. + for (auto path : sorted_typed_paths) + pos = typed_paths.find(path)->second->deserializeAndInsertFromArena(pos); + + /// Second deserialize all other paths and values and insert them into dynamic paths or shared data. auto num_paths = unalignedLoad(pos); pos += sizeof(size_t); + const auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); for (size_t i = 0; i != num_paths; ++i) { auto path_size = unalignedLoad(pos); pos += sizeof(size_t); std::string_view path(pos, path_size); pos += path_size; - /// Check if it's a typed path. In this case we should use - /// deserializeAndInsertFromArena of corresponding column. - if (auto typed_it = typed_paths.find(path); typed_it != typed_paths.end()) + /// Deserialize binary value and try to insert it to dynamic paths or shared data. + auto value_size = unalignedLoad(pos); + pos += sizeof(size_t); + std::string_view value(pos, value_size); + pos += value_size; + /// Check if we have this path in dynamic paths. + if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) { - pos = typed_it->second->deserializeAndInsertFromArena(pos); + ReadBufferFromMemory buf(value.data(), value.size()); + getDynamicSerialization()->deserializeBinary(*dynamic_it->second, buf, getFormatSettings()); } - /// If it's not a typed path, deserialize binary value and try to insert it - /// to dynamic paths or shared data. + /// Try to add a new dynamic path. + else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) + { + ReadBufferFromMemory buf(value.data(), value.size()); + getDynamicSerialization()->deserializeBinary(*dynamic_path_column, buf, getFormatSettings()); + } + /// Limit on dynamic paths is reached, add this path to shared data. + /// Serialized paths are sorted, so we can insert right away. else { - auto value_size = unalignedLoad(pos); - pos += sizeof(size_t); - std::string_view value(pos, value_size); - pos += value_size; - /// Check if we have this path in dynamic paths. - if (auto dynamic_it = dynamic_paths.find(path); dynamic_it != dynamic_paths.end()) - { - ReadBufferFromMemory buf(value.data(), value.size()); - getDynamicSerialization()->deserializeBinary(*dynamic_it->second, buf, getFormatSettings()); - } - /// Try to add a new dynamic path. - else if (auto * dynamic_path_column = tryToAddNewDynamicPath(path)) - { - ReadBufferFromMemory buf(value.data(), value.size()); - getDynamicSerialization()->deserializeBinary(*dynamic_path_column, buf, getFormatSettings()); - } - /// Limit on dynamic paths is reached, add this path to shared data later. - else - { - paths_and_values_for_shared_data.emplace_back(path, value); - } + shared_data_paths->insertData(path.data(), path.size()); + shared_data_values->insertData(value.data(), value.size()); } } - /// Sort and insert all paths from paths_and_values_for_shared_data into shared data. - std::sort(paths_and_values_for_shared_data.begin(), paths_and_values_for_shared_data.end()); - const auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); - for (const auto & [path, value] : paths_and_values_for_shared_data) - { - shared_data_paths->insertData(path.data(), path.size()); - shared_data_values->insertData(value.data(), value.size()); - } - getSharedDataOffsets().push_back(shared_data_paths->size()); - /// Insert default value in all remaining typed and dynamic paths. - - for (auto & [_, column] : typed_paths) - { - if (column->size() == current_size) - column->insertDefault(); - } - + /// Insert default value in all remaining dynamic paths. for (auto & [_, column] : dynamic_paths_ptrs) { if (column->size() == current_size) @@ -786,6 +790,11 @@ const char * ColumnObject::deserializeAndInsertFromArena(const char * pos) const char * ColumnObject::skipSerializedInArena(const char * pos) const { + /// First, skip all values of typed paths; + for (auto path : sorted_typed_paths) + pos = typed_paths.find(path)->second->skipSerializedInArena(pos); + + /// Second, skip all other paths and values. auto num_paths = unalignedLoad(pos); pos += sizeof(size_t); for (size_t i = 0; i != num_paths; ++i) @@ -794,15 +803,8 @@ const char * ColumnObject::skipSerializedInArena(const char * pos) const pos += sizeof(size_t); std::string_view path(pos, path_size); pos += path_size; - if (auto typed_it = typed_paths.find(path); typed_it != typed_paths.end()) - { - pos = typed_it->second->skipSerializedInArena(pos); - } - else - { - auto value_size = unalignedLoad(pos); - pos += sizeof(size_t) + value_size; - } + auto value_size = unalignedLoad(pos); + pos += sizeof(size_t) + value_size; } return pos; @@ -810,11 +812,51 @@ const char * ColumnObject::skipSerializedInArena(const char * pos) const void ColumnObject::updateHashWithValue(size_t n, SipHash & hash) const { - for (const auto & [_, column] : typed_paths) - column->updateHashWithValue(n, hash); - for (const auto & [_, column] : dynamic_paths_ptrs) - column->updateHashWithValue(n, hash); - shared_data->updateHashWithValue(n, hash); + for (auto path : sorted_typed_paths) + typed_paths.find(path)->second->updateHashWithValue(n, hash); + + /// The hash of the object in row should not depend on the way we store paths (in dynamic paths or in shared data) + /// and should be the same for the same objects. To support it we update hash with path and its value (if not null) in + /// sorted by path order from both dynamic paths and shared data. + const auto [shared_data_paths, shared_data_values] = getSharedDataPathsAndValues(); + const auto & shared_data_offsets = getSharedDataOffsets(); + size_t start = shared_data_offsets[static_cast(n) - 1]; + size_t end = shared_data_offsets[static_cast(n)]; + auto dynamic_paths_it = sorted_dynamic_paths.begin(); + for (size_t i = start; i != end; ++i) + { + auto path = shared_data_paths->getDataAt(i).toView(); + /// Paths in shared data are sorted. Update hash with all paths from dynamic paths that go before this path in sorted order. + while (dynamic_paths_it != sorted_dynamic_paths.end() && *dynamic_paths_it < path) + { + const auto * dynamic_column = dynamic_paths_ptrs.find(*dynamic_paths_it)->second; + if (!dynamic_column->isNullAt(n)) + { + hash.update(*dynamic_paths_it); + dynamic_column->updateHashWithValue(n, hash); + } + ++dynamic_paths_it; + } + + /// Deserialize value in temporary column to get its hash. + auto value = shared_data_values->getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto tmp_column = ColumnDynamic::create(); + getDynamicSerialization()->deserializeBinary(*tmp_column, buf, getFormatSettings()); + hash.update(path); + tmp_column->updateHashWithValue(0, hash); + } + + /// Iterate over all remaining paths in dynamic paths. + for (; dynamic_paths_it != sorted_dynamic_paths.end(); ++dynamic_paths_it) + { + const auto * dynamic_column = dynamic_paths_ptrs.find(*dynamic_paths_it)->second; + if (!dynamic_column->isNullAt(n)) + { + hash.update(*dynamic_paths_it); + dynamic_column->updateHashWithValue(n, hash); + } + } } WeakHash32 ColumnObject::getWeakHash32() const @@ -1328,8 +1370,9 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou { if (dynamic_paths.size() < max_dynamic_paths) { - dynamic_paths.emplace(path, ColumnDynamic::create(max_dynamic_types)); - dynamic_paths_ptrs.emplace(path, assert_cast(dynamic_paths.find(path)->second.get())); + auto it = dynamic_paths.emplace(path, ColumnDynamic::create(max_dynamic_types)).first; + dynamic_paths_ptrs.emplace(path, assert_cast(it->second.get())); + sorted_dynamic_paths.insert(it->first); } /// Add all remaining paths into shared data statistics until we reach its max size; else if (new_statistics.shared_data_paths_statistics.size() < Statistics::MAX_SHARED_DATA_STATISTICS_SIZE) @@ -1343,8 +1386,9 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou { for (const auto & [path, _] : path_to_total_number_of_non_null_values) { - dynamic_paths[path] = ColumnDynamic::create(max_dynamic_types); - dynamic_paths_ptrs[path] = assert_cast(dynamic_paths[path].get()); + auto it = dynamic_paths.emplace(path, ColumnDynamic::create(max_dynamic_types)).first; + dynamic_paths_ptrs[path] = assert_cast(it->second.get()); + sorted_dynamic_paths.insert(it->first); } } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index f530ed29ef3..c7f282d9079 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -238,10 +238,15 @@ private: /// Map path -> column for paths with explicitly specified types. /// This set of paths is constant and cannot be changed. PathToColumnMap typed_paths; + /// Sorted list of typed paths. Used to avoid sorting paths every time in some methods. + std::vector sorted_typed_paths; /// Map path -> column for dynamically added paths. All columns /// here are Dynamic columns. This set of paths can be extended /// during inerts into the column. PathToColumnMap dynamic_paths; + /// Sorted list of dynamic paths. Used to avoid sorting paths every time in some methods. + std::set sorted_dynamic_paths; + /// Store and use pointers to ColumnDynamic to avoid virtual calls. /// With hundreds of dynamic paths these virtual calls are noticeable. PathToDynamicColumnPtrMap dynamic_paths_ptrs; diff --git a/tests/queries/0_stateless/03231_dynamic_uniq_group_by.reference b/tests/queries/0_stateless/03231_dynamic_uniq_group_by.reference new file mode 100644 index 00000000000..aafbd72ebc4 --- /dev/null +++ b/tests/queries/0_stateless/03231_dynamic_uniq_group_by.reference @@ -0,0 +1,5 @@ +4 +5 [1,2,3] +5 2020-01-01 +5 42 +5 Hello diff --git a/tests/queries/0_stateless/03231_dynamic_uniq_group_by.sql b/tests/queries/0_stateless/03231_dynamic_uniq_group_by.sql new file mode 100644 index 00000000000..fe052027f56 --- /dev/null +++ b/tests/queries/0_stateless/03231_dynamic_uniq_group_by.sql @@ -0,0 +1,15 @@ +set allow_experimental_dynamic_type = 1; +drop table if exists test; +create table test (d Dynamic(max_types=2)) engine=Memory; +insert into test values (42), ('Hello'), ([1,2,3]), ('2020-01-01'); +insert into test values ('Hello'), ([1,2,3]), ('2020-01-01'), (42); +insert into test values ([1,2,3]), ('2020-01-01'), (42), ('Hello'); +insert into test values ('2020-01-01'), (42), ('Hello'), ([1,2,3]); +insert into test values (42); +insert into test values ('Hello'); +insert into test values ([1,2,3]); +insert into test values ('2020-01-01'); + +select uniqExact(d) from test; +select count(), d from test group by d order by d; +drop table test; diff --git a/tests/queries/0_stateless/03232_json_uniq_group_by.reference b/tests/queries/0_stateless/03232_json_uniq_group_by.reference new file mode 100644 index 00000000000..1fc82458807 --- /dev/null +++ b/tests/queries/0_stateless/03232_json_uniq_group_by.reference @@ -0,0 +1,12 @@ +11 +6 {"a":0,"b":"Hello"} +6 {"a":0,"b":[{"f":"42"}]} +6 {"a":0,"c":"Hello"} +6 {"a":0,"c":["1","2","3"]} +6 {"a":0,"d":"2020-01-01"} +6 {"a":0,"d":["1","2","3"]} +6 {"a":0,"e":"2020-01-01"} +6 {"a":0,"e":[{"f":"42"}]} +5 {"a":42,"b":"Hello","c":["1","2","3"],"d":"2020-01-01","e":[{"f":"42"}]} +5 {"a":42,"b":[{"f":"42"}],"c":"Hello","d":["1","2","3"],"e":"2020-01-01"} +12 {"a":42} diff --git a/tests/queries/0_stateless/03232_json_uniq_group_by.sql b/tests/queries/0_stateless/03232_json_uniq_group_by.sql new file mode 100644 index 00000000000..5d39390d6e1 --- /dev/null +++ b/tests/queries/0_stateless/03232_json_uniq_group_by.sql @@ -0,0 +1,39 @@ +set allow_experimental_json_type = 1; +drop table if exists test; +create table test (json JSON(a UInt32, max_dynamic_paths=2)) engine=Memory; +insert into test values ('{"a" : 42, "b" : "Hello", "c" : [1, 2, 3], "d" : "2020-01-01", "e" : [{"f" : 42}]}'); +insert into test values ('{"b" : "Hello", "c" : [1, 2, 3], "d" : "2020-01-01", "e" : [{"f" : 42}], "a" : 42}'); +insert into test values ('{"c" : [1, 2, 3], "d" : "2020-01-01", "e" : [{"f" : 42}], "a" : 42, "b" : "Hello"}'); +insert into test values ('{"d" : "2020-01-01", "e" : [{"f" : 42}], "a" : 42, "b" : "Hello", "c" : [1, 2, 3]}'); +insert into test values ('{"e" : [{"f" : 42}], "a" : 42, "b" : "Hello", "c" : [1, 2, 3], "d" : "2020-01-01"}'); +insert into test values ('{"a" : 42}'), ('{"b" : "Hello"}'), ('{"c" : [1, 2, 3]}'), ('{"d" : "2020-01-01"}'), ('{"e" : [{"f" : 42}]}'); +insert into test values ('{"b" : "Hello"}'), ('{"c" : [1, 2, 3]}'), ('{"d" : "2020-01-01"}'), ('{"e" : [{"f" : 42}]}'), ('{"a" : 42}'); +insert into test values ('{"c" : [1, 2, 3]}'), ('{"d" : "2020-01-01"}'), ('{"e" : [{"f" : 42}]}'), ('{"a" : 42}'), ('{"b" : "Hello"}'); +insert into test values ('{"d" : "2020-01-01"}'), ('{"e" : [{"f" : 42}]}'), ('{"a" : 42}'), ('{"b" : "Hello"}'), ('{"c" : [1, 2, 3]}'); +insert into test values ('{"e" : [{"f" : 42}]}'), ('{"a" : 42}'), ('{"b" : "Hello"}'), ('{"c" : [1, 2, 3]}'), ('{"d" : "2020-01-01"}'); +insert into test values ('{"a" : 42}'); +insert into test values ('{"b" : "Hello"}'); +insert into test values ('{"c" : [1, 2, 3]}'); +insert into test values ('{"d" : "2020-01-01"}'); +insert into test values ('{"e" : [{"f" : 42}]}'); + +insert into test values ('{"a" : 42, "c" : "Hello", "d" : [1, 2, 3], "e" : "2020-01-01", "b" : [{"f" : 42}]}'); +insert into test values ('{"c" : "Hello", "d" : [1, 2, 3], "e" : "2020-01-01", "b" : [{"f" : 42}], "a" : 42}'); +insert into test values ('{"d" : [1, 2, 3], "e" : "2020-01-01", "b" : [{"f" : 42}], "a" : 42, "c" : "Hello"}'); +insert into test values ('{"e" : "2020-01-01", "b" : [{"f" : 42}], "a" : 42, "c" : "Hello", "d" : [1, 2, 3]}'); +insert into test values ('{"b" : [{"f" : 42}], "a" : 42, "c" : "Hello", "d" : [1, 2, 3], "e" : "2020-01-01"}'); +insert into test values ('{"a" : 42}'), ('{"c" : "Hello"}'), ('{"d" : [1, 2, 3]}'), ('{"e" : "2020-01-01"}'), ('{"b" : [{"f" : 42}]}'); +insert into test values ('{"c" : "Hello"}'), ('{"d" : [1, 2, 3]}'), ('{"e" : "2020-01-01"}'), ('{"b" : [{"f" : 42}]}'), ('{"a" : 42}'); +insert into test values ('{"d" : [1, 2, 3]}'), ('{"e" : "2020-01-01"}'), ('{"b" : [{"f" : 42}]}'), ('{"a" : 42}'), ('{"c" : "Hello"}'); +insert into test values ('{"e" : "2020-01-01"}'), ('{"b" : [{"f" : 42}]}'), ('{"a" : 42}'), ('{"c" : "Hello"}'), ('{"d" : [1, 2, 3]}'); +insert into test values ('{"b" : [{"f" : 42}]}'), ('{"a" : 42}'), ('{"c" : "Hello"}'), ('{"d" : [1, 2, 3]}'), ('{"e" : "2020-01-01"}'); +insert into test values ('{"a" : 42}'); +insert into test values ('{"c" : "Hello"}'); +insert into test values ('{"d" : [1, 2, 3]}'); +insert into test values ('{"e" : "2020-01-01"}'); +insert into test values ('{"b" : [{"f" : 42}]}'); + +select uniqExact(json) from test; +select count(), json from test group by json order by toString(json); + +drop table test; From a44b3d02681e5aeb4f1584483928a920b5843bcf Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 3 Sep 2024 17:31:07 +0000 Subject: [PATCH 045/128] Fix sorted typed paths --- src/Columns/ColumnObject.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 920b0384448..d3af9812b5c 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -53,6 +53,7 @@ ColumnObject::ColumnObject( auto it = typed_paths.emplace(path, std::move(column)).first; sorted_typed_paths.push_back(it->first); } + std::sort(sorted_typed_paths.begin(), sorted_typed_paths.end()); dynamic_paths.reserve(dynamic_paths_.size()); dynamic_paths_ptrs.reserve(dynamic_paths_.size()); @@ -69,6 +70,7 @@ ColumnObject::ColumnObject( : max_dynamic_paths(max_dynamic_paths_), global_max_dynamic_paths(max_dynamic_paths_), max_dynamic_types(max_dynamic_types_) { typed_paths.reserve(typed_paths_.size()); + sorted_typed_paths.reserve(typed_paths_.size()); for (auto & [path, column] : typed_paths_) { if (!column->empty()) @@ -77,6 +79,8 @@ ColumnObject::ColumnObject( sorted_typed_paths.push_back(it->first); } + std::sort(sorted_typed_paths.begin(), sorted_typed_paths.end()); + MutableColumns paths_and_values; paths_and_values.emplace_back(ColumnString::create()); paths_and_values.emplace_back(ColumnString::create()); From 9633563fbdbf4aa391c89e12dd5802b5640d0ca5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 3 Sep 2024 10:07:01 +0200 Subject: [PATCH 046/128] Fix --- src/Coordination/KeeperSnapshotManager.cpp | 4 +- src/Coordination/KeeperStateMachine.cpp | 6 +- src/Coordination/KeeperStateMachine.h | 2 +- src/Coordination/KeeperStorage.cpp | 210 +++++++++++------- src/Coordination/KeeperStorage.h | 15 +- src/Coordination/ZooKeeperDataReader.cpp | 2 +- src/Coordination/tests/gtest_coordination.cpp | 28 +-- utils/keeper-data-dumper/main.cpp | 14 +- 8 files changed, 162 insertions(+), 119 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a5cb8218f1f..dd643ee4c06 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -441,7 +441,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult::deserialize(SnapshotDeserializationResultpos(0); deserialization_result.cluster_config = ClusterConfig::deserialize(*buffer); } + + storage.updateStats(); } template diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 526a38e4f9e..b3c5465fee1 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -182,11 +182,11 @@ void assertDigest( } template -struct TSA_SCOPED_LOCKABLE LockGuardWithStats final +struct LockGuardWithStats final { using LockType = std::conditional_t, std::unique_lock>; LockType lock; - explicit LockGuardWithStats(SharedMutex & mutex) TSA_ACQUIRE(mutex) + explicit LockGuardWithStats(SharedMutex & mutex) { Stopwatch watch; LockType l(mutex); @@ -194,7 +194,7 @@ struct TSA_SCOPED_LOCKABLE LockGuardWithStats final lock = std::move(l); } - ~LockGuardWithStats() TSA_RELEASE() = default; + ~LockGuardWithStats() = default; }; } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 8de14ba301b..7e6830ec1ba 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -203,7 +203,7 @@ public: // This should be used only for tests or keeper-data-dumper because it violates // TSA -- we can't acquire the lock outside of this class or return a storage under lock // in a reasonable way. - Storage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS + Storage & getStorageUnsafe() { return *storage; } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 60872e2fdc0..0add7787ee2 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -112,6 +112,22 @@ bool fixupACL( return valid_found; } +void unregisterEphemeralPath(KeeperStorageBase::Ephemerals & ephemerals, int64_t session_id, const std::string & path, bool throw_if_missing) +{ + auto ephemerals_it = ephemerals.find(session_id); + if (ephemerals_it == ephemerals.end()) + { + if (throw_if_missing) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path {}", session_id, path); + + return; + } + + ephemerals_it->second.erase(path); + if (ephemerals_it->second.empty()) + ephemerals.erase(ephemerals_it); +} + KeeperStorageBase::ResponsesForSessions processWatchesImpl( const String & path, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches, Coordination::Event event_type) { @@ -489,6 +505,7 @@ void KeeperStorage::initializeSystemNodes() } } + updateStats(); initialized = true; } @@ -612,7 +629,7 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool if (is_local) { - std::shared_lock lock(storage.storage_mutex); + std::shared_lock lock(storage.auth_mutex); return check_auth(storage.session_and_auth[session_id]); } @@ -620,6 +637,7 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool if (closed_sessions.contains(session_id)) return false; + std::shared_lock lock(storage.auth_mutex); if (check_auth(storage.session_and_auth[session_id])) return true; @@ -631,7 +649,6 @@ bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool if (check_auth(auth_it->second)) return true; - std::lock_guard lock(storage.storage_mutex); return check_auth(storage.session_and_auth[session_id]); } @@ -777,8 +794,9 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb // we need to undo ephemeral mapping modifications // CreateNodeDelta added ephemeral for session id -> we need to remove it // RemoveNodeDelta removed ephemeral for session id -> we need to add it back - for (auto & delta : rollback_deltas) + for (auto delta_it = rollback_deltas.rbegin(); delta_it != rollback_deltas.rend(); ++delta_it) { + const auto & delta = *delta_it; if (!delta.path.empty()) { std::visit( @@ -787,14 +805,17 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb if constexpr (std::same_as) { if (operation.stat.ephemeralOwner != 0) - storage.unregisterEphemeralPath(operation.stat.ephemeralOwner, delta.path); + { + std::lock_guard lock(storage.ephemeral_mutex); + unregisterEphemeralPath(storage.uncommitted_state.ephemerals, operation.stat.ephemeralOwner, delta.path, /*throw_if_missing=*/false); + } } else if constexpr (std::same_as) { if (operation.stat.ephemeralOwner() != 0) { - std::lock_guard lock(storage.ephemerals_mutex); - storage.ephemerals[operation.stat.ephemeralOwner()].emplace(delta.path); + std::lock_guard lock(storage.ephemeral_mutex); + storage.uncommitted_state.ephemerals[operation.stat.ephemeralOwner()].emplace(delta.path); } } }, @@ -802,7 +823,7 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb rollbackDelta(delta); } - else if (auto * add_auth = std::get_if(&delta.operation)) + else if (const auto * add_auth = std::get_if(&delta.operation)) { auto & uncommitted_auth = session_and_auth[add_auth->session_id]; if (uncommitted_auth.back().second == add_auth->auth_id) @@ -812,7 +833,7 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb session_and_auth.erase(add_auth->session_id); } } - else if (auto * close_session = std::get_if(&delta.operation)) + else if (const auto * close_session = std::get_if(&delta.operation)) { closed_sessions.erase(close_session->session_id); } @@ -875,16 +896,16 @@ void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t se else auth_ptr = auth.second.get(); - func(*auth_ptr); + if (!auth_ptr->scheme.empty()) + func(*auth_ptr); } }; - { - std::lock_guard lock(storage.storage_mutex); - // for committed - if (auto auth_it = storage.session_and_auth.find(session_id); auth_it != storage.session_and_auth.end()) - call_for_each_auth(auth_it->second); - } + /// both committed and uncommitted need to be under the lock to avoid fetching the same AuthID from both committed and uncommitted state + std::shared_lock lock(storage.auth_mutex); + // for committed + if (auto auth_it = storage.session_and_auth.find(session_id); auth_it != storage.session_and_auth.end()) + call_for_each_auth(auth_it->second); // for uncommitted if (auto auth_it = session_and_auth.find(session_id); auth_it != session_and_auth.end()) @@ -1034,6 +1055,7 @@ Coordination::Error KeeperStorage::commit(std::list deltas) } else if constexpr (std::same_as) { + std::lock_guard auth_lock{auth_mutex}; session_and_auth[operation.session_id].emplace_back(std::move(*operation.auth_id)); return Coordination::Error::ZOK; } @@ -1104,6 +1126,15 @@ bool KeeperStorage::createNode( addDigest(map_key->getMapped()->value, map_key->getKey().toView()); } + + if (stat.ephemeralOwner != 0) + { + ++committed_ephemeral_nodes; + std::lock_guard lock(ephemeral_mutex); + committed_ephemerals[stat.ephemeralOwner].emplace(path); + unregisterEphemeralPath(uncommitted_state.ephemerals, stat.ephemeralOwner, path, /*throw_if_missing=*/false); + } + return true; }; @@ -1133,7 +1164,6 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers [child_basename = getBaseNodeName(node_it->key)](KeeperMemNode & parent) { parent.removeChild(child_basename); - chassert(parent.stats.numChildren() == static_cast(parent.getChildren().size())); } ); @@ -1141,6 +1171,14 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers removeDigest(prev_node, path); } + + if (prev_node.stats.ephemeralOwner() != 0) + { + --committed_ephemeral_nodes; + std::lock_guard lock(ephemeral_mutex); + unregisterEphemeralPath(committed_ephemerals, prev_node.stats.ephemeralOwner(), path, /*throw_if_missing=*/true); + } + return true; } @@ -1248,19 +1286,6 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int6 return false; } -template -void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) -{ - std::lock_guard ephemerals_lock(ephemerals_mutex); - auto ephemerals_it = ephemerals.find(session_id); - if (ephemerals_it == ephemerals.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Session {} is missing ephemeral path", session_id); - - ephemerals_it->second.erase(path); - if (ephemerals_it->second.empty()) - ephemerals.erase(ephemerals_it); -} - /// Default implementations /// template T, typename Storage> Coordination::ZooKeeperResponsePtr @@ -1398,8 +1423,8 @@ std::list preprocess( if (zk_request.is_ephemeral) { - std::lock_guard lock(storage.ephemerals_mutex); - storage.ephemerals[session_id].emplace(path_created); + std::lock_guard lock(storage.ephemeral_mutex); + storage.uncommitted_state.ephemerals[session_id].emplace(path_created); } int32_t parent_cversion = zk_request.parent_cversion; @@ -1436,7 +1461,6 @@ std::list preprocess( zxid, typename Storage::CreateNodeDelta{stat, std::move(node_acls), zk_request.data}); - digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } @@ -1606,24 +1630,29 @@ std::list preprocess( auto parent_path = parentNodePath(zk_request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); - KeeperStorageBase::UpdateNodeStatDelta update_parent_delta(*parent_node); + std::optional update_parent_delta; + if (parent_node) + update_parent_delta.emplace(*parent_node); const auto add_parent_update_delta = [&] { + if (!update_parent_delta) + return; + new_deltas.emplace_back( std::string{parent_path}, zxid, - std::move(update_parent_delta) + std::move(*update_parent_delta) ); }; const auto update_parent_pzxid = [&]() { - if (!parent_node) + if (!update_parent_delta) return; - if (update_parent_delta.old_stats.pzxid < zxid) - update_parent_delta.new_stats.pzxid = zxid; + if (update_parent_delta->old_stats.pzxid < zxid) + update_parent_delta->new_stats.pzxid = zxid; }; auto node = storage.uncommitted_state.getNode(zk_request.path); @@ -1645,8 +1674,9 @@ std::list preprocess( if (zk_request.restored_from_zookeeper_log) update_parent_pzxid(); - ++update_parent_delta.new_stats.cversion; - update_parent_delta.new_stats.decreaseNumChildren(); + chassert(update_parent_delta); + ++update_parent_delta->new_stats.cversion; + update_parent_delta->new_stats.decreaseNumChildren(); add_parent_update_delta(); new_deltas.emplace_back( @@ -1656,7 +1686,11 @@ std::list preprocess( zk_request.version, node->stats, storage.uncommitted_state.getACLs(zk_request.path), std::string{node->getData()}}); if (node->stats.isEphemeral()) - storage.unregisterEphemeralPath(node->stats.ephemeralOwner(), zk_request.path); + { + std::lock_guard ephemeral_lock(storage.ephemeral_mutex); + /// try deleting the ephemeral node from the uncommitted state + unregisterEphemeralPath(storage.uncommitted_state.ephemerals, node->stats.ephemeralOwner(), zk_request.path, /*throw_if_missing=*/false); + } digest = storage.calculateNodesDigest(digest, new_deltas); @@ -1676,12 +1710,6 @@ process(const Coordination::ZooKeeperRemoveRequest & /*zk_request*/, Storage & s /// REMOVE Request /// /// EXISTS Request /// -template -bool checkAuth(const Coordination::ZooKeeperExistsRequest & zk_request, Storage & storage, int64_t session_id, bool is_local) -{ - return storage.checkACL(zk_request.getPath(), Coordination::ACL::Read, session_id, is_local); -} - template std::list preprocess( const Coordination::ZooKeeperExistsRequest & zk_request, @@ -2438,7 +2466,7 @@ void KeeperStorage::finalize() finalized = true; - ephemerals.clear(); + committed_ephemerals.clear(); watches.clear(); list_watches.clear(); @@ -2612,33 +2640,54 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { { - std::lock_guard lock(ephemerals_mutex); - auto session_ephemerals = ephemerals.find(session_id); - if (session_ephemerals != ephemerals.end()) + const auto process_ephemerals_for_session = [&](auto & current_ephemerals) { - for (const auto & ephemeral_path : session_ephemerals->second) + auto session_ephemerals = current_ephemerals.find(session_id); + if (session_ephemerals != current_ephemerals.end()) { - auto parent_node_path = parentNodePath(ephemeral_path); - auto parent_node = uncommitted_state.getNode(parent_node_path); - UpdateNodeStatDelta parent_update_delta(*parent_node); - ++parent_update_delta.new_stats.cversion; - parent_update_delta.new_stats.decreaseNumChildren(); - new_deltas.emplace_back - ( - parent_node_path.toString(), - new_last_zxid, - std::move(parent_update_delta) - ); + std::unordered_map parent_updates; + for (const auto & ephemeral_path : session_ephemerals->second) + { + auto node = uncommitted_state.getNode(ephemeral_path); - auto node = uncommitted_state.getNode(ephemeral_path); - new_deltas.emplace_back( - ephemeral_path, - transaction->zxid, - RemoveNodeDelta{.stat = node->stats, .acls = uncommitted_state.getACLs(ephemeral_path), .data = std::string{node->getData()}}); + /// maybe the node is deleted or recreated with different session_id in the uncommitted state + if (!node || node->stats.ephemeralOwner() != session_id) + continue; + + auto parent_node_path = parentNodePath(ephemeral_path).toView(); + + auto parent_update_it = parent_updates.find(parent_node_path); + if (parent_update_it == parent_updates.end()) + { + auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}); + std::tie(parent_update_it, std::ignore) = parent_updates.emplace(parent_node_path, *parent_node); + } + + auto & parent_update_delta = parent_update_it->second; + ++parent_update_delta.new_stats.cversion; + parent_update_delta.new_stats.decreaseNumChildren(); + + new_deltas.emplace_back( + ephemeral_path, + transaction->zxid, + RemoveNodeDelta{.stat = node->stats, .acls = uncommitted_state.getACLs(ephemeral_path), .data = std::string{node->getData()}}); + } + + for (auto & [parent_path, parent_update_delta] : parent_updates) + { + new_deltas.emplace_back + ( + std::string{parent_path}, + new_last_zxid, + std::move(parent_update_delta) + ); + } } - - ephemerals.erase(session_ephemerals); - } + }; + std::lock_guard ephemeral_lock(ephemeral_mutex); + process_ephemerals_for_session(committed_ephemerals); + process_ephemerals_for_session(uncommitted_state.ephemerals); + uncommitted_state.ephemerals.erase(session_id); } new_deltas.emplace_back(transaction->zxid, CloseSessionDelta{session_id}); @@ -2739,6 +2788,9 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process { std::lock_guard lock(storage_mutex); commit(std::move(deltas)); + } + { + std::lock_guard lock(auth_mutex); auto auth_it = session_and_auth.find(session_id); if (auth_it != session_and_auth.end()) session_and_auth.erase(auth_it); @@ -3087,8 +3139,8 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnStrin buf << "0x" << getHexUIntLowercase(session_id) << "\n"; } - buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCountLocked() << "):\n"; - for (const auto & [session_id, ephemeral_paths] : ephemerals) + buf << "Sessions with Ephemerals (" << getSessionWithEphemeralNodesCount() << "):\n"; + for (const auto & [session_id, ephemeral_paths] : committed_ephemerals) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; write_str_set(ephemeral_paths); @@ -3127,13 +3179,7 @@ uint64_t KeeperStorage::getTotalWatchesCount() const template uint64_t KeeperStorage::getSessionWithEphemeralNodesCount() const { - return getSessionWithEphemeralNodesCountLocked(); -} - -template -uint64_t KeeperStorage::getSessionWithEphemeralNodesCountLocked() const -{ - return ephemerals.size(); + return committed_ephemerals.size(); } template @@ -3145,11 +3191,7 @@ uint64_t KeeperStorage::getSessionsWithWatchesCount() const template uint64_t KeeperStorage::getTotalEphemeralNodesCount() const { - uint64_t ret = 0; - for (const auto & [session_id, nodes] : ephemerals) - ret += nodes.size(); - - return ret; + return committed_ephemeral_nodes; } template diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 684f75c4bdf..13f9631d2c3 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -474,9 +474,10 @@ public: int64_t session_id_counter{1}; + mutable SharedMutex auth_mutex; SessionAndAuth session_and_auth; - mutable std::shared_mutex storage_mutex; + mutable SharedMutex storage_mutex; /// Main hashtable with nodes. Contain all information about data. /// All other structures expect session_and_timeout can be restored from /// container. @@ -541,6 +542,8 @@ public: mutable std::unordered_map nodes; mutable ZxidToNodes zxid_to_nodes; + Ephemerals ephemerals; + mutable std::mutex deltas_mutex; std::list deltas TSA_GUARDED_BY(deltas_mutex); KeeperStorage & storage; @@ -570,11 +573,10 @@ public: bool checkACL(StringRef path, int32_t permissions, int64_t session_id, bool is_local); - void unregisterEphemeralPath(int64_t session_id, const std::string & path); - - mutable std::mutex ephemerals_mutex; + std::mutex ephemeral_mutex; /// Mapping session_id -> set of ephemeral nodes paths - Ephemerals ephemerals; + Ephemerals committed_ephemerals; + size_t committed_ephemeral_nodes{0}; /// Expiration queue for session, allows to get dead sessions at some point of time SessionExpiryQueue session_expiry_queue; @@ -687,7 +689,6 @@ public: /// Get all dead sessions std::vector getDeadSessions() const; - void updateStats(); const Stats & getStorageStats() const; @@ -713,8 +714,6 @@ public: void recalculateStats(); private: - uint64_t getSessionWithEphemeralNodesCountLocked() const; - void removeDigest(const Node & node, std::string_view path); void addDigest(const Node & node, std::string_view path); }; diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 2ac0009ace1..5783a803aa8 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -134,7 +134,7 @@ int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, LoggerPtr log storage.container.insertOrReplace(path, node); if (ephemeral_owner != 0) - storage.ephemerals[ephemeral_owner].insert(path); + storage.committed_ephemerals[ephemeral_owner].insert(path); storage.acl_map.addUsage(node.acl_id); } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 52cd68131e6..b2ba80e5624 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1572,8 +1572,8 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; - storage.ephemerals[3] = {"/hello2"}; - storage.ephemerals[1] = {"/hello1"}; + storage.committed_ephemerals[3] = {"/hello2"}; + storage.committed_ephemerals[1] = {"/hello1"}; storage.getSessionID(130); storage.getSessionID(130); @@ -1603,9 +1603,9 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); EXPECT_EQ(restored_storage->getZXID(), 2); - EXPECT_EQ(restored_storage->ephemerals.size(), 2); - EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); - EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); + EXPECT_EQ(restored_storage->committed_ephemerals.size(), 2); + EXPECT_EQ(restored_storage->committed_ephemerals[3].size(), 1); + EXPECT_EQ(restored_storage->committed_ephemerals[1].size(), 1); EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); } @@ -2028,7 +2028,7 @@ TYPED_TEST(CoordinationTest, TestEphemeralNodeRemove) state_machine->commit(1, entry_c->get_buf()); const auto & storage = state_machine->getStorageUnsafe(); - EXPECT_EQ(storage.ephemerals.size(), 1); + EXPECT_EQ(storage.committed_ephemerals.size(), 1); std::shared_ptr request_d = std::make_shared(); request_d->path = "/hello"; /// Delete from other session @@ -2036,7 +2036,7 @@ TYPED_TEST(CoordinationTest, TestEphemeralNodeRemove) state_machine->pre_commit(2, entry_d->get_buf()); state_machine->commit(2, entry_d->get_buf()); - EXPECT_EQ(storage.ephemerals.size(), 0); + EXPECT_EQ(storage.committed_ephemerals.size(), 0); } @@ -2536,8 +2536,8 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; TSA_SUPPRESS_WARNING_FOR_WRITE(storage.zxid) = 2; - storage.ephemerals[3] = {"/hello2"}; - storage.ephemerals[1] = {"/hello1"}; + storage.committed_ephemerals[3] = {"/hello2"}; + storage.committed_ephemerals[1] = {"/hello1"}; storage.getSessionID(130); storage.getSessionID(130); @@ -2563,9 +2563,9 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); EXPECT_EQ(restored_storage->getZXID(), 2); - EXPECT_EQ(restored_storage->ephemerals.size(), 2); - EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); - EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); + EXPECT_EQ(restored_storage->committed_ephemerals.size(), 2); + EXPECT_EQ(restored_storage->committed_ephemerals[3].size(), 1); + EXPECT_EQ(restored_storage->committed_ephemerals[1].size(), 1); EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); } @@ -2750,8 +2750,8 @@ TYPED_TEST(CoordinationTest, TestStorageSnapshotEqual) storage.session_id_counter = 5; - storage.ephemerals[3] = {"/hello"}; - storage.ephemerals[1] = {"/hello/somepath"}; + storage.committed_ephemerals[3] = {"/hello"}; + storage.committed_ephemerals[1] = {"/hello/somepath"}; for (size_t j = 0; j < 3333; ++j) storage.getSessionID(130 * j); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 60170d67a2e..0291b6e9239 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -28,13 +28,13 @@ void dumpMachine(std::shared_ptr> ma keys.pop(); std::cout << key << "\n"; auto value = storage.container.getValue(key); - std::cout << "\tStat: {version: " << value.version << - ", mtime: " << value.mtime << - ", emphemeralOwner: " << value.ephemeralOwner() << - ", czxid: " << value.czxid << - ", mzxid: " << value.mzxid << - ", numChildren: " << value.numChildren() << - ", dataLength: " << value.data_size << + std::cout << "\tStat: {version: " << value.stats.version << + ", mtime: " << value.stats.mtime << + ", emphemeralOwner: " << value.stats.ephemeralOwner() << + ", czxid: " << value.stats.czxid << + ", mzxid: " << value.stats.mzxid << + ", numChildren: " << value.stats.numChildren() << + ", dataLength: " << value.stats.data_size << "}" << std::endl; std::cout << "\tData: " << storage.container.getValue(key).getData() << std::endl; From 8896d1b78b9a96cc6f5b8349eb869d8e1c8807f0 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 4 Sep 2024 14:46:29 +0200 Subject: [PATCH 047/128] try to fix tests --- .../03231_hive_partitioning_filtering.sh | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 5c63370f38e..64e971a4891 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -9,30 +9,30 @@ DATA_DIR=$USER_FILES_PATH/$CLICKHOUSE_TEST_UNIQUE_NAME mkdir -p $DATA_DIR cp -r $CURDIR/data_hive/ $DATA_DIR -$CLICKHOUSE_CLIENT --query_id="test_03231_1" --query " +$CLICKHOUSE_CLIENT --query_id="test_03231_1_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " $CLICKHOUSE_CLIENT --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' AND current_database = currentDatabase(); + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' AND current_database = currentDatabase() and type='QueryFinish'; " -$CLICKHOUSE_CLIENT --query_id="test_03231_2" --query " +$CLICKHOUSE_CLIENT --query_id="test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " $CLICKHOUSE_CLIENT --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' AND current_database = currentDatabase(); + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' AND current_database = currentDatabase() and type='QueryFinish'; " -$CLICKHOUSE_CLIENT --query_id="test_03231_3" --query " +$CLICKHOUSE_CLIENT --query_id="test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; SYSTEM FLUSH LOGS; " $CLICKHOUSE_CLIENT --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3' AND current_database = currentDatabase(); + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3' AND current_database = currentDatabase() and type='QueryFinish'; " rm -rf $DATA_DIR From a3e233a537e3b7eb54d62cd0f567d65ce95275e4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 4 Sep 2024 15:19:56 +0200 Subject: [PATCH 048/128] Fix watch --- src/Coordination/KeeperStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 0add7787ee2..f0701326215 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1602,7 +1602,7 @@ KeeperStorageBase::ResponsesForSessions processWatches( KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) { - return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CREATED); + return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::DELETED); } template From 2fa6be55ff373b1cf847396ae80cbc8c40db5b4b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 4 Sep 2024 17:02:01 +0200 Subject: [PATCH 049/128] tests fix --- .../0_stateless/03231_hive_partitioning_filtering.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 64e971a4891..a561758c726 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -15,7 +15,7 @@ $CLICKHOUSE_CLIENT --query_id="test_03231_1_$CLICKHOUSE_TEST_UNIQUE_NAME" --quer " $CLICKHOUSE_CLIENT --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1' AND current_database = currentDatabase() and type='QueryFinish'; + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1_$CLICKHOUSE_TEST_UNIQUE_NAME' AND current_database = currentDatabase() and type='QueryFinish'; " $CLICKHOUSE_CLIENT --query_id="test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " @@ -24,7 +24,7 @@ $CLICKHOUSE_CLIENT --query_id="test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME" --quer " $CLICKHOUSE_CLIENT --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2' AND current_database = currentDatabase() and type='QueryFinish'; + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME' AND current_database = currentDatabase() and type='QueryFinish'; " $CLICKHOUSE_CLIENT --query_id="test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " @@ -33,6 +33,6 @@ $CLICKHOUSE_CLIENT --query_id="test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME" --quer " $CLICKHOUSE_CLIENT --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3' AND current_database = currentDatabase() and type='QueryFinish'; + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME' AND current_database = currentDatabase() and type='QueryFinish'; " rm -rf $DATA_DIR From e388f6f99ba477eb08e14a6f4ded9152b1d6368c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 6 Sep 2024 09:35:02 +0200 Subject: [PATCH 050/128] Remove useless log --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index 8214390a10f..16da3fe0fe9 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -209,14 +209,7 @@ void ZooKeeperCreateRequest::writeImpl(WriteBuffer & out) const size_t ZooKeeperCreateRequest::sizeImpl() const { - /// See https://github.com/ClickHouse/clickhouse-private/issues/3029 - if (path.starts_with("/clickhouse/tables/") && path.find("/parts/") != std::string::npos) - { - LOG_TRACE(getLogger(__PRETTY_FUNCTION__), "Creating part at path {}", path); - } - int32_t flags = 0; - return Coordination::size(path) + Coordination::size(data) + Coordination::size(acls) + Coordination::size(flags); } From 03c7f3817b869196780c68d67a1959e7564f0566 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 6 Sep 2024 15:41:04 +0200 Subject: [PATCH 051/128] Correct lock order --- src/Coordination/KeeperStorage.cpp | 17 +++++++++++------ src/Coordination/KeeperStorage.h | 4 ++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index f0701326215..35134064191 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -521,9 +521,11 @@ template Overloaded(Ts...) -> Overloaded; template -std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const +std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path, bool should_lock_storage) const { - std::shared_lock lock(storage.storage_mutex); + std::shared_lock lock(storage.storage_mutex, std::defer_lock); + if (should_lock_storage) + lock.lock(); if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; @@ -841,12 +843,12 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb } template -std::shared_ptr KeeperStorage::UncommittedState::getNode(StringRef path) const +std::shared_ptr KeeperStorage::UncommittedState::getNode(StringRef path, bool should_lock_storage) const { if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.node; - std::shared_ptr node = tryGetNodeFromStorage(path); + std::shared_ptr node = tryGetNodeFromStorage(path, should_lock_storage); if (node) { @@ -2648,7 +2650,7 @@ void KeeperStorage::preprocessRequest( std::unordered_map parent_updates; for (const auto & ephemeral_path : session_ephemerals->second) { - auto node = uncommitted_state.getNode(ephemeral_path); + auto node = uncommitted_state.getNode(ephemeral_path, /*should_lock_storage=*/false); /// maybe the node is deleted or recreated with different session_id in the uncommitted state if (!node || node->stats.ephemeralOwner() != session_id) @@ -2659,7 +2661,7 @@ void KeeperStorage::preprocessRequest( auto parent_update_it = parent_updates.find(parent_node_path); if (parent_update_it == parent_updates.end()) { - auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}); + auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}, /*should_lock_storage=*/false); std::tie(parent_update_it, std::ignore) = parent_updates.emplace(parent_node_path, *parent_node); } @@ -2684,6 +2686,9 @@ void KeeperStorage::preprocessRequest( } } }; + + /// storage lock should always be taken before ephemeral lock + std::shared_lock storage_lock(storage_mutex); std::lock_guard ephemeral_lock(ephemeral_mutex); process_ephemerals_for_session(committed_ephemerals); process_ephemerals_for_session(uncommitted_state.ephemerals); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 13f9631d2c3..87cb7140f0e 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -492,7 +492,7 @@ public: void rollback(int64_t rollback_zxid); void rollback(std::list rollback_deltas); - std::shared_ptr getNode(StringRef path) const; + std::shared_ptr getNode(StringRef path, bool should_lock_storage = true) const; Coordination::ACLs getACLs(StringRef path) const; void applyDeltas(const std::list & new_deltas); @@ -503,7 +503,7 @@ public: void forEachAuthInSession(int64_t session_id, std::function func) const; - std::shared_ptr tryGetNodeFromStorage(StringRef path) const; + std::shared_ptr tryGetNodeFromStorage(StringRef path, bool should_lock_storage = true) const; std::unordered_map>>> session_and_auth; std::unordered_set closed_sessions; From a903e1a726eb864049804fc906195aa669fa9ece Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 6 Sep 2024 20:24:18 +0200 Subject: [PATCH 052/128] remove logging + fixing bug --- src/Storages/StorageFile.cpp | 7 ---- src/Storages/VirtualColumnUtils.cpp | 34 ++++++++----------- .../03231_hive_partitioning_filtering.sh | 9 +++++ 3 files changed, 23 insertions(+), 27 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 655cc064fea..55bc8083ec8 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -356,18 +356,14 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user { fs::path user_files_absolute_path = fs::weakly_canonical(user_files_path); fs::path fs_table_path(table_path); - LOG_TRACE(getLogger("testing the paths"), "{} , {}", user_files_absolute_path, fs_table_path); if (fs_table_path.is_relative()) fs_table_path = user_files_absolute_path / fs_table_path; - LOG_TRACE(getLogger("testing the paths"), "fs_table_path = {}", fs_table_path); - Strings paths; /// Do not use fs::canonical or fs::weakly_canonical. /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. String path = fs::absolute(fs_table_path).lexically_normal(); /// Normalize path. - LOG_TRACE(getLogger("testing the paths"), "path = {}", path); bool can_be_directory = true; if (path.find(PartitionedSink::PARTITION_ID_WILDCARD) != std::string::npos) @@ -400,10 +396,7 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user } for (const auto & cur_path : paths) - { checkCreationIsAllowed(context, user_files_absolute_path, cur_path, can_be_directory); - LOG_TRACE(getLogger("checking all paths"), "{}", cur_path); - } return paths; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 2bd7325a789..523f236bf59 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -207,7 +207,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto return desc; } -static void addFilterDataToVirtualColumns(Block & block, const String & path, size_t idx, ColumnsWithTypeAndName partitioning_keys, const ContextPtr & context) +static void addFilterDataToVirtualColumns(Block & block, const String & path, size_t idx, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { if (block.has("_path")) block.getByName("_path").column->assumeMutableRef().insert(path); @@ -224,13 +224,20 @@ static void addFilterDataToVirtualColumns(Block & block, const String & path, si block.getByName("_file").column->assumeMutableRef().insert(file); } - for (const auto & item : partitioning_keys) + std::unordered_map keys; + if (context->getSettingsRef().use_hive_partitioning) + keys = parseHivePartitioningKeysAndValues(path); + + for (const auto & virt_column : virtual_columns) { - if (block.has(item.name)) + auto it = keys.find(virt_column.name); + if (it != keys.end()) { - auto column = block.getByName(item.name).column; - ReadBufferFromString buf(item.column->getDataAt(0).toView()); - item.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); + if (!block.has(virt_column.name)) + block.insert({virt_column.type->createColumn(), virt_column.type, virt_column.name}); + auto & column = block.getByName(virt_column.name).column; + ReadBufferFromString buf(it->second); + virt_column.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); } } @@ -259,28 +266,15 @@ std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * pr ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { Block block; - std::unordered_map keys; - ColumnsWithTypeAndName partitioning_columns; - if (context->getSettingsRef().use_hive_partitioning) - keys = parseHivePartitioningKeysAndValues(paths[0]); for (const auto & column : virtual_columns) { if (column.name == "_file" || column.name == "_path") block.insert({column.type->createColumn(), column.type, column.name}); - - auto it = keys.find(column.name); - if (it != keys.end()) - { - auto string_column = std::make_shared()->createColumn(); - string_column->insert(it->second); - block.insert({column.type->createColumn(), column.type, column.name}); - partitioning_columns.push_back({string_column->getPtr(), column.type, column.name}); - } } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); for (size_t i = 0; i != paths.size(); ++i) - addFilterDataToVirtualColumns(block, paths[i], i, partitioning_columns, context); + addFilterDataToVirtualColumns(block, paths[i], i, virtual_columns, context); filterBlockWithExpression(actions, block); diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index a561758c726..b66d2971cac 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -11,6 +11,9 @@ cp -r $CURDIR/data_hive/ $DATA_DIR $CLICKHOUSE_CLIENT --query_id="test_03231_1_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; +" + +$CLICKHOUSE_CLIENT --query " SYSTEM FLUSH LOGS; " @@ -20,6 +23,9 @@ $CLICKHOUSE_CLIENT --query " $CLICKHOUSE_CLIENT --query_id="test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; +" + +$CLICKHOUSE_CLIENT --query " SYSTEM FLUSH LOGS; " @@ -29,6 +35,9 @@ $CLICKHOUSE_CLIENT --query " $CLICKHOUSE_CLIENT --query_id="test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; +" + +$CLICKHOUSE_CLIENT --query " SYSTEM FLUSH LOGS; " From 190339c4e680582844d12f956cc352f0e908fa8f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 7 Sep 2024 17:34:59 +0200 Subject: [PATCH 053/128] Fix snapshot sync --- src/Coordination/KeeperStorage.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 35134064191..bae60139c20 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -2609,8 +2609,15 @@ void KeeperStorage::preprocessRequest( else { if (last_zxid == new_last_zxid && digest && checkDigest(*digest, current_digest)) - // we found the preprocessed request with the same ZXID, we can skip it + { + auto & last_transaction = uncommitted_transactions.back(); + // we found the preprocessed request with the same ZXID, we can get log_idx and skip preprocessing it + chassert(last_transaction.zxid == new_last_zxid && log_idx != 0); + /// initially leader preprocessed without knowing the log idx + /// on the second call we have that information and can set the log idx for the correct transaction + last_transaction.log_idx = log_idx; return; + } if (new_last_zxid <= last_zxid) throw Exception( From 2876a4e7146d53a032d74fd8bf54c9abf1dc988a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 11 Sep 2024 13:32:12 +0200 Subject: [PATCH 054/128] add retries --- .../03231_hive_partitioning_filtering.sh | 43 +++++++++++++++---- 1 file changed, 34 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index b66d2971cac..13bbfac349d 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -17,9 +17,17 @@ $CLICKHOUSE_CLIENT --query " SYSTEM FLUSH LOGS; " -$CLICKHOUSE_CLIENT --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1_$CLICKHOUSE_TEST_UNIQUE_NAME' AND current_database = currentDatabase() and type='QueryFinish'; -" +for i in {1..5}; do + count=$( $CLICKHOUSE_CLIENT --query " + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log + WHERE query_id='test_03231_1_$CLICKHOUSE_TEST_UNIQUE_NAME' AND + current_database = currentDatabase() and type='QueryFinish';" ) + if [[ "$count" == "1" ]]; then + echo "1" + break + fi + sleep 1 +done $CLICKHOUSE_CLIENT --query_id="test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; @@ -29,9 +37,17 @@ $CLICKHOUSE_CLIENT --query " SYSTEM FLUSH LOGS; " -$CLICKHOUSE_CLIENT --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME' AND current_database = currentDatabase() and type='QueryFinish'; -" +for i in {1..5}; do + count=$( $CLICKHOUSE_CLIENT --query " + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log + WHERE query_id='test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME' AND + current_database = currentDatabase() and type='QueryFinish';" ) + if [[ "$count" == "1" ]]; then + echo "1" + break + fi + sleep 1 +done $CLICKHOUSE_CLIENT --query_id="test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; @@ -41,7 +57,16 @@ $CLICKHOUSE_CLIENT --query " SYSTEM FLUSH LOGS; " -$CLICKHOUSE_CLIENT --query " - SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME' AND current_database = currentDatabase() and type='QueryFinish'; -" +for i in {1..5}; do + count=$( $CLICKHOUSE_CLIENT --query " + SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log + WHERE query_id='test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME' AND + current_database = currentDatabase() and type='QueryFinish';" ) + if [[ "$count" == "1" ]]; then + echo "1" + break + fi + sleep 1 +done + rm -rf $DATA_DIR From e8cec05d08dd53051d767d7725404abf777f6c45 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 11 Sep 2024 13:52:20 +0200 Subject: [PATCH 055/128] shellcheck --- .../0_stateless/03231_hive_partitioning_filtering.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 13bbfac349d..41f11ff869c 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -17,7 +17,7 @@ $CLICKHOUSE_CLIENT --query " SYSTEM FLUSH LOGS; " -for i in {1..5}; do +for _ in {1..5}; do count=$( $CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_1_$CLICKHOUSE_TEST_UNIQUE_NAME' AND @@ -37,7 +37,7 @@ $CLICKHOUSE_CLIENT --query " SYSTEM FLUSH LOGS; " -for i in {1..5}; do +for _ in {1..5}; do count=$( $CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME' AND @@ -57,7 +57,7 @@ $CLICKHOUSE_CLIENT --query " SYSTEM FLUSH LOGS; " -for i in {1..5}; do +for _ in {1..5}; do count=$( $CLICKHOUSE_CLIENT --query " SELECT ProfileEvents['EngineFileLikeReadFiles'] FROM system.query_log WHERE query_id='test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME' AND From a0a4858e0060082a467cf4463d794860f3bc11d9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 12 Sep 2024 18:55:35 +0000 Subject: [PATCH 056/128] Scratch build of libpqxx at 7.5.3 + patches --- contrib/libpqxx | 2 +- contrib/libpqxx-cmake/CMakeLists.txt | 39 +--------------------------- 2 files changed, 2 insertions(+), 39 deletions(-) diff --git a/contrib/libpqxx b/contrib/libpqxx index c995193a3a1..36c49cac91b 160000 --- a/contrib/libpqxx +++ b/contrib/libpqxx @@ -1 +1 @@ -Subproject commit c995193a3a14d71f4711f1f421f65a1a1db64640 +Subproject commit 36c49cac91bf22ac6b85952f630ae75ba7883f68 diff --git a/contrib/libpqxx-cmake/CMakeLists.txt b/contrib/libpqxx-cmake/CMakeLists.txt index a3317404f95..c9c4368763b 100644 --- a/contrib/libpqxx-cmake/CMakeLists.txt +++ b/contrib/libpqxx-cmake/CMakeLists.txt @@ -34,44 +34,7 @@ set (SRCS "${LIBRARY_DIR}/src/version.cxx" ) -# Need to explicitly include each header file, because in the directory include/pqxx there are also files -# like just 'array'. So if including the whole directory with `target_include_directories`, it will make -# conflicts with all includes of . -set (HDRS - "${LIBRARY_DIR}/include/pqxx/array.hxx" - "${LIBRARY_DIR}/include/pqxx/params.hxx" - "${LIBRARY_DIR}/include/pqxx/binarystring.hxx" - "${LIBRARY_DIR}/include/pqxx/composite.hxx" - "${LIBRARY_DIR}/include/pqxx/connection.hxx" - "${LIBRARY_DIR}/include/pqxx/cursor.hxx" - "${LIBRARY_DIR}/include/pqxx/dbtransaction.hxx" - "${LIBRARY_DIR}/include/pqxx/errorhandler.hxx" - "${LIBRARY_DIR}/include/pqxx/except.hxx" - "${LIBRARY_DIR}/include/pqxx/field.hxx" - "${LIBRARY_DIR}/include/pqxx/isolation.hxx" - "${LIBRARY_DIR}/include/pqxx/largeobject.hxx" - "${LIBRARY_DIR}/include/pqxx/nontransaction.hxx" - "${LIBRARY_DIR}/include/pqxx/notification.hxx" - "${LIBRARY_DIR}/include/pqxx/pipeline.hxx" - "${LIBRARY_DIR}/include/pqxx/prepared_statement.hxx" - "${LIBRARY_DIR}/include/pqxx/result.hxx" - "${LIBRARY_DIR}/include/pqxx/robusttransaction.hxx" - "${LIBRARY_DIR}/include/pqxx/row.hxx" - "${LIBRARY_DIR}/include/pqxx/separated_list.hxx" - "${LIBRARY_DIR}/include/pqxx/strconv.hxx" - "${LIBRARY_DIR}/include/pqxx/stream_from.hxx" - "${LIBRARY_DIR}/include/pqxx/stream_to.hxx" - "${LIBRARY_DIR}/include/pqxx/subtransaction.hxx" - "${LIBRARY_DIR}/include/pqxx/transaction.hxx" - "${LIBRARY_DIR}/include/pqxx/transaction_base.hxx" - "${LIBRARY_DIR}/include/pqxx/types.hxx" - "${LIBRARY_DIR}/include/pqxx/util.hxx" - "${LIBRARY_DIR}/include/pqxx/version.hxx" - "${LIBRARY_DIR}/include/pqxx/zview.hxx" -) - -add_library(_libpqxx ${SRCS} ${HDRS}) - +add_library(_libpqxx ${SRCS}) target_link_libraries(_libpqxx PUBLIC ch_contrib::libpq) target_include_directories (_libpqxx SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/include") From 5a34b9f24ec5d2a04a9693ac2b392e81cc702d04 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 12 Sep 2024 19:04:04 +0000 Subject: [PATCH 057/128] Bump to 7.6.1 --- contrib/libpqxx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libpqxx b/contrib/libpqxx index 36c49cac91b..12b5c232c4e 160000 --- a/contrib/libpqxx +++ b/contrib/libpqxx @@ -1 +1 @@ -Subproject commit 36c49cac91bf22ac6b85952f630ae75ba7883f68 +Subproject commit 12b5c232c4e046da3dd70ae23bdd20983ccaa137 From aab0d3dd9ea4290f955946a806fe2fd7c4f21f02 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 12 Sep 2024 19:24:27 +0000 Subject: [PATCH 058/128] Bump to 7.7.5 --- contrib/libpqxx | 2 +- contrib/libpqxx-cmake/CMakeLists.txt | 9 ++++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/contrib/libpqxx b/contrib/libpqxx index 12b5c232c4e..41e4c331564 160000 --- a/contrib/libpqxx +++ b/contrib/libpqxx @@ -1 +1 @@ -Subproject commit 12b5c232c4e046da3dd70ae23bdd20983ccaa137 +Subproject commit 41e4c331564167cca97ad6eccbd5b8879c2ca044 diff --git a/contrib/libpqxx-cmake/CMakeLists.txt b/contrib/libpqxx-cmake/CMakeLists.txt index c9c4368763b..fbee4d0a94e 100644 --- a/contrib/libpqxx-cmake/CMakeLists.txt +++ b/contrib/libpqxx-cmake/CMakeLists.txt @@ -8,9 +8,9 @@ endif() set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/libpqxx") set (SRCS - "${LIBRARY_DIR}/src/strconv.cxx" "${LIBRARY_DIR}/src/array.cxx" "${LIBRARY_DIR}/src/binarystring.cxx" + "${LIBRARY_DIR}/src/blob.cxx" "${LIBRARY_DIR}/src/connection.cxx" "${LIBRARY_DIR}/src/cursor.cxx" "${LIBRARY_DIR}/src/encodings.cxx" @@ -19,19 +19,22 @@ set (SRCS "${LIBRARY_DIR}/src/field.cxx" "${LIBRARY_DIR}/src/largeobject.cxx" "${LIBRARY_DIR}/src/notification.cxx" + "${LIBRARY_DIR}/src/params.cxx" "${LIBRARY_DIR}/src/pipeline.cxx" "${LIBRARY_DIR}/src/result.cxx" "${LIBRARY_DIR}/src/robusttransaction.cxx" + "${LIBRARY_DIR}/src/row.cxx" "${LIBRARY_DIR}/src/sql_cursor.cxx" + "${LIBRARY_DIR}/src/strconv.cxx" "${LIBRARY_DIR}/src/stream_from.cxx" "${LIBRARY_DIR}/src/stream_to.cxx" "${LIBRARY_DIR}/src/subtransaction.cxx" + "${LIBRARY_DIR}/src/time.cxx" "${LIBRARY_DIR}/src/transaction.cxx" "${LIBRARY_DIR}/src/transaction_base.cxx" - "${LIBRARY_DIR}/src/row.cxx" - "${LIBRARY_DIR}/src/params.cxx" "${LIBRARY_DIR}/src/util.cxx" "${LIBRARY_DIR}/src/version.cxx" + "${LIBRARY_DIR}/src/wait.cxx" ) add_library(_libpqxx ${SRCS}) From 8a89d7b2b956372e3c7a7e94ee599f1dfcae8aed Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 13 Sep 2024 10:46:38 +0800 Subject: [PATCH 059/128] allow empty inputs for arrayZip or arrayZipUnaligned --- src/Functions/array/arrayZip.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 6e1cc0f7788..36ab56006b3 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -38,13 +38,6 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.empty()) - throw Exception( - ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, - "Function {} needs at least one argument; passed {}.", - getName(), - arguments.size()); - DataTypes arguments_types; for (size_t index = 0; index < arguments.size(); ++index) { @@ -68,9 +61,16 @@ public: } ColumnPtr - executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override + executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { size_t num_arguments = arguments.size(); + if (num_arguments == 0) + { + auto res_col = result_type->createColumn(); + res_col->insertDefault(); + return ColumnConst::create(std::move(res_col), input_rows_count); + } + Columns holders(num_arguments); Columns tuple_columns(num_arguments); From f9335a2fd5e4746a8b3fa9bb704640e1a4d3e61c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 13 Sep 2024 10:50:50 +0800 Subject: [PATCH 060/128] update uts --- tests/queries/0_stateless/01045_array_zip.reference | 1 + tests/queries/0_stateless/01045_array_zip.sql | 2 +- tests/queries/0_stateless/03230_array_zip_unaligned.reference | 1 + tests/queries/0_stateless/03230_array_zip_unaligned.sql | 2 +- 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01045_array_zip.reference b/tests/queries/0_stateless/01045_array_zip.reference index 955ed98033e..85b53dd0c9f 100644 --- a/tests/queries/0_stateless/01045_array_zip.reference +++ b/tests/queries/0_stateless/01045_array_zip.reference @@ -1,2 +1,3 @@ [('a','d'),('b','e'),('c','f')] [('a','d','g'),('b','e','h'),('c','f','i')] +[()] diff --git a/tests/queries/0_stateless/01045_array_zip.sql b/tests/queries/0_stateless/01045_array_zip.sql index 0bf77747123..801df5a3230 100644 --- a/tests/queries/0_stateless/01045_array_zip.sql +++ b/tests/queries/0_stateless/01045_array_zip.sql @@ -2,7 +2,7 @@ SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']); SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']); -SELECT arrayZip(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayZip(); SELECT arrayZip('a', 'b', 'c'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/03230_array_zip_unaligned.reference b/tests/queries/0_stateless/03230_array_zip_unaligned.reference index 7067f8788e5..7327d25eab8 100644 --- a/tests/queries/0_stateless/03230_array_zip_unaligned.reference +++ b/tests/queries/0_stateless/03230_array_zip_unaligned.reference @@ -1,5 +1,6 @@ [('a','d'),('b','e'),('c','f')] Array(Tuple(Nullable(String), Nullable(String))) [('a','d','g'),('b','e','h'),('c','f','i')] +[()] [('a','d'),('b','e'),('c','f'),(NULL,'g')] [('a',1),(NULL,2),(NULL,3)] [('a',1,1.1),('b',2,2.2),('c',NULL,3.3),(NULL,NULL,4.4)] diff --git a/tests/queries/0_stateless/03230_array_zip_unaligned.sql b/tests/queries/0_stateless/03230_array_zip_unaligned.sql index 90b7aa47bfd..08d77737e54 100644 --- a/tests/queries/0_stateless/03230_array_zip_unaligned.sql +++ b/tests/queries/0_stateless/03230_array_zip_unaligned.sql @@ -2,7 +2,7 @@ SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f']) as x, toTypeName(x); SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']); -SELECT arrayZipUnaligned(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayZipUnaligned(); SELECT arrayZipUnaligned('a', 'b', 'c'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 11d2963497ad62d729b9f445f97083a490c30a43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 13 Sep 2024 11:56:47 +0800 Subject: [PATCH 061/128] fix style --- src/Functions/array/arrayZip.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 36ab56006b3..2f8c9a3af02 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -15,7 +15,6 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; -extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; } From 53e1975833a833951b110b21462a6a501cea48a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 13 Sep 2024 18:15:47 +0800 Subject: [PATCH 062/128] fix 01045_array_zip --- tests/queries/0_stateless/01045_array_zip.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01045_array_zip.reference b/tests/queries/0_stateless/01045_array_zip.reference index 85b53dd0c9f..154afa7eb89 100644 --- a/tests/queries/0_stateless/01045_array_zip.reference +++ b/tests/queries/0_stateless/01045_array_zip.reference @@ -1,3 +1,3 @@ [('a','d'),('b','e'),('c','f')] [('a','d','g'),('b','e','h'),('c','f','i')] -[()] +[] From 71553022e0c36b7740445d1806f132613a394066 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 13 Sep 2024 18:16:13 +0800 Subject: [PATCH 063/128] fix 03230_array_zip_unaligned --- tests/queries/0_stateless/03230_array_zip_unaligned.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03230_array_zip_unaligned.reference b/tests/queries/0_stateless/03230_array_zip_unaligned.reference index 7327d25eab8..d373cf47a9c 100644 --- a/tests/queries/0_stateless/03230_array_zip_unaligned.reference +++ b/tests/queries/0_stateless/03230_array_zip_unaligned.reference @@ -1,6 +1,6 @@ [('a','d'),('b','e'),('c','f')] Array(Tuple(Nullable(String), Nullable(String))) [('a','d','g'),('b','e','h'),('c','f','i')] -[()] +[] [('a','d'),('b','e'),('c','f'),(NULL,'g')] [('a',1),(NULL,2),(NULL,3)] [('a',1,1.1),('b',2,2.2),('c',NULL,3.3),(NULL,NULL,4.4)] From 2650a2062899f5f232176ee56814cce66c800139 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 13 Sep 2024 16:21:17 +0200 Subject: [PATCH 064/128] Make dedup logic O(n*log(n)) instead of O(n^2) --- .../ParallelReplicasReadingCoordinator.cpp | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 8abf735b49f..c9fb09cd0ba 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -869,8 +869,7 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa /// To get rid of duplicates for (auto && part: announcement.description) { - auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), - [&part] (const Part & other) { return other.description.info == part.info; }); + auto the_same_it = all_parts_to_read.find(Part{.description = part}); /// We have the same part - add the info about presence on the corresponding replica to it if (the_same_it != all_parts_to_read.end()) @@ -882,12 +881,28 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa if (state_initialized) continue; - auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), - [&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); }); + /// Look for the first part >= current + auto covering_it = all_parts_to_read.lower_bound(Part{.description = part}); - /// It is covering part or we have covering - skip it - if (covering_or_the_same_it != all_parts_to_read.end()) - continue; + if (covering_it != all_parts_to_read.end()) + { + /// Checks if other part covers this one or this one covers the other + auto is_covered_or_covering = [&part] (const Part & other) + { + return other.description.info.contains(part.info) || part.info.contains(other.description.info); + }; + + if (is_covered_or_covering(*covering_it)) + continue; + + /// Also look at the previous part, it could be covering the current one + if (covering_it != all_parts_to_read.begin()) + { + --covering_it; + if (is_covered_or_covering(*covering_it)) + continue; + } + } new_rows_to_read += part.rows; From 14a6b0422b6b7abb68c2dc8a55fb566bdafafe25 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 13 Sep 2024 16:33:17 +0200 Subject: [PATCH 065/128] disable optimize_count_from_files --- .../0_stateless/03231_hive_partitioning_filtering.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh index 41f11ff869c..30ae5b01a98 100755 --- a/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh +++ b/tests/queries/0_stateless/03231_hive_partitioning_filtering.sh @@ -10,7 +10,7 @@ mkdir -p $DATA_DIR cp -r $CURDIR/data_hive/ $DATA_DIR $CLICKHOUSE_CLIENT --query_id="test_03231_1_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " - SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1; + SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/column0=*/sample.parquet') WHERE column0 = 'Elizabeth' SETTINGS use_hive_partitioning=1, optimize_count_from_files=0; " $CLICKHOUSE_CLIENT --query " @@ -30,7 +30,7 @@ for _ in {1..5}; do done $CLICKHOUSE_CLIENT --query_id="test_03231_2_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " - SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1; + SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/identifier=*/email.csv') WHERE identifier = 2070 SETTINGS use_hive_partitioning=1, optimize_count_from_files=0; " $CLICKHOUSE_CLIENT --query " @@ -50,7 +50,7 @@ for _ in {1..5}; do done $CLICKHOUSE_CLIENT --query_id="test_03231_3_$CLICKHOUSE_TEST_UNIQUE_NAME" --query " - SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1; + SELECT countDistinct(_path) FROM file('$DATA_DIR/data_hive/partitioning/array=*/sample.parquet') WHERE array = [1,2,3] SETTINGS use_hive_partitioning=1, optimize_count_from_files=0; " $CLICKHOUSE_CLIENT --query " From c184aae686dc0918ec24f774c6d125e5b639e20d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 13 Sep 2024 16:40:01 +0200 Subject: [PATCH 066/128] review --- src/Storages/VirtualColumnUtils.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 523f236bf59..8d46a49bb73 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -233,8 +233,6 @@ static void addFilterDataToVirtualColumns(Block & block, const String & path, si auto it = keys.find(virt_column.name); if (it != keys.end()) { - if (!block.has(virt_column.name)) - block.insert({virt_column.type->createColumn(), virt_column.type, virt_column.name}); auto & column = block.getByName(virt_column.name).column; ReadBufferFromString buf(it->second); virt_column.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); From e13247b67ee66d510af988cf0799a7286dab4ea4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 13 Sep 2024 16:50:43 +0200 Subject: [PATCH 067/128] Fix clang-18 build --- .../MergeTree/ParallelReplicasReadingCoordinator.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index c9fb09cd0ba..ddbed5db7dc 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -869,7 +869,7 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa /// To get rid of duplicates for (auto && part: announcement.description) { - auto the_same_it = all_parts_to_read.find(Part{.description = part}); + auto the_same_it = all_parts_to_read.find(Part{.description = part, .replicas = {}}); /// We have the same part - add the info about presence on the corresponding replica to it if (the_same_it != all_parts_to_read.end()) @@ -882,14 +882,14 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa continue; /// Look for the first part >= current - auto covering_it = all_parts_to_read.lower_bound(Part{.description = part}); + auto covering_it = all_parts_to_read.lower_bound(Part{.description = part, .replicas = {}}); if (covering_it != all_parts_to_read.end()) { /// Checks if other part covers this one or this one covers the other auto is_covered_or_covering = [&part] (const Part & other) { - return other.description.info.contains(part.info) || part.info.contains(other.description.info); + return other.description.info.contains(part.info) || part.info.contains(other.description.info); }; if (is_covered_or_covering(*covering_it)) From 991279e5c626067f9f371da5115e6993ed665ee6 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 13 Sep 2024 19:23:00 +0200 Subject: [PATCH 068/128] revert --- src/Storages/VirtualColumnUtils.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 8d46a49bb73..523f236bf59 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -233,6 +233,8 @@ static void addFilterDataToVirtualColumns(Block & block, const String & path, si auto it = keys.find(virt_column.name); if (it != keys.end()) { + if (!block.has(virt_column.name)) + block.insert({virt_column.type->createColumn(), virt_column.type, virt_column.name}); auto & column = block.getByName(virt_column.name).column; ReadBufferFromString buf(it->second); virt_column.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); From ad31d86a15f865cca2b18d6240cfbf17adc26435 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 13 Sep 2024 19:58:19 +0200 Subject: [PATCH 069/128] move the block inserting --- src/Storages/VirtualColumnUtils.cpp | 44 ++++++++++++++++++----------- 1 file changed, 28 insertions(+), 16 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 523f236bf59..2daffb43c84 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -207,7 +207,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto return desc; } -static void addFilterDataToVirtualColumns(Block & block, const String & path, size_t idx, const NamesAndTypesList & virtual_columns, const ContextPtr & context) +static void addFilterDataToVirtualColumns(Block & block, const String & path, size_t idx, ColumnWithTypeAndName partitioning_column, const ContextPtr & context) { if (block.has("_path")) block.getByName("_path").column->assumeMutableRef().insert(path); @@ -224,21 +224,11 @@ static void addFilterDataToVirtualColumns(Block & block, const String & path, si block.getByName("_file").column->assumeMutableRef().insert(file); } - std::unordered_map keys; - if (context->getSettingsRef().use_hive_partitioning) - keys = parseHivePartitioningKeysAndValues(path); - - for (const auto & virt_column : virtual_columns) + if (block.has(partitioning_column.name)) { - auto it = keys.find(virt_column.name); - if (it != keys.end()) - { - if (!block.has(virt_column.name)) - block.insert({virt_column.type->createColumn(), virt_column.type, virt_column.name}); - auto & column = block.getByName(virt_column.name).column; - ReadBufferFromString buf(it->second); - virt_column.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); - } + auto & column = block.getByName(partitioning_column.name).column; + ReadBufferFromString buf(partitioning_column.column->getDataAt(0).toView()); + partitioning_column.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); } block.getByName("_idx").column->assumeMutableRef().insert(idx); @@ -266,15 +256,37 @@ std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * pr ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { Block block; + std::vector> keys_vec; + ColumnsWithTypeAndName partitioning_columns; + if (context->getSettingsRef().use_hive_partitioning) + { + for (const auto & path : paths) + keys_vec.push_back(parseHivePartitioningKeysAndValues(path)); + } for (const auto & column : virtual_columns) { if (column.name == "_file" || column.name == "_path") block.insert({column.type->createColumn(), column.type, column.name}); + else + { + for (auto & keys : keys_vec) + { + const auto & it = keys.find(column.name); + if (it != keys.end()) + { + auto string_column = std::make_shared()->createColumn(); + string_column->insert(it->second); + block.insert({column.type->createColumn(), column.type, column.name}); + partitioning_columns.push_back({string_column->getPtr(), column.type, column.name}); + keys.erase(it); + } + } + } } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); for (size_t i = 0; i != paths.size(); ++i) - addFilterDataToVirtualColumns(block, paths[i], i, virtual_columns, context); + addFilterDataToVirtualColumns(block, paths[i], i, partitioning_columns[i], context); filterBlockWithExpression(actions, block); From 0d1d750437f12ce6ab5dcec828250e387c389bc8 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 13 Sep 2024 20:43:51 +0200 Subject: [PATCH 070/128] fix crash --- src/Storages/VirtualColumnUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 2daffb43c84..667a7e2506f 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -285,7 +285,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); - for (size_t i = 0; i != paths.size(); ++i) + for (size_t i = 0; i != partitioning_columns.size(); ++i) addFilterDataToVirtualColumns(block, paths[i], i, partitioning_columns[i], context); filterBlockWithExpression(actions, block); From 7d5203f8a7d43fef85f35868616a67070bc7899a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 13 Sep 2024 21:38:48 +0200 Subject: [PATCH 071/128] add resize for partitioning_columns --- src/Storages/VirtualColumnUtils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 667a7e2506f..b5deab95f3f 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -285,6 +285,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); + partitioning_columns.resize(paths.size()); for (size_t i = 0; i != partitioning_columns.size(); ++i) addFilterDataToVirtualColumns(block, paths[i], i, partitioning_columns[i], context); From dbb1d043fec40cb3ef403645483c3aad622b4bee Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 14 Sep 2024 05:46:08 +0000 Subject: [PATCH 072/128] unification of FunctionSecretArgumentsFinder --- .../FunctionSecretArgumentsFinderTreeNode.h | 408 ++---------- src/Parsers/FunctionSecretArgumentsFinder.h | 512 ++++++++++++++- .../FunctionSecretArgumentsFinderAST.h | 583 +++--------------- 3 files changed, 642 insertions(+), 861 deletions(-) diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 439ddffe5e5..3c0e5974d16 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -3,370 +3,84 @@ #include #include #include -#include #include -#include -#include -#include - -#include namespace DB { +class FunctionTreeNode : public AbstractFunction +{ +public: + class ArgumentTreeNode : public Argument + { + public: + explicit ArgumentTreeNode(const IQueryTreeNode * argument_) : argument(argument_) {} + std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + bool isIdentifier() const override { return argument->as(); } + bool tryGetString(String * res, bool allow_identifier = true) const override + { + if (const auto * literal = argument->as()) + { + if (literal->getValue().getType() != Field::Types::String) + return false; + if (res) + *res = literal->getValue().safeGet(); + return true; + } + + if (allow_identifier) + { + if (const auto * id = argument->as()) + { + if (res) + *res = id->getIdentifier().getFullName(); + return true; + } + } + + return false; + } + private: + const IQueryTreeNode * argument = nullptr; + }; + + class ArgumentsTreeNode : public Arguments + { + public: + explicit ArgumentsTreeNode(const QueryTreeNodes * arguments_) : arguments(arguments_) {} + size_t size() const override { return arguments ? arguments->size() : 0; } + std::unique_ptr at(size_t n) const override { return std::make_unique(arguments->at(n).get()); } + private: + const QueryTreeNodes * arguments = nullptr; + }; + + explicit FunctionTreeNode(const FunctionNode & function_) : function(&function_) + { + if (const auto & nodes = function->getArguments().getNodes(); !nodes.empty()) + arguments = std::make_unique(&nodes); + } + String name() const override { return function->getFunctionName(); } +private: + const FunctionNode * function = nullptr; +}; + /// Finds arguments of a specified function which should not be displayed for most users for security reasons. /// That involves passwords and secret keys. -class FunctionSecretArgumentsFinderTreeNode +class FunctionSecretArgumentsFinderTreeNode : public FunctionSecretArgumentsFinder { public: - explicit FunctionSecretArgumentsFinderTreeNode(const FunctionNode & function_) : function(function_), arguments(function.getArguments()) + explicit FunctionSecretArgumentsFinderTreeNode(const FunctionNode & function_) + : FunctionSecretArgumentsFinder(std::make_unique(function_)) { - if (arguments.getNodes().empty()) + if (!function->hasArguments()) return; - findFunctionSecretArguments(); + findOrdinaryFunctionSecretArguments(); } - struct Result - { - /// Result constructed by default means no arguments will be hidden. - size_t start = static_cast(-1); - size_t count = 0; /// Mostly it's either 0 or 1. There are only a few cases where `count` can be greater than 1 (e.g. see `encrypt`). - /// In all known cases secret arguments are consecutive - bool are_named = false; /// Arguments like `password = 'password'` are considered as named arguments. - /// E.g. "headers" in `url('..', headers('foo' = '[HIDDEN]'))` - std::vector nested_maps; - - bool hasSecrets() const - { - return count != 0 || !nested_maps.empty(); - } - }; - FunctionSecretArgumentsFinder::Result getResult() const { return result; } - -private: - const FunctionNode & function; - const ListNode & arguments; - FunctionSecretArgumentsFinder::Result result; - - void markSecretArgument(size_t index, bool argument_is_named = false) - { - if (index >= arguments.getNodes().size()) - return; - if (!result.count) - { - result.start = index; - result.are_named = argument_is_named; - } - chassert(index >= result.start); /// We always check arguments consecutively - result.count = index + 1 - result.start; - if (!argument_is_named) - result.are_named = false; - } - - void findFunctionSecretArguments() - { - const auto & name = function.getFunctionName(); - - if ((name == "mysql") || (name == "postgresql") || (name == "mongodb")) - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) - /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((name == "s3") || (name == "cosn") || (name == "oss") || - (name == "deltaLake") || (name == "hudi") || (name == "iceberg")) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (name == "s3Cluster") - { - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ true); - } - else if ((name == "remote") || (name == "remoteSecure")) - { - /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) - findRemoteFunctionSecretArguments(); - } - else if ((name == "encrypt") || (name == "decrypt") || - (name == "aes_encrypt_mysql") || (name == "aes_decrypt_mysql") || - (name == "tryDecrypt")) - { - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) - findEncryptionFunctionSecretArguments(); - } - else if (name == "url") - { - findURLSecretArguments(); - } - } - - void findMySQLFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// mysql(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - markSecretArgument(4); - } - } - - /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should - /// always be at the end). Marks "headers" as secret, if found. - size_t excludeS3OrURLNestedMaps() - { - const auto & nodes = arguments.getNodes(); - size_t count = nodes.size(); - while (count > 0) - { - const FunctionNode * f = nodes.at(count - 1)->as(); - if (!f) - break; - if (f->getFunctionName() == "headers") - result.nested_maps.push_back(f->getFunctionName()); - else if (f->getFunctionName() != "extra_credentials") - break; - count -= 1; - } - return count; - } - - void findS3FunctionSecretArguments(bool is_cluster_function) - { - /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (url_arg_idx + 2 < count) - markSecretArgument(url_arg_idx + 2); - } - - void findURLSecretArguments() - { - if (!isNamedCollectionName(0)) - excludeS3OrURLNestedMaps(); - } - - bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const - { - if (arg_idx >= arguments.getNodes().size()) - return false; - - return tryGetStringFromArgument(arguments.getNodes()[arg_idx], res, allow_identifier); - } - - static bool tryGetStringFromArgument(const QueryTreeNodePtr argument, String * res, bool allow_identifier = true) - { - if (const auto * literal = argument->as()) - { - if (literal->getValue().getType() != Field::Types::String) - return false; - if (res) - *res = literal->getValue().safeGet(); - return true; - } - - if (allow_identifier) - { - if (const auto * id = argument->as()) - { - if (res) - *res = id->getIdentifier().getFullName(); - return true; - } - } - - return false; - } - - void findRemoteFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// remote(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - return; - } - - /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: - /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// remote('addresses_expr', db.table) - if (arguments.getNodes().size() < 3) - return; - - size_t arg_num = 1; - - /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. - const auto * table_function = arguments.getNodes()[arg_num]->as(); - if (table_function && KnownTableFunctionNames::instance().exists(table_function->getFunctionName())) - { - ++arg_num; - } - else - { - std::optional database; - std::optional qualified_table_name; - if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) - { - /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. - /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' - /// before the argument 'password'. So it's safer to wipe two arguments just in case. - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `user`. - markSecretArgument(arg_num + 2); - } - if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `sharding_key`. - markSecretArgument(arg_num + 3); - } - return; - } - - /// Skip the current argument (which is either a database name or a qualified table name). - ++arg_num; - if (database) - { - /// Skip the 'table' argument if the previous argument was a database name. - ++arg_num; - } - } - - /// Skip username. - ++arg_num; - - /// Do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); - if (can_be_password) - markSecretArgument(arg_num); - } - - /// Tries to get either a database name or a qualified table name from an argument. - /// Empty string is also allowed (it means the default database). - /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. - bool tryGetDatabaseNameOrQualifiedTableName( - size_t arg_idx, - std::optional & res_database, - std::optional & res_qualified_table_name) const - { - res_database.reset(); - res_qualified_table_name.reset(); - - String str; - if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) - return false; - - if (str.empty()) - { - res_database = ""; - return true; - } - - auto qualified_table_name = QualifiedTableName::tryParseFromString(str); - if (!qualified_table_name) - return false; - - if (qualified_table_name->database.empty()) - res_database = std::move(qualified_table_name->table); - else - res_qualified_table_name = std::move(qualified_table_name); - return true; - } - - void findEncryptionFunctionSecretArguments() - { - if (arguments.getNodes().empty()) - return; - - /// We replace all arguments after 'mode' with '[HIDDEN]': - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') - result.start = 1; - result.count = arguments.getNodes().size() - 1; - } - - - /// Whether a specified argument can be the name of a named collection? - bool isNamedCollectionName(size_t arg_idx) const - { - if (arguments.getNodes().size() <= arg_idx) - return false; - - const auto * identifier = arguments.getNodes()[arg_idx]->as(); - return identifier != nullptr; - } - - /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. - void findSecretNamedArgument(const std::string_view & key, size_t start = 0) - { - for (size_t i = start; i < arguments.getNodes().size(); ++i) - { - const auto & argument = arguments.getNodes()[i]; - const auto * equals_func = argument->as(); - if (!equals_func || (equals_func->getFunctionName() != "equals")) - continue; - - const auto * expr_list = equals_func->getArguments().as(); - if (!expr_list) - continue; - - const auto & equal_args = expr_list->getNodes(); - if (equal_args.size() != 2) - continue; - - String found_key; - if (!tryGetStringFromArgument(equal_args[0], &found_key)) - continue; - - if (found_key == key) - markSecretArgument(i, /* argument_is_named= */ true); - } - } }; } diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 002ad94f6ea..77facd715f8 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -1,10 +1,42 @@ #pragma once -#include +#include +#include +#include +#include + namespace DB { +class AbstractFunction +{ + friend class FunctionSecretArgumentsFinder; +public: + class Argument + { + public: + virtual ~Argument() = default; + virtual std::unique_ptr getFunction() const = 0; + virtual bool isIdentifier() const = 0; + virtual bool tryGetString(String * res, bool allow_identifier) const = 0; + }; + class Arguments + { + public: + virtual ~Arguments() = default; + virtual size_t size() const = 0; + virtual std::unique_ptr at(size_t n) const = 0; + }; + + virtual ~AbstractFunction() = default; + virtual String name() const = 0; + bool hasArguments() const { return !!arguments; } + +protected: + std::unique_ptr arguments; +}; + class FunctionSecretArgumentsFinder { public: @@ -23,6 +55,484 @@ public: return count != 0 || !nested_maps.empty(); } }; + + explicit FunctionSecretArgumentsFinder(std::unique_ptr && function_) : function(std::move(function_)) {} + + FunctionSecretArgumentsFinder::Result getResult() const { return result; } + +protected: + const std::unique_ptr function; + Result result; + + void markSecretArgument(size_t index, bool argument_is_named = false) + { + if (index >= function->arguments->size()) + return; + if (!result.count) + { + result.start = index; + result.are_named = argument_is_named; + } + chassert(index >= result.start); /// We always check arguments consecutively + result.count = index + 1 - result.start; + if (!argument_is_named) + result.are_named = false; + } + + void findOrdinaryFunctionSecretArguments() + { + if ((function->name() == "mysql") || (function->name() == "postgresql") || (function->name() == "mongodb")) + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) + /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || + (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg")) + { + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ false); + } + else if (function->name() == "s3Cluster") + { + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ true); + } + else if (function->name() == "azureBlobStorage") + { + /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) + findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ false); + } + else if (function->name() == "azureBlobStorageCluster") + { + /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) + findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ true); + } + else if ((function->name() == "remote") || (function->name() == "remoteSecure")) + { + /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) + findRemoteFunctionSecretArguments(); + } + else if ((function->name() == "encrypt") || (function->name() == "decrypt") || + (function->name() == "aes_encrypt_mysql") || (function->name() == "aes_decrypt_mysql") || + (function->name() == "tryDecrypt")) + { + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) + findEncryptionFunctionSecretArguments(); + } + else if (function->name() == "url") + { + findURLSecretArguments(); + } + } + + void findMySQLFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// mysql(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + markSecretArgument(4); + } + } + + /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should + /// always be at the end). Marks "headers" as secret, if found. + size_t excludeS3OrURLNestedMaps() + { + size_t count = function->arguments->size(); + while (count > 0) + { + const auto f = function->arguments->at(count - 1)->getFunction(); + if (!f) + break; + if (f->name() == "headers") + result.nested_maps.push_back(f->name()); + else if (f->name() != "extra_credentials") + break; + count -= 1; + } + return count; + } + + void findS3FunctionSecretArguments(bool is_cluster_function) + { + /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + if (!is_cluster_function && isNamedCollectionName(0)) + { + /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (url_arg_idx + 2 < count) + markSecretArgument(url_arg_idx + 2); + } + + void findAzureBlobStorageFunctionSecretArguments(bool is_cluster_function) + { + /// azureBlobStorage('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + if (!is_cluster_function && isNamedCollectionName(0)) + { + /// azureBlobStorage(named_collection, ..., account_key = 'account_key', ...) + findSecretNamedArgument("account_key", 1); + return; + } + else if (is_cluster_function && isNamedCollectionName(1)) + { + /// azureBlobStorageCluster(cluster, named_collection, ..., account_key = 'account_key', ...) + findSecretNamedArgument("account_key", 2); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case storage_account_url is not used + /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) + /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) + size_t count = function->arguments->size(); + if ((url_arg_idx + 4 <= count) && (count <= url_arg_idx + 7)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 3, &second_arg)) + { + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'account_key' with '[HIDDEN]' if account_key is used in the signature + if (url_arg_idx + 4 < count) + markSecretArgument(url_arg_idx + 4); + } + + void findURLSecretArguments() + { + if (!isNamedCollectionName(0)) + excludeS3OrURLNestedMaps(); + } + + bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const + { + if (arg_idx >= function->arguments->size()) + return false; + + return tryGetStringFromArgument(*function->arguments->at(arg_idx), res, allow_identifier); + } + + static bool tryGetStringFromArgument(const AbstractFunction::Argument & argument, String * res, bool allow_identifier = true) + { + return argument.tryGetString(res, allow_identifier); + } + + void findRemoteFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// remote(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + return; + } + + /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: + /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) + + /// But we should check the number of arguments first because we don't need to do any replacements in case of + /// remote('addresses_expr', db.table) + if (function->arguments->size() < 3) + return; + + size_t arg_num = 1; + + /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. + auto table_function = function->arguments->at(arg_num)->getFunction(); + if (table_function && KnownTableFunctionNames::instance().exists(table_function->name())) + { + ++arg_num; + } + else + { + std::optional database; + std::optional qualified_table_name; + if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) + { + /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. + /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' + /// before the argument 'password'. So it's safer to wipe two arguments just in case. + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `user`. + markSecretArgument(arg_num + 2); + } + if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `sharding_key`. + markSecretArgument(arg_num + 3); + } + return; + } + + /// Skip the current argument (which is either a database name or a qualified table name). + ++arg_num; + if (database) + { + /// Skip the 'table' argument if the previous argument was a database name. + ++arg_num; + } + } + + /// Skip username. + ++arg_num; + + /// Do our replacement: + /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); + if (can_be_password) + markSecretArgument(arg_num); + } + + /// Tries to get either a database name or a qualified table name from an argument. + /// Empty string is also allowed (it means the default database). + /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. + bool tryGetDatabaseNameOrQualifiedTableName( + size_t arg_idx, + std::optional & res_database, + std::optional & res_qualified_table_name) const + { + res_database.reset(); + res_qualified_table_name.reset(); + + String str; + if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) + return false; + + if (str.empty()) + { + res_database = ""; + return true; + } + + auto qualified_table_name = QualifiedTableName::tryParseFromString(str); + if (!qualified_table_name) + return false; + + if (qualified_table_name->database.empty()) + res_database = std::move(qualified_table_name->table); + else + res_qualified_table_name = std::move(qualified_table_name); + return true; + } + + void findEncryptionFunctionSecretArguments() + { + if (function->arguments->size() == 0) + return; + + /// We replace all arguments after 'mode' with '[HIDDEN]': + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') + result.start = 1; + result.count = function->arguments->size() - 1; + } + + void findTableEngineSecretArguments() + { + const String & engine_name = function->name(); + if (engine_name == "ExternalDistributed") + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + findExternalDistributedTableEngineSecretArguments(); + } + else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) + { + /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) + /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || + (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) + { + /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) + findS3TableEngineSecretArguments(); + } + else if (engine_name == "URL") + { + findURLSecretArguments(); + } + } + + void findExternalDistributedTableEngineSecretArguments() + { + if (isNamedCollectionName(1)) + { + /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 2); + } + else + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + markSecretArgument(5); + } + } + + void findS3TableEngineSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'secret_access_key') + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((3 <= count) && (count <= 4)) + { + String second_arg; + if (tryGetStringFromArgument(1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (count == 3) + { + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: S3('url', 'format', ...) + } + } + } + + /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (2 < count) + markSecretArgument(2); + } + + void findDatabaseEngineSecretArguments() + { + const String & engine_name = function->name(); + if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || + (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL")) + { + /// MySQL('host:port', 'database', 'user', 'password') + /// PostgreSQL('host:port', 'database', 'user', 'password') + findMySQLDatabaseSecretArguments(); + } + else if (engine_name == "S3") + { + /// S3('url', 'access_key_id', 'secret_access_key') + findS3DatabaseSecretArguments(); + } + } + + void findMySQLDatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// MySQL(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// MySQL('host:port', 'database', 'user', 'password') + markSecretArgument(3); + } + } + + void findS3DatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'password', ...) + findSecretNamedArgument("secret_access_key", 1); + } + else + { + /// S3('url', 'access_key_id', 'secret_access_key') + markSecretArgument(2); + } + } + + void findBackupNameSecretArguments() + { + const String & engine_name = function->name(); + if (engine_name == "S3") + { + /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) + markSecretArgument(2); + } + } + + /// Whether a specified argument can be the name of a named collection? + bool isNamedCollectionName(size_t arg_idx) const + { + if (function->arguments->size() <= arg_idx) + return false; + + return function->arguments->at(arg_idx)->isIdentifier(); + } + + /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. + void findSecretNamedArgument(const std::string_view & key, size_t start = 0) + { + for (size_t i = start; i < function->arguments->size(); ++i) + { + const auto & argument = function->arguments->at(i); + const auto equals_func = argument->getFunction(); + if (!equals_func || (equals_func->name() != "equals")) + continue; + + if (!equals_func->arguments || equals_func->arguments->size() != 2) + continue; + + String found_key; + if (!tryGetStringFromArgument(*equals_func->arguments->at(0), &found_key)) + continue; + + if (found_key == key) + markSecretArgument(i, /* argument_is_named= */ true); + } + } }; } diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index 94da30922cc..d2d7a63aff1 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -1,35 +1,91 @@ #pragma once #include -#include #include #include #include -#include - -#include namespace DB { - -/// Finds arguments of a specified function which should not be displayed for most users for security reasons. -/// That involves passwords and secret keys. -class FunctionSecretArgumentsFinderAST +class FunctionAST : public AbstractFunction { public: - explicit FunctionSecretArgumentsFinderAST(const ASTFunction & function_) : function(function_) + class ArgumentAST : public Argument + { + public: + explicit ArgumentAST(const IAST * argument_) : argument(argument_) {} + std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + bool isIdentifier() const override { return argument->as(); } + bool tryGetString(String * res, bool allow_identifier = true) const override + { + if (const auto * literal = argument->as()) + { + if (literal->value.getType() != Field::Types::String) + return false; + if (res) + *res = literal->value.safeGet(); + return true; + } + + if (allow_identifier) + { + if (const auto * id = argument->as()) + { + if (res) + *res = id->name(); + return true; + } + } + + return false; + } + private: + const IAST * argument = nullptr; + }; + class ArgumentsAST : public Arguments { - if (!function.arguments) + public: + explicit ArgumentsAST(const ASTs * arguments_) : arguments(arguments_) {} + size_t size() const override { return arguments ? arguments->size() : 0; } + std::unique_ptr at(size_t n) const override + { + return std::make_unique(arguments->at(n).get()); + } + private: + const ASTs * arguments = nullptr; + }; + + explicit FunctionAST(const ASTFunction & function_) : function(&function_) + { + if (!function->arguments) return; - const auto * expr_list = function.arguments->as(); + const auto * expr_list = function->arguments->as(); if (!expr_list) return; - arguments = &expr_list->children; - switch (function.kind) + arguments = std::make_unique(&expr_list->children); + } + + String name() const override { return function->name; } +private: + const ASTFunction * function = nullptr; +}; + +/// Finds arguments of a specified function which should not be displayed for most users for security reasons. +/// That involves passwords and secret keys. +class FunctionSecretArgumentsFinderAST : public FunctionSecretArgumentsFinder +{ +public: + explicit FunctionSecretArgumentsFinderAST(const ASTFunction & function_) + : FunctionSecretArgumentsFinder(std::make_unique(function_)) + { + if (!function->hasArguments()) + return; + + switch (function_.kind) { case ASTFunction::Kind::ORDINARY_FUNCTION: findOrdinaryFunctionSecretArguments(); break; case ASTFunction::Kind::WINDOW_FUNCTION: break; @@ -43,506 +99,7 @@ public: } FunctionSecretArgumentsFinder::Result getResult() const { return result; } - -private: - const ASTFunction & function; - const ASTs * arguments = nullptr; - FunctionSecretArgumentsFinder::Result result; - - void markSecretArgument(size_t index, bool argument_is_named = false) - { - if (index >= arguments->size()) - return; - if (!result.count) - { - result.start = index; - result.are_named = argument_is_named; - } - chassert(index >= result.start); /// We always check arguments consecutively - result.count = index + 1 - result.start; - if (!argument_is_named) - result.are_named = false; - } - - void findOrdinaryFunctionSecretArguments() - { - if ((function.name == "mysql") || (function.name == "postgresql") || (function.name == "mongodb")) - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) - /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") || - (function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg")) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (function.name == "s3Cluster") - { - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ true); - } - else if (function.name == "azureBlobStorage") - { - /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) - findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (function.name == "azureBlobStorageCluster") - { - /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) - findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ true); - } - else if ((function.name == "remote") || (function.name == "remoteSecure")) - { - /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) - findRemoteFunctionSecretArguments(); - } - else if ((function.name == "encrypt") || (function.name == "decrypt") || - (function.name == "aes_encrypt_mysql") || (function.name == "aes_decrypt_mysql") || - (function.name == "tryDecrypt")) - { - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) - findEncryptionFunctionSecretArguments(); - } - else if (function.name == "url") - { - findURLSecretArguments(); - } - } - - void findMySQLFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// mysql(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - markSecretArgument(4); - } - } - - /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should - /// always be at the end). Marks "headers" as secret, if found. - size_t excludeS3OrURLNestedMaps() - { - size_t count = arguments->size(); - while (count > 0) - { - const ASTFunction * f = arguments->at(count - 1)->as(); - if (!f) - break; - if (f->name == "headers") - result.nested_maps.push_back(f->name); - else if (f->name != "extra_credentials") - break; - count -= 1; - } - return count; - } - - void findS3FunctionSecretArguments(bool is_cluster_function) - { - /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (url_arg_idx + 2 < count) - markSecretArgument(url_arg_idx + 2); - } - - void findAzureBlobStorageFunctionSecretArguments(bool is_cluster_function) - { - /// azureBlobStorage('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// azureBlobStorage(named_collection, ..., account_key = 'account_key', ...) - findSecretNamedArgument("account_key", 1); - return; - } - else if (is_cluster_function && isNamedCollectionName(1)) - { - /// azureBlobStorageCluster(cluster, named_collection, ..., account_key = 'account_key', ...) - findSecretNamedArgument("account_key", 2); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case storage_account_url is not used - /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) - /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) - size_t count = arguments->size(); - if ((url_arg_idx + 4 <= count) && (count <= url_arg_idx + 7)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 3, &second_arg)) - { - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'account_key' with '[HIDDEN]' if account_key is used in the signature - if (url_arg_idx + 4 < count) - markSecretArgument(url_arg_idx + 4); - } - - void findURLSecretArguments() - { - if (!isNamedCollectionName(0)) - excludeS3OrURLNestedMaps(); - } - - bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const - { - if (arg_idx >= arguments->size()) - return false; - - return tryGetStringFromArgument(*(*arguments)[arg_idx], res, allow_identifier); - } - - static bool tryGetStringFromArgument(const IAST & argument, String * res, bool allow_identifier = true) - { - if (const auto * literal = argument.as()) - { - if (literal->value.getType() != Field::Types::String) - return false; - if (res) - *res = literal->value.safeGet(); - return true; - } - - if (allow_identifier) - { - if (const auto * id = argument.as()) - { - if (res) - *res = id->name(); - return true; - } - } - - return false; - } - - void findRemoteFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// remote(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - return; - } - - /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: - /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// remote('addresses_expr', db.table) - if (arguments->size() < 3) - return; - - size_t arg_num = 1; - - /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. - const auto * table_function = (*arguments)[arg_num]->as(); - if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) - { - ++arg_num; - } - else - { - std::optional database; - std::optional qualified_table_name; - if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) - { - /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. - /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' - /// before the argument 'password'. So it's safer to wipe two arguments just in case. - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `user`. - markSecretArgument(arg_num + 2); - } - if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `sharding_key`. - markSecretArgument(arg_num + 3); - } - return; - } - - /// Skip the current argument (which is either a database name or a qualified table name). - ++arg_num; - if (database) - { - /// Skip the 'table' argument if the previous argument was a database name. - ++arg_num; - } - } - - /// Skip username. - ++arg_num; - - /// Do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); - if (can_be_password) - markSecretArgument(arg_num); - } - - /// Tries to get either a database name or a qualified table name from an argument. - /// Empty string is also allowed (it means the default database). - /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. - bool tryGetDatabaseNameOrQualifiedTableName( - size_t arg_idx, - std::optional & res_database, - std::optional & res_qualified_table_name) const - { - res_database.reset(); - res_qualified_table_name.reset(); - - String str; - if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) - return false; - - if (str.empty()) - { - res_database = ""; - return true; - } - - auto qualified_table_name = QualifiedTableName::tryParseFromString(str); - if (!qualified_table_name) - return false; - - if (qualified_table_name->database.empty()) - res_database = std::move(qualified_table_name->table); - else - res_qualified_table_name = std::move(qualified_table_name); - return true; - } - - void findEncryptionFunctionSecretArguments() - { - if (arguments->empty()) - return; - - /// We replace all arguments after 'mode' with '[HIDDEN]': - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') - result.start = 1; - result.count = arguments->size() - 1; - } - - void findTableEngineSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "ExternalDistributed") - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - findExternalDistributedTableEngineSecretArguments(); - } - else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) - { - /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) - /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || - (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) - { - /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) - findS3TableEngineSecretArguments(); - } - else if (engine_name == "URL") - { - findURLSecretArguments(); - } - } - - void findExternalDistributedTableEngineSecretArguments() - { - if (isNamedCollectionName(1)) - { - /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 2); - } - else - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - markSecretArgument(5); - } - } - - void findS3TableEngineSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'secret_access_key') - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((3 <= count) && (count <= 4)) - { - String second_arg; - if (tryGetStringFromArgument(1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (count == 3) - { - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: S3('url', 'format', ...) - } - } - } - - /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (2 < count) - markSecretArgument(2); - } - - void findDatabaseEngineSecretArguments() - { - const String & engine_name = function.name; - if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || - (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL")) - { - /// MySQL('host:port', 'database', 'user', 'password') - /// PostgreSQL('host:port', 'database', 'user', 'password') - findMySQLDatabaseSecretArguments(); - } - else if (engine_name == "S3") - { - /// S3('url', 'access_key_id', 'secret_access_key') - findS3DatabaseSecretArguments(); - } - } - - void findMySQLDatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// MySQL(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// MySQL('host:port', 'database', 'user', 'password') - markSecretArgument(3); - } - } - - void findS3DatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'password', ...) - findSecretNamedArgument("secret_access_key", 1); - } - else - { - /// S3('url', 'access_key_id', 'secret_access_key') - markSecretArgument(2); - } - } - - void findBackupNameSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "S3") - { - /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) - markSecretArgument(2); - } - } - - /// Whether a specified argument can be the name of a named collection? - bool isNamedCollectionName(size_t arg_idx) const - { - if (arguments->size() <= arg_idx) - return false; - - const auto * identifier = (*arguments)[arg_idx]->as(); - return identifier != nullptr; - } - - /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. - void findSecretNamedArgument(const std::string_view & key, size_t start = 0) - { - for (size_t i = start; i < arguments->size(); ++i) - { - const auto & argument = (*arguments)[i]; - const auto * equals_func = argument->as(); - if (!equals_func || (equals_func->name != "equals")) - continue; - - const auto * expr_list = equals_func->arguments->as(); - if (!expr_list) - continue; - - const auto & equal_args = expr_list->children; - if (equal_args.size() != 2) - continue; - - String found_key; - if (!tryGetStringFromArgument(*equal_args[0], &found_key)) - continue; - - if (found_key == key) - markSecretArgument(i, /* argument_is_named= */ true); - } - } }; + } From 56cfa74a14f36e89c7ae9b7ab5f27f4b042cab48 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 14 Sep 2024 13:32:52 +0000 Subject: [PATCH 073/128] fix --- src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h | 7 ++++++- src/Parsers/FunctionSecretArgumentsFinderAST.h | 7 ++++++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 3c0e5974d16..8e8f56760e4 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -16,7 +16,12 @@ public: { public: explicit ArgumentTreeNode(const IQueryTreeNode * argument_) : argument(argument_) {} - std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + std::unique_ptr getFunction() const override + { + if (const auto * f = argument->as()) + return std::make_unique(*f); + return nullptr; + } bool isIdentifier() const override { return argument->as(); } bool tryGetString(String * res, bool allow_identifier = true) const override { diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index d2d7a63aff1..9430053d6a5 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -16,7 +16,12 @@ public: { public: explicit ArgumentAST(const IAST * argument_) : argument(argument_) {} - std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + std::unique_ptr getFunction() const override + { + if (const auto * f = argument->as()) + return std::make_unique(*f); + return nullptr; + } bool isIdentifier() const override { return argument->as(); } bool tryGetString(String * res, bool allow_identifier = true) const override { From 6f63a7b213170b5b561b904d44c4cafa62a88dbb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 14 Sep 2024 16:46:48 +0000 Subject: [PATCH 074/128] fix tidy --- src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h | 2 +- src/Parsers/FunctionSecretArgumentsFinderAST.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 8e8f56760e4..c598adfd98e 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -23,7 +23,7 @@ public: return nullptr; } bool isIdentifier() const override { return argument->as(); } - bool tryGetString(String * res, bool allow_identifier = true) const override + bool tryGetString(String * res, bool allow_identifier) const override { if (const auto * literal = argument->as()) { diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index 9430053d6a5..b89007c619d 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -23,7 +23,7 @@ public: return nullptr; } bool isIdentifier() const override { return argument->as(); } - bool tryGetString(String * res, bool allow_identifier = true) const override + bool tryGetString(String * res, bool allow_identifier) const override { if (const auto * literal = argument->as()) { From d223c4547faf36ba909e57c8a7d1094f33d44ea3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 16 Sep 2024 08:35:05 -0400 Subject: [PATCH 075/128] fix after master merge --- src/Parsers/FunctionSecretArgumentsFinder.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 77facd715f8..434d027c9fc 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -89,7 +89,8 @@ protected: findMySQLFunctionSecretArguments(); } else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || - (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg")) + (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg") || + (function.name == "gcs")) { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) findS3FunctionSecretArguments(/* is_cluster_function= */ false); From 04f23332c3d11a6098d69114a882f51cd5f13e9a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 16 Sep 2024 15:59:22 +0200 Subject: [PATCH 076/128] fix filter issue --- src/Storages/VirtualColumnUtils.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index b5deab95f3f..4e163be9128 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -276,7 +276,8 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const { auto string_column = std::make_shared()->createColumn(); string_column->insert(it->second); - block.insert({column.type->createColumn(), column.type, column.name}); + if (!block.has(column.name)) + block.insert({column.type->createColumn(), column.type, column.name}); partitioning_columns.push_back({string_column->getPtr(), column.type, column.name}); keys.erase(it); } @@ -286,7 +287,7 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); partitioning_columns.resize(paths.size()); - for (size_t i = 0; i != partitioning_columns.size(); ++i) + for (size_t i = 0; i != paths.size(); ++i) addFilterDataToVirtualColumns(block, paths[i], i, partitioning_columns[i], context); filterBlockWithExpression(actions, block); From 19e219758290e80261700bbe55adb48a607a0309 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 16 Sep 2024 10:38:28 -0400 Subject: [PATCH 077/128] fix --- src/Parsers/FunctionSecretArgumentsFinder.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 434d027c9fc..7836a863920 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -90,7 +90,7 @@ protected: } else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg") || - (function.name == "gcs")) + (function->name() == "gcs")) { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) findS3FunctionSecretArguments(/* is_cluster_function= */ false); From 0cdec0acf10bba22b2cbe3eee7e26d7365739587 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 16 Sep 2024 19:13:30 +0200 Subject: [PATCH 078/128] fix logical error --- src/Storages/VirtualColumnUtils.cpp | 28 ++++++++++++++++++---------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 4e163be9128..70133cf02ca 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -207,7 +207,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto return desc; } -static void addFilterDataToVirtualColumns(Block & block, const String & path, size_t idx, ColumnWithTypeAndName partitioning_column, const ContextPtr & context) +static void addFilterDataToDefaultColumns(Block & block, const String & path, size_t idx) { if (block.has("_path")) block.getByName("_path").column->assumeMutableRef().insert(path); @@ -224,16 +224,22 @@ static void addFilterDataToVirtualColumns(Block & block, const String & path, si block.getByName("_file").column->assumeMutableRef().insert(file); } - if (block.has(partitioning_column.name)) - { - auto & column = block.getByName(partitioning_column.name).column; - ReadBufferFromString buf(partitioning_column.column->getDataAt(0).toView()); - partitioning_column.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); - } - block.getByName("_idx").column->assumeMutableRef().insert(idx); } +static void addFilterDataToPartitioningColumns(Block & block, ColumnsWithTypeAndName partitioning_keys, const ContextPtr & context) +{ + for (const auto & item : partitioning_keys) + { + if (block.has(item.name)) + { + auto & column = block.getByName(item.name).column; + ReadBufferFromString buf(item.column->getDataAt(0).toView()); + item.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); + } + } +} + std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { if (!predicate || virtual_columns.empty()) @@ -286,9 +292,11 @@ ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); - partitioning_columns.resize(paths.size()); for (size_t i = 0; i != paths.size(); ++i) - addFilterDataToVirtualColumns(block, paths[i], i, partitioning_columns[i], context); + addFilterDataToDefaultColumns(block, paths[i], i); + + if (context->getSettingsRef().use_hive_partitioning) + addFilterDataToPartitioningColumns(block, partitioning_columns, context); filterBlockWithExpression(actions, block); From 676b6238d05941156089861fe1e0f4d48f56dbd7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 17 Sep 2024 10:30:39 +0200 Subject: [PATCH 079/128] Update comments --- src/Coordination/KeeperStateMachine.h | 5 ++++- src/Coordination/KeeperStorage.cpp | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 7e6830ec1ba..03204036e7e 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -126,8 +126,11 @@ protected: /// Mutex for snapshots mutable std::mutex snapshots_lock; + /// Lock for the storage + /// Storage works in thread-safe way ONLY for preprocessing/processing + /// In any other case, unique storage lock needs to be taken mutable SharedMutex storage_mutex; - /// Lock for storage and responses_queue. It's important to process requests + /// Lock for processing and responses_queue. It's important to process requests /// and push them to the responses queue while holding this lock. Otherwise /// we can get strange cases when, for example client send read request with /// watch and after that receive watch response and only receive response diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index fe7bcf71160..20049fd23e8 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -772,6 +772,7 @@ void KeeperStorage::UncommittedState::cleanup(int64_t commit_zxid) template void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) { + // we can only rollback the last zxid (if there is any) std::list rollback_deltas; { std::lock_guard lock(deltas_mutex); From cb92aaf968537cc767a87f3a4e139cf5957a298c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 17 Sep 2024 11:26:13 +0200 Subject: [PATCH 080/128] fix 03232_file_path_normalizing --- tests/queries/0_stateless/03232_file_path_normalizing.reference | 2 +- tests/queries/0_stateless/03232_file_path_normalizing.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.reference b/tests/queries/0_stateless/03232_file_path_normalizing.reference index 953db2c5dfe..3b41cf34056 100644 --- a/tests/queries/0_stateless/03232_file_path_normalizing.reference +++ b/tests/queries/0_stateless/03232_file_path_normalizing.reference @@ -1 +1 @@ -data_hive/partitioning/column0=Elizabeth/sample.parquet +data_hive/partitioning/non_existing_column=Elizabeth/sample.parquet diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.sh b/tests/queries/0_stateless/03232_file_path_normalizing.sh index e7a7a65be51..add6049f9b5 100755 --- a/tests/queries/0_stateless/03232_file_path_normalizing.sh +++ b/tests/queries/0_stateless/03232_file_path_normalizing.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL -q "SELECT substring(_path, position(_path, 'data_hive')) FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') LIMIT 1;" +$CLICKHOUSE_LOCAL -q "SELECT substring(_path, position(_path, 'data_hive')) FROM file('$CURDIR/data_hive/partitioning/non_existing_column=*/sample.parquet') LIMIT 1;" From 3a299f382d4f2be45228a8a5e7b031a08830ace5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Sep 2024 12:48:35 +0200 Subject: [PATCH 081/128] Refactor --- .../ObjectStorageQueueMetadata.cpp | 105 ++++++++++-------- .../ObjectStorageQueueMetadata.h | 14 ++- .../ObjectStorageQueueTableMetadata.cpp | 8 ++ .../ObjectStorageQueueTableMetadata.h | 3 + .../StorageObjectStorageQueue.cpp | 12 +- 5 files changed, 83 insertions(+), 59 deletions(-) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp index 2da4aa6b665..8dbf51a9cf9 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp @@ -110,16 +110,19 @@ private: ObjectStorageQueueMetadata::ObjectStorageQueueMetadata( const fs::path & zookeeper_path_, const ObjectStorageQueueTableMetadata & table_metadata_, - const ObjectStorageQueueSettings & settings_) - : settings(settings_) - , table_metadata(table_metadata_) + size_t cleanup_interval_min_ms_, + size_t cleanup_interval_max_ms_) + : table_metadata(table_metadata_) + , mode(table_metadata.getMode()) , zookeeper_path(zookeeper_path_) - , buckets_num(getBucketsNum(settings_)) + , buckets_num(getBucketsNum(table_metadata_)) + , cleanup_interval_min_ms(cleanup_interval_min_ms_) + , cleanup_interval_max_ms(cleanup_interval_max_ms_) , log(getLogger("StorageObjectStorageQueue(" + zookeeper_path_.string() + ")")) , local_file_statuses(std::make_shared()) { - if (settings.mode == ObjectStorageQueueMode::UNORDERED - && (settings.tracked_files_limit || settings.tracked_file_ttl_sec)) + if (mode == ObjectStorageQueueMode::UNORDERED + && (table_metadata.tracked_files_limit || table_metadata.tracked_file_ttl_sec)) { task = Context::getGlobalContextInstance()->getSchedulePool().createTask( "ObjectStorageQueueCleanupFunc", @@ -128,10 +131,10 @@ ObjectStorageQueueMetadata::ObjectStorageQueueMetadata( task->activate(); task->scheduleAfter( generateRescheduleInterval( - settings.cleanup_interval_min_ms, settings.cleanup_interval_max_ms)); + cleanup_interval_min_ms, cleanup_interval_max_ms)); } LOG_TRACE(log, "Mode: {}, buckets: {}, processing threads: {}, result buckets num: {}", - settings.mode.toString(), settings.buckets, settings.processing_threads_num, buckets_num); + table_metadata.mode, table_metadata.buckets, table_metadata.processing_threads_num, buckets_num); } @@ -162,7 +165,7 @@ ObjectStorageQueueMetadata::FileMetadataPtr ObjectStorageQueueMetadata::getFileM ObjectStorageQueueOrderedFileMetadata::BucketInfoPtr bucket_info) { auto file_status = local_file_statuses->get(path, /* create */true); - switch (settings.mode.value) + switch (mode) { case ObjectStorageQueueMode::ORDERED: return std::make_shared( @@ -171,39 +174,28 @@ ObjectStorageQueueMetadata::FileMetadataPtr ObjectStorageQueueMetadata::getFileM file_status, bucket_info, buckets_num, - settings.loading_retries, + table_metadata.loading_retries, log); case ObjectStorageQueueMode::UNORDERED: return std::make_shared( zookeeper_path, path, file_status, - settings.loading_retries, + table_metadata.loading_retries, log); } } -size_t ObjectStorageQueueMetadata::getBucketsNum(const ObjectStorageQueueSettings & settings) +size_t ObjectStorageQueueMetadata::getBucketsNum(const ObjectStorageQueueTableMetadata & metadata) { - if (settings.buckets) - return settings.buckets; - if (settings.processing_threads_num) - return settings.processing_threads_num; - return 0; -} - -size_t ObjectStorageQueueMetadata::getBucketsNum(const ObjectStorageQueueTableMetadata & settings) -{ - if (settings.buckets) - return settings.buckets; - if (settings.processing_threads_num) - return settings.processing_threads_num; - return 0; + if (metadata.buckets) + return metadata.buckets; + return metadata.processing_threads_num; } bool ObjectStorageQueueMetadata::useBucketsForProcessing() const { - return settings.mode == ObjectStorageQueueMode::ORDERED && (buckets_num > 1); + return mode == ObjectStorageQueueMode::ORDERED && (buckets_num > 1); } ObjectStorageQueueMetadata::Bucket ObjectStorageQueueMetadata::getBucketForPath(const std::string & path) const @@ -217,18 +209,35 @@ ObjectStorageQueueMetadata::tryAcquireBucket(const Bucket & bucket, const Proces return ObjectStorageQueueOrderedFileMetadata::tryAcquireBucket(zookeeper_path, bucket, processor, log); } -void ObjectStorageQueueMetadata::syncWithKeeper( +ObjectStorageQueueTableMetadata ObjectStorageQueueMetadata::syncWithKeeper( const fs::path & zookeeper_path, - const ObjectStorageQueueTableMetadata & table_metadata, const ObjectStorageQueueSettings & settings, + const ColumnsDescription & columns, + const std::string & format, LoggerPtr log) { - const auto table_metadata_path = zookeeper_path / "metadata"; - const auto buckets_num = getBucketsNum(settings); - const auto metadata_paths = settings.mode == ObjectStorageQueueMode::ORDERED - ? ObjectStorageQueueOrderedFileMetadata::getMetadataPaths(buckets_num) - : ObjectStorageQueueUnorderedFileMetadata::getMetadataPaths(); + ObjectStorageQueueTableMetadata table_metadata(settings, columns, format); + std::vector metadata_paths; + size_t buckets_num = 0; + if (settings.mode == ObjectStorageQueueMode::ORDERED) + { + buckets_num = getBucketsNum(table_metadata); + if (buckets_num == 0) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot have zero values of `processing_threads_num` and `buckets`"); + + LOG_TRACE(log, "Local buckets num: {}", buckets_num); + + metadata_paths = ObjectStorageQueueOrderedFileMetadata::getMetadataPaths(buckets_num); + } + else + { + metadata_paths = ObjectStorageQueueUnorderedFileMetadata::getMetadataPaths(); + } + + const auto table_metadata_path = zookeeper_path / "metadata"; auto zookeeper = getZooKeeper(); zookeeper->createAncestors(zookeeper_path); @@ -242,7 +251,7 @@ void ObjectStorageQueueMetadata::syncWithKeeper( LOG_TRACE(log, "Metadata in keeper: {}", metadata_str); table_metadata.checkEquals(metadata_from_zk); - return; + return table_metadata; } Coordination::Requests requests; @@ -256,15 +265,15 @@ void ObjectStorageQueueMetadata::syncWithKeeper( requests.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); } - if (!settings.last_processed_path.value.empty()) + if (!table_metadata.last_processed_path.empty()) { ObjectStorageQueueOrderedFileMetadata( zookeeper_path, - settings.last_processed_path, + table_metadata.last_processed_path, std::make_shared(), /* bucket_info */nullptr, buckets_num, - settings.loading_retries, + table_metadata.loading_retries, log).setProcessedAtStartRequests(requests, zookeeper); } @@ -283,7 +292,7 @@ void ObjectStorageQueueMetadata::syncWithKeeper( else if (code != Coordination::Error::ZOK) zkutil::KeeperMultiException::check(code, requests, responses); - return; + return table_metadata; } throw Exception( @@ -295,7 +304,7 @@ void ObjectStorageQueueMetadata::syncWithKeeper( void ObjectStorageQueueMetadata::cleanupThreadFunc() { /// A background task is responsible for maintaining - /// settings.tracked_files_limit and max_set_age settings for `unordered` processing mode. + /// table_metadata.tracked_files_limit and max_set_age settings for `unordered` processing mode. if (shutdown_called) return; @@ -314,7 +323,7 @@ void ObjectStorageQueueMetadata::cleanupThreadFunc() task->scheduleAfter( generateRescheduleInterval( - settings.cleanup_interval_min_ms, settings.cleanup_interval_max_ms)); + cleanup_interval_min_ms, cleanup_interval_max_ms)); } void ObjectStorageQueueMetadata::cleanupThreadFuncImpl() @@ -357,11 +366,11 @@ void ObjectStorageQueueMetadata::cleanupThreadFuncImpl() return; } - chassert(settings.tracked_files_limit || settings.tracked_file_ttl_sec); - const bool check_nodes_limit = settings.tracked_files_limit > 0; - const bool check_nodes_ttl = settings.tracked_file_ttl_sec > 0; + chassert(table_metadata.tracked_files_limit || table_metadata.tracked_file_ttl_sec); + const bool check_nodes_limit = table_metadata.tracked_files_limit > 0; + const bool check_nodes_ttl = table_metadata.tracked_file_ttl_sec > 0; - const bool nodes_limit_exceeded = nodes_num > settings.tracked_files_limit; + const bool nodes_limit_exceeded = nodes_num > table_metadata.tracked_files_limit; if ((!nodes_limit_exceeded || !check_nodes_limit) && !check_nodes_ttl) { LOG_TEST(log, "No limit exceeded"); @@ -434,9 +443,9 @@ void ObjectStorageQueueMetadata::cleanupThreadFuncImpl() wb << fmt::format("Node: {}, path: {}, timestamp: {};\n", node, metadata.file_path, metadata.last_processed_timestamp); return wb.str(); }; - LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", settings.tracked_files_limit, settings.tracked_file_ttl_sec, get_nodes_str()); + LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", table_metadata.tracked_files_limit, table_metadata.tracked_file_ttl_sec, get_nodes_str()); - size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes_num - settings.tracked_files_limit : 0; + size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes_num - table_metadata.tracked_files_limit : 0; for (const auto & node : sorted_nodes) { if (nodes_to_remove) @@ -455,7 +464,7 @@ void ObjectStorageQueueMetadata::cleanupThreadFuncImpl() else if (check_nodes_ttl) { UInt64 node_age = getCurrentTime() - node.metadata.last_processed_timestamp; - if (node_age >= settings.tracked_file_ttl_sec) + if (node_age >= table_metadata.tracked_file_ttl_sec) { LOG_TRACE(log, "Removing node at path {} ({}) because file ttl is reached", node.metadata.file_path, node.zk_path); diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h index 71d26ca7c47..0e4298adbaa 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h @@ -56,14 +56,16 @@ public: ObjectStorageQueueMetadata( const fs::path & zookeeper_path_, const ObjectStorageQueueTableMetadata & table_metadata_, - const ObjectStorageQueueSettings & settings_); + size_t cleanup_interval_min_ms_, + size_t cleanup_interval_max_ms_); ~ObjectStorageQueueMetadata(); - static void syncWithKeeper( + static ObjectStorageQueueTableMetadata syncWithKeeper( const fs::path & zookeeper_path, - const ObjectStorageQueueTableMetadata & table_metadata, const ObjectStorageQueueSettings & settings, + const ColumnsDescription & columns, + const std::string & format, LoggerPtr log); void shutdown(); @@ -78,8 +80,7 @@ public: Bucket getBucketForPath(const std::string & path) const; ObjectStorageQueueOrderedFileMetadata::BucketHolderPtr tryAcquireBucket(const Bucket & bucket, const Processor & processor); - static size_t getBucketsNum(const ObjectStorageQueueSettings & settings); - static size_t getBucketsNum(const ObjectStorageQueueTableMetadata & settings); + static size_t getBucketsNum(const ObjectStorageQueueTableMetadata & metadata); void checkTableMetadataEquals(const ObjectStorageQueueMetadata & other); @@ -90,10 +91,11 @@ private: void cleanupThreadFunc(); void cleanupThreadFuncImpl(); - ObjectStorageQueueSettings settings; ObjectStorageQueueTableMetadata table_metadata; + const ObjectStorageQueueMode mode; const fs::path zookeeper_path; const size_t buckets_num; + const size_t cleanup_interval_min_ms, cleanup_interval_max_ms; LoggerPtr log; diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp index 926d5aacda4..aeb085b9f64 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp @@ -42,6 +42,7 @@ ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata( , tracked_file_ttl_sec(engine_settings.tracked_file_ttl_sec) , buckets(engine_settings.buckets) , processing_threads_num(engine_settings.processing_threads_num) + , loading_retries(engine_settings.loading_retries) { } @@ -57,6 +58,7 @@ String ObjectStorageQueueTableMetadata::toString() const json.set("format_name", format_name); json.set("columns", columns); json.set("last_processed_file", last_processed_path); + json.set("loading_retries", loading_retries); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); @@ -64,6 +66,11 @@ String ObjectStorageQueueTableMetadata::toString() const return oss.str(); } +ObjectStorageQueueMode ObjectStorageQueueTableMetadata::getMode() const +{ + return modeFromString(mode); +} + template static auto getOrDefault( const Poco::JSON::Object::Ptr & json, @@ -90,6 +97,7 @@ ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata(const Poco::JSO , buckets(getOrDefault(json, "buckets", "", 0)) , processing_threads_num(getOrDefault(json, "processing_threads_num", "s3queue_", 1)) , last_processed_path(getOrDefault(json, "last_processed_file", "s3queue_", "")) + , loading_retries(getOrDefault(json, "loading_retries", "", 10)) { } diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h index d70b859ae1d..f1cf6428160 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h @@ -27,6 +27,7 @@ struct ObjectStorageQueueTableMetadata const UInt64 buckets; const UInt64 processing_threads_num; const String last_processed_path; + const UInt64 loading_retries; ObjectStorageQueueTableMetadata( const ObjectStorageQueueSettings & engine_settings, @@ -39,6 +40,8 @@ struct ObjectStorageQueueTableMetadata String toString() const; + ObjectStorageQueueMode getMode() const; + void checkEquals(const ObjectStorageQueueTableMetadata & from_zk) const; private: diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 8f11836a11b..b0393a33523 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -62,7 +62,7 @@ namespace return zkutil::extractZooKeeperPath(result_zk_path, true); } - void checkAndAdjustSettings( + void validateSettings( ObjectStorageQueueSettings & queue_settings, bool is_attach) { @@ -144,7 +144,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( throw Exception(ErrorCodes::BAD_QUERY_PARAMETER, "ObjectStorageQueue url must either end with '/' or contain globs"); } - checkAndAdjustSettings(*queue_settings, mode > LoadingStrictnessLevel::CREATE); + validateSettings(*queue_settings, mode > LoadingStrictnessLevel::CREATE); object_storage = configuration->createObjectStorage(context_, /* is_readonly */true); FormatFactory::instance().checkFormatName(configuration->format); @@ -164,10 +164,12 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); - ObjectStorageQueueTableMetadata table_metadata(*queue_settings, storage_metadata.getColumns(), configuration_->format); - ObjectStorageQueueMetadata::syncWithKeeper(zk_path, table_metadata, *queue_settings, log); + auto table_metadata = ObjectStorageQueueMetadata::syncWithKeeper( + zk_path, *queue_settings, storage_metadata.getColumns(), configuration_->format, log); + + auto queue_metadata = std::make_unique( + zk_path, std::move(table_metadata), queue_settings->cleanup_interval_min_ms, queue_settings->cleanup_interval_max_ms); - auto queue_metadata = std::make_unique(zk_path, std::move(table_metadata), *queue_settings); files_metadata = ObjectStorageQueueMetadataFactory::instance().getOrCreate(zk_path, std::move(queue_metadata)); task = getContext()->getSchedulePool().createTask("ObjectStorageQueueStreamingTask", [this] { threadFunc(); }); From 88b22094c82c3d62031ea34d752ae5ec0a02a9c4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Sep 2024 14:11:17 +0200 Subject: [PATCH 082/128] Update test --- .../test_postgresql_replica_database_engine_2/test.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index ae5add228f0..55f1beae12c 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -1253,8 +1253,15 @@ def test_partial_and_full_table(started_cluster): def test_quoting_publication(started_cluster): - NUM_TABLES = 5 postgres_database = "postgres-postgres" + pg_manager3 = PostgresManager() + pg_manager3.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database=postgres_database, + ) + NUM_TABLES = 5 materialized_database = "test-database" pg_manager3.create_and_fill_postgres_tables(NUM_TABLES, 10000) @@ -1346,6 +1353,7 @@ def test_quoting_publication(started_cluster): assert ( result == "postgresql-replica-5\npostgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n" + ) if __name__ == "__main__": From 3a05282bced2b8e4a557ef452afdd5c3fc62471f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Sep 2024 14:26:31 +0200 Subject: [PATCH 083/128] Update assert --- src/Interpreters/Cache/FileCache.cpp | 7 ++++++- tests/clickhouse-test | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 60db406ca72..ffe9a611014 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -718,7 +718,12 @@ FileCache::getOrSet( } } - chassert(file_segments_limit ? file_segments.back()->range().left <= result_range.right : file_segments.back()->range().contains(result_range.right)); + chassert(file_segments_limit + ? file_segments.back()->range().left <= result_range.right + : file_segments.back()->range().contains(result_range.right), + fmt::format("Unexpected state. Back: {}, result range: {}, limit: {}", + file_segments.back()->range().toString(), result_range.toString(), file_segments_limit)); + chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 75a180f356b..a005fdcc439 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -835,7 +835,7 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), - "filesystem_cache_segments_batch_size": lambda: random.choice([0, 3, 10, 50]), + "filesystem_cache_segments_batch_size": lambda: random.choice([0, 1, 2, 3, 5, 10, 50, 100]), "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint( 0, 1 ), From 813bcd896f170b3403088aad01890b07b1065f2b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 17 Sep 2024 12:30:12 +0000 Subject: [PATCH 084/128] Bump to v18.8 --- contrib/postgres | 2 +- contrib/postgres-cmake/pg_config.h | 46 ++++++++++++++++-------------- 2 files changed, 25 insertions(+), 23 deletions(-) diff --git a/contrib/postgres b/contrib/postgres index 665ff8c164d..cfd77000af2 160000 --- a/contrib/postgres +++ b/contrib/postgres @@ -1 +1 @@ -Subproject commit 665ff8c164d56d012e359735efe4d400c0564b44 +Subproject commit cfd77000af28469fcb650485bad65a35e7649e41 diff --git a/contrib/postgres-cmake/pg_config.h b/contrib/postgres-cmake/pg_config.h index ce16eab2239..0060d4960db 100644 --- a/contrib/postgres-cmake/pg_config.h +++ b/contrib/postgres-cmake/pg_config.h @@ -1,18 +1,6 @@ /* src/include/pg_config.h. Generated from pg_config.h.in by configure. */ /* src/include/pg_config.h.in. Generated from configure.in by autoheader. */ -/* Define to the type of arg 1 of 'accept' */ -#define ACCEPT_TYPE_ARG1 int - -/* Define to the type of arg 2 of 'accept' */ -#define ACCEPT_TYPE_ARG2 struct sockaddr * - -/* Define to the type of arg 3 of 'accept' */ -#define ACCEPT_TYPE_ARG3 size_t - -/* Define to the return type of 'accept' */ -#define ACCEPT_TYPE_RETURN int - /* Define if building universal (internal helper macro) */ /* #undef AC_APPLE_UNIVERSAL_BUILD */ @@ -49,6 +37,9 @@ /* Define to the default TCP port number as a string constant. */ #define DEF_PGPORT_STR "5432" +/* Define to the file name extension of dynamically-loadable modules. */ +#define DLSUFFIX ".so" + /* Define to build with GSSAPI support. (--with-gssapi) */ //#define ENABLE_GSS 0 @@ -122,6 +113,9 @@ don't. */ #define HAVE_DECL_SNPRINTF 1 +/* Define to 1 if you have the declaration of `sigwait', and to 0 if you don't. */ +#define HAVE_DECL_SIGWAIT 1 + /* Define to 1 if you have the declaration of `strlcat', and to 0 if you don't. */ #if OS_DARWIN @@ -257,6 +251,9 @@ /* Define to 1 if you have the `inet_aton' function. */ #define HAVE_INET_ATON 1 +/* Define to 1 if you have the `inet_pton' function. */ +#define HAVE_INET_PTON 1 + /* Define to 1 if the system has the type `int64'. */ /* #undef HAVE_INT64 */ @@ -323,6 +320,9 @@ /* Define to 1 if you have the `z' library (-lz). */ #define HAVE_LIBZ 1 +/* Define to 1 if you have the `zstd' library (-lzstd). */ +/* #undef HAVE_LIBZSTD */ + /* Define to 1 if constants of type 'long long int' should have the suffix LL. */ #define HAVE_LL_CONSTANTS 1 @@ -378,6 +378,9 @@ /* Define to 1 if you have the header file. */ #define HAVE_POLL_H 1 +/* Define to 1 if you have a POSIX-conforming sigwait declaration. */ +/* #undef HAVE_POSIX_DECL_SIGWAIT */ + /* Define to 1 if you have the `posix_fadvise' function. */ #define HAVE_POSIX_FADVISE 1 @@ -408,9 +411,6 @@ /* Define to 1 if you have the header file. */ #define HAVE_PWD_H 1 -/* Define to 1 if you have the `random' function. */ -#define HAVE_RANDOM 1 - /* Define to 1 if you have the header file. */ /* #undef HAVE_READLINE_H */ @@ -426,10 +426,6 @@ /* Define to 1 if you have the `rint' function. */ #define HAVE_RINT 1 -/* Define to 1 if you have the global variable - 'rl_completion_append_character'. */ -/* #undef HAVE_RL_COMPLETION_APPEND_CHARACTER */ - /* Define to 1 if you have the `rl_completion_matches' function. */ #define HAVE_RL_COMPLETION_MATCHES 1 @@ -439,6 +435,9 @@ /* Define to 1 if you have the `rl_reset_screen_size' function. */ /* #undef HAVE_RL_RESET_SCREEN_SIZE */ +/* Define to 1 if you have the `rl_variable_bind' function. */ +#define HAVE_RL_VARIABLE_BIND 1 + /* Define to 1 if you have the header file. */ #define HAVE_SECURITY_PAM_APPL_H 1 @@ -451,6 +450,9 @@ /* Define to 1 if you have the `shm_open' function. */ #define HAVE_SHM_OPEN 1 +/* Define to 1 if the system has the type `socklen_t'. */ +#define HAVE_SOCKLEN_T 1 + /* Define to 1 if you have the `sigprocmask' function. */ #define HAVE_SIGPROCMASK 1 @@ -466,9 +468,6 @@ /* Define to 1 if you have spinlocks. */ #define HAVE_SPINLOCKS 1 -/* Define to 1 if you have the `srandom' function. */ -#define HAVE_SRANDOM 1 - /* Define to 1 if you have the `SSL_CTX_set_num_tickets' function. */ /* #define HAVE_SSL_CTX_SET_NUM_TICKETS */ @@ -885,6 +884,9 @@ /* Define to select Win32-style shared memory. */ /* #undef USE_WIN32_SHARED_MEMORY */ +/* Define to 1 to build with ZSTD support. (--with-zstd) */ +/* #undef USE_ZSTD */ + /* Define to 1 if `wcstombs_l' requires . */ /* #undef WCSTOMBS_L_IN_XLOCALE */ From 5ce8604869e9c21bc5b707a985bf4247c1c5a0fd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 17 Sep 2024 12:37:23 +0000 Subject: [PATCH 085/128] Automatic style fix --- tests/clickhouse-test | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a005fdcc439..810bae86cb0 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -835,7 +835,9 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), - "filesystem_cache_segments_batch_size": lambda: random.choice([0, 1, 2, 3, 5, 10, 50, 100]), + "filesystem_cache_segments_batch_size": lambda: random.choice( + [0, 1, 2, 3, 5, 10, 50, 100] + ), "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint( 0, 1 ), From e8d50aa97ff10fc9a853a4689c9346baf23ecd82 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 17 Sep 2024 15:02:33 +0200 Subject: [PATCH 086/128] review --- src/Storages/VirtualColumnUtils.cpp | 53 ++++++++--------------------- 1 file changed, 15 insertions(+), 38 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 70133cf02ca..155420434fa 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -207,7 +207,7 @@ VirtualColumnsDescription getVirtualsForFileLikeStorage(ColumnsDescription & sto return desc; } -static void addFilterDataToDefaultColumns(Block & block, const String & path, size_t idx) +static void addPathAndFileToVirtualColumns(Block & block, const String & path, size_t idx, const FormatSettings & format_settings, bool use_hive_partitioning) { if (block.has("_path")) block.getByName("_path").column->assumeMutableRef().insert(path); @@ -224,20 +224,20 @@ static void addFilterDataToDefaultColumns(Block & block, const String & path, si block.getByName("_file").column->assumeMutableRef().insert(file); } - block.getByName("_idx").column->assumeMutableRef().insert(idx); -} - -static void addFilterDataToPartitioningColumns(Block & block, ColumnsWithTypeAndName partitioning_keys, const ContextPtr & context) -{ - for (const auto & item : partitioning_keys) + if (use_hive_partitioning) { - if (block.has(item.name)) + auto keys_and_values = parseHivePartitioningKeysAndValues(path); + for (const auto & [key, value] : keys_and_values) { - auto & column = block.getByName(item.name).column; - ReadBufferFromString buf(item.column->getDataAt(0).toView()); - item.type->getDefaultSerialization()->deserializeWholeText(column->assumeMutableRef(), buf, getFormatSettings(context)); + if (const auto * column = block.findByName(key)) + { + ReadBufferFromString buf(value); + column->type->getDefaultSerialization()->deserializeWholeText(column->column->assumeMutableRef(), buf, format_settings); + } } } + + block.getByName("_idx").column->assumeMutableRef().insert(idx); } std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) @@ -262,41 +262,18 @@ std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * pr ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { Block block; - std::vector> keys_vec; - ColumnsWithTypeAndName partitioning_columns; + NameSet common_virtuals; if (context->getSettingsRef().use_hive_partitioning) - { - for (const auto & path : paths) - keys_vec.push_back(parseHivePartitioningKeysAndValues(path)); - } + common_virtuals = getVirtualNamesForFileLikeStorage(); for (const auto & column : virtual_columns) { - if (column.name == "_file" || column.name == "_path") + if (column.name == "_file" || column.name == "_path" || !common_virtuals.contains(column.name)) block.insert({column.type->createColumn(), column.type, column.name}); - else - { - for (auto & keys : keys_vec) - { - const auto & it = keys.find(column.name); - if (it != keys.end()) - { - auto string_column = std::make_shared()->createColumn(); - string_column->insert(it->second); - if (!block.has(column.name)) - block.insert({column.type->createColumn(), column.type, column.name}); - partitioning_columns.push_back({string_column->getPtr(), column.type, column.name}); - keys.erase(it); - } - } - } } block.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); for (size_t i = 0; i != paths.size(); ++i) - addFilterDataToDefaultColumns(block, paths[i], i); - - if (context->getSettingsRef().use_hive_partitioning) - addFilterDataToPartitioningColumns(block, partitioning_columns, context); + addPathAndFileToVirtualColumns(block, paths[i], i, getFormatSettings(context), context->getSettingsRef().use_hive_partitioning); filterBlockWithExpression(actions, block); From e30ebfa23efeb7ba46f217770ceb9af0003904c4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Sep 2024 15:24:02 +0200 Subject: [PATCH 087/128] Add mode validation --- .../ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp index aeb085b9f64..e85588394ee 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp @@ -27,6 +27,12 @@ namespace return ObjectStorageQueueMode::UNORDERED; throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected ObjectStorageQueue mode: {}", mode); } + + void validateMode(const std::string & mode) + { + if (mode != "ordered" && mode != "unordered") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected ObjectStorageQueue mode: {}", mode); + } } @@ -99,6 +105,7 @@ ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata(const Poco::JSO , last_processed_path(getOrDefault(json, "last_processed_file", "s3queue_", "")) , loading_retries(getOrDefault(json, "loading_retries", "", 10)) { + validateMode(mode); } ObjectStorageQueueTableMetadata ObjectStorageQueueTableMetadata::parse(const String & metadata_str) From 3a7c68a052381b2218017b73f40a92effcd9cb93 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 17 Sep 2024 15:39:26 +0200 Subject: [PATCH 088/128] Update src/Storages/VirtualColumnUtils.cpp Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Storages/VirtualColumnUtils.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 155420434fa..14bf8ac8c13 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -262,9 +262,7 @@ std::optional createPathAndFileFilterDAG(const ActionsDAG::Node * pr ColumnPtr getFilterByPathAndFileIndexes(const std::vector & paths, const ExpressionActionsPtr & actions, const NamesAndTypesList & virtual_columns, const ContextPtr & context) { Block block; - NameSet common_virtuals; - if (context->getSettingsRef().use_hive_partitioning) - common_virtuals = getVirtualNamesForFileLikeStorage(); + NameSet common_virtuals = getVirtualNamesForFileLikeStorage(); for (const auto & column : virtual_columns) { if (column.name == "_file" || column.name == "_path" || !common_virtuals.contains(column.name)) From 8db3dddb3dc38dc4b75a1a62ece44e2c382482a6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 17 Sep 2024 14:15:33 +0200 Subject: [PATCH 089/128] Fix watches count and list request --- src/Coordination/KeeperStorage.cpp | 207 +++++++++++++++++------------ src/Coordination/KeeperStorage.h | 1 + 2 files changed, 121 insertions(+), 87 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 397e5db1f71..28bc1bf112b 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -130,7 +130,11 @@ void unregisterEphemeralPath(KeeperStorageBase::Ephemerals & ephemerals, int64_t } KeeperStorageBase::ResponsesForSessions processWatchesImpl( - const String & path, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches, Coordination::Event event_type) + const String & path, + KeeperStorageBase::Watches & watches, + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers, + Coordination::Event event_type) { KeeperStorageBase::ResponsesForSessions result; auto watch_it = watches.find(path); @@ -143,7 +147,11 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl( watch_response->type = event_type; watch_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) + { + [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase(path); + chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_response}); + } watches.erase(watch_it); } @@ -179,7 +187,11 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl( watch_list_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) + { + [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase(path_to_check); + chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_list_response}); + } list_watches.erase(watch_it); } @@ -1322,8 +1334,12 @@ std::list preprocess( } template T> -KeeperStorageBase::ResponsesForSessions -processWatches(const T & /*zk_request*/, KeeperStorageBase::DeltaRange /*deltas*/, KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) +KeeperStorageBase::ResponsesForSessions processWatches( + const T & /*zk_request*/, + KeeperStorageBase::DeltaRange /*deltas*/, + KeeperStorageBase::Watches & /*watches*/, + KeeperStorageBase::Watches & /*list_watches*/, + KeeperStorageBase::SessionAndWatcher & /*sessions_and_watchers*/) { return {}; } @@ -1371,9 +1387,10 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperCreateRequest & zk_request, KeeperStorageBase::DeltaRange /*deltas*/, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { - return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CREATED); + return processWatchesImpl(zk_request.getPath(), watches, list_watches, sessions_and_watchers, Coordination::Event::CREATED); } template @@ -1616,9 +1633,10 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperRemoveRequest & zk_request, KeeperStorageBase::DeltaRange /*deltas*/, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { - return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::DELETED); + return processWatchesImpl(zk_request.getPath(), watches, list_watches, sessions_and_watchers, Coordination::Event::DELETED); } template @@ -1903,7 +1921,8 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperRemoveRecursiveRequest & /*zk_request*/, KeeperStorageBase::DeltaRange deltas, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { KeeperStorageBase::ResponsesForSessions responses; for (const auto & delta : deltas) @@ -1911,7 +1930,7 @@ KeeperStorageBase::ResponsesForSessions processWatches( const auto * remove_delta = std::get_if(&delta.operation); if (remove_delta) { - auto new_responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); + auto new_responses = processWatchesImpl(delta.path, watches, list_watches, sessions_and_watchers, Coordination::Event::DELETED); responses.insert(responses.end(), std::make_move_iterator(new_responses.begin()), std::make_move_iterator(new_responses.end())); } } @@ -2110,9 +2129,10 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperSetRequest & zk_request, KeeperStorageBase::DeltaRange /*deltas*/, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { - return processWatchesImpl(zk_request.getPath(), watches, list_watches, Coordination::Event::CHANGED); + return processWatchesImpl(zk_request.getPath(), watches, list_watches, sessions_and_watchers, Coordination::Event::CHANGED); } template @@ -2250,15 +2270,18 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList if (path_prefix.empty()) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty"); - const auto & get_children = [&]() + const auto get_children = [&]() { if constexpr (Storage::use_rocksdb) - return container.getChildren(zk_request.path); + return std::optional{container.getChildren(zk_request.path)}; else - return node_it->value.getChildren(); + return &node_it->value.getChildren(); }; - const auto & children = get_children(); - response->names.reserve(children.size()); + + const auto children = get_children(); + response->names.reserve(children->size()); + + chassert(static_cast(node_it->value.stats.numChildren()) == children->size()); const auto add_child = [&](const auto & child) { @@ -2290,7 +2313,7 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); }; - for (const auto & child : children) + for (const auto & child : *children) { if (add_child(child)) { @@ -2572,7 +2595,8 @@ KeeperStorageBase::ResponsesForSessions processWatches( const Coordination::ZooKeeperMultiRequest & zk_request, KeeperStorageBase::DeltaRange deltas, KeeperStorageBase::Watches & watches, - KeeperStorageBase::Watches & list_watches) + KeeperStorageBase::Watches & list_watches, + KeeperStorageBase::SessionAndWatcher & sessions_and_watchers) { KeeperStorageBase::ResponsesForSessions result; @@ -2581,7 +2605,7 @@ KeeperStorageBase::ResponsesForSessions processWatches( { auto subdeltas = extractSubdeltas(deltas); auto responses = callOnConcreteRequestType( - *generic_request, [&](const auto & subrequest) { return processWatches(subrequest, subdeltas, watches, list_watches); }); + *generic_request, [&](const auto & subrequest) { return processWatches(subrequest, subdeltas, watches, list_watches, sessions_and_watchers); }); result.insert(result.end(), responses.begin(), responses.end()); } return result; @@ -2974,52 +2998,47 @@ void KeeperStorage::preprocessRequest( if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { + std::unordered_map< + std::string, + UpdateNodeStatDelta, + StringHashForHeterogeneousLookup, + StringHashForHeterogeneousLookup::transparent_key_equal> + parent_updates; + const auto process_ephemerals_for_session = [&](auto & current_ephemerals) { - const auto process_ephemerals_for_session = [&](auto & current_ephemerals) + auto session_ephemerals = current_ephemerals.find(session_id); + if (session_ephemerals != current_ephemerals.end()) { - auto session_ephemerals = current_ephemerals.find(session_id); - if (session_ephemerals != current_ephemerals.end()) + for (const auto & ephemeral_path : session_ephemerals->second) { - std::unordered_map parent_updates; - for (const auto & ephemeral_path : session_ephemerals->second) + auto node = uncommitted_state.getNode(ephemeral_path, /*should_lock_storage=*/false); + + /// maybe the node is deleted or recreated with different session_id in the uncommitted state + if (!node || node->stats.ephemeralOwner() != session_id) + continue; + + auto parent_node_path = parentNodePath(ephemeral_path).toView(); + + auto parent_update_it = parent_updates.find(parent_node_path); + if (parent_update_it == parent_updates.end()) { - auto node = uncommitted_state.getNode(ephemeral_path, /*should_lock_storage=*/false); - - /// maybe the node is deleted or recreated with different session_id in the uncommitted state - if (!node || node->stats.ephemeralOwner() != session_id) - continue; - - auto parent_node_path = parentNodePath(ephemeral_path).toView(); - - auto parent_update_it = parent_updates.find(parent_node_path); - if (parent_update_it == parent_updates.end()) - { - auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}, /*should_lock_storage=*/false); - std::tie(parent_update_it, std::ignore) = parent_updates.emplace(parent_node_path, *parent_node); - } - - auto & parent_update_delta = parent_update_it->second; - ++parent_update_delta.new_stats.cversion; - parent_update_delta.new_stats.decreaseNumChildren(); - - new_deltas.emplace_back( - ephemeral_path, - transaction->zxid, - RemoveNodeDelta{.stat = node->stats, .acls = uncommitted_state.getACLs(ephemeral_path), .data = std::string{node->getData()}}); + auto parent_node = uncommitted_state.getNode(StringRef{parent_node_path}, /*should_lock_storage=*/false); + std::tie(parent_update_it, std::ignore) = parent_updates.emplace(parent_node_path, *parent_node); } - for (auto & [parent_path, parent_update_delta] : parent_updates) - { - new_deltas.emplace_back - ( - std::string{parent_path}, - new_last_zxid, - std::move(parent_update_delta) - ); - } + auto & parent_update_delta = parent_update_it->second; + ++parent_update_delta.new_stats.cversion; + parent_update_delta.new_stats.decreaseNumChildren(); + + new_deltas.emplace_back( + ephemeral_path, + transaction->zxid, + RemoveNodeDelta{.stat = node->stats, .acls = uncommitted_state.getACLs(ephemeral_path), .data = std::string{node->getData()}}); } - }; + } + }; + { /// storage lock should always be taken before ephemeral lock std::shared_lock storage_lock(storage_mutex); std::lock_guard ephemeral_lock(ephemeral_mutex); @@ -3028,6 +3047,16 @@ void KeeperStorage::preprocessRequest( uncommitted_state.ephemerals.erase(session_id); } + for (auto & [parent_path, parent_update_delta] : parent_updates) + { + new_deltas.emplace_back + ( + parent_path, + new_last_zxid, + std::move(parent_update_delta) + ); + } + new_deltas.emplace_back(transaction->zxid, CloseSessionDelta{session_id}); new_digest = calculateNodesDigest(new_digest, new_deltas); return; @@ -3132,7 +3161,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process { if (std::holds_alternative(delta.operation)) { - auto responses = processWatchesImpl(delta.path, watches, list_watches, Coordination::Event::DELETED); + auto responses = processWatchesImpl(delta.path, watches, list_watches, sessions_and_watchers, Coordination::Event::DELETED); results.insert(results.end(), responses.begin(), responses.end()); } } @@ -3211,20 +3240,27 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process auto add_watch_result = watches_type[zk_request->getPath()].emplace(session_id); if (add_watch_result.second) + { + ++total_watches_count; sessions_and_watchers[session_id].emplace(zk_request->getPath()); + } } else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists) { auto add_watch_result = watches[zk_request->getPath()].emplace(session_id); if (add_watch_result.second) + { + ++total_watches_count; sessions_and_watchers[session_id].emplace(zk_request->getPath()); + } } } /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { - auto watch_responses = processWatches(concrete_zk_request, deltas_range, watches, list_watches); + auto watch_responses = processWatches(concrete_zk_request, deltas_range, watches, list_watches, sessions_and_watchers); + total_watches_count -= watch_responses.size(); results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } @@ -3409,33 +3445,34 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) { /// Clear all watches for this session auto watches_it = sessions_and_watchers.find(session_id); - if (watches_it != sessions_and_watchers.end()) - { - for (const auto & watch_path : watches_it->second) - { - /// Maybe it's a normal watch - auto watch = watches.find(watch_path); - if (watch != watches.end()) - { - auto & watches_for_path = watch->second; - watches_for_path.erase(session_id); - if (watches_for_path.empty()) - watches.erase(watch); - } + if (watches_it == sessions_and_watchers.end()) + return; - /// Maybe it's a list watch - auto list_watch = list_watches.find(watch_path); - if (list_watch != list_watches.end()) - { - auto & list_watches_for_path = list_watch->second; - list_watches_for_path.erase(session_id); - if (list_watches_for_path.empty()) - list_watches.erase(list_watch); - } + for (const auto & watch_path : watches_it->second) + { + /// Maybe it's a normal watch + auto watch = watches.find(watch_path); + if (watch != watches.end()) + { + auto & watches_for_path = watch->second; + watches_for_path.erase(session_id); + if (watches_for_path.empty()) + watches.erase(watch); } - sessions_and_watchers.erase(watches_it); + /// Maybe it's a list watch + auto list_watch = list_watches.find(watch_path); + if (list_watch != list_watches.end()) + { + auto & list_watches_for_path = list_watch->second; + list_watches_for_path.erase(session_id); + if (list_watches_for_path.empty()) + list_watches.erase(list_watch); + } } + + total_watches_count -= watches_it->second.size(); + sessions_and_watchers.erase(watches_it); } template @@ -3521,11 +3558,7 @@ const KeeperStorageBase::Stats & KeeperStorage::getStorageStats() con template uint64_t KeeperStorage::getTotalWatchesCount() const { - uint64_t ret = 0; - for (const auto & [session, paths] : sessions_and_watchers) - ret += paths.size(); - - return ret; + return total_watches_count; } template diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 9509ac79f2b..993ed626f96 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -636,6 +636,7 @@ public: /// Mapping session_id -> set of watched nodes paths SessionAndWatcher sessions_and_watchers; + size_t total_watches_count = 0; /// Currently active watches (node_path -> subscribed sessions) Watches watches; From d11abd634ababe3673c5c23c1ff37fddae2a33a1 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 17 Sep 2024 16:37:08 +0200 Subject: [PATCH 090/128] Update max_replication_slots --- tests/integration/compose/docker_compose_postgres.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_postgres.yml b/tests/integration/compose/docker_compose_postgres.yml index 2ef7eb17395..c55cd8a31cf 100644 --- a/tests/integration/compose/docker_compose_postgres.yml +++ b/tests/integration/compose/docker_compose_postgres.yml @@ -2,7 +2,7 @@ version: '2.3' services: postgres1: image: postgres - command: ["postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=2", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all", "-c", "max_connections=200"] + command: ["postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=4", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all", "-c", "max_connections=200"] restart: always expose: - ${POSTGRES_PORT:-5432} From aba7de5091ffc52c864cc004c44ff4be966bb126 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 16:53:32 +0200 Subject: [PATCH 091/128] Verify that there are no intersecting parts in the resulting all_parts_to_read --- .../ParallelReplicasReadingCoordinator.cpp | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index ddbed5db7dc..0a25874cfd9 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -911,6 +911,24 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa std::sort(ranges.begin(), ranges.end()); } +#ifndef NDEBUG + /// Double check that there are no intersecting parts + { + auto part_it = all_parts_to_read.begin(); + auto next_part_it = part_it; + if (next_part_it != all_parts_to_read.end()) + ++next_part_it; + while (next_part_it != all_parts_to_read.end()) + { + chassert(part_it->description.info.isDisjoint(next_part_it->description.info), + fmt::format("Parts {} and {} intersect", + part_it->description.info.getPartNameV1(), next_part_it->description.info.getPartNameV1())); + ++part_it; + ++next_part_it; + } + } +#endif + state_initialized = true; // progress_callback is not set when local plan is used for initiator From 190d3f04c9fb03e8a8c64ce1b25d5536e7835ad7 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 16:54:49 +0200 Subject: [PATCH 092/128] More optimal check for intrsecting parts in DefaultCoordinator init --- .../ParallelReplicasReadingCoordinator.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 0a25874cfd9..603584af6c2 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -376,17 +376,20 @@ void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement ann if (state_initialized) return; - for (auto && part : announcement.description) { - auto intersecting_it = std::find_if( - all_parts_to_read.begin(), - all_parts_to_read.end(), - [&part](const Part & other) { return !other.description.info.isDisjoint(part.info); }); + /// To speedup search for adjacent parts + Parts known_parts(all_parts_to_read.begin(), all_parts_to_read.end()); - if (intersecting_it != all_parts_to_read.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); + for (auto && part : announcement.description) + { + auto intersecting_it = known_parts.lower_bound(Part{.description = part, .replicas = {}}); - all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); + if (intersecting_it != known_parts.end() && !intersecting_it->description.info.isDisjoint(part.info)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); + + all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); + known_parts.emplace(Part{.description = part, .replicas = {}}); + } } std::ranges::sort( From 3674c97ebba63bc88c6bc03f630124afa314053a Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 17:49:02 +0200 Subject: [PATCH 093/128] Fix for using part after std::move from it --- src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 603584af6c2..98f28430ecc 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -387,8 +387,8 @@ void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement ann if (intersecting_it != known_parts.end() && !intersecting_it->description.info.isDisjoint(part.info)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); - all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); known_parts.emplace(Part{.description = part, .replicas = {}}); + all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); } } From 574a26c63ba24e4632b428827642b40db48424e4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 17:56:44 +0200 Subject: [PATCH 094/128] Use adjacent_find to check adjacent parts --- .../ParallelReplicasReadingCoordinator.cpp | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 98f28430ecc..26f2273d196 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -917,18 +917,15 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa #ifndef NDEBUG /// Double check that there are no intersecting parts { - auto part_it = all_parts_to_read.begin(); - auto next_part_it = part_it; - if (next_part_it != all_parts_to_read.end()) - ++next_part_it; - while (next_part_it != all_parts_to_read.end()) - { - chassert(part_it->description.info.isDisjoint(next_part_it->description.info), - fmt::format("Parts {} and {} intersect", - part_it->description.info.getPartNameV1(), next_part_it->description.info.getPartNameV1())); - ++part_it; - ++next_part_it; - } + auto intersecting_part_it = std::adjacent_find(all_parts_to_read.begin(), all_parts_to_read.end(), + [] (const Part & lhs, const Part & rhs) + { + return !lhs.description.info.isDisjoint(rhs.description.info); + }); + + if (intersecting_part_it != all_parts_to_read.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parts {} and {} intersect", + intersecting_part_it->description.info.getPartNameV1(), std::next(intersecting_part_it)->description.info.getPartNameV1()); } #endif From 80259659ff2feb457500f71f20752e03dfe11c2e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 17 Sep 2024 18:03:19 +0200 Subject: [PATCH 095/128] More asserts --- src/Interpreters/HashJoin/HashJoin.cpp | 42 +++++++++++++++++++++++--- src/Interpreters/HashJoin/HashJoin.h | 1 + 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 1b8b45b94ea..23ce4fdf0f5 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -338,12 +338,8 @@ size_t HashJoin::getTotalRowCount() const return res; } -size_t HashJoin::getTotalByteCount() const +void HashJoin::doDebugAsserts() const { - if (!data) - return 0; - -#ifndef NDEBUG size_t debug_blocks_allocated_size = 0; for (const auto & block : data->blocks) debug_blocks_allocated_size += block.allocatedBytes(); @@ -359,6 +355,15 @@ size_t HashJoin::getTotalByteCount() const if (data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size ({} != {})", data->blocks_nullmaps_allocated_size, debug_blocks_nullmaps_allocated_size); +} + +size_t HashJoin::getTotalByteCount() const +{ + if (!data) + return 0; + +#ifndef NDEBUG + doDebugAsserts(); #endif size_t res = 0; @@ -544,10 +549,17 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) have_compressed = true; } +#ifndef NDEBUG + doDebugAsserts(); +#endif data->blocks_allocated_size += block_to_save.allocatedBytes(); data->blocks.emplace_back(std::move(block_to_save)); Block * stored_block = &data->blocks.back(); +#ifndef NDEBUG + doDebugAsserts(); +#endif + if (rows) data->empty = false; @@ -634,9 +646,15 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) if (!flag_per_row && !is_inserted) { +#ifndef NDEBUG + doDebugAsserts(); +#endif LOG_TRACE(log, "Skipping inserting block with {} rows", rows); data->blocks_allocated_size -= stored_block->allocatedBytes(); data->blocks.pop_back(); +#ifndef NDEBUG + doDebugAsserts(); +#endif } if (!check_limits) @@ -687,6 +705,10 @@ void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_ stored_block = stored_block.shrinkToFit(); size_t new_size = stored_block.allocatedBytes(); +#ifndef NDEBUG + doDebugAsserts(); +#endif + if (old_size >= new_size) { if (data->blocks_allocated_size < old_size - new_size) @@ -700,6 +722,10 @@ void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_ else /// Sometimes after clone resized block can be bigger than original data->blocks_allocated_size += new_size - old_size; + +#ifndef NDEBUG + doDebugAsserts(); +#endif } auto new_total_bytes_in_join = getTotalByteCount(); @@ -1415,7 +1441,13 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) }; BlocksList sorted_blocks; visit_rows_map(sorted_blocks, map); +#ifdef NDEBUG + doDebugAsserts(); +#endif data->blocks.swap(sorted_blocks); +#ifdef NDEBUG + doDebugAsserts(); +#endif } } diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index 230343691ea..4c1ebbcdc66 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -470,6 +470,7 @@ private: void tryRerangeRightTableData() override; template void tryRerangeRightTableDataImpl(Map & map); + void doDebugAsserts() const; }; } From 665f362601a2ee1869cd5c91102d48818f7f0145 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 17 Sep 2024 16:10:03 +0000 Subject: [PATCH 096/128] Prohibit ALTER TABLE ... ADD INDEX ... TYPE inverted if setting = 0 --- src/Storages/AlterCommands.cpp | 10 +++ src/Storages/AlterCommands.h | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++ ...02346_inverted_index_experimental_flag.sql | 66 +++++++++++++++---- 4 files changed, 71 insertions(+), 12 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index ef76bc691ec..68778243371 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1142,6 +1142,16 @@ bool AlterCommands::hasFullTextIndex(const StorageInMemoryMetadata & metadata) return false; } +bool AlterCommands::hasLegacyInvertedIndex(const StorageInMemoryMetadata & metadata) +{ + for (const auto & index : metadata.secondary_indices) + { + if (index.type == INVERTED_INDEX_NAME) + return true; + } + return false; +} + bool AlterCommands::hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata) { for (const auto & index : metadata.secondary_indices) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index c4c792e7dec..be1b31f3d20 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -235,8 +235,9 @@ public: /// additional mutation command (MATERIALIZE_TTL) will be returned. MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters=false) const; - /// Check if commands have any full-text index + /// Check if commands have any full-text index or a (legacy) inverted index static bool hasFullTextIndex(const StorageInMemoryMetadata & metadata); + static bool hasLegacyInvertedIndex(const StorageInMemoryMetadata & metadata); /// Check if commands have any vector similarity index static bool hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ca619d4d208..80d61058d08 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3230,6 +3230,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is not enabled (turn on setting 'allow_experimental_full_text_index')"); + if (AlterCommands::hasLegacyInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Experimental inverted index feature is not enabled (turn on setting 'allow_experimental_inverted_index')"); + if (AlterCommands::hasVectorSimilarityIndex(new_metadata) && !settings.allow_experimental_vector_similarity_index) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental vector similarity index is disabled (turn on setting 'allow_experimental_vector_similarity_index')"); diff --git a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql index f2d294ff9e4..84188337a8d 100644 --- a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql +++ b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql @@ -1,16 +1,60 @@ --- Tests that the inverted index can only be supported when allow_experimental_full_text_index = 1. - -SET allow_experimental_full_text_index = 0; +-- Tests that CREATE TABLE and ADD INDEX respect settings 'allow_experimental_full_text_index' and `allow_experimental_inverted_index` DROP TABLE IF EXISTS tab; -CREATE TABLE tab -( - `key` UInt64, - `str` String -) -ENGINE = MergeTree -ORDER BY key; -ALTER TABLE tab ADD INDEX inv_idx(str) TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +-- Test CREATE TABLE + full_text index setting +SET allow_experimental_full_text_index = 0; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError ILLEGAL_INDEX } + +SET allow_experimental_full_text_index = 1; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError ILLEGAL_INDEX } DROP TABLE tab; + +SET allow_experimental_full_text_index = 0; -- reset to default + +-- Test CREATE TABLE + inverted index setting + +SET allow_experimental_inverted_index = 0; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError ILLEGAL_INDEX } + +SET allow_experimental_inverted_index = 1; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); +DROP TABLE tab; + +SET allow_experimental_inverted_index = 0; -- reset to default + +-- Test ADD INDEX + full_text index setting + +SET allow_experimental_full_text_index = 0; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE tab; + +SET allow_experimental_full_text_index = 1; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE tab; +SET allow_experimental_full_text_index = 0; -- reset to default + + +-- Test ADD INDEX + inverted index setting + +SET allow_experimental_inverted_index = 0; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE tab; + +SET allow_experimental_inverted_index = 1; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); +DROP TABLE tab; +SET allow_experimental_inverted_index = 0; -- reset to default From 7c5d55c6b25b1138235010eccaeabe0b7e561c5d Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 17 Sep 2024 18:10:51 +0200 Subject: [PATCH 097/128] Lint --- src/Interpreters/HashJoin/HashJoin.cpp | 21 ++------------------- 1 file changed, 2 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 23ce4fdf0f5..4f78243b09e 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -340,6 +340,7 @@ size_t HashJoin::getTotalRowCount() const void HashJoin::doDebugAsserts() const { +#ifdef NDEBUG size_t debug_blocks_allocated_size = 0; for (const auto & block : data->blocks) debug_blocks_allocated_size += block.allocatedBytes(); @@ -355,6 +356,7 @@ void HashJoin::doDebugAsserts() const if (data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size ({} != {})", data->blocks_nullmaps_allocated_size, debug_blocks_nullmaps_allocated_size); +#endif } size_t HashJoin::getTotalByteCount() const @@ -362,9 +364,7 @@ size_t HashJoin::getTotalByteCount() const if (!data) return 0; -#ifndef NDEBUG doDebugAsserts(); -#endif size_t res = 0; @@ -549,16 +549,11 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) have_compressed = true; } -#ifndef NDEBUG doDebugAsserts(); -#endif data->blocks_allocated_size += block_to_save.allocatedBytes(); data->blocks.emplace_back(std::move(block_to_save)); Block * stored_block = &data->blocks.back(); - -#ifndef NDEBUG doDebugAsserts(); -#endif if (rows) data->empty = false; @@ -646,15 +641,11 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) if (!flag_per_row && !is_inserted) { -#ifndef NDEBUG doDebugAsserts(); -#endif LOG_TRACE(log, "Skipping inserting block with {} rows", rows); data->blocks_allocated_size -= stored_block->allocatedBytes(); data->blocks.pop_back(); -#ifndef NDEBUG doDebugAsserts(); -#endif } if (!check_limits) @@ -705,9 +696,7 @@ void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_ stored_block = stored_block.shrinkToFit(); size_t new_size = stored_block.allocatedBytes(); -#ifndef NDEBUG doDebugAsserts(); -#endif if (old_size >= new_size) { @@ -723,9 +712,7 @@ void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_ /// Sometimes after clone resized block can be bigger than original data->blocks_allocated_size += new_size - old_size; -#ifndef NDEBUG doDebugAsserts(); -#endif } auto new_total_bytes_in_join = getTotalByteCount(); @@ -1441,13 +1428,9 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) }; BlocksList sorted_blocks; visit_rows_map(sorted_blocks, map); -#ifdef NDEBUG doDebugAsserts(); -#endif data->blocks.swap(sorted_blocks); -#ifdef NDEBUG doDebugAsserts(); -#endif } } From e574c49e252f29433d9c99f0a22be978b7396c39 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Sep 2024 18:19:05 +0200 Subject: [PATCH 098/128] Fix --- .../ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp index e85588394ee..25d44ed9ddb 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp @@ -48,6 +48,7 @@ ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata( , tracked_file_ttl_sec(engine_settings.tracked_file_ttl_sec) , buckets(engine_settings.buckets) , processing_threads_num(engine_settings.processing_threads_num) + , last_processed_path(engine_settings.last_processed_path) , loading_retries(engine_settings.loading_retries) { } From a210f9881968377c155c390b05ec00cfa62a6ccd Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 17 Sep 2024 18:28:27 +0200 Subject: [PATCH 099/128] Lint --- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 4f78243b09e..9a1bfe2d408 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -340,7 +340,7 @@ size_t HashJoin::getTotalRowCount() const void HashJoin::doDebugAsserts() const { -#ifdef NDEBUG +#ifndef NDEBUG size_t debug_blocks_allocated_size = 0; for (const auto & block : data->blocks) debug_blocks_allocated_size += block.allocatedBytes(); From b08e727aef1f7509b5d6409c824f19660b14dab5 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 17 Sep 2024 19:02:10 +0200 Subject: [PATCH 100/128] Count allocated bytes from scratch after rerange --- src/Interpreters/HashJoin/HashJoin.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 9a1bfe2d408..dfc41a93417 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1430,6 +1430,10 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) visit_rows_map(sorted_blocks, map); doDebugAsserts(); data->blocks.swap(sorted_blocks); + size_t new_blocks_allocated_size = 0; + for (const auto & block : data->blocks) + new_blocks_allocated_size += block.allocatedBytes(); + data->blocks_allocated_size = new_blocks_allocated_size; doDebugAsserts(); } } From 13e82d6439bd92f3e025eee7acd635a17d098a36 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Tue, 17 Sep 2024 17:45:04 +0000 Subject: [PATCH 101/128] fix double visit of uncommitted changes --- src/Coordination/KeeperStorage.cpp | 6 ++- src/Coordination/tests/gtest_coordination.cpp | 40 +++++++++++++++++++ 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 63dc39092cf..a3ba7402296 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1777,7 +1777,8 @@ private: 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 (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction + /// 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)) @@ -1811,7 +1812,8 @@ private: const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node); - if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction + /// 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)) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 46f36fe0039..15d5f460e94 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -3738,6 +3738,46 @@ TYPED_TEST(CoordinationTest, TestRemoveRecursiveInMultiRequest) ASSERT_FALSE(exists("/A/B")); ASSERT_FALSE(exists("/A/B/D")); } + + { + SCOPED_TRACE("Recursive Remove For Subtree With Updated Node"); + int create_zxid = ++zxid; + auto ops = prepare_create_tree(); + + /// First create nodes + const auto create_request = std::make_shared(ops, ACLs{}); + storage.preprocessRequest(create_request, 1, 0, create_zxid); + auto create_responses = storage.processRequest(create_request, 1, create_zxid); + ASSERT_EQ(create_responses.size(), 1); + ASSERT_TRUE(is_multi_ok(create_responses[0].response)); + + /// Small limit + int remove_zxid = ++zxid; + ops = { + zkutil::makeSetRequest("/A/B", "", -1), + zkutil::makeRemoveRecursiveRequest("/A", 3), + }; + auto remove_request = std::make_shared(ops, ACLs{}); + storage.preprocessRequest(remove_request, 1, 0, remove_zxid); + auto remove_responses = storage.processRequest(remove_request, 1, remove_zxid); + + ASSERT_EQ(remove_responses.size(), 1); + ASSERT_FALSE(is_multi_ok(remove_responses[0].response)); + + /// Big limit + remove_zxid = ++zxid; + ops[1] = zkutil::makeRemoveRecursiveRequest("/A", 4); + remove_request = std::make_shared(ops, ACLs{}); + storage.preprocessRequest(remove_request, 1, 0, remove_zxid); + remove_responses = storage.processRequest(remove_request, 1, remove_zxid); + + ASSERT_EQ(remove_responses.size(), 1); + ASSERT_TRUE(is_multi_ok(remove_responses[0].response)); + ASSERT_FALSE(exists("/A")); + ASSERT_FALSE(exists("/A/C")); + ASSERT_FALSE(exists("/A/B")); + ASSERT_FALSE(exists("/A/B/D")); + } } TYPED_TEST(CoordinationTest, TestRemoveRecursiveWatches) From 9c185374e460cfab452783c86964300e880145b3 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Tue, 17 Sep 2024 18:14:47 +0000 Subject: [PATCH 102/128] fix level sorting --- src/Coordination/KeeperStorage.h | 8 +++--- src/Coordination/tests/gtest_coordination.cpp | 26 +++++++++++++++++++ 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 6fbc4c2b168..ec502a38a72 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -613,13 +613,15 @@ public: struct PathCmp { - using is_transparent = std::true_type; - auto operator()(const std::string_view a, const std::string_view b) const { - return a.size() < b.size() || (a.size() == b.size() && a < b); + size_t level_a = std::count(a.begin(), a.end(), '/'); + size_t level_b = std::count(b.begin(), b.end(), '/'); + return level_a < level_b || (level_a == level_b && a < b); } + + using is_transparent = void; // required to make find() work with different type than key_type }; mutable std::map nodes; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 15d5f460e94..4272e504da1 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -3778,6 +3778,32 @@ TYPED_TEST(CoordinationTest, TestRemoveRecursiveInMultiRequest) ASSERT_FALSE(exists("/A/B")); ASSERT_FALSE(exists("/A/B/D")); } + + { + SCOPED_TRACE("[BUG] Recursive Remove Level Sorting"); + int new_zxid = ++zxid; + + Coordination::Requests ops = { + zkutil::makeCreateRequest("/a", "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/a/bbbbbb", "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/A", "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/A/B", "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/A/CCCCCCCCCCCC", "", zkutil::CreateMode::Persistent), + zkutil::makeRemoveRecursiveRequest("/A", 3), + }; + auto remove_request = std::make_shared(ops, ACLs{}); + storage.preprocessRequest(remove_request, 1, 0, new_zxid); + auto remove_responses = storage.processRequest(remove_request, 1, new_zxid); + + ASSERT_EQ(remove_responses.size(), 1); + ASSERT_TRUE(is_multi_ok(remove_responses[0].response)); + ASSERT_TRUE(exists("/a")); + ASSERT_TRUE(exists("/a/bbbbbb")); + ASSERT_FALSE(exists("/A")); + ASSERT_FALSE(exists("/A/B")); + ASSERT_FALSE(exists("/A/CCCCCCCCCCCC")); + } + } TYPED_TEST(CoordinationTest, TestRemoveRecursiveWatches) From 474499d240425872fadbd59ee97750a67f41390f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 17 Sep 2024 21:48:19 +0200 Subject: [PATCH 103/128] Revert "Add user-level settings min_free_diskspace_bytes_to_throw_insert and min_free_diskspace_ratio_to_throw_insert" --- .../settings/merge-tree-settings.md | 20 ------ src/Core/Settings.h | 2 - src/Core/SettingsChangesHistory.cpp | 4 +- .../MergeTree/MergeTreeDataWriter.cpp | 30 +-------- src/Storages/MergeTree/MergeTreeSettings.h | 2 - .../__init__.py | 0 .../config.d/storage_configuration.xml | 19 ------ .../test.py | 61 ------------------- 8 files changed, 3 insertions(+), 135 deletions(-) delete mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py delete mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml delete mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/test.py diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 376c1c66ad5..a13aacc76e6 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -156,26 +156,6 @@ Default value: 1000. ClickHouse artificially executes `INSERT` longer (adds ‘sleep’) so that the background merge process can merge parts faster than they are added. -## min_free_disk_bytes_to_throw_insert {#min_free_disk_bytes_to_throw_insert} - -The minimum number of bytes that should be free in disk space in order to insert data. If the number of available free bytes - `keep_free_space_bytes` is less than `min_free_disk_bytes_to_throw_insert` then an exception is thrown and the insert is not executed. Note that this setting does not take into account the amount of data that will be written by the `INSERT` operation. - -Possible values: - -- Any positive integer. - -Default value: 0 bytes. - -## min_free_disk_ratio_to_throw_insert {#min_free_disk_ratio_to_throw_insert} - -The minimum free to total disk space ratio to perform an `INSERT`. The free space is calculated by subtracting `keep_free_space_bytes` from the total available space in disk. - -Possible values: - -- Float, 0.0 - 1.0 - -Default value: 0.0 - ## inactive_parts_to_throw_insert {#inactive-parts-to-throw-insert} If the number of inactive parts in a single partition more than the `inactive_parts_to_throw_insert` value, `INSERT` is interrupted with the "Too many inactive parts (N). Parts cleaning are processing significantly slower than inserts" exception. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 95f585d1a27..fadc4079fe0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -343,8 +343,6 @@ class IColumn; M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \ M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ - M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ - M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index da0fbfad255..560f144866b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -85,9 +85,7 @@ static std::initializer_listgetVolume(0); VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); - const auto & data_settings = data.getSettings(); - const UInt64 min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; - const Float64 min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; - - if (min_bytes > 0 || min_ratio > 0.0) - { - const auto disk = data_part_volume->getDisk(); - const UInt64 total_disk_bytes = *disk->getTotalSpace(); - const UInt64 free_disk_bytes = *disk->getAvailableSpace(); - - const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_bytes); - const UInt64 needed_free_bytes = std::max(min_bytes, min_bytes_from_ratio); - - if (needed_free_bytes > free_disk_bytes) - { - throw Exception( - ErrorCodes::NOT_ENOUGH_SPACE, - "Could not perform insert: less than {} free bytes in disk space ({}). " - "Configure this limit with user settings {} or {}", - needed_free_bytes, - free_disk_bytes, - "min_free_disk_bytes_to_throw_insert", - "min_free_disk_ratio_to_throw_insert"); - } - } - auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) @@ -591,6 +564,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( if (data.storage_settings.get()->assign_part_uuids) new_data_part->uuid = UUIDHelpers::generateV4(); + const auto & data_settings = data.getSettings(); + SerializationInfo::Settings settings{data_settings->ratio_of_defaults_for_sparse_serialization, true}; SerializationInfoByName infos(columns, settings); infos.add(block); @@ -713,7 +688,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( MergeTreeDataPartType part_type; /// Size of part would not be greater than block.bytes() + epsilon size_t expected_size = block.bytes(); - // just check if there is enough space on parent volume MergeTreeData::reserveSpace(expected_size, parent_part->getDataPartStorage()); part_type = data.choosePartFormatOnDisk(expected_size, block.rows()).part_type; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index b2ebfa1dfda..dcb18155114 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -99,8 +99,6 @@ struct Settings; M(Bool, add_implicit_sign_column_constraint_for_collapsing_engine, false, "If true, add implicit constraint for sign column for CollapsingMergeTree engine.", 0) \ M(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 0) \ M(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ - M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ - M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ M(Bool, use_adaptive_write_buffer_for_dynamic_subcolumns, true, "Allow to use adaptive writer buffers during writing dynamic subcolumns to reduce memory usage", 0) \ M(UInt64, adaptive_write_buffer_initial_size, 16 * 1024, "Initial size of an adaptive write buffer", 0) \ \ diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py b/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml b/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml deleted file mode 100644 index d4031ff656c..00000000000 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - - local - /disk1/ - - - - - -
- disk1 -
-
-
-
-
-
diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py deleted file mode 100644 index 328de674de1..00000000000 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ /dev/null @@ -1,61 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.client import QueryRuntimeException - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance( - "node", - main_configs=["configs/config.d/storage_configuration.xml"], - tmpfs=["/disk1:size=7M"], - macros={"shard": 0, "replica": 1}, -) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_insert_stops_when_disk_full(start_cluster): - min_free_bytes = 3 * 1024 * 1024 # 3 MiB - - node.query( - f""" - CREATE TABLE test_table ( - id UInt32, - data String - ) ENGINE = MergeTree() - ORDER BY id - SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_throw_insert = {min_free_bytes} - """ - ) - - count = 0 - - # Insert data to fill up disk - try: - for _ in range(100000): - node.query( - "INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)" - ) - count += 1 - except QueryRuntimeException as e: - assert "Could not perform insert" in str(e) - assert "free bytes in disk space" in str(e) - - free_space = int( - node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip() - ) - assert ( - free_space <= min_free_bytes - ), f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" - - rows = int(node.query("SELECT count() from test_table").strip()) - assert rows == count - - node.query("DROP TABLE test_table") From 1bcdde3e628c570183880027770004fff196e8f1 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Tue, 17 Sep 2024 19:48:48 -0700 Subject: [PATCH 104/128] Update README.md - Meetups --- README.md | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/README.md b/README.md index 5fa04fe29e7..e74fb303d4f 100644 --- a/README.md +++ b/README.md @@ -40,17 +40,8 @@ Every month we get together with the community (users, contributors, customers, Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey Milovidov: +Upcoming meetups -* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/triangletechtalks/events/302723486/) - September 9 -* [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 -* [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 - -Other upcoming meetups - -* [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 -* [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17 -* [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 * [Bangalore Meetup](https://www.meetup.com/clickhouse-bangalore-user-group/events/303208274/) - September 18 * [Tel Aviv Meetup](https://www.meetup.com/clickhouse-meetup-israel/events/303095121) - September 22 * [Jakarta Meetup](https://www.meetup.com/clickhouse-indonesia-user-group/events/303191359/) - October 1 @@ -62,13 +53,20 @@ Other upcoming meetups * [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 * [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26 -Recently completed events +Recently completed meetups + * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 * [Seattle Meetup (Statsig)](https://www.meetup.com/clickhouse-seattle-user-group/events/302518075/) - August 27 * [Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302732666/) - August 27 * [Sydney Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302862966/) - September 5 * [Zurich Meetup](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/302267429/) - September 5 * [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 +* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/triangletechtalks/events/302723486/) - September 9 +* [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 +* [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 +* [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 +* [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 +* [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17 ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments" From 2218ebebbfaac4d5114a476eafe40f71bdd25455 Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 05:15:57 -0300 Subject: [PATCH 105/128] initial commit, tested function --- .../functions/array-functions.md | 35 +++ .../functions/array-functions.md | 37 +++ .../functions/array-functions.md | 35 +++ src/Functions/array/arrayAUCUnscaled.cpp | 212 ++++++++++++++++++ tests/fuzz/all.dict | 1 + tests/fuzz/dictionaries/functions.dict | 1 + tests/fuzz/dictionaries/old.dict | 1 + .../03237_array_auc_unscaled.reference | 25 +++ .../0_stateless/03237_array_auc_unscaled.sql | 30 +++ .../aspell-ignore/en/aspell-dict.txt | 1 + 10 files changed, 378 insertions(+) create mode 100644 src/Functions/array/arrayAUCUnscaled.cpp create mode 100644 tests/queries/0_stateless/03237_array_auc_unscaled.reference create mode 100644 tests/queries/0_stateless/03237_array_auc_unscaled.sql diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index ad971ae7554..89178cf8c5c 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -2116,6 +2116,41 @@ Result: └───────────────────────────────────────────────┘ ``` +## arrayAUC + +Calculate unscaled AUC (Area Under the Curve, which is a concept in machine learning, see more details: ), i.e. without dividing it by total true positives and total false positives. + +**Syntax** + +``` sql +arrayAUCUnscaled(arr_scores, arr_labels) +``` + +**Arguments** + +- `arr_scores` — scores prediction model gives. +- `arr_labels` — labels of samples, usually 1 for positive sample and 0 for negative sample. + +**Returned value** + +Returns unscaled AUC value with type Float64. + +**Example** + +Query: + +``` sql +select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); +``` + +Result: + +``` text +┌─arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐ +│ 3.0 │ +└───────────────────────────────────────────────────────┘ +``` + ## arrayMap(func, arr1, ...) Returns an array obtained from the original arrays by application of `func(arr1[i], ..., arrN[i])` for each element. Arrays `arr1` ... `arrN` must have the same number of elements. diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 825e3f06be2..7923e9af945 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -1654,6 +1654,43 @@ SELECT arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); └────────────────────────────────────────---──┘ ``` +## arrayAUCUnscaled {#arrayaucunscaled} + +Вычисляет площадь под кривой без нормализации. + +**Синтаксис** + +``` sql +arrayAUCUnscaled(arr_scores, arr_labels) +``` + +**Аргументы** + +- `arr_scores` — оценка, которую дает модель предсказания. +- `arr_labels` — ярлыки выборок, обычно 1 для содержательных выборок и 0 для бессодержательных выборок. + +**Возвращаемое значение** + +Значение площади под кривой без нормализации. + +Тип данных: `Float64`. + +**Пример** + +Запрос: + +``` sql +SELECT arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); +``` + +Результат: + +``` text +┌─arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐ +│ 3.0 │ +└────────────────────────────────────────---────────────┘ +``` + ## arrayProduct {#arrayproduct} Возвращает произведение элементов [массива](../../sql-reference/data-types/array.md). diff --git a/docs/zh/sql-reference/functions/array-functions.md b/docs/zh/sql-reference/functions/array-functions.md index 69db34e4a36..5ff3e6a424c 100644 --- a/docs/zh/sql-reference/functions/array-functions.md +++ b/docs/zh/sql-reference/functions/array-functions.md @@ -1221,6 +1221,41 @@ select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); └───────────────────────────────────────────────┘ ``` +## arrayAUCUnscaled {#arrayaucunscaled} + +计算没有归一化的AUC (ROC曲线下的面积,这是机器学习中的一个概念,更多细节请查看:https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve)。 + +**语法** + +``` sql +arrayAUCUnscaled(arr_scores, arr_labels) +``` + +**参数** + +- `arr_scores` — 分数预测模型给出。 +- `arr_labels` — 样本的标签,通常为 1 表示正样本,0 表示负样本。 + +**返回值** + +返回 Float64 类型的非标准化 AUC 值。 + +**示例** + +查询语句: + +``` sql +select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); +``` + +结果: + +``` text +┌─arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐ +│ 3.0 │ +└───────────────────────────────────────────────────────┘ +``` + ## arrayMap(func, arr1, ...) {#array-map} 将从 `func` 函数的原始应用中获得的数组返回给 `arr` 数组中的每个元素。 diff --git a/src/Functions/array/arrayAUCUnscaled.cpp b/src/Functions/array/arrayAUCUnscaled.cpp new file mode 100644 index 00000000000..2cf0d072218 --- /dev/null +++ b/src/Functions/array/arrayAUCUnscaled.cpp @@ -0,0 +1,212 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; +} + + +/** The function takes two arrays: scores and labels. + * Label can be one of two values: positive and negative. + * Score can be arbitrary number. + * + * These values are considered as the output of classifier. We have some true labels for objects. + * And classifier assigns some scores to objects that predict these labels in the following way: + * - we can define arbitrary threshold on score and predict that the label is positive if the score is greater than the threshold: + * + * f(object) = score + * predicted_label = score > threshold + * + * This way classifier may predict positive or negative value correctly - true positive or true negative + * or have false positive or false negative result. + * Verying the threshold we can get different probabilities of false positive or false negatives or true positives, etc... + * + * We can also calculate the True Positive Rate and the False Positive Rate: + * + * TPR (also called "sensitivity", "recall" or "probability of detection") + * is the probability of classifier to give positive result if the object has positive label: + * TPR = P(score > threshold | label = positive) + * + * FPR is the probability of classifier to give positive result if the object has negative label: + * FPR = P(score > threshold | label = negative) + * + * We can draw a curve of values of FPR and TPR with different threshold on [0..1] x [0..1] unit square. + * This curve is named "ROC curve" (Receiver Operating Characteristic). + * + * For ROC we can calculate, literally, Area Under the Curve, that will be in the range of [0..1]. + * The higher the AUC the better the classifier. + * + * AUC also is as the probability that the score for positive label is greater than the score for negative label. + * + * https://developers.google.com/machine-learning/crash-course/classification/roc-and-auc + * https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve + * + * To calculate AUC, we will draw points of (FPR, TPR) for different thresholds = score_i. + * FPR_raw = countIf(score > score_i, label = negative) = count negative labels above certain score + * TPR_raw = countIf(score > score_i, label = positive) = count positive labels above certain score + * + * Let's look at the example: + * arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); + * + * 1. We have pairs: (-, 0.1), (-, 0.4), (+, 0.35), (+, 0.8) + * + * 2. Let's sort by score: (-, 0.1), (+, 0.35), (-, 0.4), (+, 0.8) + * + * 3. Let's draw the points: + * + * threshold = 0, TPR_raw = 2, FPR_raw = 2 + * threshold = 0.1, TPR_raw = 2, FPR_raw = 1 + * threshold = 0.35, TPR_raw = 1, FPR_raw = 1 + * threshold = 0.4, TPR_raw = 1, FPR_raw = 0 + * threshold = 0.8, TPR_raw = 0, FPR_raw = 0 + * + * The "curve" will be present by a line that moves one step either towards right or top on each threshold change. + */ + +class FunctionArrayAUCUnscaled : public IFunction +{ +public: + static constexpr auto name = "arrayAUCUnscaled"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + +private: + static Float64 apply( + const IColumn & scores, + const IColumn & labels, + ColumnArray::Offset current_offset, + ColumnArray::Offset next_offset) + { + struct ScoreLabel + { + Float64 score; + bool label; + }; + + size_t size = next_offset - current_offset; + PODArrayWithStackMemory sorted_labels(size); + + for (size_t i = 0; i < size; ++i) + { + bool label = labels.getFloat64(current_offset + i) > 0; + sorted_labels[i].score = scores.getFloat64(current_offset + i); + sorted_labels[i].label = label; + } + + /// Sorting scores in descending order to traverse the ROC curve from left to right + std::sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; }); + + Float64 area = 0.0; + Float64 prev_score = sorted_labels[0].score; + size_t prev_fp = 0, prev_tp = 0; + size_t curr_fp = 0, curr_tp = 0; + for (size_t i = 0; i < size; ++i) + { + // Only increment the area when the score changes + if (sorted_labels[i].score != prev_score) + { + area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; // Trapezoidal area under curve (might degenerate to zero or to a rectangle) + prev_fp = curr_fp; + prev_tp = curr_tp; + prev_score = sorted_labels[i].score; + } + + if (sorted_labels[i].label) + curr_tp += 1; /// The curve moves one step up. + else + curr_fp += 1; /// The curve moves one step right. + } + + area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; + + return area; + } + + static void vector( + const IColumn & scores, + const IColumn & labels, + const ColumnArray::Offsets & offsets, + PaddedPODArray & result, + size_t input_rows_count) + { + result.resize(input_rows_count); + + ColumnArray::Offset current_offset = 0; + for (size_t i = 0; i < input_rows_count; ++i) + { + auto next_offset = offsets[i]; + result[i] = apply(scores, labels, current_offset, next_offset); + current_offset = next_offset; + } + } + +public: + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 2; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + for (size_t i = 0; i < getNumberOfArguments(); ++i) + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "All arguments for function {} must be an array.", getName()); + + const auto & nested_type = array_type->getNestedType(); + if (!isNativeNumber(nested_type) && !isEnum(nested_type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} cannot process values of type {}", + getName(), nested_type->getName()); + } + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst(); + ColumnPtr col2 = arguments[1].column->convertToFullColumnIfConst(); + + const ColumnArray * col_array1 = checkAndGetColumn(col1.get()); + if (!col_array1) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); + + const ColumnArray * col_array2 = checkAndGetColumn(col2.get()); + if (!col_array2) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second argument of function {}", arguments[1].column->getName(), getName()); + + if (!col_array1->hasEqualOffsets(*col_array2)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); + + auto col_res = ColumnVector::create(); + + vector( + col_array1->getData(), + col_array2->getData(), + col_array1->getOffsets(), + col_res->getData(), + input_rows_count); + + return col_res; + } +}; + + +REGISTER_FUNCTION(ArrayAUCUnscaled) +{ + factory.registerFunction(); +} + +} diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index 30af3746fca..6cb198a3e48 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -1216,6 +1216,7 @@ "argMinState" "array" "arrayAUC" +"arrayAUCUnscaled" "arrayAll" "arrayAvg" "arrayCompact" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index e562595fb67..302aab97c2d 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -529,6 +529,7 @@ "argMinState" "array" "arrayAUC" +"arrayAUCUnscaled" "arrayAll" "arrayAvg" "arrayCompact" diff --git a/tests/fuzz/dictionaries/old.dict b/tests/fuzz/dictionaries/old.dict index 61914c3b283..6ecb5503ca4 100644 --- a/tests/fuzz/dictionaries/old.dict +++ b/tests/fuzz/dictionaries/old.dict @@ -19,6 +19,7 @@ "Array" "arrayAll" "arrayAUC" +"arrayAUCUnscaled" "arrayCompact" "arrayConcat" "arrayCount" diff --git a/tests/queries/0_stateless/03237_array_auc_unscaled.reference b/tests/queries/0_stateless/03237_array_auc_unscaled.reference new file mode 100644 index 00000000000..63204682fd4 --- /dev/null +++ b/tests/queries/0_stateless/03237_array_auc_unscaled.reference @@ -0,0 +1,25 @@ +3 +3 +3 +3 +3 +3 +3 +3 +3 +1 +1 +1 +1 +1 +1 +1 +0 +0 +0 +0.5 +1 +0 +1.5 +2 +1.5 diff --git a/tests/queries/0_stateless/03237_array_auc_unscaled.sql b/tests/queries/0_stateless/03237_array_auc_unscaled.sql new file mode 100644 index 00000000000..d4f07c42118 --- /dev/null +++ b/tests/queries/0_stateless/03237_array_auc_unscaled.sql @@ -0,0 +1,30 @@ +select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); +select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8))); +select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8))); +select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1)))); +select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1)))); +select arrayAUCUnscaled(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1]); +select arrayAUCUnscaled(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1]); +select arrayAUCUnscaled(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1]); +select arrayAUCUnscaled(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1]); +select arrayAUCUnscaled(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1]); +select arrayAUCUnscaled(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1]); +select arrayAUCUnscaled(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1]); +select arrayAUCUnscaled(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]); +select arrayAUCUnscaled(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]); +select arrayAUCUnscaled([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]); +select arrayAUCUnscaled([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]); +SELECT arrayAUCUnscaled([], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUCUnscaled([1], [1]); +SELECT arrayAUCUnscaled([1], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUCUnscaled([], [1]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUCUnscaled([1, 2], [3]); -- { serverError BAD_ARGUMENTS } +SELECT arrayAUCUnscaled([1], [2, 3]); -- { serverError BAD_ARGUMENTS } +SELECT arrayAUCUnscaled([1, 1], [1, 1]); +SELECT arrayAUCUnscaled([1, 1], [0, 0]); +SELECT arrayAUCUnscaled([1, 1], [0, 1]); +SELECT arrayAUCUnscaled([0, 1], [0, 1]); +SELECT arrayAUCUnscaled([1, 0], [0, 1]); +SELECT arrayAUCUnscaled([0, 0, 1], [0, 1, 1]); +SELECT arrayAUCUnscaled([0, 1, 1], [0, 1, 1]); +SELECT arrayAUCUnscaled([0, 1, 1], [0, 0, 1]); diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3467f21c812..f658b19e8a7 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1153,6 +1153,7 @@ argMin argmax argmin arrayAUC +arrayAUCUnscaled arrayAll arrayAvg arrayCompact From 31066538527d36b5e78c1226445f51a252a61e29 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Sep 2024 09:52:19 +0200 Subject: [PATCH 106/128] Fix watches --- src/Coordination/KeeperStorage.cpp | 104 +++++++++++++++++------------ src/Coordination/KeeperStorage.h | 28 +++++++- 2 files changed, 87 insertions(+), 45 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 28bc1bf112b..493841fc50e 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -148,7 +148,8 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl( watch_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) { - [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase(path); + [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase( + KeeperStorageBase::WatchInfo{.path = path, .is_list_watch = false}); chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_response}); } @@ -188,7 +189,8 @@ KeeperStorageBase::ResponsesForSessions processWatchesImpl( watch_list_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) { - [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase(path_to_check); + [[maybe_unused]] auto erased = sessions_and_watchers[watcher_session].erase( + KeeperStorageBase::WatchInfo{.path = path_to_check, .is_list_watch = true}); chassert(erased); result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_list_response}); } @@ -828,18 +830,12 @@ void KeeperStorage::UncommittedState::rollback(std::list rollb if constexpr (std::same_as) { if (operation.stat.ephemeralOwner != 0) - { - std::lock_guard lock(storage.ephemeral_mutex); unregisterEphemeralPath(storage.uncommitted_state.ephemerals, operation.stat.ephemeralOwner, delta.path, /*throw_if_missing=*/false); - } } else if constexpr (std::same_as) { if (operation.stat.ephemeralOwner() != 0) - { - std::lock_guard lock(storage.ephemeral_mutex); storage.uncommitted_state.ephemerals[operation.stat.ephemeralOwner()].emplace(delta.path); - } } }, delta.operation); @@ -1159,7 +1155,6 @@ bool KeeperStorage::createNode( ++committed_ephemeral_nodes; std::lock_guard lock(ephemeral_mutex); committed_ephemerals[stat.ephemeralOwner].emplace(path); - unregisterEphemeralPath(uncommitted_state.ephemerals, stat.ephemeralOwner, path, /*throw_if_missing=*/false); } return true; @@ -1454,10 +1449,7 @@ std::list preprocess( return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; if (zk_request.is_ephemeral) - { - std::lock_guard lock(storage.ephemeral_mutex); storage.uncommitted_state.ephemerals[session_id].emplace(path_created); - } int32_t parent_cversion = zk_request.parent_cversion; @@ -1721,7 +1713,6 @@ std::list preprocess( if (node->stats.isEphemeral()) { - std::lock_guard ephemeral_lock(storage.ephemeral_mutex); /// try deleting the ephemeral node from the uncommitted state unregisterEphemeralPath(storage.uncommitted_state.ephemerals, node->stats.ephemeralOwner(), zk_request.path, /*throw_if_missing=*/false); } @@ -2027,7 +2018,6 @@ std::list preprocess( const auto * remove_delta = std::get_if(&delta.operation); if (remove_delta && remove_delta->stat.ephemeralOwner()) { - std::lock_guard lock(storage.ephemeral_mutex); unregisterEphemeralPath( storage.uncommitted_state.ephemerals, remove_delta->stat.ephemeralOwner(), delta.path, /*throw_if_missing=*/false); } @@ -2281,7 +2271,17 @@ Coordination::ZooKeeperResponsePtr processImpl(const Coordination::ZooKeeperList const auto children = get_children(); response->names.reserve(children->size()); - chassert(static_cast(node_it->value.stats.numChildren()) == children->size()); +#ifdef DEBUG_OR_SANITIZER_BUILD + if (!zk_request.path.starts_with(keeper_system_path) && static_cast(node_it->value.stats.numChildren()) != children->size()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Difference between numChildren ({}) and actual children size ({}) for '{}'", + node_it->value.stats.numChildren(), + children->size(), + zk_request.path); + } +#endif const auto add_child = [&](const auto & child) { @@ -3004,13 +3004,25 @@ void KeeperStorage::preprocessRequest( StringHashForHeterogeneousLookup, StringHashForHeterogeneousLookup::transparent_key_equal> parent_updates; - const auto process_ephemerals_for_session = [&](auto & current_ephemerals) + + const auto process_ephemerals_for_session + = [&](const auto & current_ephemerals, auto & processed_ephemeral_nodes, bool check_processed_nodes) { auto session_ephemerals = current_ephemerals.find(session_id); if (session_ephemerals != current_ephemerals.end()) { for (const auto & ephemeral_path : session_ephemerals->second) { + if (check_processed_nodes) + { + if (processed_ephemeral_nodes.contains(ephemeral_path)) + continue; + } + else + { + processed_ephemeral_nodes.insert(ephemeral_path); + } + auto node = uncommitted_state.getNode(ephemeral_path, /*should_lock_storage=*/false); /// maybe the node is deleted or recreated with different session_id in the uncommitted state @@ -3041,12 +3053,16 @@ void KeeperStorage::preprocessRequest( { /// storage lock should always be taken before ephemeral lock std::shared_lock storage_lock(storage_mutex); + + std::unordered_set processed_ephemeral_nodes; + process_ephemerals_for_session(uncommitted_state.ephemerals, processed_ephemeral_nodes, /*check_processed_nodes=*/false); + std::lock_guard ephemeral_lock(ephemeral_mutex); - process_ephemerals_for_session(committed_ephemerals); - process_ephemerals_for_session(uncommitted_state.ephemerals); - uncommitted_state.ephemerals.erase(session_id); + process_ephemerals_for_session(committed_ephemerals, processed_ephemeral_nodes, /*check_processed_nodes=*/true); } + uncommitted_state.ephemerals.erase(session_id); + for (auto & [parent_path, parent_update_delta] : parent_updates) { new_deltas.emplace_back @@ -3234,24 +3250,27 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process static constexpr std::array list_requests{ Coordination::OpNum::List, Coordination::OpNum::SimpleList, Coordination::OpNum::FilteredList}; - auto & watches_type = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end() - ? list_watches - : watches; + auto is_list_watch + = std::find(list_requests.begin(), list_requests.end(), zk_request->getOpNum()) != list_requests.end(); - auto add_watch_result = watches_type[zk_request->getPath()].emplace(session_id); - if (add_watch_result.second) + auto & watches_type = is_list_watch ? list_watches : watches; + + auto [watch_it, path_inserted] = watches_type.try_emplace(zk_request->getPath()); + auto [path_it, session_inserted] = watch_it->second.emplace(session_id); + if (session_inserted) { ++total_watches_count; - sessions_and_watchers[session_id].emplace(zk_request->getPath()); + sessions_and_watchers[session_id].emplace(WatchInfo{.path = watch_it->first, .is_list_watch = is_list_watch}); } } else if (response->error == Coordination::Error::ZNONODE && zk_request->getOpNum() == Coordination::OpNum::Exists) { - auto add_watch_result = watches[zk_request->getPath()].emplace(session_id); - if (add_watch_result.second) + auto [watch_it, path_inserted] = watches.try_emplace(zk_request->getPath()); + auto session_insert_info = watch_it->second.emplace(session_id); + if (session_insert_info.second) { ++total_watches_count; - sessions_and_watchers[session_id].emplace(zk_request->getPath()); + sessions_and_watchers[session_id].emplace(WatchInfo{.path = watch_it->first, .is_list_watch = false}); } } } @@ -3448,27 +3467,26 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) if (watches_it == sessions_and_watchers.end()) return; - for (const auto & watch_path : watches_it->second) + for (const auto [watch_path, is_list_watch] : watches_it->second) { - /// Maybe it's a normal watch - auto watch = watches.find(watch_path); - if (watch != watches.end()) - { - auto & watches_for_path = watch->second; - watches_for_path.erase(session_id); - if (watches_for_path.empty()) - watches.erase(watch); - } - - /// Maybe it's a list watch - auto list_watch = list_watches.find(watch_path); - if (list_watch != list_watches.end()) + if (is_list_watch) { + auto list_watch = list_watches.find(watch_path); + chassert(list_watch != list_watches.end()); auto & list_watches_for_path = list_watch->second; list_watches_for_path.erase(session_id); if (list_watches_for_path.empty()) list_watches.erase(list_watch); } + else + { + auto watch = watches.find(watch_path); + chassert(watch != watches.end()); + auto & watches_for_path = watch->second; + watches_for_path.erase(session_id); + if (watches_for_path.empty()) + watches.erase(watch); + } } total_watches_count -= watches_it->second.size(); @@ -3481,7 +3499,7 @@ void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const for (const auto & [session_id, watches_paths] : sessions_and_watchers) { buf << "0x" << getHexUIntLowercase(session_id) << "\n"; - for (const String & path : watches_paths) + for (const auto [path, is_list_watch] : watches_paths) buf << "\t" << path << "\n"; } } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 993ed626f96..03f2084df7c 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -6,6 +6,7 @@ #include #include #include +#include "Common/StringHashForHeterogeneousLookup.h" #include #include @@ -314,13 +315,36 @@ public: }; using Ephemerals = std::unordered_map>; - using SessionAndWatcher = std::unordered_map>; + struct WatchInfo + { + std::string_view path; + bool is_list_watch; + + bool operator==(const WatchInfo &) const = default; + }; + + struct WatchInfoHash + { + auto operator()(WatchInfo info) const + { + SipHash hash; + hash.update(info.path); + hash.update(info.is_list_watch); + return hash.get64(); + } + }; + + using SessionAndWatcher = std::unordered_map>; using SessionIDs = std::unordered_set; /// Just vector of SHA1 from user:password using AuthIDs = std::vector; using SessionAndAuth = std::unordered_map; - using Watches = std::unordered_map; + using Watches = std::unordered_map< + String /* path, relative of root_path */, + SessionIDs, + StringHashForHeterogeneousLookup, + StringHashForHeterogeneousLookup::transparent_key_equal>; // Applying ZooKeeper request to storage consists of two steps: // - preprocessing which, instead of applying the changes directly to storage, From 06b49d18d90307b5d89c89da2e2fcbc4263025d4 Mon Sep 17 00:00:00 2001 From: mmav Date: Wed, 18 Sep 2024 10:45:10 +0100 Subject: [PATCH 107/128] Update quantileddsketch.md Update function syntax --- .../aggregate-functions/reference/quantileddsketch.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantileddsketch.md b/docs/en/sql-reference/aggregate-functions/reference/quantileddsketch.md index fc9db7ef08d..6a9b73ab0bd 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantileddsketch.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantileddsketch.md @@ -9,7 +9,7 @@ Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a **Syntax** ``` sql -quantileDDsketch[relative_accuracy, (level)](expr) +quantileDD(relative_accuracy, [level])(expr) ``` **Arguments** From 8f350a7ec931478b64d76501715c5cd4be5d4af3 Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 08:52:58 -0300 Subject: [PATCH 108/128] remove separate function --- .../functions/array-functions.md | 38 +--- .../functions/array-functions.md | 37 --- .../functions/array-functions.md | 35 --- src/Functions/array/arrayAUC.cpp | 39 +++- src/Functions/array/arrayAUCUnscaled.cpp | 212 ------------------ tests/fuzz/all.dict | 1 - tests/fuzz/dictionaries/functions.dict | 1 - tests/fuzz/dictionaries/old.dict | 1 - .../0_stateless/03237_array_auc_unscaled.sql | 60 ++--- .../03237_array_auc_unscaled.stdout-e | 25 +++ .../aspell-ignore/en/aspell-dict.txt | 1 - 11 files changed, 86 insertions(+), 364 deletions(-) delete mode 100644 src/Functions/array/arrayAUCUnscaled.cpp create mode 100644 tests/queries/0_stateless/03237_array_auc_unscaled.stdout-e diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 89178cf8c5c..84396602d26 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -2088,13 +2088,14 @@ Calculate AUC (Area Under the Curve, which is a concept in machine learning, see **Syntax** ``` sql -arrayAUC(arr_scores, arr_labels) +arrayAUC(arr_scores, arr_labels[, scale]) ``` **Arguments** - `arr_scores` — scores prediction model gives. - `arr_labels` — labels of samples, usually 1 for positive sample and 0 for negative sample. +- `scale` - Optional. Wether to return the normalized area. Default value: true. [Bool] **Returned value** @@ -2116,41 +2117,6 @@ Result: └───────────────────────────────────────────────┘ ``` -## arrayAUC - -Calculate unscaled AUC (Area Under the Curve, which is a concept in machine learning, see more details: ), i.e. without dividing it by total true positives and total false positives. - -**Syntax** - -``` sql -arrayAUCUnscaled(arr_scores, arr_labels) -``` - -**Arguments** - -- `arr_scores` — scores prediction model gives. -- `arr_labels` — labels of samples, usually 1 for positive sample and 0 for negative sample. - -**Returned value** - -Returns unscaled AUC value with type Float64. - -**Example** - -Query: - -``` sql -select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); -``` - -Result: - -``` text -┌─arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐ -│ 3.0 │ -└───────────────────────────────────────────────────────┘ -``` - ## arrayMap(func, arr1, ...) Returns an array obtained from the original arrays by application of `func(arr1[i], ..., arrN[i])` for each element. Arrays `arr1` ... `arrN` must have the same number of elements. diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 7923e9af945..825e3f06be2 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -1654,43 +1654,6 @@ SELECT arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); └────────────────────────────────────────---──┘ ``` -## arrayAUCUnscaled {#arrayaucunscaled} - -Вычисляет площадь под кривой без нормализации. - -**Синтаксис** - -``` sql -arrayAUCUnscaled(arr_scores, arr_labels) -``` - -**Аргументы** - -- `arr_scores` — оценка, которую дает модель предсказания. -- `arr_labels` — ярлыки выборок, обычно 1 для содержательных выборок и 0 для бессодержательных выборок. - -**Возвращаемое значение** - -Значение площади под кривой без нормализации. - -Тип данных: `Float64`. - -**Пример** - -Запрос: - -``` sql -SELECT arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); -``` - -Результат: - -``` text -┌─arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐ -│ 3.0 │ -└────────────────────────────────────────---────────────┘ -``` - ## arrayProduct {#arrayproduct} Возвращает произведение элементов [массива](../../sql-reference/data-types/array.md). diff --git a/docs/zh/sql-reference/functions/array-functions.md b/docs/zh/sql-reference/functions/array-functions.md index 5ff3e6a424c..69db34e4a36 100644 --- a/docs/zh/sql-reference/functions/array-functions.md +++ b/docs/zh/sql-reference/functions/array-functions.md @@ -1221,41 +1221,6 @@ select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); └───────────────────────────────────────────────┘ ``` -## arrayAUCUnscaled {#arrayaucunscaled} - -计算没有归一化的AUC (ROC曲线下的面积,这是机器学习中的一个概念,更多细节请查看:https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve)。 - -**语法** - -``` sql -arrayAUCUnscaled(arr_scores, arr_labels) -``` - -**参数** - -- `arr_scores` — 分数预测模型给出。 -- `arr_labels` — 样本的标签,通常为 1 表示正样本,0 表示负样本。 - -**返回值** - -返回 Float64 类型的非标准化 AUC 值。 - -**示例** - -查询语句: - -``` sql -select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); -``` - -结果: - -``` text -┌─arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐ -│ 3.0 │ -└───────────────────────────────────────────────────────┘ -``` - ## arrayMap(func, arr1, ...) {#array-map} 将从 `func` 函数的原始应用中获得的数组返回给 `arr` 数组中的每个元素。 diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 7a61c9d368f..adaa52818d0 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -85,7 +85,8 @@ private: const IColumn & scores, const IColumn & labels, ColumnArray::Offset current_offset, - ColumnArray::Offset next_offset) + ColumnArray::Offset next_offset, + bool scale = true) { struct ScoreLabel { @@ -131,12 +132,15 @@ private: area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; - /// Then normalize it dividing by the area to the area of rectangle. + /// Then normalize it, if scale is true, dividing by the area to the area of rectangle. - if (curr_tp == 0 || curr_tp == size) + if (scale && (curr_tp == 0 || curr_tp == size)) return std::numeric_limits::quiet_NaN(); - return area / curr_tp / (size - curr_tp); + if (scale) + return area / curr_tp / (size - curr_tp); + else + return area; } static void vector( @@ -144,7 +148,8 @@ private: const IColumn & labels, const ColumnArray::Offsets & offsets, PaddedPODArray & result, - size_t input_rows_count) + size_t input_rows_count, + bool scale = true) { result.resize(input_rows_count); @@ -152,23 +157,23 @@ private: for (size_t i = 0; i < input_rows_count; ++i) { auto next_offset = offsets[i]; - result[i] = apply(scores, labels, current_offset, next_offset); + result[i] = apply(scores, labels, current_offset, next_offset, scale); current_offset = next_offset; } } public: String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 2; } + size_t getNumberOfArguments() const override { return 3; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - for (size_t i = 0; i < getNumberOfArguments(); ++i) + for (size_t i = 0; i < 2; ++i) { const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "All arguments for function {} must be an array.", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The two first arguments for function {} must be an array.", getName()); const auto & nested_type = array_type->getNestedType(); if (!isNativeNumber(nested_type) && !isEnum(nested_type)) @@ -176,6 +181,12 @@ public: getName(), nested_type->getName()); } + if (arguments.size() == 3) + { + if (!isBool(arguments[2])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Third argument must be a boolean (scale)"); + } + return std::make_shared(); } @@ -197,6 +208,13 @@ public: if (!col_array1->hasEqualOffsets(*col_array2)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); + // Handle third argument for scale (if passed, otherwise default to true) + bool scale = true; + if (arguments.size() == 3) + { + scale = arguments[2].column->getBool(0); // Assumes it's a scalar boolean column + } + auto col_res = ColumnVector::create(); vector( @@ -204,7 +222,8 @@ public: col_array2->getData(), col_array1->getOffsets(), col_res->getData(), - input_rows_count); + input_rows_count, + scale); return col_res; } diff --git a/src/Functions/array/arrayAUCUnscaled.cpp b/src/Functions/array/arrayAUCUnscaled.cpp deleted file mode 100644 index 2cf0d072218..00000000000 --- a/src/Functions/array/arrayAUCUnscaled.cpp +++ /dev/null @@ -1,212 +0,0 @@ -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; - extern const int BAD_ARGUMENTS; -} - - -/** The function takes two arrays: scores and labels. - * Label can be one of two values: positive and negative. - * Score can be arbitrary number. - * - * These values are considered as the output of classifier. We have some true labels for objects. - * And classifier assigns some scores to objects that predict these labels in the following way: - * - we can define arbitrary threshold on score and predict that the label is positive if the score is greater than the threshold: - * - * f(object) = score - * predicted_label = score > threshold - * - * This way classifier may predict positive or negative value correctly - true positive or true negative - * or have false positive or false negative result. - * Verying the threshold we can get different probabilities of false positive or false negatives or true positives, etc... - * - * We can also calculate the True Positive Rate and the False Positive Rate: - * - * TPR (also called "sensitivity", "recall" or "probability of detection") - * is the probability of classifier to give positive result if the object has positive label: - * TPR = P(score > threshold | label = positive) - * - * FPR is the probability of classifier to give positive result if the object has negative label: - * FPR = P(score > threshold | label = negative) - * - * We can draw a curve of values of FPR and TPR with different threshold on [0..1] x [0..1] unit square. - * This curve is named "ROC curve" (Receiver Operating Characteristic). - * - * For ROC we can calculate, literally, Area Under the Curve, that will be in the range of [0..1]. - * The higher the AUC the better the classifier. - * - * AUC also is as the probability that the score for positive label is greater than the score for negative label. - * - * https://developers.google.com/machine-learning/crash-course/classification/roc-and-auc - * https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve - * - * To calculate AUC, we will draw points of (FPR, TPR) for different thresholds = score_i. - * FPR_raw = countIf(score > score_i, label = negative) = count negative labels above certain score - * TPR_raw = countIf(score > score_i, label = positive) = count positive labels above certain score - * - * Let's look at the example: - * arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); - * - * 1. We have pairs: (-, 0.1), (-, 0.4), (+, 0.35), (+, 0.8) - * - * 2. Let's sort by score: (-, 0.1), (+, 0.35), (-, 0.4), (+, 0.8) - * - * 3. Let's draw the points: - * - * threshold = 0, TPR_raw = 2, FPR_raw = 2 - * threshold = 0.1, TPR_raw = 2, FPR_raw = 1 - * threshold = 0.35, TPR_raw = 1, FPR_raw = 1 - * threshold = 0.4, TPR_raw = 1, FPR_raw = 0 - * threshold = 0.8, TPR_raw = 0, FPR_raw = 0 - * - * The "curve" will be present by a line that moves one step either towards right or top on each threshold change. - */ - -class FunctionArrayAUCUnscaled : public IFunction -{ -public: - static constexpr auto name = "arrayAUCUnscaled"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - -private: - static Float64 apply( - const IColumn & scores, - const IColumn & labels, - ColumnArray::Offset current_offset, - ColumnArray::Offset next_offset) - { - struct ScoreLabel - { - Float64 score; - bool label; - }; - - size_t size = next_offset - current_offset; - PODArrayWithStackMemory sorted_labels(size); - - for (size_t i = 0; i < size; ++i) - { - bool label = labels.getFloat64(current_offset + i) > 0; - sorted_labels[i].score = scores.getFloat64(current_offset + i); - sorted_labels[i].label = label; - } - - /// Sorting scores in descending order to traverse the ROC curve from left to right - std::sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; }); - - Float64 area = 0.0; - Float64 prev_score = sorted_labels[0].score; - size_t prev_fp = 0, prev_tp = 0; - size_t curr_fp = 0, curr_tp = 0; - for (size_t i = 0; i < size; ++i) - { - // Only increment the area when the score changes - if (sorted_labels[i].score != prev_score) - { - area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; // Trapezoidal area under curve (might degenerate to zero or to a rectangle) - prev_fp = curr_fp; - prev_tp = curr_tp; - prev_score = sorted_labels[i].score; - } - - if (sorted_labels[i].label) - curr_tp += 1; /// The curve moves one step up. - else - curr_fp += 1; /// The curve moves one step right. - } - - area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; - - return area; - } - - static void vector( - const IColumn & scores, - const IColumn & labels, - const ColumnArray::Offsets & offsets, - PaddedPODArray & result, - size_t input_rows_count) - { - result.resize(input_rows_count); - - ColumnArray::Offset current_offset = 0; - for (size_t i = 0; i < input_rows_count; ++i) - { - auto next_offset = offsets[i]; - result[i] = apply(scores, labels, current_offset, next_offset); - current_offset = next_offset; - } - } - -public: - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 2; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - for (size_t i = 0; i < getNumberOfArguments(); ++i) - { - const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); - if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "All arguments for function {} must be an array.", getName()); - - const auto & nested_type = array_type->getNestedType(); - if (!isNativeNumber(nested_type) && !isEnum(nested_type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} cannot process values of type {}", - getName(), nested_type->getName()); - } - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst(); - ColumnPtr col2 = arguments[1].column->convertToFullColumnIfConst(); - - const ColumnArray * col_array1 = checkAndGetColumn(col1.get()); - if (!col_array1) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName()); - - const ColumnArray * col_array2 = checkAndGetColumn(col2.get()); - if (!col_array2) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of second argument of function {}", arguments[1].column->getName(), getName()); - - if (!col_array1->hasEqualOffsets(*col_array2)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); - - auto col_res = ColumnVector::create(); - - vector( - col_array1->getData(), - col_array2->getData(), - col_array1->getOffsets(), - col_res->getData(), - input_rows_count); - - return col_res; - } -}; - - -REGISTER_FUNCTION(ArrayAUCUnscaled) -{ - factory.registerFunction(); -} - -} diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index 6cb198a3e48..30af3746fca 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -1216,7 +1216,6 @@ "argMinState" "array" "arrayAUC" -"arrayAUCUnscaled" "arrayAll" "arrayAvg" "arrayCompact" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index 302aab97c2d..e562595fb67 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -529,7 +529,6 @@ "argMinState" "array" "arrayAUC" -"arrayAUCUnscaled" "arrayAll" "arrayAvg" "arrayCompact" diff --git a/tests/fuzz/dictionaries/old.dict b/tests/fuzz/dictionaries/old.dict index 6ecb5503ca4..61914c3b283 100644 --- a/tests/fuzz/dictionaries/old.dict +++ b/tests/fuzz/dictionaries/old.dict @@ -19,7 +19,6 @@ "Array" "arrayAll" "arrayAUC" -"arrayAUCUnscaled" "arrayCompact" "arrayConcat" "arrayCount" diff --git a/tests/queries/0_stateless/03237_array_auc_unscaled.sql b/tests/queries/0_stateless/03237_array_auc_unscaled.sql index d4f07c42118..4083e836067 100644 --- a/tests/queries/0_stateless/03237_array_auc_unscaled.sql +++ b/tests/queries/0_stateless/03237_array_auc_unscaled.sql @@ -1,30 +1,30 @@ -select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); -select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8))); -select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8))); -select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1)))); -select arrayAUCUnscaled([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1)))); -select arrayAUCUnscaled(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1]); -select arrayAUCUnscaled(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1]); -select arrayAUCUnscaled(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1]); -select arrayAUCUnscaled(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1]); -select arrayAUCUnscaled(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1]); -select arrayAUCUnscaled(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1]); -select arrayAUCUnscaled(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1]); -select arrayAUCUnscaled(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]); -select arrayAUCUnscaled(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]); -select arrayAUCUnscaled([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]); -select arrayAUCUnscaled([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]); -SELECT arrayAUCUnscaled([], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayAUCUnscaled([1], [1]); -SELECT arrayAUCUnscaled([1], []); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayAUCUnscaled([], [1]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayAUCUnscaled([1, 2], [3]); -- { serverError BAD_ARGUMENTS } -SELECT arrayAUCUnscaled([1], [2, 3]); -- { serverError BAD_ARGUMENTS } -SELECT arrayAUCUnscaled([1, 1], [1, 1]); -SELECT arrayAUCUnscaled([1, 1], [0, 0]); -SELECT arrayAUCUnscaled([1, 1], [0, 1]); -SELECT arrayAUCUnscaled([0, 1], [0, 1]); -SELECT arrayAUCUnscaled([1, 0], [0, 1]); -SELECT arrayAUCUnscaled([0, 0, 1], [0, 1, 1]); -SELECT arrayAUCUnscaled([0, 1, 1], [0, 1, 1]); -SELECT arrayAUCUnscaled([0, 1, 1], [0, 0, 1]); +select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], false); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), false); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), false); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))), false); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))), false); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1], false); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1], false); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1], false); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1], false); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1], false); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1], false); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1], false); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], false); +select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], false); +select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], false); +select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], false); +SELECT arrayAUC([], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([1], [1], false); +SELECT arrayAUC([1], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([], [1], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([1, 2], [3], false); -- { serverError BAD_ARGUMENTS } +SELECT arrayAUC([1], [2, 3], false); -- { serverError BAD_ARGUMENTS } +SELECT arrayAUC([1, 1], [1, 1], false); +SELECT arrayAUC([1, 1], [0, 0], false); +SELECT arrayAUC([1, 1], [0, 1], false); +SELECT arrayAUC([0, 1], [0, 1], false); +SELECT arrayAUC([1, 0], [0, 1], false); +SELECT arrayAUC([0, 0, 1], [0, 1, 1], false); +SELECT arrayAUC([0, 1, 1], [0, 1, 1], false); +SELECT arrayAUC([0, 1, 1], [0, 0, 1], false); diff --git a/tests/queries/0_stateless/03237_array_auc_unscaled.stdout-e b/tests/queries/0_stateless/03237_array_auc_unscaled.stdout-e new file mode 100644 index 00000000000..e6b5161afe3 --- /dev/null +++ b/tests/queries/0_stateless/03237_array_auc_unscaled.stdout-e @@ -0,0 +1,25 @@ +3 +3 +3 +3 +3 +3 +3 +3 +3 +1 +1 +1 +1 +1 +1 +1 +0 +nan +0 +0.5 +1 +0 +1.5 +2 +1.5 diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index f658b19e8a7..3467f21c812 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1153,7 +1153,6 @@ argMin argmax argmin arrayAUC -arrayAUCUnscaled arrayAll arrayAvg arrayCompact From 4c72fb0e32f4e872c897dcd413900d94a5d3fb27 Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 08:56:13 -0300 Subject: [PATCH 109/128] remove unnecessary file --- .../03237_array_auc_unscaled.stdout-e | 25 ------------------- 1 file changed, 25 deletions(-) delete mode 100644 tests/queries/0_stateless/03237_array_auc_unscaled.stdout-e diff --git a/tests/queries/0_stateless/03237_array_auc_unscaled.stdout-e b/tests/queries/0_stateless/03237_array_auc_unscaled.stdout-e deleted file mode 100644 index e6b5161afe3..00000000000 --- a/tests/queries/0_stateless/03237_array_auc_unscaled.stdout-e +++ /dev/null @@ -1,25 +0,0 @@ -3 -3 -3 -3 -3 -3 -3 -3 -3 -1 -1 -1 -1 -1 -1 -1 -0 -nan -0 -0.5 -1 -0 -1.5 -2 -1.5 From 4be8a0feba0646de617a366b6b7f3411158ada19 Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 08:58:14 -0300 Subject: [PATCH 110/128] fmt --- src/Functions/array/arrayAUC.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index adaa52818d0..999aa999015 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -140,7 +140,7 @@ private: if (scale) return area / curr_tp / (size - curr_tp); else - return area; + return area; } static void vector( @@ -212,7 +212,7 @@ public: bool scale = true; if (arguments.size() == 3) { - scale = arguments[2].column->getBool(0); // Assumes it's a scalar boolean column + scale = arguments[2].column->getBool(0); // Assumes it's a scalar boolean column } auto col_res = ColumnVector::create(); From e3b207d21702dc1b38ccbdb5b5b5cfa77ecf7617 Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 09:03:29 -0300 Subject: [PATCH 111/128] fmt --- src/Functions/array/arrayAUC.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 999aa999015..c770899214c 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -139,8 +139,8 @@ private: if (scale) return area / curr_tp / (size - curr_tp); - else - return area; + + return area; } static void vector( From cb2484939613238a450a3370cc29c75d0c295edf Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 18 Sep 2024 15:24:48 +0200 Subject: [PATCH 112/128] Move assert --- src/Interpreters/HashJoin/HashJoin.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index dfc41a93417..d1fa1280ed8 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -692,12 +692,12 @@ void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join, bool force_ for (auto & stored_block : data->blocks) { + doDebugAsserts(); + size_t old_size = stored_block.allocatedBytes(); stored_block = stored_block.shrinkToFit(); size_t new_size = stored_block.allocatedBytes(); - doDebugAsserts(); - if (old_size >= new_size) { if (data->blocks_allocated_size < old_size - new_size) From b94017125215599ce7ac09ad577a2431d9d078d1 Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 11:04:46 -0300 Subject: [PATCH 113/128] fix tests --- src/Functions/array/arrayAUC.cpp | 21 +++++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index c770899214c..4f37617ab79 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -14,6 +14,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -164,11 +165,21 @@ private: public: String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 3; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return false; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { + const size_t number_of_arguments = arguments.size(); + + if (number_of_arguments < 2 || number_of_arguments > 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", + getName(), number_of_arguments); + for (size_t i = 0; i < 2; ++i) { const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); @@ -181,7 +192,7 @@ public: getName(), nested_type->getName()); } - if (arguments.size() == 3) + if (number_of_arguments == 3) { if (!isBool(arguments[2])) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Third argument must be a boolean (scale)"); @@ -192,6 +203,8 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { + const size_t number_of_arguments = arguments.size(); + ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst(); ColumnPtr col2 = arguments[1].column->convertToFullColumnIfConst(); @@ -210,7 +223,7 @@ public: // Handle third argument for scale (if passed, otherwise default to true) bool scale = true; - if (arguments.size() == 3) + if (number_of_arguments == 3) { scale = arguments[2].column->getBool(0); // Assumes it's a scalar boolean column } From e0fc95c894fcd129566b7c374496423e30f94c2c Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 11:12:30 -0300 Subject: [PATCH 114/128] remove trailing spaces --- src/Functions/array/arrayAUC.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 4f37617ab79..04ebb6d5bac 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -204,7 +204,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { const size_t number_of_arguments = arguments.size(); - + ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst(); ColumnPtr col2 = arguments[1].column->convertToFullColumnIfConst(); From 3315e87e1af8e53d639380a377b77b8cd600bcce Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 18 Sep 2024 15:15:52 +0000 Subject: [PATCH 115/128] Fix METADATA_MISMATCH due to TTL with WHERE --- src/Storages/TTLDescription.cpp | 4 +++- .../03236_create_query_ttl_where.reference | 1 + .../03236_create_query_ttl_where.sql | 17 +++++++++++++++++ 3 files changed, 21 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03236_create_query_ttl_where.reference create mode 100644 tests/queries/0_stateless/03236_create_query_ttl_where.sql diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index d674f054632..48251f27ff6 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -248,7 +248,9 @@ TTLDescription TTLDescription::getTTLFromAST( if (ASTPtr where_expr_ast = ttl_element->where()) { result.where_expression_ast = where_expr_ast->clone(); - where_expression = buildExpressionAndSets(where_expr_ast, columns.getAllPhysical(), context).expression; + + ASTPtr ast = where_expr_ast->clone(); + where_expression = buildExpressionAndSets(ast, columns.getAllPhysical(), context).expression; result.where_expression_columns = where_expression->getRequiredColumnsWithTypes(); result.where_result_column = where_expression->getSampleBlock().safeGetByPosition(0).name; } diff --git a/tests/queries/0_stateless/03236_create_query_ttl_where.reference b/tests/queries/0_stateless/03236_create_query_ttl_where.reference new file mode 100644 index 00000000000..5f70943dd6d --- /dev/null +++ b/tests/queries/0_stateless/03236_create_query_ttl_where.reference @@ -0,0 +1 @@ +CREATE TABLE default.ttl\n(\n `a` UInt32,\n `timestamp` DateTime\n)\nENGINE = MergeTree\nORDER BY a\nTTL timestamp + toIntervalSecond(2) WHERE a IN (\n SELECT number\n FROM system.numbers\n LIMIT 100000\n)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/03236_create_query_ttl_where.sql b/tests/queries/0_stateless/03236_create_query_ttl_where.sql new file mode 100644 index 00000000000..4256b53cbeb --- /dev/null +++ b/tests/queries/0_stateless/03236_create_query_ttl_where.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS ttl; + +CREATE TABLE ttl +( + `a` UInt32, + `timestamp` DateTime +) +ENGINE = MergeTree +ORDER BY a +TTL timestamp + toIntervalSecond(2) WHERE a IN ( + SELECT number + FROM system.numbers + LIMIT 100000 +); + +SHOW CREATE ttl; +DROP TABLE ttl; From f4b4b3cc3548caaa5b78696b006326079849e642 Mon Sep 17 00:00:00 2001 From: maxvostrikov Date: Wed, 18 Sep 2024 17:22:36 +0200 Subject: [PATCH 116/128] added some edge cases for printf tests added some edge cases for printf tests --- .../03203_function_printf.reference | 52 +++++++++++-------- .../0_stateless/03203_function_printf.sql | 50 ++++++++++-------- 2 files changed, 60 insertions(+), 42 deletions(-) diff --git a/tests/queries/0_stateless/03203_function_printf.reference b/tests/queries/0_stateless/03203_function_printf.reference index 338ecb0183d..953cbbdb175 100644 --- a/tests/queries/0_stateless/03203_function_printf.reference +++ b/tests/queries/0_stateless/03203_function_printf.reference @@ -1,21 +1,31 @@ -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 +%d: 123 +%d: -123 +%d: 0 +%d: 9223372036854775807 +%i: 123 +%u: 123 +%o: 173 +%x: 7b +%X: 7B +%f: 0.000000 +%f: 123.456000 +%f: -123.456000 +%F: 123.456000 +%e: 1.234560e+02 +%E: 1.234560E+02 +%g: 123.456 +%G: 123.456 +%a: 0x1.edd2f1a9fbe77p+6 +%A: 0X1.EDD2F1A9FBE77P+6 +%s: abc + ┌─printf('%%s: %s', '\n\t')─┐ +1. │ %s: + │ + └───────────────────────────┘ +%s: +%%: % +%.5d: 00123 +%.2f: 123.46 +%.2e: 1.23e+02 +%.2g: 1.2e+02 +%.2s: ab diff --git a/tests/queries/0_stateless/03203_function_printf.sql b/tests/queries/0_stateless/03203_function_printf.sql index 6ff4699c8a7..3151b37da2d 100644 --- a/tests/queries/0_stateless/03203_function_printf.sql +++ b/tests/queries/0_stateless/03203_function_printf.sql @@ -1,39 +1,47 @@ -- Testing integer formats -select printf('%%d: %d', 123) = '%d: 123'; -select printf('%%i: %i', 123) = '%i: 123'; -select printf('%%u: %u', 123) = '%u: 123'; -select printf('%%o: %o', 123) = '%o: 173'; -select printf('%%x: %x', 123) = '%x: 7b'; -select printf('%%X: %X', 123) = '%X: 7B'; +select printf('%%d: %d', 123); +select printf('%%d: %d', -123); +select printf('%%d: %d', 0); +select printf('%%d: %d', 9223372036854775807); +select printf('%%i: %i', 123); +select printf('%%u: %u', 123); +select printf('%%o: %o', 123); +select printf('%%x: %x', 123); +select printf('%%X: %X', 123); -- Testing floating point formats -select printf('%%f: %f', 123.456) = '%f: 123.456000'; -select printf('%%F: %F', 123.456) = '%F: 123.456000'; -select printf('%%e: %e', 123.456) = '%e: 1.234560e+02'; -select printf('%%E: %E', 123.456) = '%E: 1.234560E+02'; -select printf('%%g: %g', 123.456) = '%g: 123.456'; -select printf('%%G: %G', 123.456) = '%G: 123.456'; -select printf('%%a: %a', 123.456) = '%a: 0x1.edd2f1a9fbe77p+6'; -select printf('%%A: %A', 123.456) = '%A: 0X1.EDD2F1A9FBE77P+6'; +select printf('%%f: %f', 0.0); +select printf('%%f: %f', 123.456); +select printf('%%f: %f', -123.456); +select printf('%%F: %F', 123.456); +select printf('%%e: %e', 123.456); +select printf('%%E: %E', 123.456); +select printf('%%g: %g', 123.456); +select printf('%%G: %G', 123.456); +select printf('%%a: %a', 123.456); +select printf('%%A: %A', 123.456); -- Testing character formats -select printf('%%s: %s', 'abc') = '%s: abc'; +select printf('%%s: %s', 'abc'); +SELECT printf('%%s: %s', '\n\t') FORMAT PrettyCompact; +select printf('%%s: %s', ''); -- Testing the %% specifier -select printf('%%%%: %%') = '%%: %'; +select printf('%%%%: %%'); -- Testing integer formats with precision -select printf('%%.5d: %.5d', 123) = '%.5d: 00123'; +select printf('%%.5d: %.5d', 123); -- Testing floating point formats with precision -select printf('%%.2f: %.2f', 123.456) = '%.2f: 123.46'; -select printf('%%.2e: %.2e', 123.456) = '%.2e: 1.23e+02'; -select printf('%%.2g: %.2g', 123.456) = '%.2g: 1.2e+02'; +select printf('%%.2f: %.2f', 123.456); +select printf('%%.2e: %.2e', 123.456); +select printf('%%.2g: %.2g', 123.456); -- Testing character formats with precision -select printf('%%.2s: %.2s', 'abc') = '%.2s: ab'; +select printf('%%.2s: %.2s', 'abc'); select printf('%%X: %X', 123.123); -- { serverError BAD_ARGUMENTS } select printf('%%A: %A', 'abc'); -- { serverError BAD_ARGUMENTS } select printf('%%s: %s', 100); -- { serverError BAD_ARGUMENTS } select printf('%%n: %n', 100); -- { serverError BAD_ARGUMENTS } +select printf('%%f: %f', 0); -- { serverError BAD_ARGUMENTS } From e818b65dc03cf0849d381c2d948deda647a59523 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 18 Sep 2024 16:43:24 +0000 Subject: [PATCH 117/128] Bump libpq to v16.4 --- contrib/postgres | 2 +- contrib/postgres-cmake/CMakeLists.txt | 3 + contrib/postgres-cmake/nodes/nodetags.h | 471 ++++++++++++++++++++++++ contrib/postgres-cmake/pg_config.h | 160 +------- 4 files changed, 485 insertions(+), 151 deletions(-) create mode 100644 contrib/postgres-cmake/nodes/nodetags.h diff --git a/contrib/postgres b/contrib/postgres index cfd77000af2..2e51f82e27f 160000 --- a/contrib/postgres +++ b/contrib/postgres @@ -1 +1 @@ -Subproject commit cfd77000af28469fcb650485bad65a35e7649e41 +Subproject commit 2e51f82e27f4be389cc239d1b8784bbf2f01d33a diff --git a/contrib/postgres-cmake/CMakeLists.txt b/contrib/postgres-cmake/CMakeLists.txt index 644e6530bbd..7f1bca79f27 100644 --- a/contrib/postgres-cmake/CMakeLists.txt +++ b/contrib/postgres-cmake/CMakeLists.txt @@ -38,12 +38,14 @@ set(SRCS "${POSTGRES_SOURCE_DIR}/src/common/fe_memutils.c" "${POSTGRES_SOURCE_DIR}/src/common/string.c" "${POSTGRES_SOURCE_DIR}/src/common/pg_get_line.c" + "${POSTGRES_SOURCE_DIR}/src/common/pg_prng.c" "${POSTGRES_SOURCE_DIR}/src/common/stringinfo.c" "${POSTGRES_SOURCE_DIR}/src/common/psprintf.c" "${POSTGRES_SOURCE_DIR}/src/common/encnames.c" "${POSTGRES_SOURCE_DIR}/src/common/logging.c" "${POSTGRES_SOURCE_DIR}/src/port/snprintf.c" + "${POSTGRES_SOURCE_DIR}/src/port/strlcat.c" "${POSTGRES_SOURCE_DIR}/src/port/strlcpy.c" "${POSTGRES_SOURCE_DIR}/src/port/strerror.c" "${POSTGRES_SOURCE_DIR}/src/port/inet_net_ntop.c" @@ -52,6 +54,7 @@ set(SRCS "${POSTGRES_SOURCE_DIR}/src/port/noblock.c" "${POSTGRES_SOURCE_DIR}/src/port/pg_strong_random.c" "${POSTGRES_SOURCE_DIR}/src/port/pgstrcasecmp.c" + "${POSTGRES_SOURCE_DIR}/src/port/pg_bitutils.c" "${POSTGRES_SOURCE_DIR}/src/port/thread.c" "${POSTGRES_SOURCE_DIR}/src/port/path.c" ) diff --git a/contrib/postgres-cmake/nodes/nodetags.h b/contrib/postgres-cmake/nodes/nodetags.h new file mode 100644 index 00000000000..f75ac7a05ee --- /dev/null +++ b/contrib/postgres-cmake/nodes/nodetags.h @@ -0,0 +1,471 @@ +/*------------------------------------------------------------------------- + * + * nodetags.h + * Generated node infrastructure code + * + * Portions Copyright (c) 1996-2023, PostgreSQL Global Development Group + * Portions Copyright (c) 1994, Regents of the University of California + * + * NOTES + * ****************************** + * *** DO NOT EDIT THIS FILE! *** + * ****************************** + * + * It has been GENERATED by src/backend/nodes/gen_node_support.pl + * + *------------------------------------------------------------------------- + */ + T_List = 1, + T_Alias = 2, + T_RangeVar = 3, + T_TableFunc = 4, + T_IntoClause = 5, + T_Var = 6, + T_Const = 7, + T_Param = 8, + T_Aggref = 9, + T_GroupingFunc = 10, + T_WindowFunc = 11, + T_SubscriptingRef = 12, + T_FuncExpr = 13, + T_NamedArgExpr = 14, + T_OpExpr = 15, + T_DistinctExpr = 16, + T_NullIfExpr = 17, + T_ScalarArrayOpExpr = 18, + T_BoolExpr = 19, + T_SubLink = 20, + T_SubPlan = 21, + T_AlternativeSubPlan = 22, + T_FieldSelect = 23, + T_FieldStore = 24, + T_RelabelType = 25, + T_CoerceViaIO = 26, + T_ArrayCoerceExpr = 27, + T_ConvertRowtypeExpr = 28, + T_CollateExpr = 29, + T_CaseExpr = 30, + T_CaseWhen = 31, + T_CaseTestExpr = 32, + T_ArrayExpr = 33, + T_RowExpr = 34, + T_RowCompareExpr = 35, + T_CoalesceExpr = 36, + T_MinMaxExpr = 37, + T_SQLValueFunction = 38, + T_XmlExpr = 39, + T_JsonFormat = 40, + T_JsonReturning = 41, + T_JsonValueExpr = 42, + T_JsonConstructorExpr = 43, + T_JsonIsPredicate = 44, + T_NullTest = 45, + T_BooleanTest = 46, + T_CoerceToDomain = 47, + T_CoerceToDomainValue = 48, + T_SetToDefault = 49, + T_CurrentOfExpr = 50, + T_NextValueExpr = 51, + T_InferenceElem = 52, + T_TargetEntry = 53, + T_RangeTblRef = 54, + T_JoinExpr = 55, + T_FromExpr = 56, + T_OnConflictExpr = 57, + T_Query = 58, + T_TypeName = 59, + T_ColumnRef = 60, + T_ParamRef = 61, + T_A_Expr = 62, + T_A_Const = 63, + T_TypeCast = 64, + T_CollateClause = 65, + T_RoleSpec = 66, + T_FuncCall = 67, + T_A_Star = 68, + T_A_Indices = 69, + T_A_Indirection = 70, + T_A_ArrayExpr = 71, + T_ResTarget = 72, + T_MultiAssignRef = 73, + T_SortBy = 74, + T_WindowDef = 75, + T_RangeSubselect = 76, + T_RangeFunction = 77, + T_RangeTableFunc = 78, + T_RangeTableFuncCol = 79, + T_RangeTableSample = 80, + T_ColumnDef = 81, + T_TableLikeClause = 82, + T_IndexElem = 83, + T_DefElem = 84, + T_LockingClause = 85, + T_XmlSerialize = 86, + T_PartitionElem = 87, + T_PartitionSpec = 88, + T_PartitionBoundSpec = 89, + T_PartitionRangeDatum = 90, + T_PartitionCmd = 91, + T_RangeTblEntry = 92, + T_RTEPermissionInfo = 93, + T_RangeTblFunction = 94, + T_TableSampleClause = 95, + T_WithCheckOption = 96, + T_SortGroupClause = 97, + T_GroupingSet = 98, + T_WindowClause = 99, + T_RowMarkClause = 100, + T_WithClause = 101, + T_InferClause = 102, + T_OnConflictClause = 103, + T_CTESearchClause = 104, + T_CTECycleClause = 105, + T_CommonTableExpr = 106, + T_MergeWhenClause = 107, + T_MergeAction = 108, + T_TriggerTransition = 109, + T_JsonOutput = 110, + T_JsonKeyValue = 111, + T_JsonObjectConstructor = 112, + T_JsonArrayConstructor = 113, + T_JsonArrayQueryConstructor = 114, + T_JsonAggConstructor = 115, + T_JsonObjectAgg = 116, + T_JsonArrayAgg = 117, + T_RawStmt = 118, + T_InsertStmt = 119, + T_DeleteStmt = 120, + T_UpdateStmt = 121, + T_MergeStmt = 122, + T_SelectStmt = 123, + T_SetOperationStmt = 124, + T_ReturnStmt = 125, + T_PLAssignStmt = 126, + T_CreateSchemaStmt = 127, + T_AlterTableStmt = 128, + T_ReplicaIdentityStmt = 129, + T_AlterTableCmd = 130, + T_AlterCollationStmt = 131, + T_AlterDomainStmt = 132, + T_GrantStmt = 133, + T_ObjectWithArgs = 134, + T_AccessPriv = 135, + T_GrantRoleStmt = 136, + T_AlterDefaultPrivilegesStmt = 137, + T_CopyStmt = 138, + T_VariableSetStmt = 139, + T_VariableShowStmt = 140, + T_CreateStmt = 141, + T_Constraint = 142, + T_CreateTableSpaceStmt = 143, + T_DropTableSpaceStmt = 144, + T_AlterTableSpaceOptionsStmt = 145, + T_AlterTableMoveAllStmt = 146, + T_CreateExtensionStmt = 147, + T_AlterExtensionStmt = 148, + T_AlterExtensionContentsStmt = 149, + T_CreateFdwStmt = 150, + T_AlterFdwStmt = 151, + T_CreateForeignServerStmt = 152, + T_AlterForeignServerStmt = 153, + T_CreateForeignTableStmt = 154, + T_CreateUserMappingStmt = 155, + T_AlterUserMappingStmt = 156, + T_DropUserMappingStmt = 157, + T_ImportForeignSchemaStmt = 158, + T_CreatePolicyStmt = 159, + T_AlterPolicyStmt = 160, + T_CreateAmStmt = 161, + T_CreateTrigStmt = 162, + T_CreateEventTrigStmt = 163, + T_AlterEventTrigStmt = 164, + T_CreatePLangStmt = 165, + T_CreateRoleStmt = 166, + T_AlterRoleStmt = 167, + T_AlterRoleSetStmt = 168, + T_DropRoleStmt = 169, + T_CreateSeqStmt = 170, + T_AlterSeqStmt = 171, + T_DefineStmt = 172, + T_CreateDomainStmt = 173, + T_CreateOpClassStmt = 174, + T_CreateOpClassItem = 175, + T_CreateOpFamilyStmt = 176, + T_AlterOpFamilyStmt = 177, + T_DropStmt = 178, + T_TruncateStmt = 179, + T_CommentStmt = 180, + T_SecLabelStmt = 181, + T_DeclareCursorStmt = 182, + T_ClosePortalStmt = 183, + T_FetchStmt = 184, + T_IndexStmt = 185, + T_CreateStatsStmt = 186, + T_StatsElem = 187, + T_AlterStatsStmt = 188, + T_CreateFunctionStmt = 189, + T_FunctionParameter = 190, + T_AlterFunctionStmt = 191, + T_DoStmt = 192, + T_InlineCodeBlock = 193, + T_CallStmt = 194, + T_CallContext = 195, + T_RenameStmt = 196, + T_AlterObjectDependsStmt = 197, + T_AlterObjectSchemaStmt = 198, + T_AlterOwnerStmt = 199, + T_AlterOperatorStmt = 200, + T_AlterTypeStmt = 201, + T_RuleStmt = 202, + T_NotifyStmt = 203, + T_ListenStmt = 204, + T_UnlistenStmt = 205, + T_TransactionStmt = 206, + T_CompositeTypeStmt = 207, + T_CreateEnumStmt = 208, + T_CreateRangeStmt = 209, + T_AlterEnumStmt = 210, + T_ViewStmt = 211, + T_LoadStmt = 212, + T_CreatedbStmt = 213, + T_AlterDatabaseStmt = 214, + T_AlterDatabaseRefreshCollStmt = 215, + T_AlterDatabaseSetStmt = 216, + T_DropdbStmt = 217, + T_AlterSystemStmt = 218, + T_ClusterStmt = 219, + T_VacuumStmt = 220, + T_VacuumRelation = 221, + T_ExplainStmt = 222, + T_CreateTableAsStmt = 223, + T_RefreshMatViewStmt = 224, + T_CheckPointStmt = 225, + T_DiscardStmt = 226, + T_LockStmt = 227, + T_ConstraintsSetStmt = 228, + T_ReindexStmt = 229, + T_CreateConversionStmt = 230, + T_CreateCastStmt = 231, + T_CreateTransformStmt = 232, + T_PrepareStmt = 233, + T_ExecuteStmt = 234, + T_DeallocateStmt = 235, + T_DropOwnedStmt = 236, + T_ReassignOwnedStmt = 237, + T_AlterTSDictionaryStmt = 238, + T_AlterTSConfigurationStmt = 239, + T_PublicationTable = 240, + T_PublicationObjSpec = 241, + T_CreatePublicationStmt = 242, + T_AlterPublicationStmt = 243, + T_CreateSubscriptionStmt = 244, + T_AlterSubscriptionStmt = 245, + T_DropSubscriptionStmt = 246, + T_PlannerGlobal = 247, + T_PlannerInfo = 248, + T_RelOptInfo = 249, + T_IndexOptInfo = 250, + T_ForeignKeyOptInfo = 251, + T_StatisticExtInfo = 252, + T_JoinDomain = 253, + T_EquivalenceClass = 254, + T_EquivalenceMember = 255, + T_PathKey = 256, + T_PathTarget = 257, + T_ParamPathInfo = 258, + T_Path = 259, + T_IndexPath = 260, + T_IndexClause = 261, + T_BitmapHeapPath = 262, + T_BitmapAndPath = 263, + T_BitmapOrPath = 264, + T_TidPath = 265, + T_TidRangePath = 266, + T_SubqueryScanPath = 267, + T_ForeignPath = 268, + T_CustomPath = 269, + T_AppendPath = 270, + T_MergeAppendPath = 271, + T_GroupResultPath = 272, + T_MaterialPath = 273, + T_MemoizePath = 274, + T_UniquePath = 275, + T_GatherPath = 276, + T_GatherMergePath = 277, + T_NestPath = 278, + T_MergePath = 279, + T_HashPath = 280, + T_ProjectionPath = 281, + T_ProjectSetPath = 282, + T_SortPath = 283, + T_IncrementalSortPath = 284, + T_GroupPath = 285, + T_UpperUniquePath = 286, + T_AggPath = 287, + T_GroupingSetData = 288, + T_RollupData = 289, + T_GroupingSetsPath = 290, + T_MinMaxAggPath = 291, + T_WindowAggPath = 292, + T_SetOpPath = 293, + T_RecursiveUnionPath = 294, + T_LockRowsPath = 295, + T_ModifyTablePath = 296, + T_LimitPath = 297, + T_RestrictInfo = 298, + T_PlaceHolderVar = 299, + T_SpecialJoinInfo = 300, + T_OuterJoinClauseInfo = 301, + T_AppendRelInfo = 302, + T_RowIdentityVarInfo = 303, + T_PlaceHolderInfo = 304, + T_MinMaxAggInfo = 305, + T_PlannerParamItem = 306, + T_AggInfo = 307, + T_AggTransInfo = 308, + T_PlannedStmt = 309, + T_Result = 310, + T_ProjectSet = 311, + T_ModifyTable = 312, + T_Append = 313, + T_MergeAppend = 314, + T_RecursiveUnion = 315, + T_BitmapAnd = 316, + T_BitmapOr = 317, + T_SeqScan = 318, + T_SampleScan = 319, + T_IndexScan = 320, + T_IndexOnlyScan = 321, + T_BitmapIndexScan = 322, + T_BitmapHeapScan = 323, + T_TidScan = 324, + T_TidRangeScan = 325, + T_SubqueryScan = 326, + T_FunctionScan = 327, + T_ValuesScan = 328, + T_TableFuncScan = 329, + T_CteScan = 330, + T_NamedTuplestoreScan = 331, + T_WorkTableScan = 332, + T_ForeignScan = 333, + T_CustomScan = 334, + T_NestLoop = 335, + T_NestLoopParam = 336, + T_MergeJoin = 337, + T_HashJoin = 338, + T_Material = 339, + T_Memoize = 340, + T_Sort = 341, + T_IncrementalSort = 342, + T_Group = 343, + T_Agg = 344, + T_WindowAgg = 345, + T_Unique = 346, + T_Gather = 347, + T_GatherMerge = 348, + T_Hash = 349, + T_SetOp = 350, + T_LockRows = 351, + T_Limit = 352, + T_PlanRowMark = 353, + T_PartitionPruneInfo = 354, + T_PartitionedRelPruneInfo = 355, + T_PartitionPruneStepOp = 356, + T_PartitionPruneStepCombine = 357, + T_PlanInvalItem = 358, + T_ExprState = 359, + T_IndexInfo = 360, + T_ExprContext = 361, + T_ReturnSetInfo = 362, + T_ProjectionInfo = 363, + T_JunkFilter = 364, + T_OnConflictSetState = 365, + T_MergeActionState = 366, + T_ResultRelInfo = 367, + T_EState = 368, + T_WindowFuncExprState = 369, + T_SetExprState = 370, + T_SubPlanState = 371, + T_DomainConstraintState = 372, + T_ResultState = 373, + T_ProjectSetState = 374, + T_ModifyTableState = 375, + T_AppendState = 376, + T_MergeAppendState = 377, + T_RecursiveUnionState = 378, + T_BitmapAndState = 379, + T_BitmapOrState = 380, + T_ScanState = 381, + T_SeqScanState = 382, + T_SampleScanState = 383, + T_IndexScanState = 384, + T_IndexOnlyScanState = 385, + T_BitmapIndexScanState = 386, + T_BitmapHeapScanState = 387, + T_TidScanState = 388, + T_TidRangeScanState = 389, + T_SubqueryScanState = 390, + T_FunctionScanState = 391, + T_ValuesScanState = 392, + T_TableFuncScanState = 393, + T_CteScanState = 394, + T_NamedTuplestoreScanState = 395, + T_WorkTableScanState = 396, + T_ForeignScanState = 397, + T_CustomScanState = 398, + T_JoinState = 399, + T_NestLoopState = 400, + T_MergeJoinState = 401, + T_HashJoinState = 402, + T_MaterialState = 403, + T_MemoizeState = 404, + T_SortState = 405, + T_IncrementalSortState = 406, + T_GroupState = 407, + T_AggState = 408, + T_WindowAggState = 409, + T_UniqueState = 410, + T_GatherState = 411, + T_GatherMergeState = 412, + T_HashState = 413, + T_SetOpState = 414, + T_LockRowsState = 415, + T_LimitState = 416, + T_IndexAmRoutine = 417, + T_TableAmRoutine = 418, + T_TsmRoutine = 419, + T_EventTriggerData = 420, + T_TriggerData = 421, + T_TupleTableSlot = 422, + T_FdwRoutine = 423, + T_Bitmapset = 424, + T_ExtensibleNode = 425, + T_ErrorSaveContext = 426, + T_IdentifySystemCmd = 427, + T_BaseBackupCmd = 428, + T_CreateReplicationSlotCmd = 429, + T_DropReplicationSlotCmd = 430, + T_StartReplicationCmd = 431, + T_ReadReplicationSlotCmd = 432, + T_TimeLineHistoryCmd = 433, + T_SupportRequestSimplify = 434, + T_SupportRequestSelectivity = 435, + T_SupportRequestCost = 436, + T_SupportRequestRows = 437, + T_SupportRequestIndexCondition = 438, + T_SupportRequestWFuncMonotonic = 439, + T_SupportRequestOptimizeWindowClause = 440, + T_Integer = 441, + T_Float = 442, + T_Boolean = 443, + T_String = 444, + T_BitString = 445, + T_ForeignKeyCacheInfo = 446, + T_IntList = 447, + T_OidList = 448, + T_XidList = 449, + T_AllocSetContext = 450, + T_GenerationContext = 451, + T_SlabContext = 452, + T_TIDBitmap = 453, + T_WindowObjectData = 454, diff --git a/contrib/postgres-cmake/pg_config.h b/contrib/postgres-cmake/pg_config.h index 0060d4960db..efe6a58ec73 100644 --- a/contrib/postgres-cmake/pg_config.h +++ b/contrib/postgres-cmake/pg_config.h @@ -66,13 +66,6 @@ reference if 'false' */ #define FLOAT8PASSBYVAL false -/* Define to 1 if gettimeofday() takes only 1 argument. */ -/* #undef GETTIMEOFDAY_1ARG */ - -#ifdef GETTIMEOFDAY_1ARG -# define gettimeofday(a,b) gettimeofday(a) -#endif - /* Define to 1 if you have the `append_history' function. */ /* #undef HAVE_APPEND_HISTORY */ @@ -113,9 +106,6 @@ don't. */ #define HAVE_DECL_SNPRINTF 1 -/* Define to 1 if you have the declaration of `sigwait', and to 0 if you don't. */ -#define HAVE_DECL_SIGWAIT 1 - /* Define to 1 if you have the declaration of `strlcat', and to 0 if you don't. */ #if OS_DARWIN @@ -139,21 +129,12 @@ /* Define to 1 if you have the header file. */ /* #undef HAVE_DLD_H */ -/* Define to 1 if you have the `dlopen' function. */ -#define HAVE_DLOPEN 1 - /* Define to 1 if you have the header file. */ /* #undef HAVE_EDITLINE_HISTORY_H */ /* Define to 1 if you have the header file. */ #define HAVE_EDITLINE_READLINE_H 1 -/* Define to 1 if you have the `fdatasync' function. */ -#define HAVE_FDATASYNC 1 - -/* Define to 1 if you have the `fls' function. */ -/* #undef HAVE_FLS */ - /* Define to 1 if you have the `fpclass' function. */ /* #undef HAVE_FPCLASS */ @@ -169,12 +150,6 @@ /* Define to 1 if fseeko (and presumably ftello) exists and is declared. */ #define HAVE_FSEEKO 1 -/* Define to 1 if your compiler understands __func__. */ -#define HAVE_FUNCNAME__FUNC 1 - -/* Define to 1 if your compiler understands __FUNCTION__. */ -/* #undef HAVE_FUNCNAME__FUNCTION */ - /* Define to 1 if you have __atomic_compare_exchange_n(int *, int *, int). */ /* #undef HAVE_GCC__ATOMIC_INT32_CAS */ @@ -194,12 +169,6 @@ /* Define to 1 if you have __sync_compare_and_swap(int64 *, int64, int64). */ /* #undef HAVE_GCC__SYNC_INT64_CAS */ -/* Define to 1 if you have the `getaddrinfo' function. */ -#define HAVE_GETADDRINFO 1 - -/* Define to 1 if you have the `gethostbyname_r' function. */ -#define HAVE_GETHOSTBYNAME_R 1 - /* Define to 1 if you have the `getifaddrs' function. */ #define HAVE_GETIFADDRS 1 @@ -218,17 +187,11 @@ /* Define to 1 if you have the `getpeerucred' function. */ /* #undef HAVE_GETPEERUCRED */ -/* Define to 1 if you have the `getpwuid_r' function. */ -#define HAVE_GETPWUID_R 1 +/* Define to 1 if you have the header file. */ +/* #undef HAVE_GSSAPI_EXT_H */ -/* Define to 1 if you have the `getrlimit' function. */ -#define HAVE_GETRLIMIT 1 - -/* Define to 1 if you have the `getrusage' function. */ -#define HAVE_GETRUSAGE 1 - -/* Define to 1 if you have the `gettimeofday' function. */ -/* #undef HAVE_GETTIMEOFDAY */ +/* Define to 1 if you have the header file. */ +/* #undef HAVE_GSSAPI_GSSAPI_EXT_H */ /* Define to 1 if you have the header file. */ //#define HAVE_GSSAPI_GSSAPI_H 0 @@ -275,18 +238,12 @@ /* Define to 1 if you have the global variable 'int timezone'. */ #define HAVE_INT_TIMEZONE 1 -/* Define to 1 if you have support for IPv6. */ -#define HAVE_IPV6 1 - /* Define to 1 if you have isinf(). */ #define HAVE_ISINF 1 /* Define to 1 if you have the header file. */ #define HAVE_LANGINFO_H 1 -/* Define to 1 if you have the header file. */ -//#define HAVE_LDAP_H 0 - /* Define to 1 if you have the `crypto' library (-lcrypto). */ #define HAVE_LIBCRYPTO 1 @@ -351,18 +308,9 @@ /* Define to 1 if you have the header file. */ #define HAVE_MEMORY_H 1 -/* Define to 1 if the system has the type `MINIDUMP_TYPE'. */ -/* #undef HAVE_MINIDUMP_TYPE */ - /* Define to 1 if you have the `mkdtemp' function. */ #define HAVE_MKDTEMP 1 -/* Define to 1 if you have the header file. */ -#define HAVE_NETINET_IN_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_NETINET_TCP_H 1 - /* Define to 1 if you have the header file. */ #define HAVE_NET_IF_H 1 @@ -372,15 +320,6 @@ /* Define to 1 if you have the header file. */ /* #undef HAVE_PAM_PAM_APPL_H */ -/* Define to 1 if you have the `poll' function. */ -#define HAVE_POLL 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_POLL_H 1 - -/* Define to 1 if you have a POSIX-conforming sigwait declaration. */ -/* #undef HAVE_POSIX_DECL_SIGWAIT */ - /* Define to 1 if you have the `posix_fadvise' function. */ #define HAVE_POSIX_FADVISE 1 @@ -399,12 +338,6 @@ /* Define to 1 if the assembler supports PPC's LWARX mutex hint bit. */ /* #undef HAVE_PPC_LWARX_MUTEX_HINT */ -/* Define to 1 if you have the `pstat' function. */ -/* #undef HAVE_PSTAT */ - -/* Define to 1 if the PS_STRINGS thing exists. */ -/* #undef HAVE_PS_STRINGS */ - /* Define to 1 if you have the `pthread_is_threaded_np' function. */ /* #undef HAVE_PTHREAD_IS_THREADED_NP */ @@ -420,9 +353,6 @@ /* Define to 1 if you have the header file. */ /* #undef HAVE_READLINE_READLINE_H */ -/* Define to 1 if you have the `readlink' function. */ -#define HAVE_READLINK 1 - /* Define to 1 if you have the `rint' function. */ #define HAVE_RINT 1 @@ -444,12 +374,6 @@ /* Define to 1 if you have the `setproctitle' function. */ /* #undef HAVE_SETPROCTITLE */ -/* Define to 1 if you have the `setsid' function. */ -#define HAVE_SETSID 1 - -/* Define to 1 if you have the `shm_open' function. */ -#define HAVE_SHM_OPEN 1 - /* Define to 1 if the system has the type `socklen_t'. */ #define HAVE_SOCKLEN_T 1 @@ -468,6 +392,9 @@ /* Define to 1 if you have spinlocks. */ #define HAVE_SPINLOCKS 1 +/* Define to 1 if you have the `SSL_CTX_set_cert_cb' function. */ +#define HAVE_SSL_CTX_SET_CERT_CB 1 + /* Define to 1 if you have the `SSL_CTX_set_num_tickets' function. */ /* #define HAVE_SSL_CTX_SET_NUM_TICKETS */ @@ -498,55 +425,19 @@ /* Define to 1 if you have the `strlcpy' function. */ /* #undef HAVE_STRLCPY */ -/* Define to 1 if you have the `strtoll' function. */ -#define HAVE_STRTOLL 1 - #if (!OS_DARWIN) #define HAVE_STRCHRNUL 1 #endif -/* Define to 1 if you have the `strtoq' function. */ -/* #undef HAVE_STRTOQ */ - -/* Define to 1 if you have the `strtoull' function. */ -#define HAVE_STRTOULL 1 - -/* Define to 1 if you have the `strtouq' function. */ -/* #undef HAVE_STRTOUQ */ - -/* Define to 1 if the system has the type `struct addrinfo'. */ -#define HAVE_STRUCT_ADDRINFO 1 - -/* Define to 1 if the system has the type `struct cmsgcred'. */ -/* #undef HAVE_STRUCT_CMSGCRED */ - /* Define to 1 if the system has the type `struct option'. */ #define HAVE_STRUCT_OPTION 1 /* Define to 1 if `sa_len' is a member of `struct sockaddr'. */ /* #undef HAVE_STRUCT_SOCKADDR_SA_LEN */ -/* Define to 1 if the system has the type `struct sockaddr_storage'. */ -#define HAVE_STRUCT_SOCKADDR_STORAGE 1 - -/* Define to 1 if `ss_family' is a member of `struct sockaddr_storage'. */ -#define HAVE_STRUCT_SOCKADDR_STORAGE_SS_FAMILY 1 - -/* Define to 1 if `ss_len' is a member of `struct sockaddr_storage'. */ -/* #undef HAVE_STRUCT_SOCKADDR_STORAGE_SS_LEN */ - -/* Define to 1 if `__ss_family' is a member of `struct sockaddr_storage'. */ -/* #undef HAVE_STRUCT_SOCKADDR_STORAGE___SS_FAMILY */ - -/* Define to 1 if `__ss_len' is a member of `struct sockaddr_storage'. */ -/* #undef HAVE_STRUCT_SOCKADDR_STORAGE___SS_LEN */ - /* Define to 1 if `tm_zone' is a member of `struct tm'. */ #define HAVE_STRUCT_TM_TM_ZONE 1 -/* Define to 1 if you have the `symlink' function. */ -#define HAVE_SYMLINK 1 - /* Define to 1 if you have the `sync_file_range' function. */ /* #undef HAVE_SYNC_FILE_RANGE */ @@ -556,45 +447,21 @@ /* Define to 1 if you have the header file. */ #define HAVE_SYS_IOCTL_H 1 -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_IPC_H 1 - /* Define to 1 if you have the header file. */ /* #undef HAVE_SYS_PERSONALITY_H */ /* Define to 1 if you have the header file. */ #define HAVE_SYS_POLL_H 1 -/* Define to 1 if you have the header file. */ -/* #undef HAVE_SYS_PSTAT_H */ - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_RESOURCE_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_SELECT_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_SEM_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_SHM_H 1 - /* Define to 1 if you have the header file. */ /* #undef HAVE_SYS_SIGNALFD_H */ /* Define to 1 if you have the header file. */ #define HAVE_SYS_SOCKET_H 1 -/* Define to 1 if you have the header file. */ -/* #undef HAVE_SYS_SOCKIO_H */ - /* Define to 1 if you have the header file. */ #define HAVE_SYS_STAT_H 1 -/* Define to 1 if you have the header file. */ -/* #undef HAVE_SYS_TAS_H */ - /* Define to 1 if you have the header file. */ #define HAVE_SYS_TIME_H 1 @@ -607,7 +474,6 @@ #endif /* Define to 1 if you have the header file. */ -#define HAVE_SYS_UN_H 1 #define _GNU_SOURCE 1 /* Needed for glibc struct ucred */ /* Define to 1 if you have the header file. */ @@ -644,9 +510,6 @@ /* Define to 1 if you have unix sockets. */ #define HAVE_UNIX_SOCKETS 1 -/* Define to 1 if you have the `unsetenv' function. */ -#define HAVE_UNSETENV 1 - /* Define to 1 if the system has the type `unsigned long long int'. */ #define HAVE_UNSIGNED_LONG_LONG_INT 1 @@ -674,6 +537,9 @@ /* Define to 1 if you have the header file. */ /* #undef HAVE_UUID_UUID_H */ +/* Define to 1 if your compiler knows the visibility("hidden") attribute. */ +/* #undef HAVE_VISIBILITY_ATTRIBUTE */ + /* Define to 1 if you have the `vsnprintf' function. */ #define HAVE_VSNPRINTF 1 @@ -686,12 +552,6 @@ /* Define to 1 if you have the `wcstombs_l' function. */ /* #undef HAVE_WCSTOMBS_L */ -/* Define to 1 if you have the header file. */ -#define HAVE_WCTYPE_H 1 - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_WINLDAP_H */ - /* Define to 1 if your compiler understands __builtin_bswap32. */ /* #undef HAVE__BUILTIN_BSWAP32 */ From 02fcd90a66ea61dee36db290a089a7cb48142ba4 Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 14:13:22 -0300 Subject: [PATCH 118/128] address some pr comments --- src/Functions/array/arrayAUC.cpp | 22 ++++++------- .../0_stateless/01064_array_auc.reference | 16 ++++++++++ tests/queries/0_stateless/01064_array_auc.sql | 18 ++++++++++- .../0_stateless/01064_array_auc.stdout-e | 32 +++++++++++++++++++ .../01202_array_auc_special.reference | 9 ++++++ .../0_stateless/01202_array_auc_special.sql | 14 ++++++++ ...rence => 01202_array_auc_special.stdout-e} | 21 ++++-------- .../0_stateless/03237_array_auc_unscaled.sql | 30 ----------------- 8 files changed, 106 insertions(+), 56 deletions(-) create mode 100644 tests/queries/0_stateless/01064_array_auc.stdout-e rename tests/queries/0_stateless/{03237_array_auc_unscaled.reference => 01202_array_auc_special.stdout-e} (50%) delete mode 100644 tests/queries/0_stateless/03237_array_auc_unscaled.sql diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 04ebb6d5bac..5577a51e198 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -87,7 +87,7 @@ private: const IColumn & labels, ColumnArray::Offset current_offset, ColumnArray::Offset next_offset, - bool scale = true) + bool scale) { struct ScoreLabel { @@ -116,10 +116,10 @@ private: size_t curr_fp = 0, curr_tp = 0; for (size_t i = 0; i < size; ++i) { - // Only increment the area when the score changes + /// Only increment the area when the score changes if (sorted_labels[i].score != prev_score) { - area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; // Trapezoidal area under curve (might degenerate to zero or to a rectangle) + area += (curr_fp - prev_fp) * (curr_tp + prev_tp) / 2.0; /// Trapezoidal area under curve (might degenerate to zero or to a rectangle) prev_fp = curr_fp; prev_tp = curr_tp; prev_score = sorted_labels[i].score; @@ -135,12 +135,12 @@ private: /// Then normalize it, if scale is true, dividing by the area to the area of rectangle. - if (scale && (curr_tp == 0 || curr_tp == size)) - return std::numeric_limits::quiet_NaN(); - if (scale) + { + if (curr_tp == 0 || curr_tp == size) + return std::numeric_limits::quiet_NaN(); return area / curr_tp / (size - curr_tp); - + } return area; } @@ -150,7 +150,7 @@ private: const ColumnArray::Offsets & offsets, PaddedPODArray & result, size_t input_rows_count, - bool scale = true) + bool scale) { result.resize(input_rows_count); @@ -195,7 +195,7 @@ public: if (number_of_arguments == 3) { if (!isBool(arguments[2])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Third argument must be a boolean (scale)"); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Third argument (scale) for function {} must be a bool.", getName()); } return std::make_shared(); @@ -221,11 +221,11 @@ public: if (!col_array1->hasEqualOffsets(*col_array2)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName()); - // Handle third argument for scale (if passed, otherwise default to true) + /// Handle third argument for scale (if passed, otherwise default to true) bool scale = true; if (number_of_arguments == 3) { - scale = arguments[2].column->getBool(0); // Assumes it's a scalar boolean column + scale = arguments[2].column->getBool(0); /// Assumes it's a scalar boolean column } auto col_res = ColumnVector::create(); diff --git a/tests/queries/0_stateless/01064_array_auc.reference b/tests/queries/0_stateless/01064_array_auc.reference index 8c17bba359a..8b5c852a38b 100644 --- a/tests/queries/0_stateless/01064_array_auc.reference +++ b/tests/queries/0_stateless/01064_array_auc.reference @@ -14,3 +14,19 @@ 0.25 0.125 0.25 +3 +3 +3 +3 +3 +3 +3 +3 +3 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01064_array_auc.sql b/tests/queries/0_stateless/01064_array_auc.sql index de05c47c51b..94767b72931 100644 --- a/tests/queries/0_stateless/01064_array_auc.sql +++ b/tests/queries/0_stateless/01064_array_auc.sql @@ -13,4 +13,20 @@ select arrayAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1]); select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]); select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]); select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]); -select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]); \ No newline at end of file +select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]); +select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], false); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), false); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), false); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))), false); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))), false); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1], false); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1], false); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1], false); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1], false); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1], false); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1], false); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1], false); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], false); +select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], false); +select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], false); +select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], false); diff --git a/tests/queries/0_stateless/01064_array_auc.stdout-e b/tests/queries/0_stateless/01064_array_auc.stdout-e new file mode 100644 index 00000000000..8b5c852a38b --- /dev/null +++ b/tests/queries/0_stateless/01064_array_auc.stdout-e @@ -0,0 +1,32 @@ +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.25 +0.25 +0.25 +0.25 +0.25 +0.125 +0.25 +3 +3 +3 +3 +3 +3 +3 +3 +3 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01202_array_auc_special.reference b/tests/queries/0_stateless/01202_array_auc_special.reference index 8f3f0cf1efe..cb25b381ff7 100644 --- a/tests/queries/0_stateless/01202_array_auc_special.reference +++ b/tests/queries/0_stateless/01202_array_auc_special.reference @@ -7,3 +7,12 @@ nan 0.75 1 0.75 +0 +0 +0 +0.5 +1 +0 +1.5 +2 +1.5 diff --git a/tests/queries/0_stateless/01202_array_auc_special.sql b/tests/queries/0_stateless/01202_array_auc_special.sql index e379050a982..f22524c2756 100644 --- a/tests/queries/0_stateless/01202_array_auc_special.sql +++ b/tests/queries/0_stateless/01202_array_auc_special.sql @@ -12,3 +12,17 @@ SELECT arrayAUC([1, 0], [0, 1]); SELECT arrayAUC([0, 0, 1], [0, 1, 1]); SELECT arrayAUC([0, 1, 1], [0, 1, 1]); SELECT arrayAUC([0, 1, 1], [0, 0, 1]); +SELECT arrayAUC([], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([1], [1], false); +SELECT arrayAUC([1], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([], [1], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([1, 2], [3], false); -- { serverError BAD_ARGUMENTS } +SELECT arrayAUC([1], [2, 3], false); -- { serverError BAD_ARGUMENTS } +SELECT arrayAUC([1, 1], [1, 1], false); +SELECT arrayAUC([1, 1], [0, 0], false); +SELECT arrayAUC([1, 1], [0, 1], false); +SELECT arrayAUC([0, 1], [0, 1], false); +SELECT arrayAUC([1, 0], [0, 1], false); +SELECT arrayAUC([0, 0, 1], [0, 1, 1], false); +SELECT arrayAUC([0, 1, 1], [0, 1, 1], false); +SELECT arrayAUC([0, 1, 1], [0, 0, 1], false); diff --git a/tests/queries/0_stateless/03237_array_auc_unscaled.reference b/tests/queries/0_stateless/01202_array_auc_special.stdout-e similarity index 50% rename from tests/queries/0_stateless/03237_array_auc_unscaled.reference rename to tests/queries/0_stateless/01202_array_auc_special.stdout-e index 63204682fd4..cb25b381ff7 100644 --- a/tests/queries/0_stateless/03237_array_auc_unscaled.reference +++ b/tests/queries/0_stateless/01202_array_auc_special.stdout-e @@ -1,19 +1,12 @@ -3 -3 -3 -3 -3 -3 -3 -3 -3 -1 -1 -1 -1 -1 +nan +nan +nan +0.5 1 +0 +0.75 1 +0.75 0 0 0 diff --git a/tests/queries/0_stateless/03237_array_auc_unscaled.sql b/tests/queries/0_stateless/03237_array_auc_unscaled.sql deleted file mode 100644 index 4083e836067..00000000000 --- a/tests/queries/0_stateless/03237_array_auc_unscaled.sql +++ /dev/null @@ -1,30 +0,0 @@ -select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], false); -select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), false); -select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), false); -select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))), false); -select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))), false); -select arrayAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1], false); -select arrayAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1], false); -select arrayAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1], false); -select arrayAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1], false); -select arrayAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1], false); -select arrayAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1], false); -select arrayAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1], false); -select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], false); -select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], false); -select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], false); -select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], false); -SELECT arrayAUC([], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayAUC([1], [1], false); -SELECT arrayAUC([1], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayAUC([], [1], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT arrayAUC([1, 2], [3], false); -- { serverError BAD_ARGUMENTS } -SELECT arrayAUC([1], [2, 3], false); -- { serverError BAD_ARGUMENTS } -SELECT arrayAUC([1, 1], [1, 1], false); -SELECT arrayAUC([1, 1], [0, 0], false); -SELECT arrayAUC([1, 1], [0, 1], false); -SELECT arrayAUC([0, 1], [0, 1], false); -SELECT arrayAUC([1, 0], [0, 1], false); -SELECT arrayAUC([0, 0, 1], [0, 1, 1], false); -SELECT arrayAUC([0, 1, 1], [0, 1, 1], false); -SELECT arrayAUC([0, 1, 1], [0, 0, 1], false); From 373927d6a5a6b89a1ccb7b1207942dc4a5e8a39e Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Sep 2024 19:25:18 +0200 Subject: [PATCH 119/128] Fix tracked_files_ttl_sec --- .../ObjectStorageQueueMetadata.cpp | 12 +++++++----- .../ObjectStorageQueueTableMetadata.cpp | 16 ++++++++-------- .../ObjectStorageQueueTableMetadata.h | 2 +- tests/integration/test_storage_s3_queue/test.py | 8 ++++++++ 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp index 8dbf51a9cf9..5814b60579a 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp @@ -122,7 +122,7 @@ ObjectStorageQueueMetadata::ObjectStorageQueueMetadata( , local_file_statuses(std::make_shared()) { if (mode == ObjectStorageQueueMode::UNORDERED - && (table_metadata.tracked_files_limit || table_metadata.tracked_file_ttl_sec)) + && (table_metadata.tracked_files_limit || table_metadata.tracked_files_ttl_sec)) { task = Context::getGlobalContextInstance()->getSchedulePool().createTask( "ObjectStorageQueueCleanupFunc", @@ -366,9 +366,9 @@ void ObjectStorageQueueMetadata::cleanupThreadFuncImpl() return; } - chassert(table_metadata.tracked_files_limit || table_metadata.tracked_file_ttl_sec); + chassert(table_metadata.tracked_files_limit || table_metadata.tracked_files_ttl_sec); const bool check_nodes_limit = table_metadata.tracked_files_limit > 0; - const bool check_nodes_ttl = table_metadata.tracked_file_ttl_sec > 0; + const bool check_nodes_ttl = table_metadata.tracked_files_ttl_sec > 0; const bool nodes_limit_exceeded = nodes_num > table_metadata.tracked_files_limit; if ((!nodes_limit_exceeded || !check_nodes_limit) && !check_nodes_ttl) @@ -443,7 +443,9 @@ void ObjectStorageQueueMetadata::cleanupThreadFuncImpl() wb << fmt::format("Node: {}, path: {}, timestamp: {};\n", node, metadata.file_path, metadata.last_processed_timestamp); return wb.str(); }; - LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", table_metadata.tracked_files_limit, table_metadata.tracked_file_ttl_sec, get_nodes_str()); + + LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", + table_metadata.tracked_files_limit, table_metadata.tracked_files_ttl_sec, get_nodes_str()); size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes_num - table_metadata.tracked_files_limit : 0; for (const auto & node : sorted_nodes) @@ -464,7 +466,7 @@ void ObjectStorageQueueMetadata::cleanupThreadFuncImpl() else if (check_nodes_ttl) { UInt64 node_age = getCurrentTime() - node.metadata.last_processed_timestamp; - if (node_age >= table_metadata.tracked_file_ttl_sec) + if (node_age >= table_metadata.tracked_files_ttl_sec) { LOG_TRACE(log, "Removing node at path {} ({}) because file ttl is reached", node.metadata.file_path, node.zk_path); diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp index 25d44ed9ddb..f329fdced02 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp @@ -45,7 +45,7 @@ ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata( , after_processing(engine_settings.after_processing.toString()) , mode(engine_settings.mode.toString()) , tracked_files_limit(engine_settings.tracked_files_limit) - , tracked_file_ttl_sec(engine_settings.tracked_file_ttl_sec) + , tracked_files_ttl_sec(engine_settings.tracked_file_ttl_sec) , buckets(engine_settings.buckets) , processing_threads_num(engine_settings.processing_threads_num) , last_processed_path(engine_settings.last_processed_path) @@ -59,7 +59,7 @@ String ObjectStorageQueueTableMetadata::toString() const json.set("after_processing", after_processing); json.set("mode", mode); json.set("tracked_files_limit", tracked_files_limit); - json.set("tracked_file_ttl_sec", tracked_file_ttl_sec); + json.set("tracked_files_ttl_sec", tracked_files_ttl_sec); json.set("processing_threads_num", processing_threads_num); json.set("buckets", buckets); json.set("format_name", format_name); @@ -100,7 +100,7 @@ ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata(const Poco::JSO , after_processing(json->getValue("after_processing")) , mode(json->getValue("mode")) , tracked_files_limit(getOrDefault(json, "tracked_files_limit", "s3queue_", 0)) - , tracked_file_ttl_sec(getOrDefault(json, "tracked_files_ttl_sec", "s3queue_", 0)) + , tracked_files_ttl_sec(getOrDefault(json, "tracked_files_ttl_sec", "", getOrDefault(json, "tracked_file_ttl_sec", "s3queue_", 0))) , buckets(getOrDefault(json, "buckets", "", 0)) , processing_threads_num(getOrDefault(json, "processing_threads_num", "s3queue_", 1)) , last_processed_path(getOrDefault(json, "last_processed_file", "s3queue_", "")) @@ -142,18 +142,18 @@ void ObjectStorageQueueTableMetadata::checkImmutableFieldsEquals(const ObjectSto if (tracked_files_limit != from_zk.tracked_files_limit) throw Exception( ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in max set size. " + "Existing table metadata in ZooKeeper differs in `tracked_files_limit`. " "Stored in ZooKeeper: {}, local: {}", from_zk.tracked_files_limit, tracked_files_limit); - if (tracked_file_ttl_sec != from_zk.tracked_file_ttl_sec) + if (tracked_files_ttl_sec != from_zk.tracked_files_ttl_sec) throw Exception( ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in max set age. " + "Existing table metadata in ZooKeeper differs in `tracked_files_ttl_sec`. " "Stored in ZooKeeper: {}, local: {}", - from_zk.tracked_file_ttl_sec, - tracked_file_ttl_sec); + from_zk.tracked_files_ttl_sec, + tracked_files_ttl_sec); if (format_name != from_zk.format_name) throw Exception( diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h index f1cf6428160..6aa1897c0f0 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h @@ -23,7 +23,7 @@ struct ObjectStorageQueueTableMetadata const String after_processing; const String mode; const UInt64 tracked_files_limit; - const UInt64 tracked_file_ttl_sec; + const UInt64 tracked_files_ttl_sec; const UInt64 buckets; const UInt64 processing_threads_num; const String last_processed_path; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index b75ad21f002..2d1152c7f8d 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -976,6 +976,14 @@ def test_max_set_age(started_cluster): ) ) + node.restart_clickhouse() + + expected_rows *= 2 + wait_for_condition(lambda: get_count() == expected_rows) + assert files_to_generate == int( + node.query(f"SELECT uniq(_path) from {dst_table_name}") + ) + def test_max_set_size(started_cluster): node = started_cluster.instances["instance"] From 006d14445eb619eb518e8c5f6879d0b41b77cd13 Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 14:59:58 -0300 Subject: [PATCH 120/128] remove stdout files --- .../0_stateless/01064_array_auc.stdout-e | 32 ------------------- .../01202_array_auc_special.stdout-e | 18 ----------- 2 files changed, 50 deletions(-) delete mode 100644 tests/queries/0_stateless/01064_array_auc.stdout-e delete mode 100644 tests/queries/0_stateless/01202_array_auc_special.stdout-e diff --git a/tests/queries/0_stateless/01064_array_auc.stdout-e b/tests/queries/0_stateless/01064_array_auc.stdout-e deleted file mode 100644 index 8b5c852a38b..00000000000 --- a/tests/queries/0_stateless/01064_array_auc.stdout-e +++ /dev/null @@ -1,32 +0,0 @@ -0.75 -0.75 -0.75 -0.75 -0.75 -0.75 -0.75 -0.75 -0.75 -0.25 -0.25 -0.25 -0.25 -0.25 -0.125 -0.25 -3 -3 -3 -3 -3 -3 -3 -3 -3 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/01202_array_auc_special.stdout-e b/tests/queries/0_stateless/01202_array_auc_special.stdout-e deleted file mode 100644 index cb25b381ff7..00000000000 --- a/tests/queries/0_stateless/01202_array_auc_special.stdout-e +++ /dev/null @@ -1,18 +0,0 @@ -nan -nan -nan -0.5 -1 -0 -0.75 -1 -0.75 -0 -0 -0 -0.5 -1 -0 -1.5 -2 -1.5 From 7f0b7a915808fe250bbb8da747ae37db5bd096da Mon Sep 17 00:00:00 2001 From: Gabriel Mendes Date: Wed, 18 Sep 2024 15:17:54 -0300 Subject: [PATCH 121/128] add tests to cover all possible flows --- .../0_stateless/01064_array_auc.reference | 16 ++++++++++++++++ tests/queries/0_stateless/01064_array_auc.sql | 16 ++++++++++++++++ .../01202_array_auc_special.reference | 9 +++++++++ .../0_stateless/01202_array_auc_special.sql | 18 ++++++++++++++++++ 4 files changed, 59 insertions(+) diff --git a/tests/queries/0_stateless/01064_array_auc.reference b/tests/queries/0_stateless/01064_array_auc.reference index 8b5c852a38b..3fd5483eb99 100644 --- a/tests/queries/0_stateless/01064_array_auc.reference +++ b/tests/queries/0_stateless/01064_array_auc.reference @@ -14,6 +14,22 @@ 0.25 0.125 0.25 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.25 +0.25 +0.25 +0.25 +0.25 +0.125 +0.25 3 3 3 diff --git a/tests/queries/0_stateless/01064_array_auc.sql b/tests/queries/0_stateless/01064_array_auc.sql index 94767b72931..adc32abcde8 100644 --- a/tests/queries/0_stateless/01064_array_auc.sql +++ b/tests/queries/0_stateless/01064_array_auc.sql @@ -14,6 +14,22 @@ select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]); select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]); select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]); select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]); +select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], true); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), true); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), true); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1))), true); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1))), true); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1], true); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1], true); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1], true); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1], true); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1], true); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1], true); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1], true); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], true); +select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], true); +select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], true); +select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], true); select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], false); select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), false); select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), false); diff --git a/tests/queries/0_stateless/01202_array_auc_special.reference b/tests/queries/0_stateless/01202_array_auc_special.reference index cb25b381ff7..8f4b9495a5c 100644 --- a/tests/queries/0_stateless/01202_array_auc_special.reference +++ b/tests/queries/0_stateless/01202_array_auc_special.reference @@ -7,6 +7,15 @@ nan 0.75 1 0.75 +nan +nan +nan +0.5 +1 +0 +0.75 +1 +0.75 0 0 0 diff --git a/tests/queries/0_stateless/01202_array_auc_special.sql b/tests/queries/0_stateless/01202_array_auc_special.sql index f22524c2756..a7276ec0620 100644 --- a/tests/queries/0_stateless/01202_array_auc_special.sql +++ b/tests/queries/0_stateless/01202_array_auc_special.sql @@ -12,6 +12,20 @@ SELECT arrayAUC([1, 0], [0, 1]); SELECT arrayAUC([0, 0, 1], [0, 1, 1]); SELECT arrayAUC([0, 1, 1], [0, 1, 1]); SELECT arrayAUC([0, 1, 1], [0, 0, 1]); +SELECT arrayAUC([], [], true); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([1], [1], true); +SELECT arrayAUC([1], [], true); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([], [1], true); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([1, 2], [3], true); -- { serverError BAD_ARGUMENTS } +SELECT arrayAUC([1], [2, 3], true); -- { serverError BAD_ARGUMENTS } +SELECT arrayAUC([1, 1], [1, 1], true); +SELECT arrayAUC([1, 1], [0, 0], true); +SELECT arrayAUC([1, 1], [0, 1], true); +SELECT arrayAUC([0, 1], [0, 1], true); +SELECT arrayAUC([1, 0], [0, 1], true); +SELECT arrayAUC([0, 0, 1], [0, 1, 1], true); +SELECT arrayAUC([0, 1, 1], [0, 1, 1], true); +SELECT arrayAUC([0, 1, 1], [0, 0, 1], true); SELECT arrayAUC([], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayAUC([1], [1], false); SELECT arrayAUC([1], [], false); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } @@ -26,3 +40,7 @@ SELECT arrayAUC([1, 0], [0, 1], false); SELECT arrayAUC([0, 0, 1], [0, 1, 1], false); SELECT arrayAUC([0, 1, 1], [0, 1, 1], false); SELECT arrayAUC([0, 1, 1], [0, 0, 1], false); +SELECT arrayAUC([0, 1, 1], [0, 0, 1], false, true); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT arrayAUC([0, 1, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT arrayAUC([0, 1, 1], [0, 0, 1], 'false'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayAUC([0, 1, 1], [0, 0, 1], 4); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } \ No newline at end of file From 818aac02c6b169d879e8ab4a6c57bc56ce4b8944 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 18 Sep 2024 18:33:55 +0100 Subject: [PATCH 122/128] fix --- src/Core/ProtocolDefines.h | 1 + src/Storages/MergeTree/RequestResponse.cpp | 4 +- .../__init__.py | 0 .../configs/remote_servers.xml | 21 ++++++ .../test_parallel_replicas_protocol/test.py | 72 +++++++++++++++++++ 5 files changed, 96 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_parallel_replicas_protocol/__init__.py create mode 100644 tests/integration/test_parallel_replicas_protocol/configs/remote_servers.xml create mode 100644 tests/integration/test_parallel_replicas_protocol/test.py diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 49c6fc1dde6..b68eff0aa5a 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -34,6 +34,7 @@ static constexpr auto DBMS_MIN_REVISION_WITH_AGGREGATE_FUNCTIONS_VERSIONING = 54 static constexpr auto DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION = 1; static constexpr auto DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; +static constexpr auto DBMS_PARALLEL_REPLICAS_MIN_VERSION_WITH_MARK_SEGMENT_SIZE_FIELD = 4; static constexpr auto DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION = 4; static constexpr auto DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS = 54453; diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index cd1cc2ad8e7..f6859b26908 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -136,7 +136,7 @@ void InitialAllRangesAnnouncement::serialize(WriteBuffer & out, UInt64 initiator writeIntBinary(mode, out); description.serialize(out); writeIntBinary(replica_num, out); - if (initiator_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + if (initiator_protocol_version >= DBMS_PARALLEL_REPLICAS_MIN_VERSION_WITH_MARK_SEGMENT_SIZE_FIELD) writeIntBinary(mark_segment_size, out); } @@ -168,7 +168,7 @@ InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffe readIntBinary(replica_num, in); size_t mark_segment_size = 128; - if (replica_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + if (replica_protocol_version >= DBMS_PARALLEL_REPLICAS_MIN_VERSION_WITH_MARK_SEGMENT_SIZE_FIELD) readIntBinary(mark_segment_size, in); return InitialAllRangesAnnouncement{mode, description, replica_num, mark_segment_size}; diff --git a/tests/integration/test_parallel_replicas_protocol/__init__.py b/tests/integration/test_parallel_replicas_protocol/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_parallel_replicas_protocol/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_protocol/configs/remote_servers.xml new file mode 100644 index 00000000000..9fc2f119898 --- /dev/null +++ b/tests/integration/test_parallel_replicas_protocol/configs/remote_servers.xml @@ -0,0 +1,21 @@ + + + + + false + + node0 + 9000 + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_parallel_replicas_protocol/test.py b/tests/integration/test_parallel_replicas_protocol/test.py new file mode 100644 index 00000000000..65b2ab00043 --- /dev/null +++ b/tests/integration/test_parallel_replicas_protocol/test.py @@ -0,0 +1,72 @@ +import pytest +import re + +from helpers.cluster import ClickHouseCluster +from random import randint + +cluster = ClickHouseCluster(__file__) + +nodes = [ + cluster.add_instance( + f"node{num}", + main_configs=["configs/remote_servers.xml"], + with_zookeeper=True, + macros={"replica": f"node{num}", "shard": "shard"}, + ) + for num in range(3) +] + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def _create_tables(table_name): + nodes[0].query( + f"DROP TABLE IF EXISTS {table_name} ON CLUSTER 'parallel_replicas'", + settings={"database_atomic_wait_for_drop_and_detach_synchronously": True}, + ) + + # big number of granules + low total size in bytes = super tiny granules = big min_marks_per_task + # => big mark_segment_size will be chosen. it is not required to be big, just not equal to the default + nodes[0].query( + f""" + CREATE TABLE {table_name} ON CLUSTER 'parallel_replicas' (value Int64) + Engine=ReplicatedMergeTree('/test_parallel_replicas/shard/{table_name}', '{{replica}}') + ORDER BY () + SETTINGS index_granularity = 1 + """ + ) + + nodes[0].query(f"INSERT INTO {table_name} SELECT 42 FROM numbers(1000)") + nodes[0].query(f"SYSTEM SYNC REPLICA ON CLUSTER 'parallel_replicas' {table_name}") + + +# now mark_segment_size is part of the protocol and is communicated to the initiator. +# let's check that the correct value is actually used by the coordinator +def test_mark_segment_size_communicated_correctly(start_cluster): + table_name = "t" + _create_tables(table_name) + + for local_plan in [0, 1]: + query_id = f"query_id_{randint(0, 1000000)}" + nodes[0].query( + f"SELECT sum(value) FROM {table_name}", + settings={ + "allow_experimental_parallel_reading_from_replicas": 2, + "max_parallel_replicas": 100, + "cluster_for_parallel_replicas": "parallel_replicas", + "parallel_replicas_mark_segment_size": 0, + "parallel_replicas_local_plan": local_plan, + "query_id": query_id, + }, + ) + + nodes[0].query("SYSTEM FLUSH LOGS") + log_line = nodes[0].grep_in_log(f"{query_id}.*Reading state is fully initialized") + assert re.search(r"mark_segment_size: (\d+)", log_line).group(1) == "16384" From b88cd7995978cb0186b042e0c2b6d7884979045b Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Thu, 12 Sep 2024 18:17:41 +0000 Subject: [PATCH 123/128] Mask azure connection string sensitive info --- src/Parsers/ASTFunction.cpp | 9 +- src/Parsers/FunctionSecretArgumentsFinder.h | 83 ++++++++++++++++--- .../ObjectStorage/Azure/Configuration.cpp | 6 +- .../test_mask_sensitive_info/test.py | 21 +++-- 4 files changed, 97 insertions(+), 22 deletions(-) diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index d42728addb7..de72641f8f8 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -722,7 +722,14 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format assert_cast(argument.get())->arguments->children[0]->formatImpl(settings, state, nested_dont_need_parens); settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : ""); } - settings.ostr << "'[HIDDEN]'"; + if (!secret_arguments.replacement.empty()) + { + settings.ostr << "'" << secret_arguments.replacement << "'"; + } + else + { + settings.ostr << "'[HIDDEN]'"; + } if (size <= secret_arguments.start + secret_arguments.count && !secret_arguments.are_named) break; /// All other arguments should also be hidden. continue; diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 7836a863920..01dbf38e97f 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -49,6 +50,11 @@ public: bool are_named = false; /// Arguments like `password = 'password'` are considered as named arguments. /// E.g. "headers" in `url('..', headers('foo' = '[HIDDEN]'))` std::vector nested_maps; + /// Full replacement of an argument. Only supported when count is 1, otherwise all arguments will be replaced with this string. + /// It's needed in cases when we don't want to hide the entire parameter, but some part of it, e.g. "connection_string" in + /// `azureBlobStorage('DefaultEndpointsProtocol=https;AccountKey=secretkey;...', ...)` should be replaced with + /// `azureBlobStorage('DefaultEndpointsProtocol=https;AccountKey=[HIDDEN];...', ...)`. + std::string replacement; bool hasSecrets() const { @@ -74,6 +80,7 @@ protected: result.are_named = argument_is_named; } chassert(index >= result.start); /// We always check arguments consecutively + chassert(result.replacement.empty()); /// We shouldn't use replacement with masking other arguments result.count = index + 1 - result.start; if (!argument_is_named) result.are_named = false; @@ -199,32 +206,39 @@ protected: void findAzureBlobStorageFunctionSecretArguments(bool is_cluster_function) { - /// azureBlobStorage('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument. + /// azureBlobStorageCluster('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument. size_t url_arg_idx = is_cluster_function ? 1 : 0; if (!is_cluster_function && isNamedCollectionName(0)) { /// azureBlobStorage(named_collection, ..., account_key = 'account_key', ...) + if (maskAzureConnectionString(-1, true, 1)) + return; findSecretNamedArgument("account_key", 1); return; } else if (is_cluster_function && isNamedCollectionName(1)) { /// azureBlobStorageCluster(cluster, named_collection, ..., account_key = 'account_key', ...) + if (maskAzureConnectionString(-1, true, 2)) + return; findSecretNamedArgument("account_key", 2); return; } - /// We should check other arguments first because we don't need to do any replacement in case storage_account_url is not used - /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) - /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) + if (maskAzureConnectionString(url_arg_idx)) + return; + + /// We should check other arguments first because we don't need to do any replacement in case of + /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, format, [account_name, account_key, ...]) + /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, format, [account_name, account_key, ...]) size_t count = function->arguments->size(); if ((url_arg_idx + 4 <= count) && (count <= url_arg_idx + 7)) { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 3, &second_arg)) + String fourth_arg; + if (tryGetStringFromArgument(url_arg_idx + 3, &fourth_arg)) { - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + if (fourth_arg == "auto" || KnownFormatNames::instance().exists(fourth_arg)) return; /// The argument after 'url' is a format: s3('url', 'format', ...) } } @@ -234,6 +248,40 @@ protected: markSecretArgument(url_arg_idx + 4); } + bool maskAzureConnectionString(ssize_t url_arg_idx, bool argument_is_named = false, size_t start = 0) + { + String url_arg; + if (argument_is_named) + { + url_arg_idx = findNamedArgument(&url_arg, "connection_string", start); + if (url_arg_idx == -1 || url_arg.empty()) + url_arg_idx = findNamedArgument(&url_arg, "storage_account_url", start); + if (url_arg_idx == -1 || url_arg.empty()) + return false; + } + else + { + if (!tryGetStringFromArgument(url_arg_idx, &url_arg)) + return false; + } + + if (!url_arg.starts_with("http")) + { + static re2::RE2 account_key_pattern = "AccountKey=.*?(;|$)"; + if (RE2::Replace(&url_arg, account_key_pattern, "AccountKey=[HIDDEN]\\1")) + { + chassert(result.count == 0); /// We shouldn't use replacement with masking other arguments + result.start = url_arg_idx; + result.are_named = argument_is_named; + result.count = 1; + result.replacement = url_arg; + return true; + } + } + + return false; + } + void findURLSecretArguments() { if (!isNamedCollectionName(0)) @@ -513,8 +561,9 @@ protected: return function->arguments->at(arg_idx)->isIdentifier(); } - /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. - void findSecretNamedArgument(const std::string_view & key, size_t start = 0) + /// Looks for an argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. + /// Returns -1 if no argument was found. + ssize_t findNamedArgument(String * res, const std::string_view & key, size_t start = 0) { for (size_t i = start; i < function->arguments->size(); ++i) { @@ -531,8 +580,22 @@ protected: continue; if (found_key == key) - markSecretArgument(i, /* argument_is_named= */ true); + { + tryGetStringFromArgument(*equals_func->arguments->at(1), res); + return i; + } } + + return -1; + } + + /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. + /// If the argument is found, it is marked as a secret. + void findSecretNamedArgument(const std::string_view & key, size_t start = 0) + { + ssize_t arg_idx = findNamedArgument(nullptr, key, start); + if (arg_idx >= 0) + markSecretArgument(arg_idx, /* argument_is_named= */ true); } }; diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index 8121f389a8d..33d6af9c26b 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -223,7 +223,7 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, { account_name = fourth_arg; account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/structure"); if (is_format_arg(sixth_arg)) { format = sixth_arg; @@ -257,10 +257,10 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, } else if (with_structure && engine_args.size() == 8) { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "account_name"); account_name = fourth_arg; account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format"); if (!is_format_arg(sixth_arg)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); format = sixth_arg; diff --git a/tests/integration/test_mask_sensitive_info/test.py b/tests/integration/test_mask_sensitive_info/test.py index 5366de39ea7..797986bbf01 100644 --- a/tests/integration/test_mask_sensitive_info/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -1,5 +1,6 @@ import pytest import random, string +import re from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV @@ -336,6 +337,10 @@ def test_create_database(): def test_table_functions(): password = new_password() azure_conn_string = cluster.env_variables["AZURITE_CONNECTION_STRING"] + account_key_pattern = re.compile("AccountKey=.*?(;|$)") + masked_azure_conn_string = re.sub( + account_key_pattern, "AccountKey=[HIDDEN]\\1", azure_conn_string + ) azure_storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] azure_account_name = "devstoreaccount1" azure_account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" @@ -467,23 +472,23 @@ def test_table_functions(): "CREATE TABLE tablefunc30 (x int) AS s3('http://minio1:9001/root/data/test9.csv.gz', 'NOSIGN', 'CSV')", "CREATE TABLE tablefunc31 (`x` int) AS s3('http://minio1:9001/root/data/test10.csv.gz', 'minio', '[HIDDEN]')", "CREATE TABLE tablefunc32 (`x` int) AS deltaLake('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", - f"CREATE TABLE tablefunc33 (x int) AS azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", - f"CREATE TABLE tablefunc34 (x int) AS azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", - f"CREATE TABLE tablefunc35 (x int) AS azureBlobStorage('{azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc33 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple.csv', 'CSV')", + f"CREATE TABLE tablefunc34 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_1.csv', 'CSV', 'none')", + f"CREATE TABLE tablefunc35 (`x` int) AS azureBlobStorage('{masked_azure_conn_string}', 'cont', 'test_simple_2.csv', 'CSV', 'none', 'auto')", f"CREATE TABLE tablefunc36 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_3.csv', '{azure_account_name}', '[HIDDEN]')", f"CREATE TABLE tablefunc37 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_4.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", f"CREATE TABLE tablefunc38 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_5.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", f"CREATE TABLE tablefunc39 (`x` int) AS azureBlobStorage('{azure_storage_account_url}', 'cont', 'test_simple_6.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc40 (x int) AS azureBlobStorage(named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", + f"CREATE TABLE tablefunc40 (`x` int) AS azureBlobStorage(named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_7.csv', format = 'CSV')", f"CREATE TABLE tablefunc41 (`x` int) AS azureBlobStorage(named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_8.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", - f"CREATE TABLE tablefunc42 (x int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_9.csv', 'CSV')", - f"CREATE TABLE tablefunc43 (x int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_10.csv', 'CSV', 'none')", - f"CREATE TABLE tablefunc44 (x int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_conn_string}', 'cont', 'test_simple_11.csv', 'CSV', 'none', 'auto')", + f"CREATE TABLE tablefunc42 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_9.csv', 'CSV')", + f"CREATE TABLE tablefunc43 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_10.csv', 'CSV', 'none')", + f"CREATE TABLE tablefunc44 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{masked_azure_conn_string}', 'cont', 'test_simple_11.csv', 'CSV', 'none', 'auto')", f"CREATE TABLE tablefunc45 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_12.csv', '{azure_account_name}', '[HIDDEN]')", f"CREATE TABLE tablefunc46 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_13.csv', '{azure_account_name}', '[HIDDEN]', 'CSV')", f"CREATE TABLE tablefunc47 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_14.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none')", f"CREATE TABLE tablefunc48 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', '{azure_storage_account_url}', 'cont', 'test_simple_15.csv', '{azure_account_name}', '[HIDDEN]', 'CSV', 'none', 'auto')", - f"CREATE TABLE tablefunc49 (x int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{azure_conn_string}', container = 'cont', blob_path = 'test_simple_16.csv', format = 'CSV')", + f"CREATE TABLE tablefunc49 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, connection_string = '{masked_azure_conn_string}', container = 'cont', blob_path = 'test_simple_16.csv', format = 'CSV')", f"CREATE TABLE tablefunc50 (`x` int) AS azureBlobStorageCluster('test_shard_localhost', named_collection_2, storage_account_url = '{azure_storage_account_url}', container = 'cont', blob_path = 'test_simple_17.csv', account_name = '{azure_account_name}', account_key = '[HIDDEN]')", "CREATE TABLE tablefunc51 (`x` int) AS iceberg('http://minio1:9001/root/data/test11.csv.gz', 'minio', '[HIDDEN]')", ], From 0fdd04254d1f2a3262729a735762ca4e5b367148 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 18 Sep 2024 18:36:26 +0000 Subject: [PATCH 124/128] Automatic style fix --- tests/integration/test_parallel_replicas_protocol/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_parallel_replicas_protocol/test.py b/tests/integration/test_parallel_replicas_protocol/test.py index 65b2ab00043..c43226c5ecc 100644 --- a/tests/integration/test_parallel_replicas_protocol/test.py +++ b/tests/integration/test_parallel_replicas_protocol/test.py @@ -68,5 +68,7 @@ def test_mark_segment_size_communicated_correctly(start_cluster): ) nodes[0].query("SYSTEM FLUSH LOGS") - log_line = nodes[0].grep_in_log(f"{query_id}.*Reading state is fully initialized") + log_line = nodes[0].grep_in_log( + f"{query_id}.*Reading state is fully initialized" + ) assert re.search(r"mark_segment_size: (\d+)", log_line).group(1) == "16384" From d2de15871ca9379de57cebd8b9142d424dd415fe Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 18 Sep 2024 19:15:13 +0000 Subject: [PATCH 125/128] Include postgres in sparse checkout script --- contrib/sparse-checkout/setup-sparse-checkout.sh | 1 + contrib/sparse-checkout/update-postgres.sh | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100755 contrib/sparse-checkout/update-postgres.sh diff --git a/contrib/sparse-checkout/setup-sparse-checkout.sh b/contrib/sparse-checkout/setup-sparse-checkout.sh index c728a03e35e..2f8b843b1bf 100755 --- a/contrib/sparse-checkout/setup-sparse-checkout.sh +++ b/contrib/sparse-checkout/setup-sparse-checkout.sh @@ -14,5 +14,6 @@ git config submodule."contrib/icu".update '!../sparse-checkout/update-icu.sh' git config submodule."contrib/boost".update '!../sparse-checkout/update-boost.sh' git config submodule."contrib/aws-s2n-tls".update '!../sparse-checkout/update-aws-s2n-tls.sh' git config submodule."contrib/protobuf".update '!../sparse-checkout/update-protobuf.sh' +git config submodule."contrib/postgres".update '!../sparse-checkout/update-postgres.sh' git config submodule."contrib/libxml2".update '!../sparse-checkout/update-libxml2.sh' git config submodule."contrib/brotli".update '!../sparse-checkout/update-brotli.sh' diff --git a/contrib/sparse-checkout/update-postgres.sh b/contrib/sparse-checkout/update-postgres.sh new file mode 100755 index 00000000000..d54d29c27c3 --- /dev/null +++ b/contrib/sparse-checkout/update-postgres.sh @@ -0,0 +1,16 @@ +#!/bin/sh + +echo "Using sparse checkout for postgres" + +FILES_TO_CHECKOUT=$(git rev-parse --git-dir)/info/sparse-checkout +echo '!/*' > $FILES_TO_CHECKOUT +echo '/src/interfaces/libpq/*' >> $FILES_TO_CHECKOUT +echo '!/src/interfaces/libpq/*/*' >> $FILES_TO_CHECKOUT +echo '/src/common/*' >> $FILES_TO_CHECKOUT +echo '!/src/port/*/*' >> $FILES_TO_CHECKOUT +echo '/src/port/*' >> $FILES_TO_CHECKOUT +echo '/src/include/*' >> $FILES_TO_CHECKOUT + +git config core.sparsecheckout true +git checkout $1 +git read-tree -mu HEAD From 228ac44a92c720704aef7d62d990fa3696949def Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 18 Sep 2024 21:27:38 +0200 Subject: [PATCH 126/128] Fix asan issue --- src/Columns/ColumnObject.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index d3af9812b5c..3577ab1ec82 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1356,6 +1356,7 @@ void ColumnObject::takeDynamicStructureFromSourceColumns(const DB::Columns & sou /// Reset current state. dynamic_paths.clear(); dynamic_paths_ptrs.clear(); + sorted_dynamic_paths.clear(); max_dynamic_paths = global_max_dynamic_paths; Statistics new_statistics(Statistics::Source::MERGE); From bb6db8926e52895e32f3c978e05ee45b4b404cfb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 18 Sep 2024 20:48:36 +0000 Subject: [PATCH 127/128] Some fixups --- src/Functions/array/arrayAUC.cpp | 23 ++++++++----------- tests/queries/0_stateless/01064_array_auc.sql | 8 +++++++ 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index 5577a51e198..68cc292e0e7 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -171,9 +171,9 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - const size_t number_of_arguments = arguments.size(); + size_t number_of_arguments = arguments.size(); if (number_of_arguments < 2 || number_of_arguments > 3) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, @@ -182,20 +182,19 @@ public: for (size_t i = 0; i < 2; ++i) { - const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); + const DataTypeArray * array_type = checkAndGetDataType(arguments[i].type.get()); if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The two first arguments for function {} must be an array.", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The two first arguments for function {} must be of type Array.", getName()); const auto & nested_type = array_type->getNestedType(); if (!isNativeNumber(nested_type) && !isEnum(nested_type)) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} cannot process values of type {}", - getName(), nested_type->getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} cannot process values of type {}", getName(), nested_type->getName()); } if (number_of_arguments == 3) { - if (!isBool(arguments[2])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Third argument (scale) for function {} must be a bool.", getName()); + if (!isBool(arguments[2].type) || arguments[2].column.get() == nullptr || !isColumnConst(*arguments[2].column)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Third argument (scale) for function {} must be of type const Bool.", getName()); } return std::make_shared(); @@ -203,7 +202,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - const size_t number_of_arguments = arguments.size(); + size_t number_of_arguments = arguments.size(); ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst(); ColumnPtr col2 = arguments[1].column->convertToFullColumnIfConst(); @@ -223,10 +222,8 @@ public: /// Handle third argument for scale (if passed, otherwise default to true) bool scale = true; - if (number_of_arguments == 3) - { - scale = arguments[2].column->getBool(0); /// Assumes it's a scalar boolean column - } + if (number_of_arguments == 3 && input_rows_count > 0) + scale = arguments[2].column->getBool(0); auto col_res = ColumnVector::create(); diff --git a/tests/queries/0_stateless/01064_array_auc.sql b/tests/queries/0_stateless/01064_array_auc.sql index adc32abcde8..5594b505223 100644 --- a/tests/queries/0_stateless/01064_array_auc.sql +++ b/tests/queries/0_stateless/01064_array_auc.sql @@ -14,6 +14,7 @@ select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]); select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]); select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]); select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]); + select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], true); select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), true); select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), true); @@ -30,6 +31,7 @@ select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], true); select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], true); select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], true); select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], true); + select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], false); select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8)), false); select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8)), false); @@ -46,3 +48,9 @@ select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1], false) select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1], false); select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0], false); select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0], false); + +-- negative tests +select arrayAUC([0, 0, 1, 1]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select arrayAUC([0.1, 0.35], [0, 0, 1, 1]); -- { serverError BAD_ARGUMENTS } +select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], materialize(true)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1], true, true); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 14823f789bcdd4296c2baa23546fa0c244aea532 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 19 Sep 2024 00:56:59 +0000 Subject: [PATCH 128/128] Save CREATE QUERY with KeeperMap engine with evaluated parameters --- src/Storages/StorageKeeperMap.cpp | 4 ++-- .../03236_keeper_map_engine_parameters.reference | 1 + .../0_stateless/03236_keeper_map_engine_parameters.sql | 8 ++++++++ 3 files changed, 11 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03236_keeper_map_engine_parameters.reference create mode 100644 tests/queries/0_stateless/03236_keeper_map_engine_parameters.sql diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index a6be9f8da04..fe64b3058d1 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -1473,8 +1473,8 @@ StoragePtr create(const StorageFactory::Arguments & args) "zk_root_path: path in the Keeper where the values will be stored (required)\n" "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)"); - const auto zk_root_path_node = evaluateConstantExpressionAsLiteral(engine_args[0], args.getLocalContext()); - auto zk_root_path = checkAndGetLiteralArgument(zk_root_path_node, "zk_root_path"); + engine_args[0] = evaluateConstantExpressionAsLiteral(engine_args[0], args.getLocalContext()); + auto zk_root_path = checkAndGetLiteralArgument(engine_args[0], "zk_root_path"); UInt64 keys_limit = 0; if (engine_args.size() > 1) diff --git a/tests/queries/0_stateless/03236_keeper_map_engine_parameters.reference b/tests/queries/0_stateless/03236_keeper_map_engine_parameters.reference new file mode 100644 index 00000000000..ad45f9cb931 --- /dev/null +++ b/tests/queries/0_stateless/03236_keeper_map_engine_parameters.reference @@ -0,0 +1 @@ +CREATE TABLE default.`03236_keeper_map_engine_parameters`\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = KeeperMap(\'/default/test2417\')\nPRIMARY KEY key diff --git a/tests/queries/0_stateless/03236_keeper_map_engine_parameters.sql b/tests/queries/0_stateless/03236_keeper_map_engine_parameters.sql new file mode 100644 index 00000000000..bb60accd712 --- /dev/null +++ b/tests/queries/0_stateless/03236_keeper_map_engine_parameters.sql @@ -0,0 +1,8 @@ +-- Tags: no-ordinary-database, no-fasttest + +DROP TABLE IF EXISTS 03236_keeper_map_engine_parameters; + +CREATE TABLE 03236_keeper_map_engine_parameters (key UInt64, value UInt64) Engine=KeeperMap('/' || currentDatabase() || '/test2417') PRIMARY KEY(key); +SHOW CREATE 03236_keeper_map_engine_parameters; + +DROP TABLE 03236_keeper_map_engine_parameters;