From 3bccce2eae52f5f746600c31f83d89f71af80a67 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 11 Nov 2023 19:37:30 +0100 Subject: [PATCH 01/79] some preparement to support RocksDB as alternative keeper storage --- programs/keeper-converter/KeeperConverter.cpp | 7 +- src/Coordination/KeeperDispatcher.cpp | 30 +- src/Coordination/KeeperDispatcher.h | 10 +- src/Coordination/KeeperServer.cpp | 24 +- src/Coordination/KeeperServer.h | 10 +- src/Coordination/KeeperSnapshotManager.cpp | 101 ++- src/Coordination/KeeperSnapshotManager.h | 36 +- src/Coordination/KeeperStateMachine.cpp | 172 +++-- src/Coordination/KeeperStateMachine.h | 201 ++++-- src/Coordination/KeeperStorage.cpp | 672 ++++++++++-------- src/Coordination/KeeperStorage.h | 174 ++--- src/Coordination/SnapshotableHashTable.h | 1 + src/Coordination/ZooKeeperDataReader.cpp | 33 +- src/Coordination/ZooKeeperDataReader.h | 12 +- src/Coordination/tests/gtest_coordination.cpp | 86 +-- 15 files changed, 900 insertions(+), 669 deletions(-) diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index 20448aafa2f..a426f67e429 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -44,16 +44,17 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) keeper_context->setDigestEnabled(true); keeper_context->setSnapshotDisk(std::make_shared("Keeper-snapshots", options["output-dir"].as())); - DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); + /// TODO(hanfei): support rocksdb here + DB::KeeperMemoryStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); DB::deserializeKeeperStorageFromSnapshotsDir(storage, options["zookeeper-snapshots-dir"].as(), logger); storage.initializeSystemNodes(); DB::deserializeLogsAndApplyToStorage(storage, options["zookeeper-logs-dir"].as(), logger); DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); - DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); + DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); - DB::KeeperSnapshotManager manager(1, keeper_context); + DB::KeeperSnapshotManager manager(1, keeper_context); auto snp = manager.serializeSnapshotToBuffer(snapshot); auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 9e0cdbd6cd3..f97fc448532 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -66,13 +66,13 @@ void KeeperDispatcher::requestThread() RaftAppendResult prev_result = nullptr; /// Requests from previous iteration. We store them to be able /// to send errors to the client. - KeeperStorage::RequestsForSessions prev_batch; + KeeperStorageBase::RequestsForSessions prev_batch; auto & shutdown_called = keeper_context->shutdown_called; while (!shutdown_called) { - KeeperStorage::RequestForSession request; + KeeperStorageBase::RequestForSession request; auto coordination_settings = configuration_and_settings->coordination_settings; uint64_t max_wait = coordination_settings->operation_timeout_ms.totalMilliseconds(); @@ -93,7 +93,7 @@ void KeeperDispatcher::requestThread() if (shutdown_called) break; - KeeperStorage::RequestsForSessions current_batch; + KeeperStorageBase::RequestsForSessions current_batch; size_t current_batch_bytes_size = 0; bool has_read_request = false; @@ -244,7 +244,7 @@ void KeeperDispatcher::responseThread() auto & shutdown_called = keeper_context->shutdown_called; while (!shutdown_called) { - KeeperStorage::ResponseForSession response_for_session; + KeeperStorageBase::ResponseForSession response_for_session; uint64_t max_wait = configuration_and_settings->coordination_settings->operation_timeout_ms.totalMilliseconds(); @@ -340,7 +340,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ return false; } - KeeperStorage::RequestForSession request_info; + KeeperStorageBase::RequestForSession request_info; request_info.request = request; using namespace std::chrono; request_info.time = duration_cast(system_clock::now().time_since_epoch()).count(); @@ -386,7 +386,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf snapshots_queue, keeper_context, snapshot_s3, - [this](uint64_t log_idx, const KeeperStorage::RequestForSession & request_for_session) + [this](uint64_t log_idx, const KeeperStorageBase::RequestForSession & request_for_session) { { /// check if we have queue of read requests depending on this request to be committed @@ -486,7 +486,7 @@ void KeeperDispatcher::shutdown() update_configuration_thread.join(); } - KeeperStorage::RequestForSession request_for_session; + KeeperStorageBase::RequestForSession request_for_session; /// Set session expired for all pending requests while (requests_queue && requests_queue->tryPop(request_for_session)) @@ -497,7 +497,7 @@ void KeeperDispatcher::shutdown() setResponse(request_for_session.session_id, response); } - KeeperStorage::RequestsForSessions close_requests; + KeeperStorageBase::RequestsForSessions close_requests; { /// Clear all registered sessions std::lock_guard lock(session_to_response_callback_mutex); @@ -511,7 +511,7 @@ void KeeperDispatcher::shutdown() auto request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close); request->xid = Coordination::CLOSE_XID; using namespace std::chrono; - KeeperStorage::RequestForSession request_info + KeeperStorageBase::RequestForSession request_info { .session_id = session, .time = duration_cast(system_clock::now().time_since_epoch()).count(), @@ -609,7 +609,7 @@ void KeeperDispatcher::sessionCleanerTask() auto request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close); request->xid = Coordination::CLOSE_XID; using namespace std::chrono; - KeeperStorage::RequestForSession request_info + KeeperStorageBase::RequestForSession request_info { .session_id = dead_session, .time = duration_cast(system_clock::now().time_since_epoch()).count(), @@ -657,16 +657,16 @@ void KeeperDispatcher::finishSession(int64_t session_id) } } -void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error) +void KeeperDispatcher::addErrorResponses(const KeeperStorageBase::RequestsForSessions & requests_for_sessions, Coordination::Error error) { for (const auto & request_for_session : requests_for_sessions) { - KeeperStorage::ResponsesForSessions responses; + KeeperStorageBase::ResponsesForSessions responses; auto response = request_for_session.request->makeResponse(); response->xid = request_for_session.request->xid; response->zxid = 0; response->error = error; - if (!responses_queue.push(DB::KeeperStorage::ResponseForSession{request_for_session.session_id, response})) + if (!responses_queue.push(DB::KeeperStorageBase::ResponseForSession{request_for_session.session_id, response})) throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push error response xid {} zxid {} error message {} to responses queue", response->xid, @@ -675,7 +675,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession } } -nuraft::ptr KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions) +nuraft::ptr KeeperDispatcher::forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions) { if (!result->has_result()) result->get(); @@ -697,7 +697,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) { /// New session id allocation is a special request, because we cannot process it in normal /// way: get request -> put to raft -> set response for registered callback. - KeeperStorage::RequestForSession request_info; + KeeperStorageBase::RequestForSession request_info; std::shared_ptr request = std::make_shared(); /// Internal session id. It's a temporary number which is unique for each client on this server /// but can be same on different servers. diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 39941f55d5e..f8dc8503380 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -27,7 +27,7 @@ using ZooKeeperResponseCallback = std::function; + using RequestsQueue = ConcurrentBoundedQueue; using SessionToResponseCallback = std::unordered_map; using ClusterUpdateQueue = ConcurrentBoundedQueue; @@ -96,11 +96,11 @@ private: /// Add error responses for requests to responses queue. /// Clears requests. - void addErrorResponses(const KeeperStorage::RequestsForSessions & requests_for_sessions, Coordination::Error error); + void addErrorResponses(const KeeperStorageBase::RequestsForSessions & requests_for_sessions, Coordination::Error error); /// Forcefully wait for result and sets errors if something when wrong. /// Clears both arguments - nuraft::ptr forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorage::RequestsForSessions & requests_for_sessions); + nuraft::ptr forceWaitAndProcessResult(RaftAppendResult & result, KeeperStorageBase::RequestsForSessions & requests_for_sessions); public: std::mutex read_request_queue_mutex; @@ -108,7 +108,7 @@ public: std::atomic our_last_committed_log_idx = 0; /// queue of read requests that can be processed after a request with specific session ID and XID is committed - std::unordered_map> read_request_queue; + std::unordered_map> read_request_queue; /// Just allocate some objects, real initialization is done by `intialize method` KeeperDispatcher(); @@ -189,7 +189,7 @@ public: Keeper4LWInfo getKeeper4LWInfo() const; - const KeeperStateMachine & getStateMachine() const + const IKeeperStateMachine & getStateMachine() const { return *server->getKeeperStateMachine(); } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index fd3db6f1032..b25d5e85431 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -116,7 +116,7 @@ KeeperServer::KeeperServer( SnapshotsQueue & snapshots_queue_, KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, - KeeperStateMachine::CommitCallback commit_callback) + IKeeperStateMachine::CommitCallback commit_callback) : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) @@ -128,7 +128,7 @@ KeeperServer::KeeperServer( if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - state_machine = nuraft::cs_new( + state_machine = nuraft::cs_new>( responses_queue_, snapshots_queue_, coordination_settings, @@ -478,7 +478,7 @@ namespace { // Serialize the request for the log entry -nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestForSession & request_for_session) +nuraft::ptr getZooKeeperLogEntry(const KeeperStorageBase::RequestForSession & request_for_session) { DB::WriteBufferFromNuraftBuffer write_buf; DB::writeIntBinary(request_for_session.session_id, write_buf); @@ -486,7 +486,7 @@ nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestFor DB::writeIntBinary(request_for_session.time, write_buf); /// we fill with dummy values to eliminate unnecessary copy later on when we will write correct values DB::writeIntBinary(static_cast(0), write_buf); /// zxid - DB::writeIntBinary(KeeperStorage::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag + DB::writeIntBinary(KeeperStorageBase::DigestVersion::NO_DIGEST, write_buf); /// digest version or NO_DIGEST flag DB::writeIntBinary(static_cast(0), write_buf); /// digest value /// if new fields are added, update KeeperStateMachine::ZooKeeperLogSerializationVersion along with parseRequest function and PreAppendLog callback handler return write_buf.getBuffer(); @@ -494,7 +494,7 @@ nuraft::ptr getZooKeeperLogEntry(const KeeperStorage::RequestFor } -void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & request_for_session) +void KeeperServer::putLocalReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) { if (!request_for_session.request->isReadRequest()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot process non-read request locally"); @@ -502,7 +502,7 @@ void KeeperServer::putLocalReadRequest(const KeeperStorage::RequestForSession & state_machine->processReadRequest(request_for_session); } -RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorage::RequestsForSessions & requests_for_sessions) +RaftAppendResult KeeperServer::putRequestBatch(const KeeperStorageBase::RequestsForSessions & requests_for_sessions) { std::vector> entries; for (const auto & request_for_session : requests_for_sessions) @@ -725,7 +725,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ auto entry_buf = entry->get_buf_ptr(); - KeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version; + IKeeperStateMachine::ZooKeeperLogSerializationVersion serialization_version; auto request_for_session = state_machine->parseRequest(*entry_buf, /*final=*/false, &serialization_version); request_for_session->zxid = next_zxid; if (!state_machine->preprocess(*request_for_session)) @@ -735,10 +735,10 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ /// older versions of Keeper can send logs that are missing some fields size_t bytes_missing = 0; - if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME) + if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME) bytes_missing += sizeof(request_for_session->time); - if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_ZXID_DIGEST) + if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_ZXID_DIGEST) bytes_missing += sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value); if (bytes_missing != 0) @@ -752,19 +752,19 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ size_t write_buffer_header_size = sizeof(request_for_session->zxid) + sizeof(request_for_session->digest->version) + sizeof(request_for_session->digest->value); - if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME) + if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME) write_buffer_header_size += sizeof(request_for_session->time); auto * buffer_start = reinterpret_cast(entry_buf->data_begin() + entry_buf->size() - write_buffer_header_size); WriteBufferFromPointer write_buf(buffer_start, write_buffer_header_size); - if (serialization_version < KeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME) + if (serialization_version < IKeeperStateMachine::ZooKeeperLogSerializationVersion::WITH_TIME) writeIntBinary(request_for_session->time, write_buf); writeIntBinary(request_for_session->zxid, write_buf); writeIntBinary(request_for_session->digest->version, write_buf); - if (request_for_session->digest->version != KeeperStorage::NO_DIGEST) + if (request_for_session->digest->version != KeeperStorageBase::NO_DIGEST) writeIntBinary(request_for_session->digest->value, write_buf); write_buf.finalize(); diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index ed58418fe5f..402dcb38742 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -24,7 +24,7 @@ private: CoordinationSettingsPtr coordination_settings; - nuraft::ptr state_machine; + nuraft::ptr state_machine; nuraft::ptr state_manager; @@ -79,26 +79,26 @@ public: SnapshotsQueue & snapshots_queue_, KeeperContextPtr keeper_context_, KeeperSnapshotManagerS3 & snapshot_manager_s3, - KeeperStateMachine::CommitCallback commit_callback); + IKeeperStateMachine::CommitCallback commit_callback); /// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings. void startup(const Poco::Util::AbstractConfiguration & config, bool enable_ipv6 = true); /// Put local read request and execute in state machine directly and response into /// responses queue - void putLocalReadRequest(const KeeperStorage::RequestForSession & request); + void putLocalReadRequest(const KeeperStorageBase::RequestForSession & request); bool isRecovering() const { return is_recovering; } bool reconfigEnabled() const { return enable_reconfiguration; } /// Put batch of requests into Raft and get result of put. Responses will be set separately into /// responses_queue. - RaftAppendResult putRequestBatch(const KeeperStorage::RequestsForSessions & requests); + RaftAppendResult putRequestBatch(const KeeperStorageBase::RequestsForSessions & requests); /// Return set of the non-active sessions std::vector getDeadSessions(); - nuraft::ptr getKeeperStateMachine() const { return state_machine; } + nuraft::ptr getKeeperStateMachine() const { return state_machine; } void forceRecovery(); diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a216272a9e1..739a31d89e7 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -68,7 +68,8 @@ namespace return base; } - void writeNode(const KeeperStorage::Node & node, SnapshotVersion version, WriteBuffer & out) + template + void writeNode(const Node & node, SnapshotVersion version, WriteBuffer & out) { writeBinary(node.getData(), out); @@ -96,7 +97,8 @@ namespace } } - void readNode(KeeperStorage::Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map) + template + void readNode(Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map) { String new_data; readBinary(new_data, in); @@ -169,7 +171,8 @@ namespace } } -void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context) +template +void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context) { writeBinary(static_cast(snapshot.version), out); serializeSnapshotMetadata(snapshot.snapshot_meta, out); @@ -179,11 +182,11 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr writeBinary(snapshot.zxid, out); if (keeper_context->digestEnabled()) { - writeBinary(static_cast(KeeperStorage::CURRENT_DIGEST_VERSION), out); + writeBinary(static_cast(Storage::CURRENT_DIGEST_VERSION), out); writeBinary(snapshot.nodes_digest, out); } else - writeBinary(static_cast(KeeperStorage::NO_DIGEST), out); + writeBinary(static_cast(Storage::NO_DIGEST), out); } writeBinary(snapshot.session_id, out); @@ -256,7 +259,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr writeBinary(session_id, out); writeBinary(timeout, out); - KeeperStorage::AuthIDs ids; + KeeperStorageBase::AuthIDs ids; if (snapshot.session_and_auth.contains(session_id)) ids = snapshot.session_and_auth.at(session_id); @@ -277,7 +280,8 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr } } -void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context) +template +void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context) { uint8_t version; readBinary(version, in); @@ -286,7 +290,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported snapshot version {}", version); deserialization_result.snapshot_meta = deserializeSnapshotMetadata(in); - KeeperStorage & storage = *deserialization_result.storage; + Storage & storage = *deserialization_result.storage; bool recalculate_digest = keeper_context->digestEnabled(); if (version >= SnapshotVersion::V5) @@ -294,11 +298,11 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial readBinary(storage.zxid, in); uint8_t digest_version; readBinary(digest_version, in); - if (digest_version != KeeperStorage::DigestVersion::NO_DIGEST) + if (digest_version != Storage::DigestVersion::NO_DIGEST) { uint64_t nodes_digest; readBinary(nodes_digest, in); - if (digest_version == KeeperStorage::CURRENT_DIGEST_VERSION) + if (digest_version == Storage::CURRENT_DIGEST_VERSION) { storage.nodes_digest = nodes_digest; recalculate_digest = false; @@ -361,7 +365,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { std::string path; readBinary(path, in); - KeeperStorage::Node node{}; + typename Storage::Node node{}; readNode(node, in, current_version, storage.acl_map); using enum Coordination::PathMatchResult; @@ -389,7 +393,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT) { LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg); - node = KeeperStorage::Node{}; + node = typename Storage::Node{}; } else throw Exception( @@ -417,7 +421,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { auto parent_path = parentNodePath(itr.key); storage.container.updateValue( - parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); }); + parent_path, [version, path = itr.key](typename Storage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); }); } } @@ -457,14 +461,14 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial size_t session_auths_size; readBinary(session_auths_size, in); - KeeperStorage::AuthIDs ids; + typename Storage::AuthIDs ids; size_t session_auth_counter = 0; while (session_auth_counter < session_auths_size) { String scheme, id; readBinary(scheme, in); readBinary(id, in); - ids.emplace_back(KeeperStorage::AuthID{scheme, id}); + ids.emplace_back(typename Storage::AuthID{scheme, id}); session_auth_counter++; } @@ -487,7 +491,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial } } -KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_) +template +KeeperStorageSnapshot::KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_) : storage(storage_) , snapshot_meta(std::make_shared(up_to_log_idx_, 0, std::make_shared())) , session_id(storage->session_id_counter) @@ -504,8 +509,9 @@ KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t session_and_auth = storage->session_and_auth; } -KeeperStorageSnapshot::KeeperStorageSnapshot( - KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_) +template +KeeperStorageSnapshot::KeeperStorageSnapshot( + Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_) : storage(storage_) , snapshot_meta(snapshot_meta_) , session_id(storage->session_id_counter) @@ -522,12 +528,14 @@ KeeperStorageSnapshot::KeeperStorageSnapshot( session_and_auth = storage->session_and_auth; } -KeeperStorageSnapshot::~KeeperStorageSnapshot() +template +KeeperStorageSnapshot::~KeeperStorageSnapshot() { storage->disableSnapshotMode(); } -KeeperSnapshotManager::KeeperSnapshotManager( +template +KeeperSnapshotManager::KeeperSnapshotManager( size_t snapshots_to_keep_, const KeeperContextPtr & keeper_context_, bool compress_snapshots_zstd_, @@ -606,7 +614,8 @@ KeeperSnapshotManager::KeeperSnapshotManager( moveSnapshotsIfNeeded(); } -SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) +template +SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx) { ReadBufferFromNuraftBuffer reader(buffer); @@ -634,7 +643,8 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::bu return {snapshot_file_name, disk}; } -nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk() +template +nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk() { while (!existing_snapshots.empty()) { @@ -655,7 +665,8 @@ nuraft::ptr KeeperSnapshotManager::deserializeLatestSnapshotBuff return nullptr; } -nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const +template +nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const { const auto & [snapshot_path, snapshot_disk] = existing_snapshots.at(up_to_log_idx); WriteBufferFromNuraftBuffer writer; @@ -664,7 +675,8 @@ nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFrom return writer.getBuffer(); } -nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const +template +nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const { std::unique_ptr writer = std::make_unique(); auto * buffer_raw_ptr = writer.get(); @@ -674,13 +686,13 @@ nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(con else compressed_writer = std::make_unique(*writer); - KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context); + KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context); compressed_writer->finalize(); return buffer_raw_ptr->getBuffer(); } - -bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) +template +bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) { static constexpr unsigned char ZSTD_COMPRESSED_MAGIC[4] = {0x28, 0xB5, 0x2F, 0xFD}; @@ -691,7 +703,8 @@ bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) return memcmp(magic_from_buffer, ZSTD_COMPRESSED_MAGIC, 4) == 0; } -SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr buffer) const +template +SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr buffer) const { bool is_zstd_compressed = isZstdCompressed(buffer); @@ -703,14 +716,15 @@ SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuff else compressed_reader = std::make_unique(*reader); - SnapshotDeserializationResult result; - result.storage = std::make_unique(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false); - KeeperStorageSnapshot::deserialize(result, *compressed_reader, keeper_context); + SnapshotDeserializationResult result; + result.storage = std::make_unique(storage_tick_time, superdigest, keeper_context, /* initialize_system_nodes */ false); + KeeperStorageSnapshot::deserialize(result, *compressed_reader, keeper_context); result.storage->initializeSystemNodes(); return result; } -SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() +template +SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() { if (existing_snapshots.empty()) return {}; @@ -721,23 +735,27 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() return deserializeSnapshotFromBuffer(buffer); } -DiskPtr KeeperSnapshotManager::getDisk() const +template +DiskPtr KeeperSnapshotManager::getDisk() const { return keeper_context->getSnapshotDisk(); } -DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const +template +DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const { return keeper_context->getLatestSnapshotDisk(); } -void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() +template +void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded() { while (existing_snapshots.size() > snapshots_to_keep) removeSnapshot(existing_snapshots.begin()->first); } -void KeeperSnapshotManager::moveSnapshotsIfNeeded() +template +void KeeperSnapshotManager::moveSnapshotsIfNeeded() { /// move snapshots to correct disks @@ -767,7 +785,8 @@ void KeeperSnapshotManager::moveSnapshotsIfNeeded() } -void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) +template +void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) { auto itr = existing_snapshots.find(log_idx); if (itr == existing_snapshots.end()) @@ -777,7 +796,8 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) existing_snapshots.erase(itr); } -SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot) +template +SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot) { auto up_to_log_idx = snapshot.snapshot_meta->get_last_log_idx(); auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); @@ -796,7 +816,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor else compressed_writer = std::make_unique(*writer); - KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context); + KeeperStorageSnapshot::serialize(snapshot, *compressed_writer, keeper_context); compressed_writer->finalize(); compressed_writer->sync(); @@ -809,4 +829,7 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStor return {snapshot_file_name, disk}; } +template struct KeeperStorageSnapshot; +template class KeeperSnapshotManager; + } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 9bb287b9276..b0a9425881f 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -29,16 +29,19 @@ enum SnapshotVersion : uint8_t static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V5; /// What is stored in binary snapshot +template struct SnapshotDeserializationResult { /// Storage - KeeperStoragePtr storage; + std::unique_ptr storage; /// Snapshot metadata (up_to_log_idx and so on) SnapshotMetadataPtr snapshot_meta; /// Cluster config ClusterConfigPtr cluster_config; }; +struct IKeeperStorageSnapshot {}; + /// In memory keeper snapshot. Keeper Storage based on a hash map which can be /// turned into snapshot mode. This operation is fast and KeeperStorageSnapshot /// class do it in constructor. It also copies iterators from storage hash table @@ -47,21 +50,22 @@ struct SnapshotDeserializationResult /// /// This representation of snapshot have to be serialized into NuRaft /// buffer and send over network or saved to file. -struct KeeperStorageSnapshot +template +struct KeeperStorageSnapshot : IKeeperStorageSnapshot { public: - KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr); + KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr); KeeperStorageSnapshot( - KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr); + Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr); ~KeeperStorageSnapshot(); - static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context); + static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context); - static void deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context); + static void deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in, KeeperContextPtr keeper_context); - KeeperStorage * storage; + Storage * storage; SnapshotVersion version = CURRENT_SNAPSHOT_VERSION; /// Snapshot metadata @@ -72,11 +76,11 @@ public: /// so we have for loop for (i = 0; i < snapshot_container_size; ++i) { doSmth(begin + i); } size_t snapshot_container_size; /// Iterator to the start of the storage - KeeperStorage::Container::const_iterator begin; + Storage::Container::const_iterator begin; /// Active sessions and their timeouts SessionAndTimeout session_and_timeout; /// Sessions credentials - KeeperStorage::SessionAndAuth session_and_auth; + Storage::SessionAndAuth session_and_auth; /// ACLs cache for better performance. Without we cannot deserialize storage. std::unordered_map acl_map; /// Cluster config from snapshot, can be empty @@ -93,14 +97,12 @@ struct SnapshotFileInfo DiskPtr disk; }; -using KeeperStorageSnapshotPtr = std::shared_ptr; +using KeeperStorageSnapshotPtr = std::shared_ptr; using CreateSnapshotCallback = std::function; - -using SnapshotMetaAndStorage = std::pair; - /// Class responsible for snapshots serialization and deserialization. Each snapshot /// has it's path on disk and log index. +template class KeeperSnapshotManager { public: @@ -112,18 +114,18 @@ public: size_t storage_tick_time_ = 500); /// Restore storage from latest available snapshot - SnapshotDeserializationResult restoreFromLatestSnapshot(); + SnapshotDeserializationResult restoreFromLatestSnapshot(); /// Compress snapshot and serialize it to buffer - nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const; + nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const; /// Serialize already compressed snapshot to disk (return path) SnapshotFileInfo serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); /// Serialize snapshot directly to disk - SnapshotFileInfo serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot); + SnapshotFileInfo serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot); - SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; + SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; /// Deserialize snapshot with log index up_to_log_idx from disk into compressed nuraft buffer. nuraft::ptr deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index b6185fa2e36..a3c9378a173 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -40,7 +40,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -KeeperStateMachine::KeeperStateMachine( +IKeeperStateMachine::IKeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const CoordinationSettingsPtr & coordination_settings_, @@ -50,12 +50,6 @@ KeeperStateMachine::KeeperStateMachine( const std::string & superdigest_) : commit_callback(commit_callback_) , coordination_settings(coordination_settings_) - , snapshot_manager( - coordination_settings->snapshots_to_keep, - keeper_context_, - coordination_settings->compress_snapshots_with_zstd_format, - superdigest_, - coordination_settings->dead_session_check_period_ms.totalMilliseconds()) , responses_queue(responses_queue_) , snapshots_queue(snapshots_queue_) , min_request_size_to_cache(coordination_settings_->min_request_size_for_cache) @@ -67,6 +61,32 @@ KeeperStateMachine::KeeperStateMachine( { } +template +KeeperStateMachine::KeeperStateMachine( + ResponsesQueue & responses_queue_, + SnapshotsQueue & snapshots_queue_, + const CoordinationSettingsPtr & coordination_settings_, + const KeeperContextPtr & keeper_context_, + KeeperSnapshotManagerS3 * snapshot_manager_s3_, + IKeeperStateMachine::CommitCallback commit_callback_, + const std::string & superdigest_) + : IKeeperStateMachine( + responses_queue_, + snapshots_queue_, + coordination_settings_, + keeper_context_, + snapshot_manager_s3_, + commit_callback_, + superdigest_), + snapshot_manager( + coordination_settings->snapshots_to_keep, + keeper_context_, + coordination_settings->compress_snapshots_with_zstd_format, + superdigest_, + coordination_settings->dead_session_check_period_ms.totalMilliseconds()) +{ +} + namespace { @@ -77,7 +97,8 @@ bool isLocalDisk(const IDisk & disk) } -void KeeperStateMachine::init() +template +void KeeperStateMachine::init() { /// Do everything without mutexes, no other threads exist. LOG_DEBUG(log, "Totally have {} snapshots", snapshot_manager.totalSnapshots()); @@ -129,7 +150,7 @@ void KeeperStateMachine::init() } if (!storage) - storage = std::make_unique( + storage = std::make_unique( coordination_settings->dead_session_check_period_ms.totalMilliseconds(), superdigest, keeper_context); } @@ -137,12 +158,12 @@ namespace { void assertDigest( - const KeeperStorage::Digest & first, - const KeeperStorage::Digest & second, + const KeeperStorageBase::Digest & first, + const KeeperStorageBase::Digest & second, const Coordination::ZooKeeperRequest & request, bool committing) { - if (!KeeperStorage::checkDigest(first, second)) + if (!KeeperStorageBase::checkDigest(first, second)) { LOG_FATAL( &Poco::Logger::get("KeeperStateMachine"), @@ -160,7 +181,8 @@ void assertDigest( } -nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data) +template +nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data, /*final=*/false); if (!request_for_session->zxid) @@ -175,10 +197,10 @@ nuraft::ptr KeeperStateMachine::pre_commit(uint64_t log_idx, nur return result; } -std::shared_ptr KeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version) +std::shared_ptr IKeeperStateMachine::parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version) { ReadBufferFromNuraftBuffer buffer(data); - auto request_for_session = std::make_shared(); + auto request_for_session = std::make_shared(); readIntBinary(request_for_session->session_id, buffer); int32_t length; @@ -251,7 +273,7 @@ std::shared_ptr KeeperStateMachine::parseReque request_for_session->digest.emplace(); readIntBinary(request_for_session->digest->version, buffer); - if (request_for_session->digest->version != KeeperStorage::DigestVersion::NO_DIGEST || !buffer.eof()) + if (request_for_session->digest->version != KeeperStorageBase::DigestVersion::NO_DIGEST || !buffer.eof()) readIntBinary(request_for_session->digest->value, buffer); } @@ -267,7 +289,8 @@ std::shared_ptr KeeperStateMachine::parseReque return request_for_session; } -bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session) +template +bool KeeperStateMachine::preprocess(const KeeperStorageBase::RequestForSession & request_for_session) { const auto op_num = request_for_session.request->getOpNum(); if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig) @@ -301,10 +324,11 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req return true; } -void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session) +template +void KeeperStateMachine::reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) { std::lock_guard _(storage_and_responses_lock); - KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session); + KeeperStorageBase::ResponseForSession response = processReconfiguration(request_for_session); if (!responses_queue.push(response)) { ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed); @@ -314,8 +338,9 @@ void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& req } } -KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( - const KeeperStorage::RequestForSession & request_for_session) +template +KeeperStorageBase::ResponseForSession KeeperStateMachine::processReconfiguration( + const KeeperStorageBase::RequestForSession & request_for_session) { ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest); @@ -324,7 +349,7 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( const int64_t zxid = request_for_session.zxid; using enum Coordination::Error; - auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorage::ResponseForSession + auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorageBase::ResponseForSession { auto res = std::make_shared(); res->xid = request.xid; @@ -381,7 +406,8 @@ KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration( return { session_id, std::move(response) }; } -nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) +template +nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data) { auto request_for_session = parseRequest(data, true); if (!request_for_session->zxid) @@ -392,7 +418,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n if (!keeper_context->local_logs_preprocessed) preprocess(*request_for_session); - auto try_push = [this](const KeeperStorage::ResponseForSession& response) + auto try_push = [this](const KeeperStorageBase::ResponseForSession& response) { if (!responses_queue.push(response)) { @@ -412,7 +438,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n std::shared_ptr response = std::make_shared(); response->internal_id = session_id_request.internal_id; response->server_id = session_id_request.server_id; - KeeperStorage::ResponseForSession response_for_session; + KeeperStorageBase::ResponseForSession response_for_session; response_for_session.session_id = -1; response_for_session.response = response; @@ -431,7 +457,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n } std::lock_guard lock(storage_and_responses_lock); - KeeperStorage::ResponsesForSessions responses_for_sessions + KeeperStorageBase::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid); for (auto & response_for_session : responses_for_sessions) try_push(response_for_session); @@ -448,7 +474,8 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n return nullptr; } -bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) +template +bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx()); nuraft::ptr latest_snapshot_ptr; @@ -475,7 +502,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { /// deserialize and apply snapshot to storage std::lock_guard lock(storage_and_responses_lock); - SnapshotDeserializationResult snapshot_deserialization_result; + SnapshotDeserializationResult snapshot_deserialization_result; if (latest_snapshot_ptr) snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr); else @@ -496,7 +523,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) } -void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr & new_conf) +void IKeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr & new_conf) { std::lock_guard lock(cluster_config_lock); auto tmp = new_conf->serialize(); @@ -504,7 +531,7 @@ void KeeperStateMachine::commit_config(const uint64_t log_idx, nuraft::ptr +void KeeperStateMachine::rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) { if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; @@ -525,7 +553,8 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession storage->rollbackRequest(request_for_session.zxid, allow_missing); } -void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing) +template +void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) { if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) return; @@ -533,14 +562,15 @@ void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSe storage->rollbackRequest(request_for_session.zxid, allow_missing); } -nuraft::ptr KeeperStateMachine::last_snapshot() +nuraft::ptr IKeeperStateMachine::last_snapshot() { /// Just return the latest snapshot. std::lock_guard lock(snapshots_lock); return latest_snapshot_meta; } -void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_result::handler_type & when_done) +template +void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_result::handler_type & when_done) { LOG_DEBUG(log, "Creating snapshot {}", s.get_last_log_idx()); @@ -549,12 +579,13 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res CreateSnapshotTask snapshot_task; { /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking. std::lock_guard lock(storage_and_responses_lock); - snapshot_task.snapshot = std::make_shared(storage.get(), snapshot_meta_copy, getClusterConfig()); + snapshot_task.snapshot = std::make_shared>(storage.get(), snapshot_meta_copy, getClusterConfig()); } /// create snapshot task for background execution (in snapshot thread) - snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot) + snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot_) { + auto * snapshot = typeid_cast *>(snapshot_.get()); nuraft::ptr exception(nullptr); bool ret = true; try @@ -601,7 +632,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res /// Turn off "snapshot mode" and clear outdate part of storage state storage->clearGarbageAfterSnapshot(); LOG_TRACE(log, "Cleared garbage after snapshot"); - snapshot.reset(); + snapshot_.reset(); } } catch (...) @@ -638,7 +669,8 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res LOG_WARNING(log, "Cannot push snapshot task into queue"); } -void KeeperStateMachine::save_logical_snp_obj( +template +void KeeperStateMachine::save_logical_snp_obj( nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool /*is_first_obj*/, bool /*is_last_obj*/) { LOG_DEBUG(log, "Saving snapshot {} obj_id {}", s.get_last_log_idx(), obj_id); @@ -703,7 +735,7 @@ static int bufferFromFile(Poco::Logger * log, const std::string & path, nuraft:: return 0; } -int KeeperStateMachine::read_logical_snp_obj( +int IKeeperStateMachine::read_logical_snp_obj( nuraft::snapshot & s, void *& /*user_snp_ctx*/, uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) { LOG_DEBUG(log, "Reading snapshot {} obj_id {}", s.get_last_log_idx(), obj_id); @@ -743,7 +775,8 @@ int KeeperStateMachine::read_logical_snp_obj( return 1; } -void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session) +template +void KeeperStateMachine::processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) { /// Pure local request, just process it with storage std::lock_guard lock(storage_and_responses_lock); @@ -754,103 +787,120 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", response.session_id); } -void KeeperStateMachine::shutdownStorage() +template +void KeeperStateMachine::shutdownStorage() { std::lock_guard lock(storage_and_responses_lock); storage->finalize(); } -std::vector KeeperStateMachine::getDeadSessions() +template +std::vector KeeperStateMachine::getDeadSessions() { std::lock_guard lock(storage_and_responses_lock); return storage->getDeadSessions(); } -int64_t KeeperStateMachine::getNextZxid() const +template +int64_t KeeperStateMachine::getNextZxid() const { std::lock_guard lock(storage_and_responses_lock); return storage->getNextZXID(); } -KeeperStorage::Digest KeeperStateMachine::getNodesDigest() const +template +KeeperStorageBase::Digest KeeperStateMachine::getNodesDigest() const { std::lock_guard lock(storage_and_responses_lock); return storage->getNodesDigest(false); } -uint64_t KeeperStateMachine::getLastProcessedZxid() const +template +uint64_t KeeperStateMachine::getLastProcessedZxid() const { std::lock_guard lock(storage_and_responses_lock); return storage->getZXID(); } -uint64_t KeeperStateMachine::getNodesCount() const +template +uint64_t KeeperStateMachine::getNodesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getNodesCount(); } -uint64_t KeeperStateMachine::getTotalWatchesCount() const +template +uint64_t KeeperStateMachine::getTotalWatchesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getTotalWatchesCount(); } -uint64_t KeeperStateMachine::getWatchedPathsCount() const +template +uint64_t KeeperStateMachine::getWatchedPathsCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getWatchedPathsCount(); } -uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const +template +uint64_t KeeperStateMachine::getSessionsWithWatchesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getSessionsWithWatchesCount(); } -uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const +template +uint64_t KeeperStateMachine::getTotalEphemeralNodesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getTotalEphemeralNodesCount(); } -uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const +template +uint64_t KeeperStateMachine::getSessionWithEphemeralNodesCount() const { std::lock_guard lock(storage_and_responses_lock); return storage->getSessionWithEphemeralNodesCount(); } -void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const +template +void KeeperStateMachine::dumpWatches(WriteBufferFromOwnString & buf) const { std::lock_guard lock(storage_and_responses_lock); storage->dumpWatches(buf); } -void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const +template +void KeeperStateMachine::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { std::lock_guard lock(storage_and_responses_lock); storage->dumpWatchesByPath(buf); } -void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const +template +void KeeperStateMachine::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { std::lock_guard lock(storage_and_responses_lock); storage->dumpSessionsAndEphemerals(buf); } -uint64_t KeeperStateMachine::getApproximateDataSize() const +template +uint64_t KeeperStateMachine::getApproximateDataSize() const { std::lock_guard lock(storage_and_responses_lock); return storage->getApproximateDataSize(); } -uint64_t KeeperStateMachine::getKeyArenaSize() const +template +uint64_t KeeperStateMachine::getKeyArenaSize() const { std::lock_guard lock(storage_and_responses_lock); return storage->getArenaDataSize(); } -uint64_t KeeperStateMachine::getLatestSnapshotBufSize() const +template +uint64_t KeeperStateMachine::getLatestSnapshotBufSize() const { std::lock_guard lock(snapshots_lock); if (latest_snapshot_buf) @@ -858,7 +908,7 @@ uint64_t KeeperStateMachine::getLatestSnapshotBufSize() const return 0; } -ClusterConfigPtr KeeperStateMachine::getClusterConfig() const +ClusterConfigPtr IKeeperStateMachine::getClusterConfig() const { std::lock_guard lock(cluster_config_lock); if (cluster_config) @@ -870,11 +920,15 @@ ClusterConfigPtr KeeperStateMachine::getClusterConfig() const return nullptr; } -void KeeperStateMachine::recalculateStorageStats() +template +void KeeperStateMachine::recalculateStorageStats() { std::lock_guard lock(storage_and_responses_lock); LOG_INFO(log, "Recalculating storage stats"); storage->recalculateStats(); LOG_INFO(log, "Done recalculating storage stats"); } + +template class KeeperStateMachine; + } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index aad5d3aafd4..9ae3905cddd 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -12,27 +12,25 @@ namespace DB { -using ResponsesQueue = ConcurrentBoundedQueue; +using ResponsesQueue = ConcurrentBoundedQueue; using SnapshotsQueue = ConcurrentBoundedQueue; -/// ClickHouse Keeper state machine. Wrapper for KeeperStorage. -/// Responsible for entries commit, snapshots creation and so on. -class KeeperStateMachine : public nuraft::state_machine +class IKeeperStateMachine : public nuraft::state_machine { public: - using CommitCallback = std::function; + using CommitCallback = std::function; - KeeperStateMachine( + IKeeperStateMachine( ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const CoordinationSettingsPtr & coordination_settings_, const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, - CommitCallback commit_callback_ = {}, - const std::string & superdigest_ = ""); + CommitCallback commit_callback_, + const std::string & superdisgest_); /// Read state from the latest snapshot - void init(); + virtual void init() = 0; enum ZooKeeperLogSerializationVersion { @@ -49,89 +47,66 @@ public: /// /// final - whether it's the final time we will fetch the request so we can safely remove it from cache /// serialization_version - information about which fields were parsed from the buffer so we can modify the buffer accordingly - std::shared_ptr parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr); + std::shared_ptr parseRequest(nuraft::buffer & data, bool final, ZooKeeperLogSerializationVersion * serialization_version = nullptr); - bool preprocess(const KeeperStorage::RequestForSession & request_for_session); + virtual bool preprocess(const KeeperStorageBase::RequestForSession & request_for_session) = 0; - nuraft::ptr pre_commit(uint64_t log_idx, nuraft::buffer & data) override; - - nuraft::ptr commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT - - /// Save new cluster config to our snapshot (copy of the config stored in StateManager) void commit_config(const uint64_t log_idx, nuraft::ptr & new_conf) override; /// NOLINT void rollback(uint64_t log_idx, nuraft::buffer & data) override; // allow_missing - whether the transaction we want to rollback can be missing from storage // (can happen in case of exception during preprocessing) - void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); - - void rollbackRequestNoLock( - const KeeperStorage::RequestForSession & request_for_session, - bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS; + virtual void rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) = 0; uint64_t last_commit_index() override { return last_committed_idx; } - /// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state. - bool apply_snapshot(nuraft::snapshot & s) override; - nuraft::ptr last_snapshot() override; /// Create new snapshot from current state. - void create_snapshot(nuraft::snapshot & s, nuraft::async_result::handler_type & when_done) override; + void create_snapshot(nuraft::snapshot & s, nuraft::async_result::handler_type & when_done) override = 0; /// Save snapshot which was send by leader to us. After that we will apply it in apply_snapshot. - void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override; + void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override = 0; - /// Better name is `serialize snapshot` -- save existing snapshot (created by create_snapshot) into - /// in-memory buffer data_out. int read_logical_snp_obj( nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr & data_out, bool & is_last_obj) override; - // This should be used only for tests or keeper-data-dumper because it violates - // TSA -- we can't acquire the lock outside of this class or return a storage under lock - // in a reasonable way. - KeeperStorage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS - { - return *storage; - } - - void shutdownStorage(); + virtual void shutdownStorage() = 0; ClusterConfigPtr getClusterConfig() const; - /// Process local read request - void processReadRequest(const KeeperStorage::RequestForSession & request_for_session); + virtual void processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) = 0; - std::vector getDeadSessions(); + virtual std::vector getDeadSessions() = 0; - int64_t getNextZxid() const; + virtual int64_t getNextZxid() const = 0; - KeeperStorage::Digest getNodesDigest() const; + virtual KeeperStorageBase::Digest getNodesDigest() const = 0; /// Introspection functions for 4lw commands - uint64_t getLastProcessedZxid() const; + virtual uint64_t getLastProcessedZxid() const = 0; - uint64_t getNodesCount() const; - uint64_t getTotalWatchesCount() const; - uint64_t getWatchedPathsCount() const; - uint64_t getSessionsWithWatchesCount() const; + virtual uint64_t getNodesCount() const = 0; + virtual uint64_t getTotalWatchesCount() const = 0; + virtual uint64_t getWatchedPathsCount() const = 0; + virtual uint64_t getSessionsWithWatchesCount() const = 0; - void dumpWatches(WriteBufferFromOwnString & buf) const; - void dumpWatchesByPath(WriteBufferFromOwnString & buf) const; - void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const; + virtual void dumpWatches(WriteBufferFromOwnString & buf) const = 0; + virtual void dumpWatchesByPath(WriteBufferFromOwnString & buf) const = 0; + virtual void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const = 0; - uint64_t getSessionWithEphemeralNodesCount() const; - uint64_t getTotalEphemeralNodesCount() const; - uint64_t getApproximateDataSize() const; - uint64_t getKeyArenaSize() const; - uint64_t getLatestSnapshotBufSize() const; + virtual uint64_t getSessionWithEphemeralNodesCount() const = 0; + virtual uint64_t getTotalEphemeralNodesCount() const = 0; + virtual uint64_t getApproximateDataSize() const = 0; + virtual uint64_t getKeyArenaSize() const = 0; + virtual uint64_t getLatestSnapshotBufSize() const = 0; - void recalculateStorageStats(); + virtual void recalculateStorageStats() = 0; - void reconfigure(const KeeperStorage::RequestForSession& request_for_session); + virtual void reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) = 0; -private: +protected: CommitCallback commit_callback; /// In our state machine we always have a single snapshot which is stored /// in memory in compressed (serialized) format. @@ -141,12 +116,7 @@ private: CoordinationSettingsPtr coordination_settings; - /// Main state machine logic - KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock); - /// Save/Load and Serialize/Deserialize logic for snapshots. - KeeperSnapshotManager snapshot_manager; - /// Put processed responses into this queue ResponsesQueue & responses_queue; @@ -163,7 +133,7 @@ private: /// for request. mutable std::mutex storage_and_responses_lock; - std::unordered_map>> parsed_request_cache; + std::unordered_map>> parsed_request_cache; uint64_t min_request_size_to_cache{0}; /// we only need to protect the access to the map itself /// requests can be modified from anywhere without lock because a single request @@ -188,8 +158,105 @@ private: KeeperSnapshotManagerS3 * snapshot_manager_s3; - KeeperStorage::ResponseForSession processReconfiguration( - const KeeperStorage::RequestForSession& request_for_session) - TSA_REQUIRES(storage_and_responses_lock); + virtual KeeperStorageBase::ResponseForSession processReconfiguration( + const KeeperStorageBase::RequestForSession& request_for_session) + TSA_REQUIRES(storage_and_responses_lock) = 0; + +}; + +/// ClickHouse Keeper state machine. Wrapper for KeeperStorage. +/// Responsible for entries commit, snapshots creation and so on. +template +class KeeperStateMachine : public IKeeperStateMachine +{ +public: + /// using CommitCallback = std::function; + + KeeperStateMachine( + ResponsesQueue & responses_queue_, + SnapshotsQueue & snapshots_queue_, + const CoordinationSettingsPtr & coordination_settings_, + const KeeperContextPtr & keeper_context_, + KeeperSnapshotManagerS3 * snapshot_manager_s3_, + CommitCallback commit_callback_ = {}, + const std::string & superdigest_ = ""); + + /// Read state from the latest snapshot + void init() override; + + bool preprocess(const KeeperStorageBase::RequestForSession & request_for_session) override; + + nuraft::ptr pre_commit(uint64_t log_idx, nuraft::buffer & data) override; + + nuraft::ptr commit(const uint64_t log_idx, nuraft::buffer & data) override; /// NOLINT + + // allow_missing - whether the transaction we want to rollback can be missing from storage + // (can happen in case of exception during preprocessing) + void rollbackRequest(const KeeperStorageBase::RequestForSession & request_for_session, bool allow_missing) override; + + void rollbackRequestNoLock( + const KeeperStorageBase::RequestForSession & request_for_session, + bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS; + + /// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state. + bool apply_snapshot(nuraft::snapshot & s) override; + + /// Create new snapshot from current state. + void create_snapshot(nuraft::snapshot & s, nuraft::async_result::handler_type & when_done) override; + + /// Save snapshot which was send by leader to us. After that we will apply it in apply_snapshot. + void save_logical_snp_obj(nuraft::snapshot & s, uint64_t & obj_id, nuraft::buffer & data, bool is_first_obj, bool is_last_obj) override; + + // This should be used only for tests or keeper-data-dumper because it violates + // TSA -- we can't acquire the lock outside of this class or return a storage under lock + // in a reasonable way. + Storage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS + { + return *storage; + } + + void shutdownStorage() override; + + /// Process local read request + void processReadRequest(const KeeperStorageBase::RequestForSession & request_for_session) override; + + std::vector getDeadSessions() override; + + int64_t getNextZxid() const override; + + KeeperStorageBase::Digest getNodesDigest() const override; + + /// Introspection functions for 4lw commands + uint64_t getLastProcessedZxid() const override; + + uint64_t getNodesCount() const override; + uint64_t getTotalWatchesCount() const override; + uint64_t getWatchedPathsCount() const override; + uint64_t getSessionsWithWatchesCount() const override; + + void dumpWatches(WriteBufferFromOwnString & buf) const override; + void dumpWatchesByPath(WriteBufferFromOwnString & buf) const override; + void dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const override; + + uint64_t getSessionWithEphemeralNodesCount() const override; + uint64_t getTotalEphemeralNodesCount() const override; + uint64_t getApproximateDataSize() const override; + uint64_t getKeyArenaSize() const override; + uint64_t getLatestSnapshotBufSize() const override; + + void recalculateStorageStats() override; + + void reconfigure(const KeeperStorageBase::RequestForSession& request_for_session) override; + +private: + /// Main state machine logic + std::unique_ptr storage; //TSA_PT_GUARDED_BY(storage_and_responses_lock); + + /// Save/Load and Serialize/Deserialize logic for snapshots. + KeeperSnapshotManager snapshot_manager; + + KeeperStorageBase::ResponseForSession processReconfiguration( + const KeeperStorageBase::RequestForSession& request_for_session) + TSA_REQUIRES(storage_and_responses_lock) override; }; } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index a4bb65a2a72..e925067b6a3 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -61,10 +61,11 @@ String getSHA1(const String & userdata) return String{digest_id.begin(), digest_id.end()}; } +template bool fixupACL( const std::vector & request_acls, int64_t session_id, - const KeeperStorage::UncommittedState & uncommitted_state, + const UncommittedState & uncommitted_state, std::vector & result_acls) { if (request_acls.empty()) @@ -77,7 +78,7 @@ bool fixupACL( { uncommitted_state.forEachAuthInSession( session_id, - [&](const KeeperStorage::AuthID & auth_id) + [&](const KeeperStorageBase::AuthID & auth_id) { valid_found = true; Coordination::ACL new_acl = request_acl; @@ -108,10 +109,10 @@ bool fixupACL( return valid_found; } -KeeperStorage::ResponsesForSessions processWatchesImpl( - const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type) +KeeperStorageBase::ResponsesForSessions processWatchesImpl( + const String & path, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches, Coordination::Event event_type) { - KeeperStorage::ResponsesForSessions result; + KeeperStorageBase::ResponsesForSessions result; auto watch_it = watches.find(path); if (watch_it != watches.end()) { @@ -122,7 +123,7 @@ KeeperStorage::ResponsesForSessions processWatchesImpl( watch_response->type = event_type; watch_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) - result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_response}); + result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_response}); watches.erase(watch_it); } @@ -158,7 +159,7 @@ KeeperStorage::ResponsesForSessions processWatchesImpl( watch_list_response->state = Coordination::State::CONNECTED; for (auto watcher_session : watch_it->second) - result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_list_response}); + result.push_back(KeeperStorageBase::ResponseForSession{watcher_session, watch_list_response}); list_watches.erase(watch_it); } @@ -193,31 +194,31 @@ uint64_t calculateDigest(std::string_view path, std::string_view data, const Coo } -void KeeperStorage::Node::setData(String new_data) +void KeeperMemNode::setData(String new_data) { size_bytes = size_bytes - data.size() + new_data.size(); data = std::move(new_data); } -void KeeperStorage::Node::addChild(StringRef child_path, bool update_size) +void KeeperMemNode::addChild(StringRef child_path, bool update_size) { if (update_size) [[likely]] size_bytes += sizeof child_path; children.insert(child_path); } -void KeeperStorage::Node::removeChild(StringRef child_path) +void KeeperMemNode::removeChild(StringRef child_path) { size_bytes -= sizeof child_path; children.erase(child_path); } -void KeeperStorage::Node::invalidateDigestCache() const +void KeeperMemNode::invalidateDigestCache() const { cached_digest.reset(); } -UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const +UInt64 KeeperMemNode::getDigest(const std::string_view path) const { if (!cached_digest) cached_digest = calculateDigest(path, data, stat); @@ -225,7 +226,7 @@ UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const return *cached_digest; }; -void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) +void KeeperMemNode::shallowCopy(const KeeperMemNode & other) { stat = other.stat; seq_num = other.seq_num; @@ -233,14 +234,15 @@ void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) cached_digest = other.cached_digest; } -void KeeperStorage::Node::recalculateSize() +void KeeperMemNode::recalculateSize() { - size_bytes = sizeof(Node); + size_bytes = sizeof(KeeperMemNode); size_bytes += children.size() * sizeof(decltype(children)::value_type); size_bytes += data.size(); } -KeeperStorage::KeeperStorage( +template +KeeperStorage::KeeperStorage( int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, const bool initialize_system_nodes) : session_expiry_queue(tick_time_ms), keeper_context(keeper_context_), superdigest(superdigest_) { @@ -252,7 +254,8 @@ KeeperStorage::KeeperStorage( initializeSystemNodes(); } -void KeeperStorage::initializeSystemNodes() +template +void KeeperStorage::initializeSystemNodes() { if (initialized) throw Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage system nodes initialized twice"); @@ -314,12 +317,13 @@ struct Overloaded : Ts... template Overloaded(Ts...) -> Overloaded; -std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const +template +std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const { if (auto node_it = storage.container.find(path); node_it != storage.container.end()) { const auto & committed_node = node_it->value; - auto node = std::make_shared(); + auto node = std::make_shared::Node>(); node->shallowCopy(committed_node); return node; } @@ -327,7 +331,8 @@ std::shared_ptr KeeperStorage::UncommittedState::tryGetNode return nullptr; } -void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) +template +void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) { assert(!delta.path.empty()); if (!nodes.contains(delta.path)) @@ -374,7 +379,8 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) delta.operation); } -void KeeperStorage::UncommittedState::addDelta(Delta new_delta) +template +void KeeperStorage::UncommittedState::addDelta(Delta new_delta) { const auto & added_delta = deltas.emplace_back(std::move(new_delta)); @@ -390,13 +396,15 @@ void KeeperStorage::UncommittedState::addDelta(Delta new_delta) } } -void KeeperStorage::UncommittedState::addDeltas(std::vector new_deltas) +template +void KeeperStorage::UncommittedState::addDeltas(std::vector new_deltas) { for (auto & delta : new_deltas) addDelta(std::move(delta)); } -void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) +template +void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) { assert(deltas.empty() || deltas.front().zxid >= commit_zxid); @@ -453,7 +461,8 @@ void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) } } -void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) +template +void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) { // we can only rollback the last zxid (if there is any) // if there is a delta with a larger zxid, we have invalid state @@ -545,7 +554,8 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) } } -std::shared_ptr KeeperStorage::UncommittedState::getNode(StringRef path) const +template +std::shared_ptr KeeperStorage::UncommittedState::getNode(StringRef path) const { if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.node; @@ -553,7 +563,8 @@ std::shared_ptr KeeperStorage::UncommittedState::getNode(St return tryGetNodeFromStorage(path); } -Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const +template +Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const { if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) return node_it->second.acls; @@ -565,7 +576,8 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) cons return storage.acl_map.convertNumber(node_it->value.acl_id); } -void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, std::function func) const +template +void KeeperStorage::UncommittedState::forEachAuthInSession(int64_t session_id, std::function func) const { const auto call_for_each_auth = [&func](const auto & auth_ids) { @@ -604,7 +616,8 @@ namespace } -void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_log_idx) +template +void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_log_idx) { std::unordered_set zxids_to_apply; for (const auto & transaction : uncommitted_transactions) @@ -630,7 +643,8 @@ void KeeperStorage::applyUncommittedState(KeeperStorage & other, int64_t last_lo } } -Coordination::Error KeeperStorage::commit(int64_t commit_zxid) +template +Coordination::Error KeeperStorage::commit(int64_t commit_zxid) { // Deltas are added with increasing ZXIDs // If there are no deltas for the commit_zxid (e.g. read requests), we instantly return @@ -644,7 +658,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) auto result = std::visit( [&, &path = delta.path](DeltaType & operation) -> Coordination::Error { - if constexpr (std::same_as) + if constexpr (std::same_as) { if (!createNode( path, @@ -656,7 +670,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { auto node_it = container.find(path); if (node_it == container.end()) @@ -671,14 +685,14 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { if (!removeNode(path, operation.version)) onStorageInconsistency(); return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { auto node_it = container.find(path); if (node_it == container.end()) @@ -692,18 +706,18 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) uint64_t acl_id = acl_map.convertACLs(operation.acls); acl_map.addUsage(acl_id); - container.updateValue(path, [acl_id](KeeperStorage::Node & node) { node.acl_id = acl_id; }); + container.updateValue(path, [acl_id](Node & node) { node.acl_id = acl_id; }); return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) return operation.error; - else if constexpr (std::same_as) + else if constexpr (std::same_as) { finish_subdelta = true; return Coordination::Error::ZOK; } - else if constexpr (std::same_as) + else if constexpr (std::same_as) { session_and_auth[operation.session_id].emplace_back(std::move(operation.auth_id)); return Coordination::Error::ZOK; @@ -726,7 +740,8 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) return Coordination::Error::ZOK; } -bool KeeperStorage::createNode( +template +bool KeeperStorage::createNode( const std::string & path, String data, const Coordination::Stat & stat, @@ -745,7 +760,7 @@ bool KeeperStorage::createNode( if (container.contains(path)) return false; - KeeperStorage::Node created_node; + Node created_node; uint64_t acl_id = acl_map.convertACLs(node_acls); acl_map.addUsage(acl_id); @@ -759,7 +774,7 @@ bool KeeperStorage::createNode( auto child_path = getBaseNodeName(map_key->getKey()); container.updateValue( parent_path, - [child_path](KeeperStorage::Node & parent) + [child_path](Node & parent) { parent.addChild(child_path); chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); @@ -770,7 +785,8 @@ bool KeeperStorage::createNode( return true; }; -bool KeeperStorage::removeNode(const std::string & path, int32_t version) +template +bool KeeperStorage::removeNode(const std::string & path, int32_t version) { auto node_it = container.find(path); if (node_it == container.end()) @@ -787,7 +803,7 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t version) container.updateValue( parentNodePath(path), - [child_basename = getBaseNodeName(node_it->key)](KeeperStorage::Node & parent) + [child_basename = getBaseNodeName(node_it->key)](Node & parent) { parent.removeChild(child_basename); chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); @@ -800,54 +816,62 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t version) return true; } +template struct KeeperStorageRequestProcessor { Coordination::ZooKeeperRequestPtr zk_request; explicit KeeperStorageRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) : zk_request(zk_request_) { } - virtual Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const = 0; - virtual std::vector - preprocess(KeeperStorage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const + + virtual Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const = 0; + + virtual std::vector + preprocess(Storage & /*storage*/, int64_t /*zxid*/, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const { return {}; } // process the request using locally committed data virtual Coordination::ZooKeeperResponsePtr - processLocal(KeeperStorage & /*storage*/, int64_t /*zxid*/) const + processLocal(Storage & /*storage*/, int64_t /*zxid*/) const { throw Exception{DB::ErrorCodes::LOGICAL_ERROR, "Cannot process the request locally"}; } - virtual KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & /*watches*/, KeeperStorage::Watches & /*list_watches*/) const + virtual Storage::ResponsesForSessions + processWatches(Storage::Watches & /*watches*/, Storage::Watches & /*list_watches*/) const { return {}; } - virtual bool checkAuth(KeeperStorage & /*storage*/, int64_t /*session_id*/, bool /*is_local*/) const { return true; } + + virtual bool checkAuth(Storage & /*storage*/, int64_t /*session_id*/, bool /*is_local*/) const { return true; } virtual ~KeeperStorageRequestProcessor() = default; }; -struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageHeartbeatRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + Coordination::ZooKeeperResponsePtr - process(KeeperStorage & /* storage */, int64_t /* zxid */) const override + process(Storage & /* storage */, int64_t /* zxid */) const override { - return zk_request->makeResponse(); + return this->zk_request->makeResponse(); } }; -struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + Coordination::ZooKeeperResponsePtr - process(KeeperStorage & /* storage */, int64_t /* zxid */) const override + process(Storage & /* storage */, int64_t /* zxid */) const override { - auto response = zk_request->makeResponse(); + auto response = this->zk_request->makeResponse(); dynamic_cast(*response).path - = dynamic_cast(*zk_request).path; + = dynamic_cast(*this->zk_request).path; return response; } }; @@ -855,7 +879,8 @@ struct KeeperStorageSyncRequestProcessor final : public KeeperStorageRequestProc namespace { -Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_local) +template +Coordination::ACLs getNodeACLs(Storage & storage, StringRef path, bool is_local) { if (is_local) { @@ -883,7 +908,8 @@ void handleSystemNodeModification(const KeeperContext & keeper_context, std::str } -bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session_id, bool is_local) +template +bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session_id, bool is_local) { const auto node_acls = getNodeACLs(*this, path, is_local); if (node_acls.empty()) @@ -910,7 +936,8 @@ bool KeeperStorage::checkACL(StringRef path, int32_t permission, int64_t session return false; } -void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) +template +void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::string & path) { auto ephemerals_it = ephemerals.find(session_id); if (ephemerals_it == ephemerals.end()) @@ -921,43 +948,44 @@ void KeeperStorage::unregisterEphemeralPath(int64_t session_id, const std::strin ephemerals.erase(ephemerals_it); } -struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorageBase::ResponsesForSessions + processWatches(Storage::Watches & watches, Storage::Watches & list_watches) const override { - return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED); + return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED); } - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - auto path = zk_request->getPath(); + auto path = this->zk_request->getPath(); return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local); } - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperCreateRequest); - Coordination::ZooKeeperCreateRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperCreateRequest & request = dynamic_cast(*this->zk_request); - std::vector new_deltas; + std::vector new_deltas; auto parent_path = parentNodePath(request.path); auto parent_node = storage.uncommitted_state.getNode(parent_path); if (parent_node == nullptr) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; else if (parent_node->stat.ephemeralOwner != 0) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}}; std::string path_created = request.path; if (request.is_sequential) { if (request.not_exists) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; auto seq_num = parent_node->seq_num; @@ -973,30 +1001,30 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr auto error_msg = fmt::format("Trying to create a node inside the internal Keeper path ({}) which is not allowed. Path: {}", keeper_system_path, path_created); handleSystemNodeModification(keeper_context, error_msg); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } if (storage.uncommitted_state.getNode(path_created)) { - if (zk_request->getOpNum() == Coordination::OpNum::CreateIfNotExists) + if (this->zk_request->getOpNum() == Coordination::OpNum::CreateIfNotExists) return new_deltas; - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; } if (getBaseNodeName(path_created).size == 0) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; Coordination::ACLs node_acls; if (!fixupACL(request.acls, session_id, storage.uncommitted_state, node_acls)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; if (request.is_ephemeral) storage.ephemerals[session_id].emplace(path_created); int32_t parent_cversion = request.parent_cversion; - auto parent_update = [parent_cversion, zxid](KeeperStorage::Node & node) + auto parent_update = [parent_cversion, zxid](Storage::Node & node) { /// Increment sequential number even if node is not sequential ++node.seq_num; @@ -1010,7 +1038,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr ++node.stat.numChildren; }; - new_deltas.emplace_back(std::string{parent_path}, zxid, KeeperStorage::UpdateNodeDelta{std::move(parent_update)}); + new_deltas.emplace_back(std::string{parent_path}, zxid, typename Storage::UpdateNodeDelta{std::move(parent_update)}); Coordination::Stat stat; stat.czxid = zxid; @@ -1028,20 +1056,20 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr new_deltas.emplace_back( std::move(path_created), zxid, - KeeperStorage::CreateNodeDelta{stat, request.is_sequential, std::move(node_acls), request.data}); + typename Storage::CreateNodeDelta{stat, request.is_sequential, std::move(node_acls), request.data}); digest = storage.calculateNodesDigest(digest, new_deltas); return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperCreateResponse & response = dynamic_cast(*response_ptr); if (storage.uncommitted_state.deltas.begin()->zxid != zxid) { - response.path_created = zk_request->getPath(); + response.path_created = this->zk_request->getPath(); response.error = Coordination::Error::ZOK; return response_ptr; } @@ -1057,7 +1085,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr deltas.begin(), deltas.end(), [zxid](const auto & delta) - { return delta.zxid == zxid && std::holds_alternative(delta.operation); }); + { return delta.zxid == zxid && std::holds_alternative(delta.operation); }); response.path_created = create_delta_it->path; response.error = Coordination::Error::ZOK; @@ -1065,37 +1093,38 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr } }; -struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); - Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperGetRequest & request = dynamic_cast(*this->zk_request); if (request.path == Coordination::keeper_api_feature_flags_path || request.path == Coordination::keeper_config_path) return {}; if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; return {}; } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperGetResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperGetRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperGetRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1134,40 +1163,42 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperGetRequest); return processImpl(storage, zxid); } }; -struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(parentNodePath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); + return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); - Coordination::ZooKeeperRemoveRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperRemoveRequest & request = dynamic_cast(*this->zk_request); - std::vector new_deltas; + std::vector new_deltas; if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path); handleSystemNodeModification(keeper_context, error_msg); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } const auto update_parent_pzxid = [&]() @@ -1179,9 +1210,9 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr new_deltas.emplace_back( std::string{parent_path}, zxid, - KeeperStorage::UpdateNodeDelta + typename Storage::UpdateNodeDelta { - [zxid](KeeperStorage::Node & parent) + [zxid](Storage::Node & parent) { if (parent.stat.pzxid < zxid) parent.stat.pzxid = zxid; @@ -1196,12 +1227,12 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr { if (request.restored_from_zookeeper_log) update_parent_pzxid(); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; } else if (request.version != -1 && request.version != node->stat.version) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; else if (node->stat.numChildren != 0) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; if (request.restored_from_zookeeper_log) update_parent_pzxid(); @@ -1209,13 +1240,13 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr new_deltas.emplace_back( std::string{parentNodePath(request.path)}, zxid, - KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent) + typename Storage::UpdateNodeDelta{[](typename Storage::Node & parent) { ++parent.stat.cversion; --parent.stat.numChildren; }}); - new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->stat.ephemeralOwner}); + new_deltas.emplace_back(request.path, zxid, typename Storage::RemoveNodeDelta{request.version, node->stat.ephemeralOwner}); if (node->stat.ephemeralOwner != 0) storage.unregisterEphemeralPath(node->stat.ephemeralOwner, request.path); @@ -1225,44 +1256,45 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperRemoveResponse & response = dynamic_cast(*response_ptr); response.error = storage.commit(zxid); return response_ptr; } - KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorageBase::ResponsesForSessions + processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override { - return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); + return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); } }; -struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); - Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*this->zk_request); if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; return {}; } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperExistsResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperExistsRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1291,55 +1323,57 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperExistsRequest); return processImpl(storage, zxid); } }; -struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Write, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Write, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperSetRequest); - Coordination::ZooKeeperSetRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperSetRequest & request = dynamic_cast(*this->zk_request); - std::vector new_deltas; + std::vector new_deltas; if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path); handleSystemNodeModification(keeper_context, error_msg); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; auto node = storage.uncommitted_state.getNode(request.path); if (request.version != -1 && request.version != node->stat.version) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; new_deltas.emplace_back( request.path, zxid, - KeeperStorage::UpdateNodeDelta{ - [zxid, data = request.data, time](KeeperStorage::Node & value) + typename Storage::UpdateNodeDelta{ + [zxid, data = request.data, time](typename Storage::Node & value) { value.stat.version++; value.stat.mzxid = zxid; @@ -1352,9 +1386,9 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce new_deltas.emplace_back( parentNodePath(request.path).toString(), zxid, - KeeperStorage::UpdateNodeDelta + typename Storage::UpdateNodeDelta { - [](KeeperStorage::Node & parent) + [](Storage::Node & parent) { parent.stat.cversion++; } @@ -1365,13 +1399,13 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { auto & container = storage.container; - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperSetResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperSetRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperSetRequest & request = dynamic_cast(*this->zk_request); if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) { @@ -1389,40 +1423,41 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce return response_ptr; } - KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorageBase::ResponsesForSessions + processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override { - return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); + return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); } }; -struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Read, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); - Coordination::ZooKeeperListRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperListRequest & request = dynamic_cast(*this->zk_request); if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; return {}; } - template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperListResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperListRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperListRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1487,63 +1522,64 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperListRequest); return processImpl(storage, zxid); } }; -struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestProcessor { explicit KeeperStorageCheckRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequestProcessor(zk_request_) + : KeeperStorageRequestProcessor(zk_request_) { - check_not_exists = zk_request->getOpNum() == Coordination::OpNum::CheckNotExists; + check_not_exists = this->zk_request->getOpNum() == Coordination::OpNum::CheckNotExists; } - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - auto path = zk_request->getPath(); + auto path = this->zk_request->getPath(); return storage.checkACL(check_not_exists ? parentNodePath(path) : path, Coordination::ACL::Read, session_id, is_local); } - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); - Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*this->zk_request); auto node = storage.uncommitted_state.getNode(request.path); if (check_not_exists) { if (node && (request.version == -1 || request.version == node->stat.version)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; } else { if (!node) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; if (request.version != -1 && request.version != node->stat.version) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; } return {}; } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperCheckResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperCheckRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1585,12 +1621,12 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperCheckRequest); return processImpl(storage, zxid); @@ -1601,55 +1637,56 @@ private: }; -struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Admin, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override { - Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*this->zk_request); if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) { auto error_msg = fmt::format("Trying to update an internal Keeper path ({}) which is not allowed", request.path); handleSystemNodeModification(keeper_context, error_msg); - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } auto & uncommitted_state = storage.uncommitted_state; if (!uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; auto node = uncommitted_state.getNode(request.path); if (request.version != -1 && request.version != node->stat.aversion) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZBADVERSION}}; Coordination::ACLs node_acls; if (!fixupACL(request.acls, session_id, uncommitted_state, node_acls)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZINVALIDACL}}; - std::vector new_deltas + std::vector new_deltas { { request.path, zxid, - KeeperStorage::SetACLDelta{std::move(node_acls), request.version} + typename Storage::SetACLDelta{std::move(node_acls), request.version} }, { request.path, zxid, - KeeperStorage::UpdateNodeDelta + typename Storage::UpdateNodeDelta { - [](KeeperStorage::Node & n) { ++n.stat.aversion; } + [](typename Storage::Node & n) { ++n.stat.aversion; } } } }; @@ -1659,11 +1696,11 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperSetACLResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperSetACLRequest & request = dynamic_cast(*this->zk_request); if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) { @@ -1681,32 +1718,33 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr } }; -struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestProcessor { - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { - return storage.checkACL(zk_request->getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local); + return storage.checkACL(this->zk_request->getPath(), Coordination::ACL::Admin | Coordination::ACL::Read, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); if (!storage.uncommitted_state.getNode(request.path)) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; return {}; } template - Coordination::ZooKeeperResponsePtr processImpl(KeeperStorage & storage, int64_t zxid) const + Coordination::ZooKeeperResponsePtr processImpl(Storage & storage, int64_t zxid) const { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperGetACLResponse & response = dynamic_cast(*response_ptr); - Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperGetACLRequest & request = dynamic_cast(*this->zk_request); if constexpr (!local) { @@ -1735,23 +1773,24 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr return response_ptr; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { return processImpl(storage, zxid); } }; -struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestProcessor { using OperationType = Coordination::ZooKeeperMultiRequest::OperationType; std::optional operation_type; - bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { for (const auto & concrete_request : concrete_requests) if (!concrete_request->checkAuth(storage, session_id, is_local)) @@ -1759,11 +1798,11 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return true; } - std::vector concrete_requests; + std::vector>> concrete_requests; explicit KeeperStorageMultiRequestProcessor(const Coordination::ZooKeeperRequestPtr & zk_request_) - : KeeperStorageRequestProcessor(zk_request_) + : KeeperStorageRequestProcessor(zk_request_) { - Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*zk_request); + Coordination::ZooKeeperMultiRequest & request = dynamic_cast(*this->zk_request); concrete_requests.reserve(request.requests.size()); const auto check_operation_type = [&](OperationType type) @@ -1781,34 +1820,34 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro case Coordination::OpNum::Create: case Coordination::OpNum::CreateIfNotExists: check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Remove: check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Set: check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Check: case Coordination::OpNum::CheckNotExists: check_operation_type(OperationType::Write); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Get: check_operation_type(OperationType::Read); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::Exists: check_operation_type(OperationType::Read); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; case Coordination::OpNum::List: case Coordination::OpNum::FilteredList: case Coordination::OpNum::SimpleList: check_operation_type(OperationType::Read); - concrete_requests.push_back(std::make_shared(sub_zk_request)); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; default: throw DB::Exception( @@ -1821,8 +1860,8 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro assert(request.requests.empty() || operation_type.has_value()); } - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t time, uint64_t & digest, const KeeperContext & keeper_context) const override { ProfileEvents::increment(ProfileEvents::KeeperMultiRequest); std::vector response_errors; @@ -1834,7 +1873,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro if (!new_deltas.empty()) { - if (auto * error = std::get_if(&new_deltas.back().operation); + if (auto * error = std::get_if(&new_deltas.back().operation); error && *operation_type == OperationType::Write) { storage.uncommitted_state.rollback(zxid); @@ -1845,10 +1884,10 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro response_errors.push_back(Coordination::Error::ZRUNTIMEINCONSISTENCY); } - return {KeeperStorage::Delta{zxid, KeeperStorage::FailedMultiDelta{std::move(response_errors)}}}; + return {typename Storage::Delta{zxid, typename Storage::FailedMultiDelta{std::move(response_errors)}}}; } } - new_deltas.emplace_back(zxid, KeeperStorage::SubDeltaEnd{}); + new_deltas.emplace_back(zxid, typename Storage::SubDeltaEnd{}); response_errors.push_back(Coordination::Error::ZOK); // manually add deltas so that the result of previous request in the transaction is used in the next request @@ -1860,15 +1899,15 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return {}; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); auto & deltas = storage.uncommitted_state.deltas; // the deltas will have at least SubDeltaEnd or FailedMultiDelta assert(!deltas.empty()); - if (auto * failed_multi = std::get_if(&deltas.front().operation)) + if (auto * failed_multi = std::get_if(&deltas.front().operation)) { for (size_t i = 0; i < concrete_requests.size(); ++i) { @@ -1890,10 +1929,10 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - Coordination::ZooKeeperResponsePtr processLocal(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr processLocal(Storage & storage, int64_t zxid) const override { ProfileEvents::increment(ProfileEvents::KeeperMultiReadRequest); - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperMultiResponse & response = dynamic_cast(*response_ptr); for (size_t i = 0; i < concrete_requests.size(); ++i) @@ -1905,10 +1944,10 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro return response_ptr; } - KeeperStorage::ResponsesForSessions - processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override + KeeperStorageBase::ResponsesForSessions + processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override { - KeeperStorage::ResponsesForSessions result; + typename Storage::ResponsesForSessions result; for (const auto & generic_request : concrete_requests) { auto responses = generic_request->processWatches(watches, list_watches); @@ -1918,47 +1957,50 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro } }; -struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageCloseRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - Coordination::ZooKeeperResponsePtr process(KeeperStorage &, int64_t) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + Coordination::ZooKeeperResponsePtr process(Storage &, int64_t) const override { throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Called process on close request"); } }; -struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor +template +struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProcessor { - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector - preprocess(KeeperStorage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & /*digest*/, const KeeperContext & /*keeper_context*/) const override { - Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast(*zk_request); - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperAuthRequest & auth_request = dynamic_cast(*this->zk_request); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); if (auth_request.scheme != "digest" || std::count(auth_request.data.begin(), auth_request.data.end(), ':') != 1) - return {KeeperStorage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; + return {typename Storage::Delta{zxid, Coordination::Error::ZAUTHFAILED}}; - std::vector new_deltas; - auto auth_digest = KeeperStorage::generateDigest(auth_request.data); + std::vector new_deltas; + auto auth_digest = Storage::generateDigest(auth_request.data); if (auth_digest == storage.superdigest) { - KeeperStorage::AuthID auth{"super", ""}; - new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(auth)}); + typename Storage::AuthID auth{"super", ""}; + new_deltas.emplace_back(zxid, typename Storage::AddAuthDelta{session_id, std::move(auth)}); } else { - KeeperStorage::AuthID new_auth{auth_request.scheme, auth_digest}; + typename Storage::AuthID new_auth{auth_request.scheme, auth_digest}; if (!storage.uncommitted_state.hasACL(session_id, false, [&](const auto & auth_id) { return new_auth == auth_id; })) - new_deltas.emplace_back(zxid, KeeperStorage::AddAuthDelta{session_id, std::move(new_auth)}); + new_deltas.emplace_back(zxid, typename Storage::AddAuthDelta{session_id, std::move(new_auth)}); } return new_deltas; } - Coordination::ZooKeeperResponsePtr process(KeeperStorage & storage, int64_t zxid) const override + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override { - Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse(); + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); Coordination::ZooKeeperAuthResponse & auth_response = dynamic_cast(*response_ptr); if (const auto result = storage.commit(zxid); result != Coordination::Error::ZOK) @@ -1968,7 +2010,8 @@ struct KeeperStorageAuthRequestProcessor final : public KeeperStorageRequestProc } }; -void KeeperStorage::finalize() +template +void KeeperStorage::finalize() { if (finalized) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage already finalized"); @@ -1983,16 +2026,17 @@ void KeeperStorage::finalize() session_expiry_queue.clear(); } -bool KeeperStorage::isFinalized() const +template +bool KeeperStorage::isFinalized() const { return finalized; } - +template class KeeperStorageRequestProcessorsFactory final : private boost::noncopyable { public: - using Creator = std::function; + using Creator = std::function>(const Coordination::ZooKeeperRequestPtr &)>; using OpNumToRequest = std::unordered_map; static KeeperStorageRequestProcessorsFactory & instance() @@ -2001,7 +2045,7 @@ public: return factory; } - KeeperStorageRequestProcessorPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const + std::shared_ptr> get(const Coordination::ZooKeeperRequestPtr & zk_request) const { auto request_it = op_num_to_request.find(zk_request->getOpNum()); if (request_it == op_num_to_request.end()) @@ -2021,39 +2065,41 @@ private: KeeperStorageRequestProcessorsFactory(); }; -template -void registerKeeperRequestProcessor(KeeperStorageRequestProcessorsFactory & factory) +template +void registerKeeperRequestProcessor(Factory & factory) { factory.registerRequest( num, [](const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared(zk_request); }); } -KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() +template +KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory() { - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); - registerKeeperRequestProcessor(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); } -UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const +template +UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector & new_deltas) const { if (!keeper_context->digestEnabled()) return current_digest; @@ -2088,7 +2134,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec auto updated_node_it = updated_nodes.find(delta.path); if (updated_node_it == updated_nodes.end()) { - node = std::make_shared(); + node = std::make_shared(); node->shallowCopy(*uncommitted_state.getNode(delta.path)); current_digest -= node->getDigest(delta.path); updated_nodes.emplace(delta.path, node); @@ -2114,7 +2160,8 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec return current_digest; } -void KeeperStorage::preprocessRequest( +template +void KeeperStorage::preprocessRequest( const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id, int64_t time, @@ -2173,7 +2220,7 @@ void KeeperStorage::preprocessRequest( uncommitted_state.addDeltas(std::move(new_deltas)); }); - KeeperStorageRequestProcessorPtr request_processor = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); + auto request_processor = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); if (zk_request->getOpNum() == Coordination::OpNum::Close) /// Close request is special { @@ -2215,7 +2262,8 @@ void KeeperStorage::preprocessRequest( new_deltas = request_processor->preprocess(*this, transaction.zxid, session_id, time, new_digest, *keeper_context); } -KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( +template +KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id, std::optional new_last_zxid, @@ -2242,7 +2290,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( uncommitted_transactions.pop_front(); } - KeeperStorage::ResponsesForSessions results; + ResponsesForSessions results; /// ZooKeeper update sessions expirity for each request, not only for heartbeats session_expiry_queue.addNewSessionOrUpdate(session_id, session_and_timeout[session_id]); @@ -2280,7 +2328,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat) /// Heartbeat request is also special { - KeeperStorageRequestProcessorPtr storage_request = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); + auto storage_request = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); auto response = storage_request->process(*this, zxid); response->xid = zk_request->xid; response->zxid = getZXID(); @@ -2289,7 +2337,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( } else /// normal requests proccession { - KeeperStorageRequestProcessorPtr request_processor = KeeperStorageRequestProcessorsFactory::instance().get(zk_request); + auto request_processor = KeeperStorageRequestProcessorsFactory>::instance().get(zk_request); Coordination::ZooKeeperResponsePtr response; if (is_local) @@ -2352,7 +2400,8 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( return results; } -void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) +template +void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) { if (allow_missing && (uncommitted_transactions.empty() || uncommitted_transactions.back().zxid < rollback_zxid)) return; @@ -2378,7 +2427,8 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) } } -KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const +template +KeeperStorageBase::Digest KeeperStorage::getNodesDigest(bool committed) const { if (!keeper_context->digestEnabled()) return {.version = DigestVersion::NO_DIGEST}; @@ -2389,13 +2439,15 @@ KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const return uncommitted_transactions.back().nodes_digest; } -void KeeperStorage::removeDigest(const Node & node, const std::string_view path) +template +void KeeperStorage::removeDigest(const Node & node, const std::string_view path) { if (keeper_context->digestEnabled()) nodes_digest -= node.getDigest(path); } -void KeeperStorage::addDigest(const Node & node, const std::string_view path) +template +void KeeperStorage::addDigest(const Node & node, const std::string_view path) { if (keeper_context->digestEnabled()) { @@ -2404,7 +2456,8 @@ void KeeperStorage::addDigest(const Node & node, const std::string_view path) } } -void KeeperStorage::clearDeadWatches(int64_t session_id) +template +void KeeperStorage::clearDeadWatches(int64_t session_id) { /// Clear all watches for this session auto watches_it = sessions_and_watchers.find(session_id); @@ -2437,7 +2490,8 @@ void KeeperStorage::clearDeadWatches(int64_t session_id) } } -void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const +template +void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const { for (const auto & [session_id, watches_paths] : sessions_and_watchers) { @@ -2447,7 +2501,8 @@ void KeeperStorage::dumpWatches(WriteBufferFromOwnString & buf) const } } -void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const +template +void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const { auto write_int_container = [&buf](const auto & session_ids) { @@ -2470,7 +2525,8 @@ void KeeperStorage::dumpWatchesByPath(WriteBufferFromOwnString & buf) const } } -void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const +template +void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) const { auto write_str_set = [&buf](const std::unordered_set & ephemeral_paths) { @@ -2495,7 +2551,8 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co } } -uint64_t KeeperStorage::getTotalWatchesCount() const +template +uint64_t KeeperStorage::getTotalWatchesCount() const { uint64_t ret = 0; for (const auto & [session, paths] : sessions_and_watchers) @@ -2504,12 +2561,14 @@ uint64_t KeeperStorage::getTotalWatchesCount() const return ret; } -uint64_t KeeperStorage::getSessionsWithWatchesCount() const +template +uint64_t KeeperStorage::getSessionsWithWatchesCount() const { return sessions_and_watchers.size(); } -uint64_t KeeperStorage::getTotalEphemeralNodesCount() const +template +uint64_t KeeperStorage::getTotalEphemeralNodesCount() const { uint64_t ret = 0; for (const auto & [session_id, nodes] : ephemerals) @@ -2518,17 +2577,20 @@ uint64_t KeeperStorage::getTotalEphemeralNodesCount() const return ret; } -void KeeperStorage::recalculateStats() +template +void KeeperStorage::recalculateStats() { container.recalculateDataSize(); } -String KeeperStorage::generateDigest(const String & userdata) +template +String KeeperStorage::generateDigest(const String & userdata) { std::vector user_password; boost::split(user_password, userdata, [](char character) { return character == ':'; }); return user_password[0] + ":" + base64Encode(getSHA1(userdata)); } +template class KeeperStorage>; } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index ec5df74efb6..2eff9f6651b 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -16,62 +16,55 @@ namespace DB { -struct KeeperStorageRequestProcessor; -using KeeperStorageRequestProcessorPtr = std::shared_ptr; using ResponseCallback = std::function; using ChildrenSet = absl::flat_hash_set; using SessionAndTimeout = std::unordered_map; -struct KeeperStorageSnapshot; +struct KeeperMemNode +{ + uint64_t acl_id = 0; /// 0 -- no ACL by default + bool is_sequental = false; + Coordination::Stat stat{}; + int32_t seq_num = 0; + uint64_t size_bytes; // save size to avoid calculate every time -/// Keeper state machine almost equal to the ZooKeeper's state machine. -/// Implements all logic of operations, data changes, sessions allocation. -/// In-memory and not thread safe. -class KeeperStorage + KeeperMemNode() : size_bytes(sizeof(KeeperMemNode)) { } + + /// Object memory size + uint64_t sizeInBytes() const { return size_bytes; } + + void setData(String new_data); + + const auto & getData() const noexcept { return data; } + + void addChild(StringRef child_path, bool update_size = true); + + void removeChild(StringRef child_path); + + const auto & getChildren() const noexcept { return children; } + + // Invalidate the calculated digest so it's recalculated again on the next + // getDigest call + void invalidateDigestCache() const; + + // get the calculated digest of the node + UInt64 getDigest(std::string_view path) const; + + // copy only necessary information for preprocessing and digest calculation + // (e.g. we don't need to copy list of children) + void shallowCopy(const KeeperMemNode & other); + + void recalculateSize(); + +private: + String data; + ChildrenSet children{}; + mutable std::optional cached_digest; +}; + +class KeeperStorageBase { public: - struct Node - { - uint64_t acl_id = 0; /// 0 -- no ACL by default - bool is_sequental = false; - Coordination::Stat stat{}; - int32_t seq_num = 0; - uint64_t size_bytes; // save size to avoid calculate every time - - Node() : size_bytes(sizeof(Node)) { } - - /// Object memory size - uint64_t sizeInBytes() const { return size_bytes; } - - void setData(String new_data); - - const auto & getData() const noexcept { return data; } - - void addChild(StringRef child_path, bool update_size = true); - - void removeChild(StringRef child_path); - - const auto & getChildren() const noexcept { return children; } - - // Invalidate the calculated digest so it's recalculated again on the next - // getDigest call - void invalidateDigestCache() const; - - // get the calculated digest of the node - UInt64 getDigest(std::string_view path) const; - - // copy only necessary information for preprocessing and digest calculation - // (e.g. we don't need to copy list of children) - void shallowCopy(const Node & other); - - void recalculateSize(); - - private: - String data; - ChildrenSet children{}; - mutable std::optional cached_digest; - }; - enum DigestVersion : uint8_t { NO_DIGEST = 0, @@ -79,7 +72,11 @@ public: V2 = 2 // added system nodes that modify the digest on startup so digest from V0 is invalid }; - static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V2; + struct Digest + { + DigestVersion version{DigestVersion::NO_DIGEST}; + uint64_t value{0}; + }; struct ResponseForSession { @@ -88,11 +85,34 @@ public: }; using ResponsesForSessions = std::vector; - struct Digest + struct RequestForSession { - DigestVersion version{DigestVersion::NO_DIGEST}; - uint64_t value{0}; + int64_t session_id; + int64_t time{0}; + Coordination::ZooKeeperRequestPtr request; + int64_t zxid{0}; + std::optional digest; + int64_t log_idx{0}; }; + using RequestsForSessions = std::vector; + + struct AuthID + { + std::string scheme; + std::string id; + + bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; } + }; + + // using Container = SnapshotableHashTable; + using Ephemerals = std::unordered_map>; + using SessionAndWatcher = std::unordered_map>; + using SessionIDs = std::unordered_set; + + /// Just vector of SHA1 from user:password + using AuthIDs = std::vector; + using SessionAndAuth = std::unordered_map; + using Watches = std::unordered_map; static bool checkDigest(const Digest & first, const Digest & second) { @@ -105,38 +125,22 @@ public: return first.value == second.value; } +}; + +/// Keeper state machine almost equal to the ZooKeeper's state machine. +/// Implements all logic of operations, data changes, sessions allocation. +/// In-memory and not thread safe. +template +class KeeperStorage : public KeeperStorageBase +{ +public: + using Container = Container_; + using Node = Container::Node; + + static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V2; + static String generateDigest(const String & userdata); - struct RequestForSession - { - int64_t session_id; - int64_t time{0}; - Coordination::ZooKeeperRequestPtr request; - int64_t zxid{0}; - std::optional digest; - int64_t log_idx{0}; - }; - - struct AuthID - { - std::string scheme; - std::string id; - - bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; } - }; - - using RequestsForSessions = std::vector; - - using Container = SnapshotableHashTable; - using Ephemerals = std::unordered_map>; - using SessionAndWatcher = std::unordered_map>; - using SessionIDs = std::unordered_set; - - /// Just vector of SHA1 from user:password - using AuthIDs = std::vector; - using SessionAndAuth = std::unordered_map; - using Watches = std::unordered_map; - int64_t session_id_counter{1}; SessionAndAuth session_and_auth; @@ -307,7 +311,7 @@ public: std::unordered_map, Hash, Equal> deltas_for_path; std::list deltas; - KeeperStorage & storage; + KeeperStorage & storage; }; UncommittedState uncommitted_state{*this}; @@ -487,6 +491,6 @@ private: void addDigest(const Node & node, std::string_view path); }; -using KeeperStoragePtr = std::unique_ptr; +using KeeperMemoryStorage = KeeperStorage>; } diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 093126237ef..2bdef189e6b 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -121,6 +121,7 @@ private: public: + using Node = V; using iterator = typename List::iterator; using const_iterator = typename List::const_iterator; using ValueUpdater = std::function; diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 3c1550f08c8..e783582af23 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -43,7 +43,8 @@ void deserializeSnapshotMagic(ReadBuffer & in) throw Exception(ErrorCodes::CORRUPTED_DATA, "Incorrect magic header in file, expected {}, got {}", SNP_HEADER, magic_header); } -int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in) +template +int64_t deserializeSessionAndTimeout(Storage & storage, ReadBuffer & in) { int32_t count; Coordination::read(count, in); @@ -62,7 +63,8 @@ int64_t deserializeSessionAndTimeout(KeeperStorage & storage, ReadBuffer & in) return max_session_id; } -void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in) +template +void deserializeACLMap(Storage & storage, ReadBuffer & in) { int32_t count; Coordination::read(count, in); @@ -90,7 +92,8 @@ void deserializeACLMap(KeeperStorage & storage, ReadBuffer & in) } } -int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * log) +template +int64_t deserializeStorageData(Storage & storage, ReadBuffer & in, Poco::Logger * log) { int64_t max_zxid = 0; std::string path; @@ -98,7 +101,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L size_t count = 0; while (path != "/") { - KeeperStorage::Node node{}; + typename Storage::Node node{}; String data; Coordination::read(data, in); node.setData(std::move(data)); @@ -140,14 +143,15 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L if (itr.key != "/") { auto parent_path = parentNodePath(itr.key); - storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; }); + storage.container.updateValue(parent_path, [my_path = itr.key] (typename Storage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; }); } } return max_zxid; } -void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log) +template +void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, Poco::Logger * log) { LOG_INFO(log, "Deserializing storage snapshot {}", snapshot_path); int64_t zxid = getZxidFromName(snapshot_path); @@ -186,7 +190,8 @@ void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::st LOG_INFO(log, "Finished, snapshot ZXID {}", storage.zxid); } -void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, Poco::Logger * log) +template +void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, Poco::Logger * log) { namespace fs = std::filesystem; std::map existing_snapshots; @@ -474,7 +479,8 @@ bool hasErrorsInMultiRequest(Coordination::ZooKeeperRequestPtr request) } -bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*log*/) +template +bool deserializeTxn(Storage & storage, ReadBuffer & in, Poco::Logger * /*log*/) { int64_t checksum; Coordination::read(checksum, in); @@ -529,7 +535,8 @@ bool deserializeTxn(KeeperStorage & storage, ReadBuffer & in, Poco::Logger * /*l return true; } -void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, Poco::Logger * log) +template +void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, Poco::Logger * log) { ReadBufferFromFile reader(log_path); @@ -553,7 +560,8 @@ void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string LOG_INFO(log, "Finished {} deserialization, totally read {} records", log_path, counter); } -void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log) +template +void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, Poco::Logger * log) { namespace fs = std::filesystem; std::map existing_logs; @@ -589,4 +597,9 @@ void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string } } +template void deserializeKeeperStorageFromSnapshot(KeeperMemoryStorage & storage, const std::string & snapshot_path, Poco::Logger * log); +template void deserializeKeeperStorageFromSnapshotsDir(KeeperMemoryStorage & storage, const std::string & path, Poco::Logger * log); +template void deserializeLogAndApplyToStorage(KeeperMemoryStorage & storage, const std::string & log_path, Poco::Logger * log); +template void deserializeLogsAndApplyToStorage(KeeperMemoryStorage & storage, const std::string & path, Poco::Logger * log); + } diff --git a/src/Coordination/ZooKeeperDataReader.h b/src/Coordination/ZooKeeperDataReader.h index 8fd86ba99e2..928e560926b 100644 --- a/src/Coordination/ZooKeeperDataReader.h +++ b/src/Coordination/ZooKeeperDataReader.h @@ -5,12 +5,16 @@ namespace DB { -void deserializeKeeperStorageFromSnapshot(KeeperStorage & storage, const std::string & snapshot_path, Poco::Logger * log); +template +void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & snapshot_path, Poco::Logger * log); -void deserializeKeeperStorageFromSnapshotsDir(KeeperStorage & storage, const std::string & path, Poco::Logger * log); +template +void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, Poco::Logger * log); -void deserializeLogAndApplyToStorage(KeeperStorage & storage, const std::string & log_path, Poco::Logger * log); +template +void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_path, Poco::Logger * log); -void deserializeLogsAndApplyToStorage(KeeperStorage & storage, const std::string & path, Poco::Logger * log); +template +void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, Poco::Logger * log); } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 418b5225fa4..517b55af866 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1320,7 +1320,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) EXPECT_EQ(hello.getApproximateDataSize(), 0); /// Node - using Node = DB::KeeperStorage::Node; + using Node = DB::KeeperMemoryStorage::Node; DB::SnapshotableHashTable world; Node n1; n1.setData("1234"); @@ -1359,9 +1359,9 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) EXPECT_EQ(world.getApproximateDataSize(), 0); } -void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) +void addNode(DB::KeeperMemoryStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) { - using Node = DB::KeeperStorage::Node; + using Node = DB::KeeperMemoryStorage::Node; Node node{}; node.setData(data); node.stat.ephemeralOwner = ephemeral_owner; @@ -1383,9 +1383,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); addNode(storage, "/hello", "world", 1); addNode(storage, "/hello/somepath", "somedata", 3); storage.session_id_counter = 5; @@ -1395,7 +1395,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) storage.getSessionID(130); storage.getSessionID(130); - DB::KeeperStorageSnapshot snapshot(&storage, 2); + DB::KeeperStorageSnapshot snapshot(&storage, 2); EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2); EXPECT_EQ(snapshot.session_id, 7); @@ -1433,9 +1433,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); storage.getSessionID(130); for (size_t i = 0; i < 50; ++i) @@ -1443,7 +1443,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } - DB::KeeperStorageSnapshot snapshot(&storage, 50); + DB::KeeperStorageSnapshot snapshot(&storage, 50); EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50); EXPECT_EQ(snapshot.snapshot_container_size, 54); @@ -1476,9 +1476,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); storage.getSessionID(130); for (size_t j = 1; j <= 5; ++j) @@ -1488,7 +1488,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } - DB::KeeperStorageSnapshot snapshot(&storage, j * 50); + DB::KeeperStorageSnapshot snapshot(&storage, j * 50); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, j * 50); EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin" + params.extension)); @@ -1517,15 +1517,15 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperMemoryStorage storage(500, "", keeper_context); for (size_t i = 0; i < 50; ++i) { addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } { - DB::KeeperStorageSnapshot snapshot(&storage, 50); + DB::KeeperStorageSnapshot snapshot(&storage, 50); for (size_t i = 0; i < 50; ++i) { addNode(storage, "/hello_" + std::to_string(i), "wlrd_" + std::to_string(i)); @@ -1571,14 +1571,14 @@ TEST_P(CoordinationTest, TestStorageSnapshotBroken) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperMemoryStorage storage(500, "", keeper_context); for (size_t i = 0; i < 50; ++i) { addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } { - DB::KeeperStorageSnapshot snapshot(&storage, 50); + DB::KeeperStorageSnapshot snapshot(&storage, 50); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); } @@ -1602,7 +1602,7 @@ nuraft::ptr getBufferFromZKRequest(int64_t session_id, int64_t z auto time = duration_cast(system_clock::now().time_since_epoch()).count(); DB::writeIntBinary(time, buf); DB::writeIntBinary(zxid, buf); - DB::writeIntBinary(DB::KeeperStorage::DigestVersion::NO_DIGEST, buf); + DB::writeIntBinary(DB::KeeperMemoryStorage::DigestVersion::NO_DIGEST, buf); return buf.getBuffer(); } @@ -1629,7 +1629,7 @@ void testLogAndStateMachine( ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); DB::KeeperLogStore changelog( DB::LogFileSettings{ @@ -1672,7 +1672,7 @@ void testLogAndStateMachine( } SnapshotsQueue snapshots_queue1{1}; - auto restore_machine = std::make_shared(queue, snapshots_queue1, settings, keeper_context, nullptr); + auto restore_machine = std::make_shared>(queue, snapshots_queue1, settings, keeper_context, nullptr); restore_machine->init(); EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance); @@ -1791,7 +1791,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); std::shared_ptr request_c = std::make_shared(); @@ -1825,11 +1825,11 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; - String digest = KeeperStorage::generateDigest(user_auth_data); + String digest = KeeperMemoryStorage::generateDigest(user_auth_data); std::shared_ptr auth_req = std::make_shared(); auth_req->scheme = "digest"; @@ -1877,11 +1877,11 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; - String digest = KeeperStorage::generateDigest(user_auth_data); + String digest = KeeperMemoryStorage::generateDigest(user_auth_data); std::shared_ptr auth_req = std::make_shared(); auth_req->scheme = "digest"; @@ -2104,9 +2104,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) ChangelogDirTest test("./snapshots"); setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); addNode(storage, "/hello", "world", 1); addNode(storage, "/hello/somepath", "somedata", 3); storage.session_id_counter = 5; @@ -2116,13 +2116,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) storage.getSessionID(130); storage.getSessionID(130); - DB::KeeperStorageSnapshot snapshot(&storage, 2); + DB::KeeperStorageSnapshot snapshot(&storage, 2); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 2); EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension)); - DB::KeeperSnapshotManager new_manager(3, keeper_context, !params.enable_compression); + DB::KeeperSnapshotManager new_manager(3, keeper_context, !params.enable_compression); auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); @@ -2308,9 +2308,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual) std::optional snapshot_hash; for (size_t i = 0; i < 15; ++i) { - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperStorage storage(500, "", keeper_context); + DB::KeeperMemoryStorage storage(500, "", keeper_context); addNode(storage, "/hello", ""); for (size_t j = 0; j < 5000; ++j) { @@ -2326,7 +2326,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual) for (size_t j = 0; j < 3333; ++j) storage.getSessionID(130 * j); - DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); + DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); auto buf = manager.serializeSnapshotToBuffer(snapshot); @@ -2389,7 +2389,7 @@ TEST_P(CoordinationTest, TestUncommittedStateBasicCrud) using namespace DB; using namespace Coordination; - DB::KeeperStorage storage{500, "", keeper_context}; + DB::KeeperMemoryStorage storage{500, "", keeper_context}; constexpr std::string_view path = "/test"; @@ -2506,7 +2506,7 @@ TEST_P(CoordinationTest, TestListRequestTypes) using namespace DB; using namespace Coordination; - KeeperStorage storage{500, "", keeper_context}; + KeeperMemoryStorage storage{500, "", keeper_context}; int32_t zxid = 0; @@ -2660,7 +2660,7 @@ TEST_P(CoordinationTest, TestDurableState) TEST_P(CoordinationTest, TestFeatureFlags) { using namespace Coordination; - KeeperStorage storage{500, "", keeper_context}; + KeeperMemoryStorage storage{500, "", keeper_context}; auto request = std::make_shared(); request->path = DB::keeper_api_feature_flags_path; auto responses = storage.processRequest(request, 0, std::nullopt, true, true); @@ -2679,7 +2679,7 @@ TEST_P(CoordinationTest, TestSystemNodeModify) // On INIT we abort when a system path is modified keeper_context->setServerState(KeeperContext::Phase::RUNNING); - KeeperStorage storage{500, "", keeper_context}; + KeeperMemoryStorage storage{500, "", keeper_context}; const auto assert_create = [&](const std::string_view path, const auto expected_code) { auto request = std::make_shared(); @@ -2771,7 +2771,7 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest) using namespace DB; using namespace Coordination; - KeeperStorage storage{500, "", keeper_context}; + KeeperMemoryStorage storage{500, "", keeper_context}; int32_t zxid = 0; @@ -2850,7 +2850,7 @@ TEST_P(CoordinationTest, TestReapplyingDeltas) create_request->path = "/test/data"; create_request->is_sequential = true; - const auto process_create = [](KeeperStorage & storage, const auto & request, int64_t zxid) + const auto process_create = [](KeeperMemoryStorage & storage, const auto & request, int64_t zxid) { storage.preprocessRequest(request, 1, 0, zxid); auto responses = storage.processRequest(request, 1, zxid); @@ -2871,19 +2871,19 @@ TEST_P(CoordinationTest, TestReapplyingDeltas) process_create(storage, create_request, zxid); }; - KeeperStorage storage1{500, "", keeper_context}; + KeeperMemoryStorage storage1{500, "", keeper_context}; commit_initial_data(storage1); for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) storage1.preprocessRequest(create_request, 1, 0, zxid, /*check_acl=*/true, /*digest=*/std::nullopt, /*log_idx=*/zxid); /// create identical new storage - KeeperStorage storage2{500, "", keeper_context}; + KeeperMemoryStorage storage2{500, "", keeper_context}; commit_initial_data(storage2); storage1.applyUncommittedState(storage2, initial_zxid); - const auto commit_unprocessed = [&](KeeperStorage & storage) + const auto commit_unprocessed = [&](KeeperMemoryStorage & storage) { for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) { @@ -2896,7 +2896,7 @@ TEST_P(CoordinationTest, TestReapplyingDeltas) commit_unprocessed(storage1); commit_unprocessed(storage2); - const auto get_children = [&](KeeperStorage & storage) + const auto get_children = [&](KeeperMemoryStorage & storage) { const auto list_request = std::make_shared(); list_request->path = "/test"; From 512705a25fac3f034a3fabcacec0fb526bd04ca0 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sun, 12 Nov 2023 22:21:43 +0100 Subject: [PATCH 02/79] fix unit test --- src/Coordination/KeeperSnapshotManager.h | 10 ++++++---- src/Coordination/KeeperStateMachine.cpp | 16 ++++++++-------- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index b0a9425881f..3d6c8f6ec43 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -40,8 +40,6 @@ struct SnapshotDeserializationResult ClusterConfigPtr cluster_config; }; -struct IKeeperStorageSnapshot {}; - /// In memory keeper snapshot. Keeper Storage based on a hash map which can be /// turned into snapshot mode. This operation is fast and KeeperStorageSnapshot /// class do it in constructor. It also copies iterators from storage hash table @@ -51,7 +49,7 @@ struct IKeeperStorageSnapshot {}; /// This representation of snapshot have to be serialized into NuRaft /// buffer and send over network or saved to file. template -struct KeeperStorageSnapshot : IKeeperStorageSnapshot +struct KeeperStorageSnapshot { public: KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr); @@ -59,6 +57,9 @@ public: KeeperStorageSnapshot( Storage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr); + KeeperStorageSnapshot(const KeeperStorageSnapshot&) = delete; + KeeperStorageSnapshot(KeeperStorageSnapshot&&) = default; + ~KeeperStorageSnapshot(); static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out, KeeperContextPtr keeper_context); @@ -97,7 +98,8 @@ struct SnapshotFileInfo DiskPtr disk; }; -using KeeperStorageSnapshotPtr = std::shared_ptr; +/// TODO(hanfei): use +using KeeperStorageSnapshotPtr = std::unique_ptr>>; using CreateSnapshotCallback = std::function; /// Class responsible for snapshots serialization and deserialization. Each snapshot diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index a3c9378a173..097adf68e50 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -579,13 +579,13 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft:: CreateSnapshotTask snapshot_task; { /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking. std::lock_guard lock(storage_and_responses_lock); - snapshot_task.snapshot = std::make_shared>(storage.get(), snapshot_meta_copy, getClusterConfig()); + snapshot_task.snapshot = std::make_unique>>(KeeperStorageSnapshot(storage.get(), snapshot_meta_copy, getClusterConfig())); } /// create snapshot task for background execution (in snapshot thread) snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot_) { - auto * snapshot = typeid_cast *>(snapshot_.get()); + auto && snapshot = std::get>(*snapshot_); nuraft::ptr exception(nullptr); bool ret = true; try @@ -593,29 +593,29 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft:: { /// Read storage data without locks and create snapshot std::lock_guard lock(snapshots_lock); - if (latest_snapshot_meta && snapshot->snapshot_meta->get_last_log_idx() <= latest_snapshot_meta->get_last_log_idx()) + if (latest_snapshot_meta && snapshot.snapshot_meta->get_last_log_idx() <= latest_snapshot_meta->get_last_log_idx()) { LOG_INFO( log, "Will not create a snapshot with last log idx {} because a snapshot with bigger last log idx ({}) is already " "created", - snapshot->snapshot_meta->get_last_log_idx(), + snapshot.snapshot_meta->get_last_log_idx(), latest_snapshot_meta->get_last_log_idx()); } else { - latest_snapshot_meta = snapshot->snapshot_meta; + latest_snapshot_meta = snapshot.snapshot_meta; /// we rely on the fact that the snapshot disk cannot be changed during runtime if (isLocalDisk(*keeper_context->getLatestSnapshotDisk())) { - auto snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); + auto snapshot_info = snapshot_manager.serializeSnapshotToDisk(snapshot); latest_snapshot_info = std::move(snapshot_info); latest_snapshot_buf = nullptr; } else { - auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(*snapshot); - auto snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx()); + auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(snapshot); + auto snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot.snapshot_meta->get_last_log_idx()); latest_snapshot_info = std::move(snapshot_info); latest_snapshot_buf = std::move(snapshot_buf); } From bea4e28691ffc78cd549bd89794bf2eb7a416c88 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sun, 12 Nov 2023 23:46:53 +0100 Subject: [PATCH 03/79] fix special build --- src/Coordination/KeeperStateMachine.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 9ae3905cddd..e7b3ef6022d 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -27,7 +27,7 @@ public: const KeeperContextPtr & keeper_context_, KeeperSnapshotManagerS3 * snapshot_manager_s3_, CommitCallback commit_callback_, - const std::string & superdisgest_); + const std::string & superdigest_); /// Read state from the latest snapshot virtual void init() = 0; From d67e3019af6a0a5f536237c1a9626a298d0fe740 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 13 Nov 2023 13:53:22 +0100 Subject: [PATCH 04/79] try to fix data race --- src/Coordination/KeeperSnapshotManager.h | 3 +-- src/Coordination/KeeperStateMachine.cpp | 18 +++++++++--------- src/Coordination/KeeperStorage.h | 14 +++++++++++--- src/Coordination/ZooKeeperDataReader.cpp | 4 ++-- utils/keeper-data-dumper/main.cpp | 4 ++-- 5 files changed, 25 insertions(+), 18 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 3d6c8f6ec43..fc7e65dd98a 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -98,8 +98,7 @@ struct SnapshotFileInfo DiskPtr disk; }; -/// TODO(hanfei): use -using KeeperStorageSnapshotPtr = std::unique_ptr>>; +using KeeperStorageSnapshotPtr = std::variant>>; using CreateSnapshotCallback = std::function; /// Class responsible for snapshots serialization and deserialization. Each snapshot diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 097adf68e50..5e0e3c90eef 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -579,43 +579,43 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft:: CreateSnapshotTask snapshot_task; { /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking. std::lock_guard lock(storage_and_responses_lock); - snapshot_task.snapshot = std::make_unique>>(KeeperStorageSnapshot(storage.get(), snapshot_meta_copy, getClusterConfig())); + snapshot_task.snapshot = std::make_unique>(KeeperStorageSnapshot(storage.get(), snapshot_meta_copy, getClusterConfig())); } /// create snapshot task for background execution (in snapshot thread) snapshot_task.create_snapshot = [this, when_done](KeeperStorageSnapshotPtr && snapshot_) { - auto && snapshot = std::get>(*snapshot_); nuraft::ptr exception(nullptr); bool ret = true; try { + auto && snapshot = std::get>>(std::move(snapshot_)); { /// Read storage data without locks and create snapshot std::lock_guard lock(snapshots_lock); - if (latest_snapshot_meta && snapshot.snapshot_meta->get_last_log_idx() <= latest_snapshot_meta->get_last_log_idx()) + if (latest_snapshot_meta && snapshot->snapshot_meta->get_last_log_idx() <= latest_snapshot_meta->get_last_log_idx()) { LOG_INFO( log, "Will not create a snapshot with last log idx {} because a snapshot with bigger last log idx ({}) is already " "created", - snapshot.snapshot_meta->get_last_log_idx(), + snapshot->snapshot_meta->get_last_log_idx(), latest_snapshot_meta->get_last_log_idx()); } else { - latest_snapshot_meta = snapshot.snapshot_meta; + latest_snapshot_meta = snapshot->snapshot_meta; /// we rely on the fact that the snapshot disk cannot be changed during runtime if (isLocalDisk(*keeper_context->getLatestSnapshotDisk())) { - auto snapshot_info = snapshot_manager.serializeSnapshotToDisk(snapshot); + auto snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot); latest_snapshot_info = std::move(snapshot_info); latest_snapshot_buf = nullptr; } else { - auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(snapshot); - auto snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot.snapshot_meta->get_last_log_idx()); + auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(*snapshot); + auto snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx()); latest_snapshot_info = std::move(snapshot_info); latest_snapshot_buf = std::move(snapshot_buf); } @@ -632,7 +632,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft:: /// Turn off "snapshot mode" and clear outdate part of storage state storage->clearGarbageAfterSnapshot(); LOG_TRACE(log, "Cleared garbage after snapshot"); - snapshot_.reset(); + snapshot.reset(); } } catch (...) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 2eff9f6651b..6fc36a7d04a 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -104,7 +104,6 @@ public: bool operator==(const AuthID & other) const { return scheme == other.scheme && id == other.id; } }; - // using Container = SnapshotableHashTable; using Ephemerals = std::unordered_map>; using SessionAndWatcher = std::unordered_map>; using SessionIDs = std::unordered_set; @@ -449,10 +448,19 @@ 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) + { + /// TODO: remove garbage log. + LOG_DEBUG(&Poco::Logger::get("KeeperStorage"), "enable snapshot mode {}", up_to_version); + container.enableSnapshotMode(up_to_version); + } /// Turn off snapshot mode. - void disableSnapshotMode() { container.disableSnapshotMode(); } + void disableSnapshotMode() + { + LOG_DEBUG(&Poco::Logger::get("KeeperStorage"), "disable snapshot mode"); + container.disableSnapshotMode(); + } Container::const_iterator getSnapshotIteratorBegin() const { return container.begin(); } diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index e783582af23..20080e1cdf6 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -190,10 +190,11 @@ void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & LOG_INFO(log, "Finished, snapshot ZXID {}", storage.zxid); } +namespace fs = std::filesystem; + template void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, Poco::Logger * log) { - namespace fs = std::filesystem; std::map existing_snapshots; for (const auto & p : fs::directory_iterator(path)) { @@ -563,7 +564,6 @@ void deserializeLogAndApplyToStorage(Storage & storage, const std::string & log_ template void deserializeLogsAndApplyToStorage(Storage & storage, const std::string & path, Poco::Logger * log) { - namespace fs = std::filesystem; std::map existing_logs; for (const auto & p : fs::directory_iterator(path)) { diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index aa8c0efbb26..abde98e6b7d 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -13,7 +13,7 @@ using namespace Coordination; using namespace DB; -void dumpMachine(std::shared_ptr machine) +void dumpMachine(std::shared_ptr> machine) { auto & storage = machine->getStorageUnsafe(); std::queue keys; @@ -67,7 +67,7 @@ int main(int argc, char *argv[]) keeper_context->setLogDisk(std::make_shared("LogDisk", argv[2])); keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", argv[1])); - auto state_machine = std::make_shared(queue, snapshots_queue, settings, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, settings, keeper_context, nullptr); state_machine->init(); size_t last_commited_index = state_machine->last_commit_index(); From e4c3b4c885d2d4a0295caddba0e9a8edf949bb0b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 14 Nov 2023 14:39:10 +0100 Subject: [PATCH 05/79] try to fix data race again --- src/Coordination/KeeperSnapshotManager.h | 2 +- src/Coordination/KeeperStateMachine.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index fc7e65dd98a..edc2607b072 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -98,7 +98,7 @@ struct SnapshotFileInfo DiskPtr disk; }; -using KeeperStorageSnapshotPtr = std::variant>>; +using KeeperStorageSnapshotPtr = std::variant>>; using CreateSnapshotCallback = std::function; /// Class responsible for snapshots serialization and deserialization. Each snapshot diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 5e0e3c90eef..fbc8a4b6ec0 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -579,7 +579,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft:: CreateSnapshotTask snapshot_task; { /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking. std::lock_guard lock(storage_and_responses_lock); - snapshot_task.snapshot = std::make_unique>(KeeperStorageSnapshot(storage.get(), snapshot_meta_copy, getClusterConfig())); + snapshot_task.snapshot = std::make_shared>(storage.get(), snapshot_meta_copy, getClusterConfig()); } /// create snapshot task for background execution (in snapshot thread) @@ -589,7 +589,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft:: bool ret = true; try { - auto && snapshot = std::get>>(std::move(snapshot_)); + auto && snapshot = std::get>>(std::move(snapshot_)); { /// Read storage data without locks and create snapshot std::lock_guard lock(snapshots_lock); From 13e5e6d8e7241ec9bb98c195899d969e33ca1c9b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 27 Dec 2023 17:54:33 +0100 Subject: [PATCH 06/79] refine find interface --- src/Coordination/CoordinationSettings.h | 3 +- src/Coordination/KeeperContext.cpp | 112 ++++++++++++++++++ src/Coordination/KeeperContext.h | 13 ++ src/Coordination/KeeperStorage.cpp | 34 +++--- src/Coordination/KeeperStorage.h | 26 ++++ src/Coordination/SnapshotableHashTable.h | 14 ++- src/Coordination/tests/gtest_coordination.cpp | 2 +- 7 files changed, 182 insertions(+), 22 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index a58f2b04797..a4679ae7a87 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -52,7 +52,8 @@ struct Settings; M(UInt64, log_file_overallocate_size, 50 * 1024 * 1024, "If max_log_file_size is not set to 0, this value will be added to it for preallocating bytes on disk. If a log record is larger than this value, it could lead to uncaught out-of-space issues so a larger value is preferred", 0) \ M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests, set to 0 to disable", 0) \ M(UInt64, raft_limits_reconnect_limit, 50, "If connection to a peer is silent longer than this limit * (multiplied by heartbeat interval), we re-establish the connection.", 0) \ - M(Bool, async_replication, false, "Enable async replication. All write and read guarantees are preserved while better performance is achieved. Settings is disabled by default to not break backwards compatibility.", 0) + M(Bool, async_replication, false, "Enable async replication. All write and read guarantees are preserved while better performance is achieved. Settings is disabled by default to not break backwards compatibility.", 0) \ + M(Bool, use_rocksdb, false, "Use rocksdb as backend storage", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index c3cb166abee..29569b03c49 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -4,12 +4,17 @@ #include #include #include +#include #include #include #include #include #include +#include +#include +#include + namespace DB { @@ -17,6 +22,8 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; +extern const int LOCICAL_ERROR; +extern const int ROCKSDB_ERROR; } @@ -33,6 +40,93 @@ KeeperContext::KeeperContext(bool standalone_keeper_) system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } +using RocksDBOptions = std::unordered_map; + +static RocksDBOptions getOptionsFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & path) +{ + RocksDBOptions options; + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(path, keys); + + for (const auto & key : keys) + { + const String key_path = path + "." + key; + options[key] = config.getString(key_path); + } + + return options; +} + +static rocksdb::Options getRocksDBOptionsFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + rocksdb::Status status; + rocksdb::Options base; + + base.create_if_missing = true; + base.compression = rocksdb::CompressionType::kZSTD; + base.statistics = rocksdb::CreateDBStatistics(); + /// It is too verbose by default, and in fact we don't care about rocksdb logs at all. + base.info_log_level = rocksdb::ERROR_LEVEL; + + rocksdb::Options merged = base; + rocksdb::BlockBasedTableOptions table_options; + + if (config.has("keeper_server.rocksdb.options")) + { + auto config_options = getOptionsFromConfig(config, "keeper_server.rocksdb.options"); + status = rocksdb::GetDBOptionsFromMap(merged, config_options, &merged); + if (!status.ok()) + { + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.options' : {}", + status.ToString()); + } + } + if (config.has("rocksdb.column_family_options")) + { + auto column_family_options = getOptionsFromConfig(config, "rocksdb.column_family_options"); + status = rocksdb::GetColumnFamilyOptionsFromMap(merged, column_family_options, &merged); + if (!status.ok()) + { + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.column_family_options' at: {}", status.ToString()); + } + } + if (config.has("rocksdb.block_based_table_options")) + { + auto block_based_table_options = getOptionsFromConfig(config, "rocksdb.block_based_table_options"); + status = rocksdb::GetBlockBasedTableOptionsFromMap(table_options, block_based_table_options, &table_options); + if (!status.ok()) + { + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Fail to merge rocksdb options from 'rocksdb.block_based_table_options' at: {}", status.ToString()); + } + } + + merged.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); + return merged; +} + +KeeperContext::Storage KeeperContext::getRocksDBPathFromConfig(const Poco::Util::AbstractConfiguration & config) const +{ + const auto create_local_disk = [](const auto & path) + { + if (fs::exists(path)) + fs::remove(path); + fs::create_directories(path); + + return std::make_shared("LocalRocksDBDisk", path); + }; + if (config.has("keeper_server.rocksdb_path")) + return create_local_disk(config.getString("keeper_server.rocksdb_path")); + + if (config.has("keeper_server.storage_path")) + return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")} / "rocksdb"); + + if (standalone_keeper) + return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "rocksdb"); + else + return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/rocksdb"); +} + void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_) { dispatcher = dispatcher_; @@ -64,6 +158,9 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, initializeFeatureFlags(config); initializeDisks(config); + + if (config.getBool("keeper_server.coordination_settings.use_rocksdb", false)) + rocksdb_options = std::make_shared(getRocksDBOptionsFromConfig(config)); } namespace @@ -99,6 +196,8 @@ void KeeperContext::initializeDisks(const Poco::Util::AbstractConfiguration & co { disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance(), diskValidator); + rocksdb_storage = getRocksDBPathFromConfig(config); + log_storage = getLogsPathFromConfig(config); if (config.has("keeper_server.latest_log_storage_disk")) @@ -267,6 +366,19 @@ void KeeperContext::dumpConfiguration(WriteBufferFromOwnString & buf) const } } +DiskPtr KeeperContext::getTemporaryRocksDBDisk() const +{ + DiskPtr rocksdb_disk = getDisk(rocksdb_storage); + if (!rocksdb_disk) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "rocksdb storage is not initialized"); + } + auto uuid_str = formatUUID(UUIDHelpers::generateV4()); + String path_to_create = "rocks_" + std::string(uuid_str.data(), uuid_str.size()); + rocksdb_disk->createDirectory(path_to_create); + return std::make_shared("LocalTmpRocksDBDisk", fullPath(rocksdb_disk, path_to_create)); +} + KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { const auto create_local_disk = [](const auto & path) diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 1af34b19ccf..1b3d18e8e53 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -6,6 +6,11 @@ #include #include +namespace rocksdb +{ +struct Options; +} + namespace DB { @@ -53,6 +58,10 @@ public: constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; } + DiskPtr getTemporaryRocksDBDisk() const; + + std::shared_ptr getRocksDBOptions() const { return rocksdb_options; } + /// set to true when we have preprocessed or committed all the logs /// that were already present locally during startup std::atomic local_logs_preprocessed = false; @@ -64,6 +73,7 @@ private: void initializeFeatureFlags(const Poco::Util::AbstractConfiguration & config); void initializeDisks(const Poco::Util::AbstractConfiguration & config); + Storage getRocksDBPathFromConfig(const Poco::Util::AbstractConfiguration & config) const; Storage getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const; Storage getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const; Storage getStatePathFromConfig(const Poco::Util::AbstractConfiguration & config) const; @@ -77,12 +87,15 @@ private: std::shared_ptr disk_selector; + Storage rocksdb_storage; Storage log_storage; Storage latest_log_storage; Storage snapshot_storage; Storage latest_snapshot_storage; Storage state_file_storage; + std::shared_ptr rocksdb_options; + std::vector old_log_disk_names; std::vector old_snapshot_disk_names; diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 80c6c0ad5a4..094d3664cd4 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -261,7 +261,7 @@ void KeeperStorage::initializeSystemNodes() throw Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage system nodes initialized twice"); // insert root system path if it isn't already inserted - if (container.find(keeper_system_path) == container.end()) + if (container.find(keeper_system_path) == nullptr) { Node system_node; container.insert(keeper_system_path, system_node); @@ -320,7 +320,7 @@ Overloaded(Ts...) -> Overloaded; template std::shared_ptr KeeperStorage::UncommittedState::tryGetNodeFromStorage(StringRef path) const { - if (auto node_it = storage.container.find(path); node_it != storage.container.end()) + if (auto node_it = storage.container.find(path); node_it) { const auto & committed_node = node_it->value; auto node = std::make_shared::Node>(); @@ -570,7 +570,7 @@ Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef return node_it->second.acls; auto node_it = storage.container.find(path); - if (node_it == storage.container.end()) + if (node_it == nullptr) return {}; return storage.acl_map.convertNumber(node_it->value.acl_id); @@ -673,7 +673,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) else if constexpr (std::same_as) { auto node_it = container.find(path); - if (node_it == container.end()) + if (node_it == nullptr) onStorageInconsistency(); if (operation.version != -1 && operation.version != node_it->value.stat.version) @@ -695,7 +695,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) else if constexpr (std::same_as) { auto node_it = container.find(path); - if (node_it == container.end()) + if (node_it == nullptr) onStorageInconsistency(); if (operation.version != -1 && operation.version != node_it->value.stat.aversion) @@ -751,7 +751,7 @@ bool KeeperStorage::createNode( auto parent_path = parentNodePath(path); auto node_it = container.find(parent_path); - if (node_it == container.end()) + if (node_it == nullptr) return false; if (node_it->value.stat.ephemeralOwner != 0) @@ -789,7 +789,7 @@ template bool KeeperStorage::removeNode(const std::string & path, int32_t version) { auto node_it = container.find(path); - if (node_it == container.end()) + if (node_it == nullptr) return false; if (version != -1 && version != node_it->value.stat.version) @@ -885,7 +885,7 @@ Coordination::ACLs getNodeACLs(Storage & storage, StringRef path, bool is_local) if (is_local) { auto node_it = storage.container.find(path); - if (node_it == storage.container.end()) + if (node_it == nullptr) return {}; return storage.acl_map.convertNumber(node_it->value.acl_id); @@ -1146,7 +1146,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce auto & container = storage.container; auto node_it = container.find(request.path); - if (node_it == container.end()) + if (node_it == nullptr) { if constexpr (local) response.error = Coordination::Error::ZNONODE; @@ -1308,7 +1308,7 @@ struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestPr auto & container = storage.container; auto node_it = container.find(request.path); - if (node_it == container.end()) + if (node_it == nullptr) { if constexpr (local) response.error = Coordination::Error::ZNONODE; @@ -1415,7 +1415,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce } auto node_it = container.find(request.path); - if (node_it == container.end()) + if (node_it == nullptr) onStorageInconsistency(); response.stat = node_it->value.stat; @@ -1472,7 +1472,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc auto & container = storage.container; auto node_it = container.find(request.path); - if (node_it == container.end()) + if (node_it == nullptr) { if constexpr (local) response.error = Coordination::Error::ZNONODE; @@ -1503,7 +1503,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc auto child_path = (std::filesystem::path(request.path) / child.toView()).generic_string(); auto child_it = container.find(child_path); - if (child_it == container.end()) + if (child_it == nullptr) onStorageInconsistency(); const auto is_ephemeral = child_it->value.stat.ephemeralOwner != 0; @@ -1604,14 +1604,14 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro if (check_not_exists) { - if (node_it != container.end() && (request.version == -1 || request.version == node_it->value.stat.version)) + if (node_it && (request.version == -1 || request.version == node_it->value.stat.version)) on_error(Coordination::Error::ZNODEEXISTS); else response.error = Coordination::Error::ZOK; } else { - if (node_it == container.end()) + if (node_it == nullptr) on_error(Coordination::Error::ZNONODE); else if (request.version != -1 && request.version != node_it->value.stat.version) on_error(Coordination::Error::ZBADVERSION); @@ -1710,7 +1710,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr } auto node_it = storage.container.find(request.path); - if (node_it == storage.container.end()) + if (node_it == nullptr) onStorageInconsistency(); response.stat = node_it->value.stat; response.error = Coordination::Error::ZOK; @@ -1758,7 +1758,7 @@ struct KeeperStorageGetACLRequestProcessor final : public KeeperStorageRequestPr auto & container = storage.container; auto node_it = container.find(request.path); - if (node_it == container.end()) + if (node_it == nullptr) { if constexpr (local) response.error = Coordination::Error::ZNONODE; diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 6fc36a7d04a..288792bb488 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -20,6 +20,29 @@ using ResponseCallback = std::function; using SessionAndTimeout = std::unordered_map; +struct KeeperRocksNode +{ + struct KeeperRocksNodeInfo + { + uint64_t acl_id = 0; /// 0 -- no ACL by default + bool is_sequental = false; + Coordination::Stat stat{}; + int32_t seq_num = 0; + UInt64 digest = 0; /// we cached digest for this node. + } meta; + + uint64_t sizeInBytes() const { return data.size() + sizeof(KeeperRocksNodeInfo); } + void setData(String new_data) {data = new_data;} + const auto & getData() const noexcept { return data; } + void shallowCopy(const KeeperRocksNode & other) + { + *this = other; + } +private: + /// TODO(hanfei): Maybe we can store data and meta seperately. + String data; +}; + struct KeeperMemNode { uint64_t acl_id = 0; /// 0 -- no ACL by default @@ -136,6 +159,8 @@ public: using Container = Container_; using Node = Container::Node; + /// static constexpr bool use_rocksdb = std::is_same_v; + static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V2; static String generateDigest(const String & userdata); @@ -500,5 +525,6 @@ private: }; using KeeperMemoryStorage = KeeperStorage>; +//using KeeperRocksStorage = KeeperStorage>; } diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 2bdef189e6b..c5ad43cc69a 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -119,6 +119,14 @@ private: } } + struct KVPair + { + StringRef key; + V value; + }; + + using KVPointer = std::shared_ptr; + public: using Node = V; @@ -258,12 +266,12 @@ public: return ret; } - const_iterator find(StringRef key) const + KVPointer find(StringRef key) const { auto map_it = map.find(key); if (map_it != map.end()) - return map_it->getMapped(); - return list.end(); + return std::make_shared(KVPair{map_it->getMapped()->key, map_it->getMapped()->value}); + return nullptr; } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 3d7d62b173b..6153ac12d06 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -2950,7 +2950,7 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest) create_path("/test_node"); auto node_it = storage.container.find("/test_node"); - ASSERT_NE(node_it, storage.container.end()); + ASSERT_NE(node_it, nullptr); auto node_version = node_it->value.stat.version; { From cba6629b1cc9418f15551dae7af3ba5b5ca6507e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 29 Dec 2023 11:46:43 +0100 Subject: [PATCH 07/79] fix build --- src/Coordination/KeeperContext.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index b9fba780b66..c79bea2d7c8 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -11,9 +11,12 @@ #include #include +#include "config.h" +#if USE_ROCKSDB #include #include #include +#endif namespace DB { @@ -40,6 +43,7 @@ KeeperContext::KeeperContext(bool standalone_keeper_) system_nodes_with_data[keeper_api_version_path] = toString(static_cast(KeeperApiVersion::WITH_MULTI_READ)); } +#if USE_ROCKSDB using RocksDBOptions = std::unordered_map; static RocksDBOptions getOptionsFromConfig(const Poco::Util::AbstractConfiguration & config, const std::string & path) @@ -104,6 +108,7 @@ static rocksdb::Options getRocksDBOptionsFromConfig(const Poco::Util::AbstractCo merged.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); return merged; } +#endif KeeperContext::Storage KeeperContext::getRocksDBPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { @@ -161,8 +166,10 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, initializeFeatureFlags(config); initializeDisks(config); + #if USE_ROCKSDB if (config.getBool("keeper_server.coordination_settings.use_rocksdb", false)) rocksdb_options = std::make_shared(getRocksDBOptionsFromConfig(config)); + #endif } namespace From 2e5f7c7851564cf8cf40ecb88f60ea76bb471f0a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 9 Jan 2024 16:26:53 +0100 Subject: [PATCH 08/79] fix build --- 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 c1b7d80935c..b9b2995db4e 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -271,7 +271,7 @@ void KeeperStorage::initializeSystemNodes() // update root and the digest based on it auto current_root_it = container.find("/"); - assert(current_root_it != container.end()); + assert(current_root_it); removeDigest(current_root_it->value, "/"); auto updated_root_it = container.updateValue( "/", From b05db179973b396d4aacd247d818742be260d712 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 17 Jan 2024 15:32:30 +0100 Subject: [PATCH 09/79] support rocksdb container and see how ci is going --- src/Coordination/KeeperServer.cpp | 32 +- src/Coordination/KeeperSnapshotManager.cpp | 51 +-- src/Coordination/KeeperSnapshotManager.h | 10 + src/Coordination/KeeperStateMachine.cpp | 3 + src/Coordination/KeeperStorage.cpp | 224 +++++++++---- src/Coordination/KeeperStorage.h | 51 ++- src/Coordination/RocksDBContainer.h | 368 +++++++++++++++++++++ 7 files changed, 631 insertions(+), 108 deletions(-) create mode 100644 src/Coordination/RocksDBContainer.h diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index db8f34afa03..dceb01d9d96 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -127,14 +127,30 @@ KeeperServer::KeeperServer( if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); - state_machine = nuraft::cs_new>( - responses_queue_, - snapshots_queue_, - coordination_settings, - keeper_context, - config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, - commit_callback, - checkAndGetSuperdigest(configuration_and_settings_->super_digest)); + if (coordination_settings->use_rocksdb) + { +#if USE_ROCKSDB + state_machine = nuraft::cs_new>( + responses_queue_, + snapshots_queue_, + coordination_settings, + keeper_context, + config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, + commit_callback, + checkAndGetSuperdigest(configuration_and_settings_->super_digest)); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "RocksDB support is disabled because ClickHouse was built without RocksDB support."); +#endif + } + else + state_machine = nuraft::cs_new>( + responses_queue_, + snapshots_queue_, + coordination_settings, + keeper_context, + config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, + commit_callback, + checkAndGetSuperdigest(configuration_and_settings_->super_digest)); state_manager = nuraft::cs_new( server_id, diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 7e377421e28..cccd24214cb 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -415,31 +415,34 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult(itr.value.getChildren().size())) + if (itr.key != "/") { -#ifdef NDEBUG - /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. - LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", itr.value.stat.numChildren, itr.value.getChildren().size(), itr.key); -#else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", - itr.value.stat.numChildren, itr.value.getChildren().size(), itr.key); -#endif + auto parent_path = parentNodePath(itr.key); + storage.container.updateValue( + parent_path, [version, path = itr.key](typename Storage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); }); + } + } + + for (const auto & itr : storage.container) + { + if (itr.key != "/") + { + if (itr.value.stat.numChildren != static_cast(itr.value.getChildren().size())) + { + #ifdef NDEBUG + /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. + LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" + " is different from actual children size {} for node {}", itr.value.stat.numChildren, itr.value.getChildren().size(), itr.key); + #else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" + " is different from actual children size {} for node {}", + itr.value.stat.numChildren, itr.value.getChildren().size(), itr.key); + #endif + } } } } @@ -845,5 +848,9 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const K template struct KeeperStorageSnapshot; template class KeeperSnapshotManager; +#if USE_ROCKSDB +template struct KeeperStorageSnapshot; +template class KeeperSnapshotManager; +#endif } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index edc2607b072..50ce94159b6 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -51,6 +51,12 @@ struct SnapshotDeserializationResult template struct KeeperStorageSnapshot { +#if USE_ROCKSDB + static constexpr bool use_rocksdb = std::is_same_v; +#else + static constexpr bool use_rocksdb = false; +#endif + public: KeeperStorageSnapshot(Storage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr); @@ -98,7 +104,11 @@ struct SnapshotFileInfo DiskPtr disk; }; +#if USE_ROCKSDB +using KeeperStorageSnapshotPtr = std::variant>, std::shared_ptr>>; +#else using KeeperStorageSnapshotPtr = std::variant>>; +#endif using CreateSnapshotCallback = std::function; /// Class responsible for snapshots serialization and deserialization. Each snapshot diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index a4b27ca06fb..0c50d9a8b77 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -939,5 +939,8 @@ void KeeperStateMachine::recalculateStorageStats() } template class KeeperStateMachine; +#if USE_ROCKSDB +template class KeeperStateMachine; +#endif } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index b9b2995db4e..0d9efc87869 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -194,6 +194,40 @@ uint64_t calculateDigest(std::string_view path, std::string_view data, const Coo } +void KeeperRocksNode::invalidateDigestCache() const +{ + if (serialized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "We modify node after serialized it"); + digest = 0; +} + +UInt64 KeeperRocksNode::getDigest(std::string_view path) const +{ + if (!digest) + digest = calculateDigest(path, data, stat); + return digest; +} + +String KeeperRocksNode::getEncodedString() +{ + if (serialized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "We modify node after serialized it"); + serialized = true; + + WriteBufferFromOwnString buffer; + const KeeperRocksNodeInfo & node_info = *this; + writePODBinary(node_info, buffer); + return buffer.str(); +} + +void KeeperRocksNode::decodeFromString(const String &buffer_str) +{ + ReadBufferFromOwnString buffer(buffer_str); + KeeperRocksNodeInfo & node_info = *this; + readPODBinary(node_info, buffer); + readStringBinary(data, buffer); +} + void KeeperMemNode::setData(String new_data) { size_bytes = size_bytes - data.size() + new_data.size(); @@ -248,7 +282,8 @@ KeeperStorage::KeeperStorage( { Node root_node; container.insert("/", root_node); - addDigest(root_node, "/"); + if constexpr (!use_rocksdb) + addDigest(root_node, "/"); if (initialize_system_nodes) initializeSystemNodes(); @@ -267,21 +302,25 @@ void KeeperStorage::initializeSystemNodes() container.insert(keeper_system_path, system_node); // store digest for the empty node because we won't update // its stats - addDigest(system_node, keeper_system_path); + if constexpr (!use_rocksdb) + addDigest(system_node, keeper_system_path); // update root and the digest based on it auto current_root_it = container.find("/"); assert(current_root_it); - removeDigest(current_root_it->value, "/"); + if constexpr (!use_rocksdb) + removeDigest(current_root_it->value, "/"); auto updated_root_it = container.updateValue( "/", [](auto & node) { ++node.stat.numChildren; - node.addChild(getBaseNodeName(keeper_system_path)); + if constexpr (!use_rocksdb) + node.addChild(getBaseNodeName(keeper_system_path)); } ); - addDigest(updated_root_it->value, "/"); + if constexpr (!use_rocksdb) + addDigest(updated_root_it->value, "/"); } // insert child system nodes @@ -290,17 +329,22 @@ void KeeperStorage::initializeSystemNodes() assert(path.starts_with(keeper_system_path)); Node child_system_node; child_system_node.setData(data); - auto [map_key, _] = container.insert(std::string{path}, child_system_node); - /// Take child path from key owned by map. - auto child_path = getBaseNodeName(map_key->getKey()); - container.updateValue( - parentNodePath(StringRef(path)), - [child_path](auto & parent) - { - // don't update stats so digest is okay - parent.addChild(child_path); - } - ); + if constexpr (use_rocksdb) + container.insert(std::string{path}, child_system_node); + else + { + auto [map_key, _] = container.insert(std::string{path}, child_system_node); + /// Take child path from key owned by map. + auto child_path = getBaseNodeName(map_key->getKey()); + container.updateValue( + parentNodePath(StringRef(path)), + [child_path](auto & parent) + { + // don't update stats so digest is okay + parent.addChild(child_path); + } + ); + } } initialized = true; @@ -679,9 +723,11 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) if (operation.version != -1 && operation.version != node_it->value.stat.version) onStorageInconsistency(); - removeDigest(node_it->value, path); + if constexpr (!use_rocksdb) + removeDigest(node_it->value, path); auto updated_node = container.updateValue(path, operation.update_fn); - addDigest(updated_node->value, path); + if constexpr (!use_rocksdb) + addDigest(updated_node->value, path); return Coordination::Error::ZOK; } @@ -769,19 +815,26 @@ bool KeeperStorage::createNode( created_node.stat = stat; created_node.setData(std::move(data)); created_node.is_sequental = is_sequental; - auto [map_key, _] = container.insert(path, created_node); - /// Take child path from key owned by map. - auto child_path = getBaseNodeName(map_key->getKey()); - container.updateValue( - parent_path, - [child_path](Node & parent) - { - parent.addChild(child_path); - chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); - } - ); + if constexpr (use_rocksdb) + { + container.insert(path, created_node); + } + else + { + auto [map_key, _] = container.insert(path, created_node); + /// Take child path from key owned by map. + auto child_path = getBaseNodeName(map_key->getKey()); + container.updateValue( + parent_path, + [child_path](KeeperMemNode & parent) + { + parent.addChild(child_path); + chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); + } + ); - addDigest(map_key->getMapped()->value, map_key->getKey().toView()); + addDigest(map_key->getMapped()->value, map_key->getKey().toView()); + } return true; }; @@ -801,18 +854,22 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers auto prev_node = node_it->value; acl_map.removeUsage(prev_node.acl_id); - container.updateValue( - parentNodePath(path), - [child_basename = getBaseNodeName(node_it->key)](Node & parent) - { - parent.removeChild(child_basename); - chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); - } - ); + if constexpr (use_rocksdb) + container.erase(path); + else { + container.updateValue( + parentNodePath(path), + [child_basename = getBaseNodeName(node_it->key)](KeeperMemNode & parent) + { + parent.removeChild(child_basename); + chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); + } + ); - container.erase(path); + container.erase(path); - removeDigest(prev_node, path); + removeDigest(prev_node, path); + } return true; } @@ -1485,35 +1542,65 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc if (path_prefix.empty()) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: path cannot be empty"); - const auto & children = node_it->value.getChildren(); - response.names.reserve(children.size()); - - const auto add_child = [&](const auto child) + if constexpr (Storage::use_rocksdb) { - using enum Coordination::ListRequestType; - - auto list_request_type = ALL; - if (auto * filtered_list = dynamic_cast(&request)) + const auto & children = container.getChildren(request.path); + response.names.reserve(children.size()); + const auto add_child = [&](const auto & child) { - list_request_type = filtered_list->list_request_type; + 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; + + const auto is_ephemeral = child.value.stat.ephemeralOwner != 0; + return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); + }; + + for (const auto & child : children) + { + if (add_child(child)) + response.names.push_back(child.key.toString()); } - - if (list_request_type == ALL) - return true; - - auto child_path = (std::filesystem::path(request.path) / child.toView()).generic_string(); - auto child_it = container.find(child_path); - if (child_it == nullptr) - onStorageInconsistency(); - - const auto is_ephemeral = child_it->value.stat.ephemeralOwner != 0; - return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); - }; - - for (const auto child : children) + } + else { - if (add_child(child)) - response.names.push_back(child.toString()); + const auto & children = node_it->value.getChildren(); + 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; + + auto child_path = (std::filesystem::path(request.path) / child.toView()).generic_string(); + auto child_it = container.find(child_path); + if (child_it == nullptr) + onStorageInconsistency(); + + const auto is_ephemeral = child_it->value.stat.ephemeralOwner != 0; + return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); + }; + + for (const auto child : children) + { + if (add_child(child)) + response.names.push_back(child.toString()); + } } response.stat = node_it->value.stat; @@ -2040,9 +2127,9 @@ public: using Creator = std::function>(const Coordination::ZooKeeperRequestPtr &)>; using OpNumToRequest = std::unordered_map; - static KeeperStorageRequestProcessorsFactory & instance() + static KeeperStorageRequestProcessorsFactory & instance() { - static KeeperStorageRequestProcessorsFactory factory; + static KeeperStorageRequestProcessorsFactory factory; return factory; } @@ -2593,5 +2680,8 @@ String KeeperStorage::generateDigest(const String & userdata) } template class KeeperStorage>; +#if USE_ROCKSDB +template class KeeperStorage>; +#endif } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index d222f8ca0a3..0eeedb8dd0c 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -13,6 +14,11 @@ #include +#include "config.h" +#if USE_ROCKSDB +#include +#endif + namespace DB { @@ -20,16 +26,23 @@ using ResponseCallback = std::function; using SessionAndTimeout = std::unordered_map; -struct KeeperRocksNode +struct KeeperRocksNodeInfo { - struct KeeperRocksNodeInfo - { - uint64_t acl_id = 0; /// 0 -- no ACL by default - bool is_sequental = false; - Coordination::Stat stat{}; - int32_t seq_num = 0; - UInt64 digest = 0; /// we cached digest for this node. - } meta; + uint64_t acl_id = 0; /// 0 -- no ACL by default + bool is_sequental = false; + Coordination::Stat stat{}; + int32_t seq_num = 0; + mutable UInt64 digest = 0; /// we cached digest for this node. +}; + +struct KeeperRocksNode:KeeperRocksNodeInfo +{ +#if USE_ROCKSDB + friend struct RocksDBContainer; +#endif + using Meta = KeeperRocksNodeInfo; + + uint64_t size_bytes = 0; // only for compatible, should be deprecated uint64_t sizeInBytes() const { return data.size() + sizeof(KeeperRocksNodeInfo); } void setData(String new_data) {data = new_data;} @@ -38,7 +51,17 @@ struct KeeperRocksNode { *this = other; } + void invalidateDigestCache() const; + UInt64 getDigest(std::string_view path) const; + String getEncodedString(); + void decodeFromString(const String & buffer_str); + void recalculateSize() {} + void reset() + { + serialized = false; + } private: + bool serialized = false; String data; }; @@ -158,7 +181,11 @@ public: using Container = Container_; using Node = Container::Node; - /// static constexpr bool use_rocksdb = std::is_same_v; +#if USE_ROCKSDB + static constexpr bool use_rocksdb = std::is_same_v>; +#else + static constexpr bool use_rocksdb = false; +#endif static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V2; @@ -521,6 +548,8 @@ private: }; using KeeperMemoryStorage = KeeperStorage>; -//using KeeperRocksStorage = KeeperStorage>; +#if USE_ROCKSDB +using KeeperRocksStorage = KeeperStorage>; +#endif } diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h new file mode 100644 index 00000000000..b357bcb2dab --- /dev/null +++ b/src/Coordination/RocksDBContainer.h @@ -0,0 +1,368 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ROCKSDB_ERROR; +} + +/// The key-value format of rocks db will be +/// - key: Int8 (depth of the path) + String (path) +/// - value: SizeOf(keeperRocksNodeInfo) (meta of the node) + String (data) + +template +struct RocksDBContainer +{ + using Node = Node_; + /// using const_iterator = std::unique_ptr; + +private: + UInt8 getKeyDepth(const std::string & key) + { + UInt8 depth = 0; + for (size_t i = 0; i < key.size(); i++) + { + if (key[i] == '/' && i + 1 != key.size()) + depth ++; + } + return depth; + } + + std::string getEncodedKey(const std::string & key, bool child_prefix = false) + { + WriteBufferFromOwnString key_buffer; + UInt16 depth = getKeyDepth(key) + child_prefix; + writeIntBinary(depth, key_buffer); + writeStringBinary(key, key_buffer); + return key_buffer.str(); + } + + + struct KVPair + { + StringRef key; + Node value; + }; + + /// using KVPointer = std::shared_ptr; + using ValueUpdater = std::function; + +public: + + struct const_iterator + { + std::shared_ptr iter; + + KVPair pair; + + const_iterator() + { + } + + const_iterator(KVPair && pair_) : pair(std::move(pair_)) {} + + const_iterator(rocksdb::Iterator * iter_) : iter(iter_) + { + iter->SeekToFirst(); + } + + const KVPair & operator * () const + { + return pair; + } + + const KVPair * operator->() const + { + return &pair; + } + + bool operator != (const const_iterator & other) const + { + return pair.key.toView() != other->key.toView(); + } + + bool operator == (std::nullptr_t) const + { + return pair.key.empty(); + } + + bool operator != (std::nullptr_t) const + { + return !pair.key.empty(); + } + + operator bool() const + { + return !pair.key.empty(); + } + + const_iterator & operator ++() + { + iter->Next(); + if (iter->Valid()) + { + pair.key = iter->key().ToString(); + pair.value.reset(); + ReadBufferFromOwnString buffer(iter->value().ToStringView()); + typename Node::Meta & meta = pair.value; + readPODBinary(meta, buffer); + readStringBinary(pair.value.data, buffer); + } + else + { + pair.key = StringRef{}; + } + return *this; + } + }; + + void initialize(const KeeperContextPtr & context) + { + + DiskPtr disk = context->getTemporaryRocksDBDisk(); + if (disk == nullptr) + { + return; + } + auto options = context->getRocksDBOptions(); + if (options == nullptr) + { + return; + } + rocksdb_dir = disk->getPath(); + rocksdb::DB * db; + auto status = rocksdb::DB::Open(*options, rocksdb_dir, &db); + if (!status.ok()) + { + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Failed to open rocksdb path at: {}: {}", + rocksdb_dir, status.ToString()); + } + rocksdb_ptr = std::unique_ptr(db); + /// storage_ptr = storage_; + } + + ~RocksDBContainer() + { + rocksdb_ptr->Close(); + rocksdb_ptr = nullptr; + + fs::remove_all(rocksdb_dir); + } + + std::vector getChildren(const std::string & key) + { + rocksdb::ReadOptions read_options; + read_options.total_order_seek = true; + + auto iter = std::unique_ptr(rocksdb_ptr->NewIterator(read_options)); + rocksdb::Slice prefix = getEncodedKey(key, true); + std::vector result; + for (iter->Seek(prefix); iter->Valid() && iter->key().starts_with(prefix); iter->Next()) + { + Node node; + ReadBufferFromOwnString buffer(iter->value().ToStringView()); + typename Node::Meta & meta = node; + readPODBinary(meta, buffer); + result.push_back(KVPair{iter->key().ToString(), node}); + } + return result; + } + + bool contains(const std::string & path) + { + const std::string & encoded_key = getEncodedKey(path); + std::string buffer_str; + rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &buffer_str); + if (status.IsNotFound()) + return false; + if (!status.ok()) + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during find. The error message is {}.", status.ToString()); + return true; + } + + const_iterator find(StringRef key_) + { + /// rocksdb::PinnableSlice slice; + const std::string & encoded_key = getEncodedKey(key_.toString()); + std::string buffer_str; + rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &buffer_str); + if (status.IsNotFound()) + return {}; + if (!status.ok()) + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during find. The error message is {}.", status.ToString()); + ReadBufferFromOwnString buffer(buffer_str); + KVPair kv; + kv.key = key_; + typename Node::Meta & meta = kv.value; + readPODBinary(meta, buffer); + readStringBinary(kv.value.data, buffer); + return const_iterator(std::move(kv)); + } + + const_iterator updateValue(StringRef key_, ValueUpdater updater) + { + /// rocksdb::PinnableSlice slice; + const std::string & key = key_.toString(); + const std::string & encoded_key = getEncodedKey(key); + std::string buffer_str; + rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &buffer_str); + if (!status.ok()) + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during find. The error message is {}.", status.ToString()); + Node node; + node.decodeFromString(buffer_str); + /// storage->removeDigest(node, key); + updater(node); + insertOrReplace(key, node); + return const_iterator(KVPair{key_, node}); + } + + bool insert(const std::string & key, Node & value) + { + std::string value_str; + const std::string & encoded_key = getEncodedKey(key); + rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &value_str); + if (status.ok()) + { + return false; + } + else if (status.IsNotFound()) + { + // storage->addDigest(value, key); + status = rocksdb_ptr->Put(rocksdb::WriteOptions(), encoded_key, value.getEncodedString()); + if (status.ok()) + { + counter++; + return true; + } + } + + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during insert. The error message is {}.", status.ToString()); + } + + void insertOrReplace(const std::string & key, Node & value) + { + const std::string & encoded_key = getEncodedKey(key); + /// storage->addDigest(value, key); + std::string value_str; + rocksdb::Status status = rocksdb_ptr->Get(rocksdb::ReadOptions(), encoded_key, &value_str); + bool increase_counter = false; + if (status.IsNotFound()) + increase_counter = true; + else if (!status.ok()) + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during get. The error message is {}.", status.ToString()); + + status = rocksdb_ptr->Put(rocksdb::WriteOptions(), encoded_key, value.getEncodedString()); + if (status.ok()) + counter += increase_counter; + else + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during insert. The error message is {}.", status.ToString()); + + } + + bool erase(const std::string & key) + { + /// storage->removeDigest(value, key); + auto status = rocksdb_ptr->Delete(rocksdb::WriteOptions(), key); + if (status.IsNotFound()) + return false; + if (status.ok()) + { + counter --; + return true; + } + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during erase. The error message is {}.", status.ToString()); + } + + void recalculateDataSize() {} + + uint64_t getApproximateDataSize() const + { + /// use statistics from rocksdb + return counter * sizeof(Node); + } + + void enableSnapshotMode(size_t version) + { + chassert(!snapshot_mode); + snapshot_mode = true; + snapshot_up_to_version = version; + ++current_version; + + snapshot = rocksdb_ptr->GetSnapshot(); + } + + void disableSnapshotMode() + { + chassert(snapshot_mode); + snapshot_mode = false; + rocksdb_ptr->ReleaseSnapshot(snapshot); + } + + void clearOutdatedNodes() {} + + std::pair snapshotSizeWithVersion() const + { + return std::make_pair(counter, current_version); + } + + const_iterator begin() const + { + rocksdb::ReadOptions read_options; + read_options.total_order_seek = true; + if (snapshot_mode) + read_options.snapshot = snapshot; + return const_iterator(rocksdb_ptr->NewIterator(read_options)); + } + + const_iterator end() const + { + return const_iterator(); + } + + size_t size() const + { + return counter; + } + + uint64_t getArenaDataSize() const + { + return 0; + } + + uint64_t keyArenaSize() const + { + return 0; + } + +private: + String rocksdb_dir; + + std::unique_ptr rocksdb_ptr; + + /// Storage* storage_ptr; + + const rocksdb::Snapshot * snapshot; + + bool snapshot_mode{false}; + size_t current_version{0}; + size_t snapshot_up_to_version{0}; + size_t counter{0}; + +}; + +} From 89ff11e967179b6005773b859136b9f909ba86b2 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 17 Jan 2024 15:40:57 +0100 Subject: [PATCH 10/79] fix style --- src/Coordination/KeeperStorage.cpp | 3 ++- src/Coordination/RocksDBContainer.h | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 0d9efc87869..c115546ca71 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -856,7 +856,8 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t vers if constexpr (use_rocksdb) container.erase(path); - else { + else + { container.updateValue( parentNodePath(path), [child_basename = getBaseNodeName(node_it->key)](KeeperMemNode & parent) diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index b357bcb2dab..a0eaa0b6ae7 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -17,7 +17,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int ROCKSDB_ERROR; } From 8cddb8163480e6c77d7748d07c922d7ec7aba76d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 17 Jan 2024 17:39:25 +0100 Subject: [PATCH 11/79] fix build --- src/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 083b959c4b6..9340e070eb2 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -131,6 +131,7 @@ endif() if (TARGET ch_contrib::rocksdb) add_headers_and_sources(dbms Storages/RocksDB) + add_headers_and_sources(dbms Coordination) endif() if (TARGET ch_contrib::aws_s3) From 60674094436e8eee02f61593f23d75b17dd5de3d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 5 Feb 2024 08:57:30 +0100 Subject: [PATCH 12/79] fix ut --- src/Coordination/KeeperSnapshotManager.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 710c3be77b9..86158ee5ab6 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -427,9 +427,12 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult Date: Mon, 18 Mar 2024 01:07:03 +0100 Subject: [PATCH 13/79] fix --- src/Coordination/KeeperSnapshotManager.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 5d07067ad26..c5623e746ff 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -885,14 +885,16 @@ SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const K return {snapshot_file_name, disk}; } -size_t KeeperSnapshotManager::getLatestSnapshotIndex() const +template +size_t KeeperSnapshotManager::getLatestSnapshotIndex() const { if (!existing_snapshots.empty()) return existing_snapshots.rbegin()->first; return 0; } -SnapshotFileInfo KeeperSnapshotManager::getLatestSnapshotInfo() const +template +SnapshotFileInfo KeeperSnapshotManager::getLatestSnapshotInfo() const { if (!existing_snapshots.empty()) { From 5f4d2f2119d5a338af0fe3ee65314eb73ed368f7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 18 Mar 2024 09:23:51 +0100 Subject: [PATCH 14/79] fix build --- src/Coordination/KeeperStorage.cpp | 1 + src/Coordination/ZooKeeperDataReader.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index b1d6655a55d..992cfe8f767 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index f3dc7ffe0b7..99d71b85e78 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -196,7 +196,7 @@ void deserializeKeeperStorageFromSnapshot(Storage & storage, const std::string & LOG_INFO(log, "Finished, snapshot ZXID {}", storage.zxid); } -/// namespace fs = std::filesystem; +namespace fs = std::filesystem; template void deserializeKeeperStorageFromSnapshotsDir(Storage & storage, const std::string & path, LoggerPtr log) From 587b176a2e05cf10c859e18d7c06bfa75ef7bbdb Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 19 Mar 2024 16:06:53 +0100 Subject: [PATCH 15/79] fix keeper build --- programs/keeper/CMakeLists.txt | 1 + src/CMakeLists.txt | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 70e0f229fd4..e66df8e724e 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -192,6 +192,7 @@ if (BUILD_STANDALONE_KEEPER) PRIVATE ch_contrib::abseil_swiss_tables ch_contrib::nuraft + ch_contrib::rocksdb ch_contrib::lz4 ch_contrib::zstd ch_contrib::cityhash diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index fd15d2df1e9..73aa409e995 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -133,7 +133,6 @@ endif() if (TARGET ch_contrib::rocksdb) add_headers_and_sources(dbms Storages/RocksDB) - add_headers_and_sources(dbms Coordination) endif() if (TARGET ch_contrib::aws_s3) From 930568c8c68ad8463c38888c6843a25ba3e38429 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 19 Mar 2024 20:41:34 +0100 Subject: [PATCH 16/79] fix debug build --- src/Coordination/KeeperSnapshotManager.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index c5623e746ff..9f653b7dfc4 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -477,7 +477,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult Date: Wed, 20 Mar 2024 11:58:02 +0100 Subject: [PATCH 17/79] fix clang tidy --- src/Coordination/RocksDBContainer.h | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index 4bfd57e7682..a2ff6ed3f3c 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -71,13 +71,11 @@ public: std::shared_ptr pair; - const_iterator() - { - } + const_iterator() = default; - const_iterator(std::shared_ptr pair_) : pair(std::move(pair_)) {} + explicit const_iterator(std::shared_ptr pair_) : pair(std::move(pair_)) {} - const_iterator(rocksdb::Iterator * iter_) : iter(iter_) + explicit const_iterator(rocksdb::Iterator * iter_) : iter(iter_) { iter->SeekToFirst(); } @@ -112,7 +110,7 @@ public: return iter != nullptr; } - operator bool() const + explicit operator bool() const { return iter != nullptr; } From 0f48238a27c9167ade17bb32aa9f39710305f4f8 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 4 Apr 2024 13:47:04 +0200 Subject: [PATCH 18/79] copy ut for rocks keeper --- src/Coordination/KeeperContext.cpp | 19 + src/Coordination/KeeperContext.h | 3 + src/Coordination/KeeperSnapshotManager.cpp | 1 - src/Coordination/KeeperStorage.cpp | 9 +- src/Coordination/RocksDBContainer.h | 107 +- src/Coordination/tests/gtest_rocks_keeper.cpp | 1150 +++++++++++++++++ 6 files changed, 1263 insertions(+), 26 deletions(-) create mode 100644 src/Coordination/tests/gtest_rocks_keeper.cpp diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 8fc4661a779..4ae3c295824 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -367,6 +368,12 @@ void KeeperContext::dumpConfiguration(WriteBufferFromOwnString & buf) const } } + +void KeeperContext::setRocksDBDisk(DiskPtr disk) +{ + rocksdb_storage = std::move(disk); +} + DiskPtr KeeperContext::getTemporaryRocksDBDisk() const { DiskPtr rocksdb_disk = getDisk(rocksdb_storage); @@ -380,6 +387,18 @@ DiskPtr KeeperContext::getTemporaryRocksDBDisk() const return std::make_shared("LocalTmpRocksDBDisk", fullPath(rocksdb_disk, path_to_create)); } +void KeeperContext::setRocksDBOptions(std::shared_ptr rocksdb_options_) +{ + if (rocksdb_options_ != nullptr) + rocksdb_options = rocksdb_options_; + else + { + #if USE_ROCKSDB + rocksdb_options = std::make_shared(getRocksDBOptionsFromConfig(Poco::Util::JSONConfiguration())); + #endif + } +} + KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const { const auto create_local_disk = [](const auto & path) diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index f84d087cb41..c9abd13b6e9 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -67,7 +67,10 @@ public: constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; } + void setRocksDBDisk(DiskPtr disk); DiskPtr getTemporaryRocksDBDisk() const; + + void setRocksDBOptions(std::shared_ptr rocksdb_options_); std::shared_ptr getRocksDBOptions() const { return rocksdb_options; } UInt64 getKeeperMemorySoftLimit() const { return memory_soft_limit; } diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 9f653b7dfc4..593c358e80d 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -258,7 +258,6 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot snapshot.zxid) break; - writeBinary(path, out); writeNode(node, snapshot.version, out); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 992cfe8f767..905fd06359c 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -261,7 +261,10 @@ void KeeperRocksNode::decodeFromString(const String &buffer_str) readPODBinary(node_info, buffer); readVarUInt(data_size, buffer); if (data_size) + { + data = std::unique_ptr(new char[data_size]); buffer.readStrict(data.get(), data_size); + } } KeeperMemNode & KeeperMemNode::operator=(const KeeperMemNode & other) @@ -445,6 +448,8 @@ KeeperStorage::KeeperStorage( int64_t tick_time_ms, const String & superdigest_, const KeeperContextPtr & keeper_context_, const bool initialize_system_nodes) : session_expiry_queue(tick_time_ms), keeper_context(keeper_context_), superdigest(superdigest_) { + if constexpr (use_rocksdb) + container.initialize(keeper_context); Node root_node; container.insert("/", root_node); if constexpr (!use_rocksdb) @@ -1757,14 +1762,14 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc if (list_request_type == ALL) return true; - const auto is_ephemeral = child.value.isEphemeral(); + const auto is_ephemeral = child.second.isEphemeral(); return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); }; for (const auto & child : children) { if (add_child(child)) - response.names.push_back(child.key.toString()); + response.names.push_back(child.first); } } else diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index a2ff6ed3f3c..683678cfcd1 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -18,6 +18,7 @@ namespace DB namespace ErrorCodes { extern const int ROCKSDB_ERROR; + extern const int LOGICAL_ERROR; } namespace fs = std::filesystem; @@ -32,9 +33,27 @@ struct RocksDBContainer using Node = Node_; private: - UInt8 getKeyDepth(const std::string & key) + std::string toHex(std::string origin) { - UInt8 depth = 0; + auto toHexChar = [](char v) + { + if (v <= 9) + return v + '0'; + return 'A' + v - 10; + }; + std::string result; + result.reserve(2 * origin.size()); + for (size_t i = 0; i < origin.size(); ++i) { + unsigned char c = origin[i]; + result.push_back(toHexChar(c >> 4)); + result.push_back(toHexChar(c & 0xf)); + } + return result; + } + + UInt16 getKeyDepth(const std::string & key) + { + UInt16 depth = 0; for (size_t i = 0; i < key.size(); i++) { if (key[i] == '/' && i + 1 != key.size()) @@ -46,12 +65,17 @@ private: std::string getEncodedKey(const std::string & key, bool child_prefix = false) { WriteBufferFromOwnString key_buffer; - UInt16 depth = getKeyDepth(key) + child_prefix; + UInt16 depth = getKeyDepth(key) + (child_prefix ? 1 : 0); writeIntBinary(depth, key_buffer); - writeStringBinary(key, key_buffer); + writeString(key, key_buffer); return key_buffer.str(); } + static std::string_view getDecodedKey(const std::string_view & key) + { + return std::string_view(key.begin() + 2, key.end()); + } + struct KVPair { @@ -59,7 +83,6 @@ private: Node value; }; - /// using KVPointer = std::shared_ptr; using ValueUpdater = std::function; public: @@ -77,7 +100,7 @@ public: explicit const_iterator(rocksdb::Iterator * iter_) : iter(iter_) { - iter->SeekToFirst(); + updatePairFromIter(); } const KVPair & operator * () const @@ -92,11 +115,15 @@ public: bool operator != (const const_iterator & other) const { - return pair->key.toView() != other->key.toView(); + return !(*this == other); } bool operator == (const const_iterator & other) const { + if (pair == nullptr && other == nullptr) + return true; + if (pair == nullptr || other == nullptr) + return false; return pair->key.toView() == other->key.toView() && iter == other.iter; } @@ -118,17 +145,26 @@ public: const_iterator & operator ++() { iter->Next(); + updatePairFromIter(); + return *this; + } + + private: + void updatePairFromIter() + { if (iter && iter->Valid()) { auto new_pair = std::make_shared(); - new_pair->key = iter->key().ToString(); - new_pair->value.reset(); + new_pair->key = StringRef(getDecodedKey(iter->key().ToStringView())); ReadBufferFromOwnString buffer(iter->value().ToStringView()); typename Node::Meta & meta = new_pair->value; readPODBinary(meta, buffer); readVarUInt(new_pair->value.data_size, buffer); if (new_pair->value.data_size) + { + new_pair->value.data = std::unique_ptr(new char[new_pair->value.data_size]); buffer.readStrict(new_pair->value.data.get(), new_pair->value.data_size); + } pair = new_pair; } else @@ -136,23 +172,25 @@ public: pair = nullptr; iter = nullptr; } - return *this; } }; + bool initialized = false; + const const_iterator end_ptr; void initialize(const KeeperContextPtr & context) { - DiskPtr disk = context->getTemporaryRocksDBDisk(); if (disk == nullptr) { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get rocksdb disk"); return; } auto options = context->getRocksDBOptions(); if (options == nullptr) { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get rocksdb options"); return; } rocksdb_dir = disk->getPath(); @@ -165,24 +203,33 @@ public: } rocksdb_ptr = std::unique_ptr(db); /// storage_ptr = storage_; + initialized = true; } ~RocksDBContainer() { - rocksdb_ptr->Close(); - rocksdb_ptr = nullptr; + if (initialized) + { + rocksdb_ptr->Close(); + rocksdb_ptr = nullptr; - fs::remove_all(rocksdb_dir); + fs::remove_all(rocksdb_dir); + } } - std::vector getChildren(const std::string & key) + std::vector> getChildren(const std::string & key) { rocksdb::ReadOptions read_options; read_options.total_order_seek = true; + size_t len = key.size() + 2; + if (!key.ends_with('/')) + len ++; + auto iter = std::unique_ptr(rocksdb_ptr->NewIterator(read_options)); - rocksdb::Slice prefix = getEncodedKey(key, true); - std::vector result; + std::string encoded_string = getEncodedKey(key, true); + rocksdb::Slice prefix(encoded_string); + std::vector> result; for (iter->Seek(prefix); iter->Valid() && iter->key().starts_with(prefix); iter->Next()) { Node node; @@ -190,8 +237,11 @@ public: typename Node::Meta & meta = node; /// We do not read data here readPODBinary(meta, buffer); - result.emplace_back(iter->key().ToString(), std::move(node)); + std::string real_key(iter->key().data() + len, iter->key().size() - len); + // std::cout << "real key: " << real_key << std::endl; + result.emplace_back(std::move(real_key), std::move(node)); } + return result; } @@ -203,7 +253,7 @@ public: if (status.IsNotFound()) return false; if (!status.ok()) - throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during find. The error message is {}.", status.ToString()); + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during executing contains. The error message is {}.", status.ToString()); return true; } @@ -216,7 +266,7 @@ public: if (status.IsNotFound()) return end(); if (!status.ok()) - throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during find. The error message is {}.", status.ToString()); + throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during executing find. The error message is {}.", status.ToString()); ReadBufferFromOwnString buffer(buffer_str); auto kv = std::make_shared(); kv->key = key_; @@ -225,7 +275,10 @@ public: /// TODO: Sometimes we don't need to load data. readVarUInt(kv->value.data_size, buffer); if (kv->value.data_size) + { + kv->value.data = std::unique_ptr(new char[kv->value.data_size]); buffer.readStrict(kv->value.data.get(), kv->value.data_size); + } return const_iterator(kv); } @@ -306,7 +359,8 @@ public: bool erase(const std::string & key) { /// storage->removeDigest(value, key); - auto status = rocksdb_ptr->Delete(rocksdb::WriteOptions(), key); + const std::string & encoded_key = getEncodedKey(key); + auto status = rocksdb_ptr->Delete(rocksdb::WriteOptions(), encoded_key); if (status.IsNotFound()) return false; if (status.ok()) @@ -331,6 +385,7 @@ public: chassert(!snapshot_mode); snapshot_mode = true; snapshot_up_to_version = version; + snapshot_size = counter; ++current_version; snapshot = rocksdb_ptr->GetSnapshot(); @@ -347,7 +402,10 @@ public: std::pair snapshotSizeWithVersion() const { - return std::make_pair(counter, current_version); + if (!snapshot_mode) + return std::make_pair(counter, current_version); + else + return std::make_pair(snapshot_size, current_version); } const_iterator begin() const @@ -356,7 +414,9 @@ public: read_options.total_order_seek = true; if (snapshot_mode) read_options.snapshot = snapshot; - return const_iterator(rocksdb_ptr->NewIterator(read_options)); + auto * iter = rocksdb_ptr->NewIterator(read_options); + iter->SeekToFirst(); + return const_iterator(iter); } const_iterator end() const @@ -391,6 +451,7 @@ private: bool snapshot_mode{false}; size_t current_version{0}; size_t snapshot_up_to_version{0}; + size_t snapshot_size{0}; size_t counter{0}; }; diff --git a/src/Coordination/tests/gtest_rocks_keeper.cpp b/src/Coordination/tests/gtest_rocks_keeper.cpp new file mode 100644 index 00000000000..6281f04a637 --- /dev/null +++ b/src/Coordination/tests/gtest_rocks_keeper.cpp @@ -0,0 +1,1150 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "config.h" + +#if USE_NURAFT and USE_ROCKSDB + +namespace fs = std::filesystem; +struct ChangelogDirTest +{ + std::string path; + bool drop; + explicit ChangelogDirTest(std::string path_, bool drop_ = true) : path(path_), drop(drop_) + { + EXPECT_FALSE(fs::exists(path)) << "Path " << path << " already exists, remove it to run test"; + fs::create_directory(path); + } + + ~ChangelogDirTest() + { + if (fs::exists(path) && drop) + fs::remove_all(path); + } +}; + +class RocksKeeperTest : public ::testing::Test +{ +protected: + DB::KeeperContextPtr keeper_context; + LoggerPtr log{getLogger("RocksKeeperTest")}; + + void SetUp() override + { + Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); + + auto settings = std::make_shared(); + settings->use_rocksdb = true; + keeper_context = std::make_shared(true, settings); + keeper_context->setLocalLogsPreprocessed(); + keeper_context->setRocksDBOptions(nullptr); + } + + void setSnapshotDirectory(const std::string & path) + { + keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", path)); + } + + void setRocksDBDirectory(const std::string & path) + { + keeper_context->setRocksDBDisk(std::make_shared("RocksDisk", path)); + } +}; + +void addNode(DB::KeeperRocksStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) +{ + using Node = DB::KeeperRocksStorage::Node; + Node node{}; + node.setData(data); + node.setEphemeralOwner(ephemeral_owner); + storage.container.insertOrReplace(path, node); + storage.container.updateValue( + DB::parentNodePath(StringRef{path}), + [&](auto & parent) + { + parent.increaseNumChildren(); + }); +} + +namespace +{ +void waitDurableLogs(nuraft::log_store & log_store) +{ + while (log_store.last_durable_index() != log_store.next_slot() - 1) + std::this_thread::sleep_for(std::chrono::milliseconds(200)); +} +} + +TEST_F(RocksKeeperTest, TestStorageSnapshotSimple) +{ + ChangelogDirTest test("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setSnapshotDirectory("./snapshots"); + setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, keeper_context); + + DB::KeeperRocksStorage storage(500, "", keeper_context); + addNode(storage, "/hello1", "world", 1); + addNode(storage, "/hello2", "somedata", 3); + storage.session_id_counter = 5; + storage.zxid = 2; + storage.ephemerals[3] = {"/hello2"}; + storage.ephemerals[1] = {"/hello1"}; + storage.getSessionID(130); + storage.getSessionID(130); + + DB::KeeperStorageSnapshot snapshot(&storage, 2); + + EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2); + EXPECT_EQ(snapshot.session_id, 7); + EXPECT_EQ(snapshot.snapshot_container_size, 6); + EXPECT_EQ(snapshot.session_and_timeout.size(), 2); + + auto buf = manager.serializeSnapshotToBuffer(snapshot); + manager.serializeSnapshotBufferToDisk(*buf, 2); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin.zstd")); + + + auto debuf = manager.deserializeSnapshotBufferFromDisk(2); + + auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); + + EXPECT_EQ(restored_storage->container.size(), 6); + EXPECT_EQ(restored_storage->container.find("/")->value.numChildren(), 3); + EXPECT_EQ(restored_storage->container.find("/hello1")->value.numChildren(), 0); + EXPECT_EQ(restored_storage->container.find("/hello2")->value.numChildren(), 0); + + EXPECT_EQ(restored_storage->container.find("/")->value.getData(), ""); + EXPECT_EQ(restored_storage->container.find("/hello1")->value.getData(), "world"); + EXPECT_EQ(restored_storage->container.find("/hello2")->value.getData(), "somedata"); + EXPECT_EQ(restored_storage->session_id_counter, 7); + EXPECT_EQ(restored_storage->zxid, 2); + EXPECT_EQ(restored_storage->ephemerals.size(), 2); + EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); + EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); + EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); +} + +TEST_F(RocksKeeperTest, TestStorageSnapshotMoreWrites) +{ + ChangelogDirTest test("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setSnapshotDirectory("./snapshots"); + setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, keeper_context); + + DB::KeeperRocksStorage storage(500, "", keeper_context); + storage.getSessionID(130); + + for (size_t i = 0; i < 50; ++i) + { + addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); + } + + DB::KeeperStorageSnapshot snapshot(&storage, 50); + EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50); + EXPECT_EQ(snapshot.snapshot_container_size, 54); + + for (size_t i = 50; i < 100; ++i) + { + addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); + } + + EXPECT_EQ(storage.container.size(), 104); + + auto buf = manager.serializeSnapshotToBuffer(snapshot); + manager.serializeSnapshotBufferToDisk(*buf, 50); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin.zstd")); + + + auto debuf = manager.deserializeSnapshotBufferFromDisk(50); + auto [restored_storage, meta, _] = manager.deserializeSnapshotFromBuffer(debuf); + + EXPECT_EQ(restored_storage->container.size(), 54); + for (size_t i = 0; i < 50; ++i) + { + EXPECT_EQ(restored_storage->container.find("/hello_" + std::to_string(i))->value.getData(), "world_" + std::to_string(i)); + } +} + +TEST_F(RocksKeeperTest, TestStorageSnapshotManySnapshots) +{ + ChangelogDirTest test("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setSnapshotDirectory("./snapshots"); + setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, keeper_context); + + DB::KeeperRocksStorage storage(500, "", keeper_context); + storage.getSessionID(130); + + for (size_t j = 1; j <= 5; ++j) + { + for (size_t i = (j - 1) * 50; i < j * 50; ++i) + { + addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); + } + + DB::KeeperStorageSnapshot snapshot(&storage, j * 50); + auto buf = manager.serializeSnapshotToBuffer(snapshot); + manager.serializeSnapshotBufferToDisk(*buf, j * 50); + EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin.zstd")); + } + + EXPECT_FALSE(fs::exists("./snapshots/snapshot_50.bin.zstd")); + EXPECT_FALSE(fs::exists("./snapshots/snapshot_100.bin.zstd")); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_150.bin.zstd")); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_200.bin.zstd")); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin.zstd")); + + + auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); + + EXPECT_EQ(restored_storage->container.size(), 254); + + for (size_t i = 0; i < 250; ++i) + { + EXPECT_EQ(restored_storage->container.find("/hello_" + std::to_string(i))->value.getData(), "world_" + std::to_string(i)); + } +} + +TEST_F(RocksKeeperTest, TestStorageSnapshotMode) +{ + ChangelogDirTest test("./snapshots"); + setSnapshotDirectory("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, keeper_context); + DB::KeeperRocksStorage storage(500, "", keeper_context); + for (size_t i = 0; i < 50; ++i) + { + addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); + } + + { + DB::KeeperStorageSnapshot snapshot(&storage, 50); + for (size_t i = 0; i < 50; ++i) + { + addNode(storage, "/hello_" + std::to_string(i), "wlrd_" + std::to_string(i)); + } + for (size_t i = 0; i < 50; ++i) + { + EXPECT_EQ(storage.container.find("/hello_" + std::to_string(i))->value.getData(), "wlrd_" + std::to_string(i)); + } + for (size_t i = 0; i < 50; ++i) + { + if (i % 2 == 0) + storage.container.erase("/hello_" + std::to_string(i)); + } + EXPECT_EQ(storage.container.size(), 29); + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 54); + EXPECT_EQ(storage.container.snapshotSizeWithVersion().second, 1); + auto buf = manager.serializeSnapshotToBuffer(snapshot); + manager.serializeSnapshotBufferToDisk(*buf, 50); + } + EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin.zstd")); + EXPECT_EQ(storage.container.size(), 29); + storage.clearGarbageAfterSnapshot(); + for (size_t i = 0; i < 50; ++i) + { + if (i % 2 != 0) + EXPECT_EQ(storage.container.find("/hello_" + std::to_string(i))->value.getData(), "wlrd_" + std::to_string(i)); + else + EXPECT_FALSE(storage.container.contains("/hello_" + std::to_string(i))); + } + + auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); + + for (size_t i = 0; i < 50; ++i) + { + EXPECT_EQ(restored_storage->container.find("/hello_" + std::to_string(i))->value.getData(), "world_" + std::to_string(i)); + } +} + +TEST_F(RocksKeeperTest, TestStorageSnapshotBroken) +{ + ChangelogDirTest test("./snapshots"); + setSnapshotDirectory("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, keeper_context); + DB::KeeperRocksStorage storage(500, "", keeper_context); + for (size_t i = 0; i < 50; ++i) + { + addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); + } + { + DB::KeeperStorageSnapshot snapshot(&storage, 50); + auto buf = manager.serializeSnapshotToBuffer(snapshot); + manager.serializeSnapshotBufferToDisk(*buf, 50); + } + EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin.zstd")); + + /// Let's corrupt file + DB::WriteBufferFromFile plain_buf( + "./snapshots/snapshot_50.bin.zstd", DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + plain_buf.truncate(34); + plain_buf.sync(); + + EXPECT_THROW(manager.restoreFromLatestSnapshot(), DB::Exception); +} + +nuraft::ptr getBufferFromZKRequest(int64_t session_id, int64_t zxid, const Coordination::ZooKeeperRequestPtr & request); + +nuraft::ptr +getLogEntryFromZKRequest(size_t term, int64_t session_id, int64_t zxid, const Coordination::ZooKeeperRequestPtr & request); + +static void testLogAndStateMachine( + DB::CoordinationSettingsPtr settings, + uint64_t total_logs, + bool enable_compression = true) +{ + using namespace Coordination; + using namespace DB; + + ChangelogDirTest snapshots("./snapshots"); + ChangelogDirTest logs("./logs"); + ChangelogDirTest rocks("./rocksdb"); + + auto get_keeper_context = [&] + { + auto local_keeper_context = std::make_shared(true, settings); + local_keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", "./snapshots")); + local_keeper_context->setLogDisk(std::make_shared("LogDisk", "./logs")); + local_keeper_context->setRocksDBDisk(std::make_shared("RocksDisk", "./rocksdb")); + local_keeper_context->setRocksDBOptions(nullptr); + return local_keeper_context; + }; + + ResponsesQueue queue(std::numeric_limits::max()); + SnapshotsQueue snapshots_queue{1}; + + auto keeper_context = get_keeper_context(); + auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); + + state_machine->init(); + DB::KeeperLogStore changelog( + DB::LogFileSettings{ + .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, + DB::FlushSettings(), + keeper_context); + changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items); + + for (size_t i = 1; i < total_logs + 1; ++i) + { + std::shared_ptr request = std::make_shared(); + request->path = "/hello_" + std::to_string(i); + auto entry = getLogEntryFromZKRequest(0, 1, i, request); + changelog.append(entry); + changelog.end_of_append_batch(0, 0); + + waitDurableLogs(changelog); + + state_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); + state_machine->commit(i, changelog.entry_at(i)->get_buf()); + bool snapshot_created = false; + if (i % settings->snapshot_distance == 0) + { + nuraft::snapshot s(i, 0, std::make_shared()); + nuraft::async_result::handler_type when_done + = [&snapshot_created](bool & ret, nuraft::ptr & /*exception*/) + { + snapshot_created = ret; + }; + + state_machine->create_snapshot(s, when_done); + CreateSnapshotTask snapshot_task; + bool pop_result = snapshots_queue.pop(snapshot_task); + EXPECT_TRUE(pop_result); + + snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); + } + + if (snapshot_created && changelog.size() > settings->reserved_log_items) + changelog.compact(i - settings->reserved_log_items); + } + + SnapshotsQueue snapshots_queue1{1}; + keeper_context = get_keeper_context(); + auto restore_machine = std::make_shared>(queue, snapshots_queue1, keeper_context, nullptr); + restore_machine->init(); + EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance); + + DB::KeeperLogStore restore_changelog( + DB::LogFileSettings{ + .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, + DB::FlushSettings(), + keeper_context); + restore_changelog.init(restore_machine->last_commit_index() + 1, settings->reserved_log_items); + + EXPECT_EQ(restore_changelog.size(), std::min(settings->reserved_log_items + total_logs % settings->snapshot_distance, total_logs)); + EXPECT_EQ(restore_changelog.next_slot(), total_logs + 1); + if (total_logs > settings->reserved_log_items + 1) + EXPECT_EQ( + restore_changelog.start_index(), total_logs - total_logs % settings->snapshot_distance - settings->reserved_log_items + 1); + else + EXPECT_EQ(restore_changelog.start_index(), 1); + + for (size_t i = restore_machine->last_commit_index() + 1; i < restore_changelog.next_slot(); ++i) + { + restore_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); + restore_machine->commit(i, changelog.entry_at(i)->get_buf()); + } + + auto & source_storage = state_machine->getStorageUnsafe(); + auto & restored_storage = restore_machine->getStorageUnsafe(); + + EXPECT_EQ(source_storage.container.size(), restored_storage.container.size()); + for (size_t i = 1; i < total_logs + 1; ++i) + { + auto path = "/hello_" + std::to_string(i); + EXPECT_EQ(source_storage.container.find(path)->value.getData(), restored_storage.container.find(path)->value.getData()); + } +} + +TEST_F(RocksKeeperTest, TestStateMachineAndLogStore) +{ + using namespace Coordination; + using namespace DB; + + { + CoordinationSettingsPtr settings = std::make_shared(); + settings->snapshot_distance = 10; + settings->reserved_log_items = 10; + settings->rotate_log_storage_interval = 10; + + testLogAndStateMachine(settings, 37); + } + { + CoordinationSettingsPtr settings = std::make_shared(); + settings->snapshot_distance = 10; + settings->reserved_log_items = 10; + settings->rotate_log_storage_interval = 10; + testLogAndStateMachine(settings, 11); + } + { + CoordinationSettingsPtr settings = std::make_shared(); + settings->snapshot_distance = 10; + settings->reserved_log_items = 10; + settings->rotate_log_storage_interval = 10; + testLogAndStateMachine(settings, 40); + } + { + CoordinationSettingsPtr settings = std::make_shared(); + settings->snapshot_distance = 10; + settings->reserved_log_items = 20; + settings->rotate_log_storage_interval = 30; + testLogAndStateMachine(settings, 40); + } + { + CoordinationSettingsPtr settings = std::make_shared(); + settings->snapshot_distance = 10; + settings->reserved_log_items = 0; + settings->rotate_log_storage_interval = 10; + testLogAndStateMachine(settings, 40); + } + { + CoordinationSettingsPtr settings = std::make_shared(); + settings->snapshot_distance = 1; + settings->reserved_log_items = 1; + settings->rotate_log_storage_interval = 32; + testLogAndStateMachine(settings, 32); + } + { + CoordinationSettingsPtr settings = std::make_shared(); + settings->snapshot_distance = 10; + settings->reserved_log_items = 7; + settings->rotate_log_storage_interval = 1; + testLogAndStateMachine(settings, 33); + } + { + CoordinationSettingsPtr settings = std::make_shared(); + settings->snapshot_distance = 37; + settings->reserved_log_items = 1000; + settings->rotate_log_storage_interval = 5000; + testLogAndStateMachine(settings, 33); + } + { + CoordinationSettingsPtr settings = std::make_shared(); + settings->snapshot_distance = 37; + settings->reserved_log_items = 1000; + settings->rotate_log_storage_interval = 5000; + testLogAndStateMachine(settings, 45); + } +} + +TEST_F(RocksKeeperTest, TestEphemeralNodeRemove) +{ + using namespace Coordination; + using namespace DB; + + ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + ResponsesQueue queue(std::numeric_limits::max()); + SnapshotsQueue snapshots_queue{1}; + + auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); + state_machine->init(); + + std::shared_ptr request_c = std::make_shared(); + request_c->path = "/hello"; + request_c->is_ephemeral = true; + auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c); + state_machine->pre_commit(1, entry_c->get_buf()); + state_machine->commit(1, entry_c->get_buf()); + const auto & storage = state_machine->getStorageUnsafe(); + + EXPECT_EQ(storage.ephemerals.size(), 1); + std::shared_ptr request_d = std::make_shared(); + request_d->path = "/hello"; + /// Delete from other session + auto entry_d = getLogEntryFromZKRequest(0, 2, state_machine->getNextZxid(), request_d); + state_machine->pre_commit(2, entry_d->get_buf()); + state_machine->commit(2, entry_d->get_buf()); + + EXPECT_EQ(storage.ephemerals.size(), 0); +} + +TEST_F(RocksKeeperTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitted) +{ + using namespace Coordination; + using namespace DB; + + ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + ResponsesQueue queue(std::numeric_limits::max()); + SnapshotsQueue snapshots_queue{1}; + + auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); + state_machine->init(); + + String user_auth_data = "test_user:test_password"; + String digest = KeeperRocksStorage::generateDigest(user_auth_data); + + std::shared_ptr auth_req = std::make_shared(); + auth_req->scheme = "digest"; + auth_req->data = user_auth_data; + + // Add auth data to the session + auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req); + state_machine->pre_commit(1, auth_entry->get_buf()); + + // Create a node with 'auth' scheme for ACL + String node_path = "/hello"; + std::shared_ptr create_req = std::make_shared(); + create_req->path = node_path; + // When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can + // create nodes with this ACL. + create_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}}; + auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); + state_machine->pre_commit(2, create_entry->get_buf()); + + const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; + ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); + + // commit log entries + state_machine->commit(1, auth_entry->get_buf()); + state_machine->commit(2, create_entry->get_buf()); + + auto node = uncommitted_state.getNode(node_path); + ASSERT_NE(node, nullptr); + auto acls = uncommitted_state.getACLs(node_path); + ASSERT_EQ(acls.size(), 1); + EXPECT_EQ(acls[0].scheme, "digest"); + EXPECT_EQ(acls[0].id, digest); + EXPECT_EQ(acls[0].permissions, 31); +} + +TEST_F(RocksKeeperTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) +{ + using namespace Coordination; + using namespace DB; + + ChangelogDirTest snapshots("./snapshots"); + setSnapshotDirectory("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + ResponsesQueue queue(std::numeric_limits::max()); + SnapshotsQueue snapshots_queue{1}; + + auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); + state_machine->init(); + + String user_auth_data = "test_user:test_password"; + String digest = KeeperRocksStorage::generateDigest(user_auth_data); + + std::shared_ptr auth_req = std::make_shared(); + auth_req->scheme = "digest"; + auth_req->data = user_auth_data; + + // Add auth data to the session + auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req); + state_machine->pre_commit(1, auth_entry->get_buf()); + + // Create a node + String node_path = "/hello"; + std::shared_ptr create_req = std::make_shared(); + create_req->path = node_path; + auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); + state_machine->pre_commit(2, create_entry->get_buf()); + + // Set ACL with 'auth' scheme for ACL + std::shared_ptr set_acl_req = std::make_shared(); + set_acl_req->path = node_path; + // When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can + // set this ACL. + set_acl_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}}; + auto set_acl_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), set_acl_req); + state_machine->pre_commit(3, set_acl_entry->get_buf()); + + // commit all entries + state_machine->commit(1, auth_entry->get_buf()); + state_machine->commit(2, create_entry->get_buf()); + state_machine->commit(3, set_acl_entry->get_buf()); + + const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; + auto node = uncommitted_state.getNode(node_path); + + ASSERT_NE(node, nullptr); + auto acls = uncommitted_state.getACLs(node_path); + ASSERT_EQ(acls.size(), 1); + EXPECT_EQ(acls[0].scheme, "digest"); + EXPECT_EQ(acls[0].id, digest); + EXPECT_EQ(acls[0].permissions, 31); +} + +TEST_F(RocksKeeperTest, TestStorageSnapshotEqual) +{ + ChangelogDirTest test("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setSnapshotDirectory("./snapshots"); + setRocksDBDirectory("./rocksdb"); + + std::optional snapshot_hash; + for (size_t i = 0; i < 15; ++i) + { + DB::KeeperSnapshotManager manager(3, keeper_context); + + DB::KeeperRocksStorage storage(500, "", keeper_context); + addNode(storage, "/hello", ""); + for (size_t j = 0; j < 100; ++j) + { + addNode(storage, "/hello_" + std::to_string(j), "world", 1); + addNode(storage, "/hello/somepath_" + std::to_string(j), "somedata", 3); + } + + storage.session_id_counter = 5; + + storage.ephemerals[3] = {"/hello"}; + storage.ephemerals[1] = {"/hello/somepath"}; + + for (size_t j = 0; j < 3333; ++j) + storage.getSessionID(130 * j); + + DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); + + auto buf = manager.serializeSnapshotToBuffer(snapshot); + + auto new_hash = sipHash128(reinterpret_cast(buf->data()), buf->size()); + if (!snapshot_hash.has_value()) + { + snapshot_hash = new_hash; + } + else + { + EXPECT_EQ(*snapshot_hash, new_hash); + } + } +} + +TEST_F(RocksKeeperTest, TestStorageSnapshotDifferentCompressions) +{ + ChangelogDirTest test("./snapshots"); + setSnapshotDirectory("./snapshots"); + ChangelogDirTest rocks("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, keeper_context); + + DB::KeeperRocksStorage storage(500, "", keeper_context); + addNode(storage, "/hello1", "world", 1); + addNode(storage, "/hello2", "somedata", 3); + storage.session_id_counter = 5; + storage.zxid = 2; + storage.ephemerals[3] = {"/hello2"}; + storage.ephemerals[1] = {"/hello1"}; + storage.getSessionID(130); + storage.getSessionID(130); + + DB::KeeperStorageSnapshot snapshot(&storage, 2); + + auto buf = manager.serializeSnapshotToBuffer(snapshot); + manager.serializeSnapshotBufferToDisk(*buf, 2); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin.zstd")); + + DB::KeeperSnapshotManager new_manager(3, keeper_context, false); + + auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); + + auto [restored_storage, snapshot_meta, _] = new_manager.deserializeSnapshotFromBuffer(debuf); + + EXPECT_EQ(restored_storage->container.size(), 6); + EXPECT_EQ(restored_storage->container.find("/")->value.numChildren(), 3); + EXPECT_EQ(restored_storage->container.find("/hello1")->value.numChildren(), 0); + EXPECT_EQ(restored_storage->container.find("/hello2")->value.numChildren(), 0); + + EXPECT_EQ(restored_storage->container.find("/")->value.getData(), ""); + EXPECT_EQ(restored_storage->container.find("/hello1")->value.getData(), "world"); + EXPECT_EQ(restored_storage->container.find("/hello2")->value.getData(), "somedata"); + EXPECT_EQ(restored_storage->session_id_counter, 7); + EXPECT_EQ(restored_storage->zxid, 2); + EXPECT_EQ(restored_storage->ephemerals.size(), 2); + EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); + EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); + EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); +} + +template +ResponseType getSingleResponse(const auto & responses) +{ + EXPECT_FALSE(responses.empty()); + return dynamic_cast(*responses[0].response); +} + +TEST_F(RocksKeeperTest, TestUncommittedStateBasicCrud) +{ + ChangelogDirTest test("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + using namespace DB; + using namespace Coordination; + + DB::KeeperRocksStorage storage{500, "", keeper_context}; + + constexpr std::string_view path = "/test"; + + const auto get_committed_data = [&]() -> std::optional + { + auto request = std::make_shared(); + request->path = path; + auto responses = storage.processRequest(request, 0, std::nullopt, true, true); + const auto & get_response = getSingleResponse(responses); + + if (get_response.error != Error::ZOK) + return std::nullopt; + + return get_response.data; + }; + + const auto preprocess_get = [&](int64_t zxid) + { + auto get_request = std::make_shared(); + get_request->path = path; + storage.preprocessRequest(get_request, 0, 0, zxid); + return get_request; + }; + + const auto create_request = std::make_shared(); + create_request->path = path; + create_request->data = "initial_data"; + storage.preprocessRequest(create_request, 0, 0, 1); + storage.preprocessRequest(create_request, 0, 0, 2); + + ASSERT_FALSE(get_committed_data()); + + const auto after_create_get = preprocess_get(3); + + ASSERT_FALSE(get_committed_data()); + + const auto set_request = std::make_shared(); + set_request->path = path; + set_request->data = "new_data"; + storage.preprocessRequest(set_request, 0, 0, 4); + + const auto after_set_get = preprocess_get(5); + + ASSERT_FALSE(get_committed_data()); + + const auto remove_request = std::make_shared(); + remove_request->path = path; + storage.preprocessRequest(remove_request, 0, 0, 6); + storage.preprocessRequest(remove_request, 0, 0, 7); + + const auto after_remove_get = preprocess_get(8); + + ASSERT_FALSE(get_committed_data()); + + { + const auto responses = storage.processRequest(create_request, 0, 1); + const auto & create_response = getSingleResponse(responses); + ASSERT_EQ(create_response.error, Error::ZOK); + } + + { + const auto responses = storage.processRequest(create_request, 0, 2); + const auto & create_response = getSingleResponse(responses); + ASSERT_EQ(create_response.error, Error::ZNODEEXISTS); + } + + { + const auto responses = storage.processRequest(after_create_get, 0, 3); + const auto & get_response = getSingleResponse(responses); + ASSERT_EQ(get_response.error, Error::ZOK); + ASSERT_EQ(get_response.data, "initial_data"); + } + + ASSERT_EQ(get_committed_data(), "initial_data"); + + { + const auto responses = storage.processRequest(set_request, 0, 4); + const auto & create_response = getSingleResponse(responses); + ASSERT_EQ(create_response.error, Error::ZOK); + } + + { + const auto responses = storage.processRequest(after_set_get, 0, 5); + const auto & get_response = getSingleResponse(responses); + ASSERT_EQ(get_response.error, Error::ZOK); + ASSERT_EQ(get_response.data, "new_data"); + } + + ASSERT_EQ(get_committed_data(), "new_data"); + + { + const auto responses = storage.processRequest(remove_request, 0, 6); + const auto & create_response = getSingleResponse(responses); + ASSERT_EQ(create_response.error, Error::ZOK); + } + + { + const auto responses = storage.processRequest(remove_request, 0, 7); + const auto & remove_response = getSingleResponse(responses); + ASSERT_EQ(remove_response.error, Error::ZNONODE); + } + + { + const auto responses = storage.processRequest(after_remove_get, 0, 8); + const auto & get_response = getSingleResponse(responses); + ASSERT_EQ(get_response.error, Error::ZNONODE); + } + + ASSERT_FALSE(get_committed_data()); +} + +TEST_F(RocksKeeperTest, TestListRequestTypes) +{ + ChangelogDirTest test("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + using namespace DB; + using namespace Coordination; + KeeperRocksStorage storage{500, "", keeper_context}; + + int32_t zxid = 0; + + static constexpr std::string_view test_path = "/list_request_type/node"; + + const auto create_path = [&](const auto & path, bool is_ephemeral, bool is_sequential = true) + { + const auto create_request = std::make_shared(); + int new_zxid = ++zxid; + create_request->path = path; + create_request->is_sequential = is_sequential; + create_request->is_ephemeral = is_ephemeral; + storage.preprocessRequest(create_request, 1, 0, new_zxid); + auto responses = storage.processRequest(create_request, 1, new_zxid); + + EXPECT_GE(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path; + const auto & create_response = dynamic_cast(*responses[0].response); + return create_response.path_created; + }; + + create_path(parentNodePath(StringRef{test_path}).toString(), false, false); + + static constexpr size_t persistent_num = 5; + std::unordered_set expected_persistent_children; + for (size_t i = 0; i < persistent_num; ++i) + { + expected_persistent_children.insert(getBaseNodeName(create_path(test_path, false)).toString()); + } + ASSERT_EQ(expected_persistent_children.size(), persistent_num); + + static constexpr size_t ephemeral_num = 5; + std::unordered_set expected_ephemeral_children; + for (size_t i = 0; i < ephemeral_num; ++i) + { + expected_ephemeral_children.insert(getBaseNodeName(create_path(test_path, true)).toString()); + } + ASSERT_EQ(expected_ephemeral_children.size(), ephemeral_num); + + const auto get_children = [&](const auto list_request_type) + { + const auto list_request = std::make_shared(); + int new_zxid = ++zxid; + list_request->path = parentNodePath(StringRef{test_path}).toString(); + list_request->list_request_type = list_request_type; + storage.preprocessRequest(list_request, 1, 0, new_zxid); + auto responses = storage.processRequest(list_request, 1, new_zxid); + + EXPECT_GE(responses.size(), 1); + const auto & list_response = dynamic_cast(*responses[0].response); + EXPECT_EQ(list_response.error, Coordination::Error::ZOK); + return list_response.names; + }; + + const auto persistent_children = get_children(ListRequestType::PERSISTENT_ONLY); + EXPECT_EQ(persistent_children.size(), persistent_num); + for (const auto & child : persistent_children) + { + EXPECT_TRUE(expected_persistent_children.contains(child)) << "Missing persistent child " << child; + } + + const auto ephemeral_children = get_children(ListRequestType::EPHEMERAL_ONLY); + EXPECT_EQ(ephemeral_children.size(), ephemeral_num); + for (const auto & child : ephemeral_children) + { + EXPECT_TRUE(expected_ephemeral_children.contains(child)) << "Missing ephemeral child " << child; + } + + const auto all_children = get_children(ListRequestType::ALL); + EXPECT_EQ(all_children.size(), ephemeral_num + persistent_num); + for (const auto & child : all_children) + { + EXPECT_TRUE(expected_ephemeral_children.contains(child) || expected_persistent_children.contains(child)) + << "Missing child " << child; + } +} + +TEST_F(RocksKeeperTest, TestFeatureFlags) +{ + ChangelogDirTest test("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + using namespace Coordination; + KeeperMemoryStorage storage{500, "", keeper_context}; + auto request = std::make_shared(); + request->path = DB::keeper_api_feature_flags_path; + auto responses = storage.processRequest(request, 0, std::nullopt, true, true); + const auto & get_response = getSingleResponse(responses); + DB::KeeperFeatureFlags feature_flags; + feature_flags.setFeatureFlags(get_response.data); + ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::FILTERED_LIST)); + ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ)); + ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS)); +} + +TEST_F(RocksKeeperTest, TestSystemNodeModify) +{ + ChangelogDirTest test("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + using namespace Coordination; + int64_t zxid{0}; + + // On INIT we abort when a system path is modified + keeper_context->setServerState(KeeperContext::Phase::RUNNING); + KeeperRocksStorage storage{500, "", keeper_context}; + const auto assert_create = [&](const std::string_view path, const auto expected_code) + { + auto request = std::make_shared(); + request->path = path; + storage.preprocessRequest(request, 0, 0, zxid); + auto responses = storage.processRequest(request, 0, zxid); + ASSERT_FALSE(responses.empty()); + + const auto & response = responses[0]; + ASSERT_EQ(response.response->error, expected_code) << "Unexpected error for path " << path; + + ++zxid; + }; + + assert_create("/keeper", Error::ZBADARGUMENTS); + assert_create("/keeper/with_child", Error::ZBADARGUMENTS); + assert_create(DB::keeper_api_version_path, Error::ZBADARGUMENTS); + + assert_create("/keeper_map", Error::ZOK); + assert_create("/keeper1", Error::ZOK); + assert_create("/keepe", Error::ZOK); + assert_create("/keeper1/test", Error::ZOK); +} + +TEST_F(RocksKeeperTest, TestCheckNotExistsRequest) +{ + ChangelogDirTest test("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + using namespace DB; + using namespace Coordination; + + KeeperRocksStorage storage{500, "", keeper_context}; + + int32_t zxid = 0; + + const auto create_path = [&](const auto & path) + { + const auto create_request = std::make_shared(); + int new_zxid = ++zxid; + create_request->path = path; + storage.preprocessRequest(create_request, 1, 0, new_zxid); + auto responses = storage.processRequest(create_request, 1, new_zxid); + + EXPECT_GE(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path; + }; + + const auto check_request = std::make_shared(); + check_request->path = "/test_node"; + check_request->not_exists = true; + + { + SCOPED_TRACE("CheckNotExists returns ZOK"); + int new_zxid = ++zxid; + storage.preprocessRequest(check_request, 1, 0, new_zxid); + auto responses = storage.processRequest(check_request, 1, new_zxid); + EXPECT_GE(responses.size(), 1); + auto error = responses[0].response->error; + EXPECT_EQ(error, Coordination::Error::ZOK) << "CheckNotExists returned invalid result: " << errorMessage(error); + } + + 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; + + { + SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS"); + int new_zxid = ++zxid; + storage.preprocessRequest(check_request, 1, 0, new_zxid); + auto responses = storage.processRequest(check_request, 1, new_zxid); + EXPECT_GE(responses.size(), 1); + auto error = responses[0].response->error; + EXPECT_EQ(error, Coordination::Error::ZNODEEXISTS) << "CheckNotExists returned invalid result: " << errorMessage(error); + } + + { + SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS for same version"); + int new_zxid = ++zxid; + check_request->version = node_version; + storage.preprocessRequest(check_request, 1, 0, new_zxid); + auto responses = storage.processRequest(check_request, 1, new_zxid); + EXPECT_GE(responses.size(), 1); + auto error = responses[0].response->error; + EXPECT_EQ(error, Coordination::Error::ZNODEEXISTS) << "CheckNotExists returned invalid result: " << errorMessage(error); + } + + { + SCOPED_TRACE("CheckNotExists returns ZOK for different version"); + int new_zxid = ++zxid; + check_request->version = node_version + 1; + storage.preprocessRequest(check_request, 1, 0, new_zxid); + auto responses = storage.processRequest(check_request, 1, new_zxid); + EXPECT_GE(responses.size(), 1); + auto error = responses[0].response->error; + EXPECT_EQ(error, Coordination::Error::ZOK) << "CheckNotExists returned invalid result: " << errorMessage(error); + } +} + +TEST_F(RocksKeeperTest, TestReapplyingDeltas) +{ + ChangelogDirTest test("./rocksdb"); + setRocksDBDirectory("./rocksdb"); + + using namespace DB; + using namespace Coordination; + + static constexpr int64_t initial_zxid = 100; + + const auto create_request = std::make_shared(); + create_request->path = "/test/data"; + create_request->is_sequential = true; + + const auto process_create = [](KeeperRocksStorage & storage, const auto & request, int64_t zxid) + { + storage.preprocessRequest(request, 1, 0, zxid); + auto responses = storage.processRequest(request, 1, zxid); + EXPECT_GE(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Error::ZOK); + }; + + const auto commit_initial_data = [&](auto & storage) + { + int64_t zxid = 1; + + const auto root_create = std::make_shared(); + root_create->path = "/test"; + process_create(storage, root_create, zxid); + ++zxid; + + for (; zxid <= initial_zxid; ++zxid) + process_create(storage, create_request, zxid); + }; + + KeeperRocksStorage storage1{500, "", keeper_context}; + commit_initial_data(storage1); + + for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) + storage1.preprocessRequest(create_request, 1, 0, zxid, /*check_acl=*/true, /*digest=*/std::nullopt, /*log_idx=*/zxid); + + /// create identical new storage + KeeperRocksStorage storage2{500, "", keeper_context}; + commit_initial_data(storage2); + + storage1.applyUncommittedState(storage2, initial_zxid); + + const auto commit_unprocessed = [&](KeeperRocksStorage & storage) + { + for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) + { + auto responses = storage.processRequest(create_request, 1, zxid); + EXPECT_GE(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Error::ZOK); + } + }; + + commit_unprocessed(storage1); + commit_unprocessed(storage2); + + const auto get_children = [&](KeeperRocksStorage & storage) + { + const auto list_request = std::make_shared(); + list_request->path = "/test"; + auto responses = storage.processRequest(list_request, 1, std::nullopt, /*check_acl=*/true, /*is_local=*/true); + EXPECT_EQ(responses.size(), 1); + const auto * list_response = dynamic_cast(responses[0].response.get()); + EXPECT_TRUE(list_response); + return list_response->names; + }; + + auto children1 = get_children(storage1); + std::unordered_set children1_set(children1.begin(), children1.end()); + + auto children2 = get_children(storage2); + std::unordered_set children2_set(children2.begin(), children2.end()); + + ASSERT_TRUE(children1_set == children2_set); +} + +#endif From 96f35f9a4746654ace7bb5a54fef8f503be9ddcb Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 4 Apr 2024 18:32:53 +0200 Subject: [PATCH 19/79] fix style --- src/Coordination/RocksDBContainer.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index 683678cfcd1..4076e39655a 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -33,6 +33,7 @@ struct RocksDBContainer using Node = Node_; private: + /// this is only for debug. std::string toHex(std::string origin) { auto toHexChar = [](char v) @@ -43,7 +44,8 @@ private: }; std::string result; result.reserve(2 * origin.size()); - for (size_t i = 0; i < origin.size(); ++i) { + for (size_t i = 0; i < origin.size(); ++i) + { unsigned char c = origin[i]; result.push_back(toHexChar(c >> 4)); result.push_back(toHexChar(c & 0xf)); From a63ee58fc4cea606cececc8edbc2bec58649861d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 5 Apr 2024 12:16:40 +0200 Subject: [PATCH 20/79] fix special build --- src/Coordination/RocksDBContainer.h | 3 +-- src/Coordination/tests/gtest_rocks_keeper.cpp | 8 ++++---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index 4076e39655a..9d60a49ba23 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -44,9 +44,8 @@ private: }; std::string result; result.reserve(2 * origin.size()); - for (size_t i = 0; i < origin.size(); ++i) + for (unsigned char c : origin) { - unsigned char c = origin[i]; result.push_back(toHexChar(c >> 4)); result.push_back(toHexChar(c & 0xf)); } diff --git a/src/Coordination/tests/gtest_rocks_keeper.cpp b/src/Coordination/tests/gtest_rocks_keeper.cpp index 15f97676db3..d42eb66d683 100644 --- a/src/Coordination/tests/gtest_rocks_keeper.cpp +++ b/src/Coordination/tests/gtest_rocks_keeper.cpp @@ -1,4 +1,8 @@ #include +#include "config.h" + +#if USE_NURAFT and USE_ROCKSDB + #include #include #include @@ -10,10 +14,6 @@ #include #include -#include "config.h" - -#if USE_NURAFT and USE_ROCKSDB - namespace fs = std::filesystem; struct ChangelogDirTest { From 55f4235d834301a6acd77964e9601a09e4b7e2ba Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 5 Apr 2024 14:59:05 +0200 Subject: [PATCH 21/79] enable rocksdb and check ci --- src/Coordination/KeeperContext.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 4ae3c295824..949e699f436 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -160,8 +160,11 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, initializeDisks(config); #if USE_ROCKSDB - if (config.getBool("keeper_server.coordination_settings.use_rocksdb", false)) + if (config.getBool("keeper_server.coordination_settings.use_rocksdb", true)) + { rocksdb_options = std::make_shared(getRocksDBOptionsFromConfig(config)); + digest_enabled = false; /// TODO: support digest + } #endif } From e93872c826be150291115b6f4fb81a2fe6000384 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 6 Apr 2024 18:22:08 +0200 Subject: [PATCH 22/79] try test --- src/Coordination/CoordinationSettings.h | 2 +- src/Coordination/KeeperServer.cpp | 7 +++---- src/Coordination/KeeperSnapshotManagerS3.cpp | 2 -- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index c5bd672055f..4a898d1975c 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -55,7 +55,7 @@ struct Settings; M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests, set to 0 to disable", 0) \ M(UInt64, raft_limits_reconnect_limit, 50, "If connection to a peer is silent longer than this limit * (multiplied by heartbeat interval), we re-establish the connection.", 0) \ M(Bool, async_replication, false, "Enable async replication. All write and read guarantees are preserved while better performance is achieved. Settings is disabled by default to not break backwards compatibility.", 0) \ - M(Bool, use_rocksdb, false, "Use rocksdb as backend storage", 0) \ + M(Bool, use_rocksdb, true, "Use rocksdb as backend storage", 0) \ M(UInt64, latest_logs_cache_size_threshold, 1 * 1024 * 1024 * 1024, "Maximum total size of in-memory cache of latest log entries.", 0) \ M(UInt64, commit_logs_cache_size_threshold, 500 * 1024 * 1024, "Maximum total size of in-memory cache of log entries needed next for commit.", 0) \ M(UInt64, disk_move_retries_wait_ms, 1000, "How long to wait between retries after a failure which happened while a file was being moved between disks.", 0) \ diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 43688ca9cda..951d3729289 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -131,10 +131,10 @@ KeeperServer::KeeperServer( if (keeper_context->getCoordinationSettings()->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); +#if USE_ROCKSDB const auto & coordination_settings = keeper_context->getCoordinationSettings(); if (coordination_settings->use_rocksdb) { -#if USE_ROCKSDB state_machine = nuraft::cs_new>( responses_queue_, snapshots_queue_, @@ -142,11 +142,10 @@ KeeperServer::KeeperServer( config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, commit_callback, checkAndGetSuperdigest(configuration_and_settings_->super_digest)); -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "RocksDB support is disabled because ClickHouse was built without RocksDB support."); -#endif + LOG_WARNING(log, "Use RocksDB as Keeper backend storage."); } else +#endif state_machine = nuraft::cs_new>( responses_queue_, snapshots_queue_, diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index b984b8ad18e..b54e2ceaa15 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -25,8 +25,6 @@ #include -namespace fs = std::filesystem; - namespace DB { From 801dcca2e62e68d8e6110a8ef33fc22b451913ce Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 9 Apr 2024 14:56:26 +0200 Subject: [PATCH 23/79] fix tests --- src/Coordination/KeeperContext.cpp | 2 +- src/Coordination/RocksDBContainer.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index 949e699f436..b65bb02ca5d 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -123,7 +123,7 @@ KeeperContext::Storage KeeperContext::getRocksDBPathFromConfig(const Poco::Util: const auto create_local_disk = [](const auto & path) { if (fs::exists(path)) - fs::remove(path); + fs::remove_all(path); fs::create_directories(path); return std::make_shared("LocalRocksDBDisk", path); diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index 9d60a49ba23..a3ff5143c6a 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -353,7 +353,7 @@ public: void insertOrReplace(KeyPtr key_data, size_t key_size, Node value) { - std::string key(key_data.release(), key_size); + std::string key(key_data.get(), key_size); insertOrReplace(key, value); } From a966a94de6665095991064455470eb3a69d18c1c Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 10 Apr 2024 15:04:31 +0200 Subject: [PATCH 24/79] fix a bug in get children --- src/Coordination/RocksDBContainer.h | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index a3ff5143c6a..30950248061 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -218,14 +218,15 @@ public: } } - std::vector> getChildren(const std::string & key) + std::vector> getChildren(const std::string & key_) { rocksdb::ReadOptions read_options; read_options.total_order_seek = true; - size_t len = key.size() + 2; + std::string key = key_; if (!key.ends_with('/')) - len ++; + key += '/'; + size_t len = key.size() + 2; auto iter = std::unique_ptr(rocksdb_ptr->NewIterator(read_options)); std::string encoded_string = getEncodedKey(key, true); From b5e74548b3c43116d900ee4ed573ed969c10678d Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 10 Apr 2024 15:22:57 +0200 Subject: [PATCH 25/79] fix clang tidy --- src/Coordination/KeeperSnapshotManagerS3.cpp | 2 ++ src/Coordination/RocksDBContainer.h | 4 +--- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index b54e2ceaa15..b984b8ad18e 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -25,6 +25,8 @@ #include +namespace fs = std::filesystem; + namespace DB { diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index 30950248061..ba0f7f74b4e 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -21,8 +21,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace fs = std::filesystem; - /// The key-value format of rocks db will be /// - key: Int8 (depth of the path) + String (path) /// - value: SizeOf(keeperRocksNodeInfo) (meta of the node) + String (data) @@ -214,7 +212,7 @@ public: rocksdb_ptr->Close(); rocksdb_ptr = nullptr; - fs::remove_all(rocksdb_dir); + std::filesystem::remove_all(rocksdb_dir); } } From 91ca441eed9962537ad3f3d5897810e9e876cb49 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 3 May 2024 17:57:35 +0200 Subject: [PATCH 26/79] address part of comments --- src/Coordination/RocksDBContainer.h | 22 +--------------------- 1 file changed, 1 insertion(+), 21 deletions(-) diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index ba0f7f74b4e..75cc2a9e555 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -31,24 +31,6 @@ struct RocksDBContainer using Node = Node_; private: - /// this is only for debug. - std::string toHex(std::string origin) - { - auto toHexChar = [](char v) - { - if (v <= 9) - return v + '0'; - return 'A' + v - 10; - }; - std::string result; - result.reserve(2 * origin.size()); - for (unsigned char c : origin) - { - result.push_back(toHexChar(c >> 4)); - result.push_back(toHexChar(c & 0xf)); - } - return result; - } UInt16 getKeyDepth(const std::string & key) { @@ -184,13 +166,11 @@ public: if (disk == nullptr) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get rocksdb disk"); - return; } auto options = context->getRocksDBOptions(); if (options == nullptr) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get rocksdb options"); - return; } rocksdb_dir = disk->getPath(); rocksdb::DB * db; @@ -365,7 +345,7 @@ public: return false; if (status.ok()) { - counter --; + counter--; return true; } throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during erase. The error message is {}.", status.ToString()); From aa41ad12f66a56cb61b9a71bc8add9da20179808 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 6 May 2024 15:38:29 +0200 Subject: [PATCH 27/79] address comments --- src/Coordination/KeeperContext.h | 2 +- src/Coordination/KeeperStorage.cpp | 84 +- src/Coordination/KeeperStorage.h | 7 + src/Coordination/RocksDBContainer.h | 21 + src/Coordination/tests/gtest_coordination.cpp | 1012 ++++++++------- src/Coordination/tests/gtest_rocks_keeper.cpp | 1150 ----------------- 6 files changed, 627 insertions(+), 1649 deletions(-) delete mode 100644 src/Coordination/tests/gtest_rocks_keeper.cpp diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index c9abd13b6e9..38013725f56 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -70,7 +70,7 @@ public: void setRocksDBDisk(DiskPtr disk); DiskPtr getTemporaryRocksDBDisk() const; - void setRocksDBOptions(std::shared_ptr rocksdb_options_); + void setRocksDBOptions(std::shared_ptr rocksdb_options_ = nullptr); std::shared_ptr getRocksDBOptions() const { return rocksdb_options; } UInt64 getKeeperMemorySoftLimit() const { return memory_soft_limit; } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 905fd06359c..73d735c0af8 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1745,63 +1745,53 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc if (path_prefix.empty()) throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty"); - if constexpr (Storage::use_rocksdb) + const auto & get_children = [&]() { - const auto & children = container.getChildren(request.path); - response.names.reserve(children.size()); - const auto add_child = [&](const auto & child) + if constexpr (Storage::use_rocksdb) + return container.getChildren(request.path); + else + return node_it->value.getChildren(); + }; + const auto & children = get_children(); + response.names.reserve(children.size()); + + const auto add_child = [&](const auto & child) + { + using enum Coordination::ListRequestType; + + auto list_request_type = ALL; + if (auto * filtered_list = dynamic_cast(&request)) { - 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; - - const auto is_ephemeral = child.second.isEphemeral(); - return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); - }; - - for (const auto & child : children) - { - if (add_child(child)) - response.names.push_back(child.first); + list_request_type = filtered_list->list_request_type; } - } - else - { - const auto & children = node_it->value.getChildren(); - response.names.reserve(children.size()); - const auto add_child = [&](const auto child) + if (list_request_type == ALL) + return true; + + bool is_ephemeral; + if constexpr (!Storage::use_rocksdb) { - 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; - 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(); - - const auto is_ephemeral = child_it->value.isEphemeral(); - return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); - }; - - for (const auto child : children) + is_ephemeral = child_it->value.isEphemeral(); + } + else { - if (add_child(child)) + is_ephemeral = child.second.isEphemeral(); + } + + return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); + }; + + for (const auto & child : children) + { + if (add_child(child)) + { + if constexpr (Storage::use_rocksdb) + response.names.push_back(child.first); + else response.names.push_back(child.toString()); } } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 1d47061a17c..c2fd196b95e 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -93,6 +93,13 @@ struct KeeperRocksNodeInfo ephemeral_or_children_data.children_info.num_children = num_children; } + /// dummy interface for test + void addChild(StringRef) {} + auto getChildren() const + { + return std::vector(numChildren()); + } + void increaseNumChildren() { chassert(!isEphemeral()); diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index 75cc2a9e555..a4a236f332e 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -31,6 +31,20 @@ struct RocksDBContainer using Node = Node_; private: + /// MockNode is only use in test to mock `getChildren()` and `getData()` + struct MockNode + { + std::vector children; + std::string data; + MockNode(size_t children_num, std::string_view data_) + : children(std::vector(children_num)), + data(data_) + { + } + + std::vector getChildren() { return children; } + std::string getData() { return data; } + }; UInt16 getKeyDepth(const std::string & key) { @@ -262,6 +276,13 @@ public: return const_iterator(kv); } + MockNode getValue(StringRef key) + { + auto it = find(key); + chassert(it != end()); + return MockNode(it->value.numChildren(), it->value.getData()); + } + const_iterator updateValue(StringRef key_, ValueUpdater updater) { /// rocksdb::PinnableSlice slice; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index c303db11474..9f0937572a9 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -60,10 +60,22 @@ struct CompressionParam std::string extension; }; -class CoordinationTest : public ::testing::TestWithParam +template +struct TestParam { -protected: - DB::KeeperContextPtr keeper_context = std::make_shared(true, std::make_shared()); + using Storage = Storage_; + static constexpr bool enable_compression = enable_compression_; +}; + +template +class CoordinationTest : public ::testing::Test +{ +public: + using Storage = typename TestType::Storage; + static constexpr bool enable_compression = TestType::enable_compression; + std::string extension; + + DB::KeeperContextPtr keeper_context; LoggerPtr log{getLogger("CoordinationTest")}; void SetUp() override @@ -72,7 +84,12 @@ protected: Poco::Logger::root().setChannel(channel); Poco::Logger::root().setLevel("trace"); + auto settings = std::make_shared(); + settings->use_rocksdb = true; + keeper_context = std::make_shared(true, settings); keeper_context->setLocalLogsPreprocessed(); + keeper_context->setRocksDBOptions(); + extension = enable_compression ? ".zstd" : ""; } void setLogDirectory(const std::string & path) { keeper_context->setLogDisk(std::make_shared("LogDisk", path)); } @@ -82,13 +99,24 @@ protected: keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", path)); } + void setRocksDBDirectory(const std::string & path) + { + keeper_context->setRocksDBDisk(std::make_shared("RocksDisk", path)); + } + void setStateFileDirectory(const std::string & path) { keeper_context->setStateFileDisk(std::make_shared("StateFile", path)); } }; -TEST_P(CoordinationTest, RaftServerConfigParse) +using Implementation = testing::Types, + TestParam, + TestParam, + TestParam>; +TYPED_TEST_SUITE(CoordinationTest, Implementation); + +TYPED_TEST(CoordinationTest, RaftServerConfigParse) { auto parse = Coordination::RaftServerConfig::parse; using Cfg = std::optional; @@ -113,7 +141,7 @@ TEST_P(CoordinationTest, RaftServerConfigParse) (Cfg{{1, "2001:0db8:85a3:0000:0000:8a2e:0370:7334:80"}})); } -TEST_P(CoordinationTest, RaftServerClusterConfigParse) +TYPED_TEST(CoordinationTest, RaftServerClusterConfigParse) { auto parse = Coordination::parseRaftServers; using Cfg = DB::RaftServerConfig; @@ -129,14 +157,14 @@ TEST_P(CoordinationTest, RaftServerClusterConfigParse) (Servers{Cfg{1, "host:80"}, Cfg{2, "host:81"}})); } -TEST_P(CoordinationTest, BuildTest) +TYPED_TEST(CoordinationTest, BuildTest) { DB::InMemoryLogStore store; DB::SummingStateMachine machine; EXPECT_EQ(1, 1); } -TEST_P(CoordinationTest, BufferSerde) +TYPED_TEST(CoordinationTest, BufferSerde) { Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Get); request->xid = 3; @@ -260,13 +288,13 @@ nuraft::ptr getBuffer(int64_t number) return ret; } -TEST_P(CoordinationTest, TestSummingRaft1) +TYPED_TEST(CoordinationTest, TestSummingRaft1) { ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); - setStateFileDirectory("."); + this->setLogDirectory("./logs"); + this->setStateFileDirectory("."); - SummingRaftServer s1(1, "localhost", 44444, keeper_context); + SummingRaftServer s1(1, "localhost", 44444, this->keeper_context); SCOPE_EXIT(if (std::filesystem::exists("./state")) std::filesystem::remove("./state");); /// Single node is leader @@ -279,7 +307,7 @@ TEST_P(CoordinationTest, TestSummingRaft1) while (s1.state_machine->getValue() != 143) { - LOG_INFO(log, "Waiting s1 to apply entry"); + LOG_INFO(this->log, "Waiting s1 to apply entry"); std::this_thread::sleep_for(std::chrono::milliseconds(100)); } @@ -295,16 +323,16 @@ DB::LogEntryPtr getLogEntry(const std::string & s, size_t term) return nuraft::cs_new(term, bufwriter.getBuffer()); } -TEST_P(CoordinationTest, ChangelogTestSimple) +TYPED_TEST(CoordinationTest, ChangelogTestSimple) { - auto params = GetParam(); + /// ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -327,16 +355,15 @@ void waitDurableLogs(nuraft::log_store & log_store) } -TEST_P(CoordinationTest, ChangelogTestFile) +TYPED_TEST(CoordinationTest, ChangelogTestFile) { - auto params = GetParam(); ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -344,9 +371,9 @@ TEST_P(CoordinationTest, ChangelogTestFile) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); for (const auto & p : fs::directory_iterator("./logs")) - EXPECT_EQ(p.path(), "./logs/changelog_1_5.bin" + params.extension); + EXPECT_EQ(p.path(), "./logs/changelog_1_5.bin" + this->extension); changelog.append(entry); changelog.append(entry); @@ -357,20 +384,20 @@ TEST_P(CoordinationTest, ChangelogTestFile) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); } -TEST_P(CoordinationTest, ChangelogReadWrite) +TYPED_TEST(CoordinationTest, ChangelogReadWrite) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 1000}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) @@ -385,9 +412,9 @@ TEST_P(CoordinationTest, ChangelogReadWrite) waitDurableLogs(changelog); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 1000}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 10); EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term()); @@ -403,16 +430,16 @@ TEST_P(CoordinationTest, ChangelogReadWrite) EXPECT_EQ(10, entries_from_range->size()); } -TEST_P(CoordinationTest, ChangelogWriteAt) +TYPED_TEST(CoordinationTest, ChangelogWriteAt) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 1000}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -435,9 +462,9 @@ TEST_P(CoordinationTest, ChangelogWriteAt) EXPECT_EQ(changelog.next_slot(), 8); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1000}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 1000}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), changelog.size()); @@ -447,16 +474,16 @@ TEST_P(CoordinationTest, ChangelogWriteAt) } -TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) +TYPED_TEST(CoordinationTest, ChangelogTestAppendAfterRead) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 7; ++i) { @@ -469,13 +496,13 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 7); @@ -488,8 +515,8 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_EQ(changelog_reader.size(), 10); waitDurableLogs(changelog_reader); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); size_t logs_count = 0; for (const auto & _ [[maybe_unused]] : fs::directory_iterator("./logs")) @@ -504,9 +531,9 @@ TEST_P(CoordinationTest, ChangelogTestAppendAfterRead) waitDurableLogs(changelog_reader); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); logs_count = 0; for (const auto & _ [[maybe_unused]] : fs::directory_iterator("./logs")) @@ -533,16 +560,16 @@ void assertFileDeleted(std::string path) } -TEST_P(CoordinationTest, ChangelogTestCompaction) +TYPED_TEST(CoordinationTest, ChangelogTestCompaction) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 3; ++i) @@ -556,7 +583,7 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) EXPECT_EQ(changelog.size(), 3); - keeper_context->setLastCommitIndex(2); + this->keeper_context->setLastCommitIndex(2); changelog.compact(2); EXPECT_EQ(changelog.size(), 1); @@ -564,7 +591,7 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) EXPECT_EQ(changelog.next_slot(), 4); EXPECT_EQ(changelog.last_entry()->get_term(), 20); // nothing should be deleted - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); auto e1 = getLogEntry("hello world", 30); changelog.append(e1); @@ -578,15 +605,15 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); - keeper_context->setLastCommitIndex(6); + this->keeper_context->setLastCommitIndex(6); changelog.compact(6); std::this_thread::sleep_for(std::chrono::microseconds(1000)); - assertFileDeleted("./logs/changelog_1_5.bin" + params.extension); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); + assertFileDeleted("./logs/changelog_1_5.bin" + this->extension); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); EXPECT_EQ(changelog.size(), 1); EXPECT_EQ(changelog.start_index(), 7); @@ -594,9 +621,9 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) EXPECT_EQ(changelog.last_entry()->get_term(), 60); /// And we able to read it DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader.init(7, 0); EXPECT_EQ(changelog_reader.size(), 1); @@ -605,16 +632,16 @@ TEST_P(CoordinationTest, ChangelogTestCompaction) EXPECT_EQ(changelog_reader.last_entry()->get_term(), 60); } -TEST_P(CoordinationTest, ChangelogTestBatchOperations) +TYPED_TEST(CoordinationTest, ChangelogTestBatchOperations) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -630,9 +657,9 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) auto entries = changelog.pack(1, 5); DB::KeeperLogStore apply_changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); apply_changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) @@ -660,18 +687,18 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperations) EXPECT_EQ(apply_changelog.entry_at(12)->get_term(), 40); } -TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) +TYPED_TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); nuraft::ptr entries; { DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 10; ++i) { @@ -688,11 +715,11 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) } ChangelogDirTest test1("./logs1"); - setLogDirectory("./logs1"); + this->setLogDirectory("./logs1"); DB::KeeperLogStore changelog_new( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_new.init(1, 0); EXPECT_EQ(changelog_new.size(), 0); @@ -715,23 +742,23 @@ TEST_P(CoordinationTest, ChangelogTestBatchOperationsEmpty) EXPECT_EQ(changelog_new.next_slot(), 11); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader.init(5, 0); } -TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) +TYPED_TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) @@ -743,13 +770,13 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + this->extension)); EXPECT_EQ(changelog.size(), 33); @@ -763,19 +790,19 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + this->extension)); DB::KeeperLogStore changelog_read( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_read.init(1, 0); EXPECT_EQ(changelog_read.size(), 7); EXPECT_EQ(changelog_read.start_index(), 1); @@ -783,16 +810,16 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_EQ(changelog_read.last_entry()->get_term(), 5555); } -TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) +TYPED_TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) @@ -804,13 +831,13 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + this->extension)); EXPECT_EQ(changelog.size(), 33); @@ -824,19 +851,19 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + this->extension)); DB::KeeperLogStore changelog_read( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_read.init(1, 0); EXPECT_EQ(changelog_read.size(), 11); EXPECT_EQ(changelog_read.start_index(), 1); @@ -844,16 +871,16 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_EQ(changelog_read.last_entry()->get_term(), 5555); } -TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) +TYPED_TEST(CoordinationTest, ChangelogTestWriteAtAllFiles) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 33; ++i) { @@ -864,13 +891,13 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + this->extension)); EXPECT_EQ(changelog.size(), 33); @@ -884,26 +911,26 @@ TEST_P(CoordinationTest, ChangelogTestWriteAtAllFiles) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin" + this->extension)); } -TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) +TYPED_TEST(CoordinationTest, ChangelogTestStartNewLogAfterRead) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -915,19 +942,19 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) EXPECT_EQ(changelog.size(), 35); waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin" + this->extension)); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader.init(1, 0); auto entry = getLogEntry("36_hello_world", 360); @@ -937,14 +964,14 @@ TEST_P(CoordinationTest, ChangelogTestStartNewLogAfterRead) EXPECT_EQ(changelog_reader.size(), 36); waitDurableLogs(changelog_reader); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_36_40.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_36_40.bin" + this->extension)); } namespace @@ -965,18 +992,18 @@ void assertBrokenFileRemoved(const fs::path & directory, const fs::path & filena } -TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) +TYPED_TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) { static const fs::path log_folder{"./logs"}; - auto params = GetParam(); + ChangelogDirTest test(log_folder); - setLogDirectory(log_folder); + this->setLogDirectory(log_folder); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -988,36 +1015,36 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) EXPECT_EQ(changelog.size(), 35); waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin" + this->extension)); DB::WriteBufferFromFile plain_buf( - "./logs/changelog_11_15.bin" + params.extension, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + "./logs/changelog_11_15.bin" + this->extension, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(0); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader.init(1, 0); changelog_reader.end_of_append_batch(0, 0); EXPECT_EQ(changelog_reader.size(), 10); EXPECT_EQ(changelog_reader.last_entry()->get_term(), 90); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); - assertBrokenFileRemoved(log_folder, "changelog_16_20.bin" + params.extension); - assertBrokenFileRemoved(log_folder, "changelog_21_25.bin" + params.extension); - assertBrokenFileRemoved(log_folder, "changelog_26_30.bin" + params.extension); - assertBrokenFileRemoved(log_folder, "changelog_31_35.bin" + params.extension); + assertBrokenFileRemoved(log_folder, "changelog_16_20.bin" + this->extension); + assertBrokenFileRemoved(log_folder, "changelog_21_25.bin" + this->extension); + assertBrokenFileRemoved(log_folder, "changelog_26_30.bin" + this->extension); + assertBrokenFileRemoved(log_folder, "changelog_31_35.bin" + this->extension); auto entry = getLogEntry("h", 7777); changelog_reader.append(entry); @@ -1027,35 +1054,35 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) waitDurableLogs(changelog_reader); - EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin" + this->extension)); - assertBrokenFileRemoved(log_folder, "changelog_16_20.bin" + params.extension); - assertBrokenFileRemoved(log_folder, "changelog_21_25.bin" + params.extension); - assertBrokenFileRemoved(log_folder, "changelog_26_30.bin" + params.extension); - assertBrokenFileRemoved(log_folder, "changelog_31_35.bin" + params.extension); + assertBrokenFileRemoved(log_folder, "changelog_16_20.bin" + this->extension); + assertBrokenFileRemoved(log_folder, "changelog_21_25.bin" + this->extension); + assertBrokenFileRemoved(log_folder, "changelog_26_30.bin" + this->extension); + assertBrokenFileRemoved(log_folder, "changelog_31_35.bin" + this->extension); DB::KeeperLogStore changelog_reader2( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader2.init(1, 0); EXPECT_EQ(changelog_reader2.size(), 11); EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); } /// Truncating all entries -TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) +TYPED_TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 20}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -1066,22 +1093,22 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) changelog.end_of_append_batch(0, 0); waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + this->extension)); DB::WriteBufferFromFile plain_buf( - "./logs/changelog_1_20.bin" + params.extension, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + "./logs/changelog_1_20.bin" + this->extension, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(30); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 20}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 0); - EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); - assertBrokenFileRemoved("./logs", "changelog_21_40.bin" + params.extension); + EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + this->extension)); + assertBrokenFileRemoved("./logs", "changelog_21_40.bin" + this->extension); auto entry = getLogEntry("hello_world", 7777); changelog_reader.append(entry); changelog_reader.end_of_append_batch(0, 0); @@ -1092,9 +1119,9 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); DB::KeeperLogStore changelog_reader2( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 1}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 1}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader2.init(1, 0); EXPECT_EQ(changelog_reader2.size(), 1); EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); @@ -1103,15 +1130,15 @@ TEST_P(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) /// Truncating only some entries from the end /// For compressed logs we have no reliable way of knowing how many log entries were lost /// after we truncate some bytes from the end -TEST_F(CoordinationTest, ChangelogTestReadAfterBrokenTruncate3) +TYPED_TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate3) { ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( DB::LogFileSettings{.force_sync = true, .compress_logs = false, .rotate_interval = 20}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -1133,7 +1160,7 @@ TEST_F(CoordinationTest, ChangelogTestReadAfterBrokenTruncate3) DB::KeeperLogStore changelog_reader( DB::LogFileSettings{.force_sync = true, .compress_logs = false, .rotate_interval = 20}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_reader.init(1, 0); EXPECT_EQ(changelog_reader.size(), 19); @@ -1150,10 +1177,10 @@ TEST_F(CoordinationTest, ChangelogTestReadAfterBrokenTruncate3) EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); } -TEST_F(CoordinationTest, ChangelogTestMixedLogTypes) +TYPED_TEST(CoordinationTest, ChangelogTestMixedLogTypes) { ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); std::vector changelog_files; @@ -1185,7 +1212,7 @@ TEST_F(CoordinationTest, ChangelogTestMixedLogTypes) DB::KeeperLogStore changelog( DB::LogFileSettings{.force_sync = true, .compress_logs = false, .rotate_interval = 20}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -1206,7 +1233,7 @@ TEST_F(CoordinationTest, ChangelogTestMixedLogTypes) DB::KeeperLogStore changelog_compressed( DB::LogFileSettings{.force_sync = true, .compress_logs = true, .rotate_interval = 20}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_compressed.init(1, 0); verify_changelog_files(); @@ -1228,7 +1255,7 @@ TEST_F(CoordinationTest, ChangelogTestMixedLogTypes) DB::KeeperLogStore changelog( DB::LogFileSettings{.force_sync = true, .compress_logs = false, .rotate_interval = 20}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); verify_changelog_files(); @@ -1246,16 +1273,16 @@ TEST_F(CoordinationTest, ChangelogTestMixedLogTypes) } } -TEST_P(CoordinationTest, ChangelogTestLostFiles) +TYPED_TEST(CoordinationTest, ChangelogTestLostFiles) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 20}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -1266,30 +1293,30 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles) changelog.end_of_append_batch(0, 0); waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin" + this->extension)); - fs::remove("./logs/changelog_1_20.bin" + params.extension); + fs::remove("./logs/changelog_1_20.bin" + this->extension); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 20}, DB::FlushSettings(), - keeper_context); + this->keeper_context); /// It should print error message, but still able to start changelog_reader.init(5, 0); - assertBrokenFileRemoved("./logs", "changelog_21_40.bin" + params.extension); + assertBrokenFileRemoved("./logs", "changelog_21_40.bin" + this->extension); } -TEST_P(CoordinationTest, ChangelogTestLostFiles2) +TYPED_TEST(CoordinationTest, ChangelogTestLostFiles2) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 10}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t i = 0; i < 35; ++i) @@ -1301,24 +1328,24 @@ TEST_P(CoordinationTest, ChangelogTestLostFiles2) waitDurableLogs(changelog); - EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_21_30.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_31_40.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_21_30.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_31_40.bin" + this->extension)); // we have a gap in our logs, we need to remove all the logs after the gap - fs::remove("./logs/changelog_21_30.bin" + params.extension); + fs::remove("./logs/changelog_21_30.bin" + this->extension); DB::KeeperLogStore changelog_reader( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 10}, DB::FlushSettings(), - keeper_context); + this->keeper_context); /// It should print error message, but still able to start changelog_reader.init(5, 0); - EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_10.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_11_20.bin" + this->extension)); - assertBrokenFileRemoved("./logs", "changelog_31_40.bin" + params.extension); + assertBrokenFileRemoved("./logs", "changelog_31_40.bin" + this->extension); } struct IntNode { @@ -1334,7 +1361,7 @@ struct IntNode bool operator!=(const int & rhs) const { return rhs != this->value; } }; -TEST_P(CoordinationTest, SnapshotableHashMapSimple) +TYPED_TEST(CoordinationTest, SnapshotableHashMapSimple) { DB::SnapshotableHashTable hello; EXPECT_TRUE(hello.insert("hello", 5).second); @@ -1349,7 +1376,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapSimple) EXPECT_EQ(hello.size(), 0); } -TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) +TYPED_TEST(CoordinationTest, SnapshotableHashMapTrySnapshot) { DB::SnapshotableHashTable map_snp; EXPECT_TRUE(map_snp.insert("/hello", 7).second); @@ -1426,7 +1453,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) map_snp.disableSnapshotMode(); } -TEST_P(CoordinationTest, SnapshotableHashMapDataSize) +TYPED_TEST(CoordinationTest, SnapshotableHashMapDataSize) { /// int DB::SnapshotableHashTable hello; @@ -1503,9 +1530,10 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) EXPECT_EQ(world.getApproximateDataSize(), 0); } -void addNode(DB::KeeperMemoryStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) +template +void addNode(Storage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) { - using Node = DB::KeeperMemoryStorage::Node; + using Node = typename Storage::Node; Node node{}; node.setData(data); node.setEphemeralOwner(ephemeral_owner); @@ -1521,15 +1549,20 @@ void addNode(DB::KeeperMemoryStorage & storage, const std::string & path, const }); } -TEST_P(CoordinationTest, TestStorageSnapshotSimple) +TYPED_TEST(CoordinationTest, TestStorageSnapshotSimple) { - auto params = GetParam(); + ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); + this->setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + using Storage = typename TestFixture::Storage; - DB::KeeperMemoryStorage storage(500, "", keeper_context); + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, this->keeper_context, this->enable_compression); + + Storage storage(500, "", this->keeper_context); addNode(storage, "/hello1", "world", 1); addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; @@ -1539,7 +1572,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) storage.getSessionID(130); storage.getSessionID(130); - DB::KeeperStorageSnapshot snapshot(&storage, 2); + DB::KeeperStorageSnapshot snapshot(&storage, 2); EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2); EXPECT_EQ(snapshot.session_id, 7); @@ -1548,7 +1581,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 2); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + this->extension)); auto debuf = manager.deserializeSnapshotBufferFromDisk(2); @@ -1571,15 +1604,20 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); } -TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) +TYPED_TEST(CoordinationTest, TestStorageSnapshotMoreWrites) { - auto params = GetParam(); + ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); + this->setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + using Storage = typename TestFixture::Storage; - DB::KeeperMemoryStorage storage(500, "", keeper_context); + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, this->keeper_context, this->enable_compression); + + Storage storage(500, "", this->keeper_context); storage.getSessionID(130); for (size_t i = 0; i < 50; ++i) @@ -1587,7 +1625,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } - DB::KeeperStorageSnapshot snapshot(&storage, 50); + DB::KeeperStorageSnapshot snapshot(&storage, 50); EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50); EXPECT_EQ(snapshot.snapshot_container_size, 54); @@ -1600,7 +1638,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + this->extension)); auto debuf = manager.deserializeSnapshotBufferFromDisk(50); @@ -1614,15 +1652,20 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) } -TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) +TYPED_TEST(CoordinationTest, TestStorageSnapshotManySnapshots) { - auto params = GetParam(); + ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); + this->setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + using Storage = typename TestFixture::Storage; - DB::KeeperMemoryStorage storage(500, "", keeper_context); + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, this->keeper_context, this->enable_compression); + + Storage storage(500, "", this->keeper_context); storage.getSessionID(130); for (size_t j = 1; j <= 5; ++j) @@ -1632,17 +1675,17 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } - DB::KeeperStorageSnapshot snapshot(&storage, j * 50); + DB::KeeperStorageSnapshot snapshot(&storage, j * 50); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, j * 50); - EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin" + params.extension)); + EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin" + this->extension)); } - EXPECT_FALSE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); - EXPECT_FALSE(fs::exists("./snapshots/snapshot_100.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_150.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_200.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin" + params.extension)); + EXPECT_FALSE(fs::exists("./snapshots/snapshot_50.bin" + this->extension)); + EXPECT_FALSE(fs::exists("./snapshots/snapshot_100.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_150.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_200.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin" + this->extension)); auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); @@ -1655,21 +1698,26 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) } } -TEST_P(CoordinationTest, TestStorageSnapshotMode) +TYPED_TEST(CoordinationTest, TestStorageSnapshotMode) { - auto params = GetParam(); - ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperMemoryStorage storage(500, "", keeper_context); + ChangelogDirTest test("./snapshots"); + this->setSnapshotDirectory("./snapshots"); + + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, this->keeper_context, this->enable_compression); + Storage storage(500, "", this->keeper_context); for (size_t i = 0; i < 50; ++i) { addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } { - DB::KeeperStorageSnapshot snapshot(&storage, 50); + DB::KeeperStorageSnapshot snapshot(&storage, 50); for (size_t i = 0; i < 50; ++i) { addNode(storage, "/hello_" + std::to_string(i), "wlrd_" + std::to_string(i)); @@ -1684,12 +1732,15 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) storage.container.erase("/hello_" + std::to_string(i)); } EXPECT_EQ(storage.container.size(), 29); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 105); + if constexpr (Storage::use_rocksdb) + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 54); + else + EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 105); EXPECT_EQ(storage.container.snapshotSizeWithVersion().second, 1); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); } - EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + this->extension)); EXPECT_EQ(storage.container.size(), 29); storage.clearGarbageAfterSnapshot(); EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 29); @@ -1709,28 +1760,33 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) } } -TEST_P(CoordinationTest, TestStorageSnapshotBroken) +TYPED_TEST(CoordinationTest, TestStorageSnapshotBroken) { - auto params = GetParam(); - ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); - DB::KeeperMemoryStorage storage(500, "", keeper_context); + ChangelogDirTest test("./snapshots"); + this->setSnapshotDirectory("./snapshots"); + + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, this->keeper_context, this->enable_compression); + Storage storage(500, "", this->keeper_context); for (size_t i = 0; i < 50; ++i) { addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); } { - DB::KeeperStorageSnapshot snapshot(&storage, 50); + DB::KeeperStorageSnapshot snapshot(&storage, 50); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); } - EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + this->extension)); /// Let's corrupt file DB::WriteBufferFromFile plain_buf( - "./snapshots/snapshot_50.bin" + params.extension, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + "./snapshots/snapshot_50.bin" + this->extension, DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(34); plain_buf.sync(); @@ -1757,6 +1813,7 @@ getLogEntryFromZKRequest(size_t term, int64_t session_id, int64_t zxid, const Co return nuraft::cs_new(term, buffer); } +template void testLogAndStateMachine( DB::CoordinationSettingsPtr settings, uint64_t total_logs, @@ -1767,12 +1824,15 @@ void testLogAndStateMachine( ChangelogDirTest snapshots("./snapshots"); ChangelogDirTest logs("./logs"); + ChangelogDirTest rocks("./rocksdb"); auto get_keeper_context = [&] { auto local_keeper_context = std::make_shared(true, settings); local_keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", "./snapshots")); local_keeper_context->setLogDisk(std::make_shared("LogDisk", "./logs")); + local_keeper_context->setRocksDBDisk(std::make_shared("RocksDisk", "./rocksdb")); + local_keeper_context->setRocksDBOptions(); return local_keeper_context; }; @@ -1780,7 +1840,7 @@ void testLogAndStateMachine( SnapshotsQueue snapshots_queue{1}; auto keeper_context = get_keeper_context(); - auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); state_machine->init(); DB::KeeperLogStore changelog( @@ -1827,7 +1887,7 @@ void testLogAndStateMachine( SnapshotsQueue snapshots_queue1{1}; keeper_context = get_keeper_context(); - auto restore_machine = std::make_shared>(queue, snapshots_queue1, keeper_context, nullptr); + auto restore_machine = std::make_shared>(queue, snapshots_queue1, keeper_context, nullptr); restore_machine->init(); EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance); @@ -1863,11 +1923,12 @@ void testLogAndStateMachine( } } -TEST_P(CoordinationTest, TestStateMachineAndLogStore) +TYPED_TEST(CoordinationTest, TestStateMachineAndLogStore) { using namespace Coordination; using namespace DB; - auto params = GetParam(); + + using Storage = typename TestFixture::Storage; { CoordinationSettingsPtr settings = std::make_shared(); @@ -1875,78 +1936,83 @@ TEST_P(CoordinationTest, TestStateMachineAndLogStore) settings->reserved_log_items = 10; settings->rotate_log_storage_interval = 10; - testLogAndStateMachine(settings, 37, params.enable_compression); + testLogAndStateMachine(settings, 37, this->enable_compression); } { CoordinationSettingsPtr settings = std::make_shared(); settings->snapshot_distance = 10; settings->reserved_log_items = 10; settings->rotate_log_storage_interval = 10; - testLogAndStateMachine(settings, 11, params.enable_compression); + testLogAndStateMachine(settings, 11, this->enable_compression); } { CoordinationSettingsPtr settings = std::make_shared(); settings->snapshot_distance = 10; settings->reserved_log_items = 10; settings->rotate_log_storage_interval = 10; - testLogAndStateMachine(settings, 40, params.enable_compression); + testLogAndStateMachine(settings, 40, this->enable_compression); } { CoordinationSettingsPtr settings = std::make_shared(); settings->snapshot_distance = 10; settings->reserved_log_items = 20; settings->rotate_log_storage_interval = 30; - testLogAndStateMachine(settings, 40, params.enable_compression); + testLogAndStateMachine(settings, 40, this->enable_compression); } { CoordinationSettingsPtr settings = std::make_shared(); settings->snapshot_distance = 10; settings->reserved_log_items = 0; settings->rotate_log_storage_interval = 10; - testLogAndStateMachine(settings, 40, params.enable_compression); + testLogAndStateMachine(settings, 40, this->enable_compression); } { CoordinationSettingsPtr settings = std::make_shared(); settings->snapshot_distance = 1; settings->reserved_log_items = 1; settings->rotate_log_storage_interval = 32; - testLogAndStateMachine(settings, 32, params.enable_compression); + testLogAndStateMachine(settings, 32, this->enable_compression); } { CoordinationSettingsPtr settings = std::make_shared(); settings->snapshot_distance = 10; settings->reserved_log_items = 7; settings->rotate_log_storage_interval = 1; - testLogAndStateMachine(settings, 33, params.enable_compression); + testLogAndStateMachine(settings, 33, this->enable_compression); } { CoordinationSettingsPtr settings = std::make_shared(); settings->snapshot_distance = 37; settings->reserved_log_items = 1000; settings->rotate_log_storage_interval = 5000; - testLogAndStateMachine(settings, 33, params.enable_compression); + testLogAndStateMachine(settings, 33, this->enable_compression); } { CoordinationSettingsPtr settings = std::make_shared(); settings->snapshot_distance = 37; settings->reserved_log_items = 1000; settings->rotate_log_storage_interval = 5000; - testLogAndStateMachine(settings, 45, params.enable_compression); + testLogAndStateMachine(settings, 45, this->enable_compression); } } -TEST_P(CoordinationTest, TestEphemeralNodeRemove) +TYPED_TEST(CoordinationTest, TestEphemeralNodeRemove) { using namespace Coordination; using namespace DB; ChangelogDirTest snapshots("./snapshots"); - setSnapshotDirectory("./snapshots"); + this->setSnapshotDirectory("./snapshots"); + + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, this->keeper_context, nullptr); state_machine->init(); std::shared_ptr request_c = std::make_shared(); @@ -1969,17 +2035,23 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) } -TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitted) +TYPED_TEST(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitted) { using namespace Coordination; using namespace DB; ChangelogDirTest snapshots("./snapshots"); - setSnapshotDirectory("./snapshots"); + this->setSnapshotDirectory("./snapshots"); + + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, this->keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; @@ -2019,18 +2091,18 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte EXPECT_EQ(acls[0].permissions, 31); } -TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) +TYPED_TEST(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) { using namespace Coordination; using namespace DB; ChangelogDirTest snapshots("./snapshots"); - setSnapshotDirectory("./snapshots"); + this->setSnapshotDirectory("./snapshots"); ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; - auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); + auto state_machine = std::make_shared>(queue, snapshots_queue, this->keeper_context, nullptr); state_machine->init(); String user_auth_data = "test_user:test_password"; @@ -2077,17 +2149,17 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) } -TEST_P(CoordinationTest, TestRotateIntervalChanges) +TYPED_TEST(CoordinationTest, TestRotateIntervalChanges) { using namespace Coordination; - auto params = GetParam(); + ChangelogDirTest snapshots("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); { DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -2103,12 +2175,12 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) } - EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + this->extension)); DB::KeeperLogStore changelog_1( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 10}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 10}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_1.init(0, 50); for (size_t i = 0; i < 55; ++i) { @@ -2121,13 +2193,13 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) waitDurableLogs(changelog_1); - EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_1_100.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + this->extension)); DB::KeeperLogStore changelog_2( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 7}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 7}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_2.init(98, 55); for (size_t i = 0; i < 17; ++i) @@ -2141,20 +2213,20 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) waitDurableLogs(changelog_2); - keeper_context->setLastCommitIndex(105); + this->keeper_context->setLastCommitIndex(105); changelog_2.compact(105); std::this_thread::sleep_for(std::chrono::microseconds(1000)); - assertFileDeleted("./logs/changelog_1_100.bin" + params.extension); - EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_111_117.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_118_124.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_125_131.bin" + params.extension)); + assertFileDeleted("./logs/changelog_1_100.bin" + this->extension); + EXPECT_TRUE(fs::exists("./logs/changelog_101_110.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_111_117.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_118_124.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_125_131.bin" + this->extension)); DB::KeeperLogStore changelog_3( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 5}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 5}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog_3.init(116, 3); for (size_t i = 0; i < 17; ++i) { @@ -2167,20 +2239,20 @@ TEST_P(CoordinationTest, TestRotateIntervalChanges) waitDurableLogs(changelog_3); - keeper_context->setLastCommitIndex(125); + this->keeper_context->setLastCommitIndex(125); changelog_3.compact(125); std::this_thread::sleep_for(std::chrono::microseconds(1000)); - assertFileDeleted("./logs/changelog_101_110.bin" + params.extension); - assertFileDeleted("./logs/changelog_111_117.bin" + params.extension); - assertFileDeleted("./logs/changelog_118_124.bin" + params.extension); + assertFileDeleted("./logs/changelog_101_110.bin" + this->extension); + assertFileDeleted("./logs/changelog_111_117.bin" + this->extension); + assertFileDeleted("./logs/changelog_118_124.bin" + this->extension); - EXPECT_TRUE(fs::exists("./logs/changelog_125_131.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_132_136.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_137_141.bin" + params.extension)); - EXPECT_TRUE(fs::exists("./logs/changelog_142_146.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_125_131.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_132_136.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_137_141.bin" + this->extension)); + EXPECT_TRUE(fs::exists("./logs/changelog_142_146.bin" + this->extension)); } -TEST_P(CoordinationTest, TestSessionExpiryQueue) +TYPED_TEST(CoordinationTest, TestSessionExpiryQueue) { using namespace Coordination; SessionExpiryQueue queue(500); @@ -2198,16 +2270,15 @@ TEST_P(CoordinationTest, TestSessionExpiryQueue) } -TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) +TYPED_TEST(CoordinationTest, TestCompressedLogsMultipleRewrite) { using namespace Coordination; - auto test_params = GetParam(); ChangelogDirTest logs("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -2222,9 +2293,9 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) waitDurableLogs(changelog); DB::KeeperLogStore changelog1( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog1.init(0, 3); for (size_t i = 55; i < 70; ++i) { @@ -2238,9 +2309,9 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) waitDurableLogs(changelog1); DB::KeeperLogStore changelog2( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog2.init(0, 3); for (size_t i = 70; i < 80; ++i) { @@ -2252,16 +2323,21 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) } } -TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) +TYPED_TEST(CoordinationTest, TestStorageSnapshotDifferentCompressions) { - auto params = GetParam(); + ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); + this->setSnapshotDirectory("./snapshots"); - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + using Storage = typename TestFixture::Storage; - DB::KeeperMemoryStorage storage(500, "", keeper_context); + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + DB::KeeperSnapshotManager manager(3, this->keeper_context, this->enable_compression); + + Storage storage(500, "", this->keeper_context); addNode(storage, "/hello1", "world", 1); addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; @@ -2271,13 +2347,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) storage.getSessionID(130); storage.getSessionID(130); - DB::KeeperStorageSnapshot snapshot(&storage, 2); + DB::KeeperStorageSnapshot snapshot(&storage, 2); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 2); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + this->extension)); - DB::KeeperSnapshotManager new_manager(3, keeper_context, !params.enable_compression); + DB::KeeperSnapshotManager new_manager(3, this->keeper_context, !this->enable_compression); auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); @@ -2299,17 +2375,17 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); } -TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) +TYPED_TEST(CoordinationTest, ChangelogInsertThreeTimesSmooth) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); { - LOG_INFO(log, "================First time====================="); + LOG_INFO(this->log, "================First time====================="); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -2319,11 +2395,11 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) } { - LOG_INFO(log, "================Second time====================="); + LOG_INFO(this->log, "================Second time====================="); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -2333,11 +2409,11 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) } { - LOG_INFO(log, "================Third time====================="); + LOG_INFO(this->log, "================Third time====================="); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -2347,11 +2423,11 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) } { - LOG_INFO(log, "================Fourth time====================="); + LOG_INFO(this->log, "================Fourth time====================="); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog.append(entry); @@ -2362,18 +2438,18 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesSmooth) } -TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) +TYPED_TEST(CoordinationTest, ChangelogInsertMultipleTimesSmooth) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); for (size_t i = 0; i < 36; ++i) { - LOG_INFO(log, "================First time====================="); + LOG_INFO(this->log, "================First time====================="); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (size_t j = 0; j < 7; ++j) { @@ -2385,24 +2461,24 @@ TEST_P(CoordinationTest, ChangelogInsertMultipleTimesSmooth) } DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); EXPECT_EQ(changelog.next_slot(), 36 * 7 + 1); } -TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) +TYPED_TEST(CoordinationTest, ChangelogInsertThreeTimesHard) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); { - LOG_INFO(log, "================First time====================="); + LOG_INFO(this->log, "================First time====================="); DB::KeeperLogStore changelog1( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog1.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog1.append(entry); @@ -2412,11 +2488,11 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) } { - LOG_INFO(log, "================Second time====================="); + LOG_INFO(this->log, "================Second time====================="); DB::KeeperLogStore changelog2( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog2.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog2.append(entry); @@ -2426,11 +2502,11 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) } { - LOG_INFO(log, "================Third time====================="); + LOG_INFO(this->log, "================Third time====================="); DB::KeeperLogStore changelog3( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog3.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog3.append(entry); @@ -2440,11 +2516,11 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) } { - LOG_INFO(log, "================Fourth time====================="); + LOG_INFO(this->log, "================Fourth time====================="); DB::KeeperLogStore changelog4( - DB::LogFileSettings{.force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog4.init(1, 0); auto entry = getLogEntry("hello_world", 1000); changelog4.append(entry); @@ -2454,18 +2530,23 @@ TEST_P(CoordinationTest, ChangelogInsertThreeTimesHard) } } -TEST_P(CoordinationTest, TestStorageSnapshotEqual) +TYPED_TEST(CoordinationTest, TestStorageSnapshotEqual) { - auto params = GetParam(); + ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); + this->setSnapshotDirectory("./snapshots"); + + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); std::optional snapshot_hash; for (size_t i = 0; i < 15; ++i) { - DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); + DB::KeeperSnapshotManager manager(3, this->keeper_context, this->enable_compression); - DB::KeeperMemoryStorage storage(500, "", keeper_context); + Storage storage(500, "", this->keeper_context); addNode(storage, "/hello", ""); for (size_t j = 0; j < 5000; ++j) { @@ -2481,7 +2562,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual) for (size_t j = 0; j < 3333; ++j) storage.getSessionID(130 * j); - DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); + DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); auto buf = manager.serializeSnapshotToBuffer(snapshot); @@ -2498,17 +2579,16 @@ TEST_P(CoordinationTest, TestStorageSnapshotEqual) } -TEST_P(CoordinationTest, TestLogGap) +TYPED_TEST(CoordinationTest, TestLogGap) { using namespace Coordination; - auto test_params = GetParam(); ChangelogDirTest logs("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); DB::KeeperLogStore changelog( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(0, 3); for (size_t i = 1; i < 55; ++i) @@ -2521,13 +2601,13 @@ TEST_P(CoordinationTest, TestLogGap) } DB::KeeperLogStore changelog1( - DB::LogFileSettings{.force_sync = true, .compress_logs = test_params.enable_compression, .rotate_interval = 100}, + DB::LogFileSettings{.force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog1.init(61, 3); /// Logs discarded - EXPECT_FALSE(fs::exists("./logs/changelog_1_100.bin" + test_params.extension)); + EXPECT_FALSE(fs::exists("./logs/changelog_1_100.bin" + this->extension)); EXPECT_EQ(changelog1.start_index(), 61); EXPECT_EQ(changelog1.next_slot(), 61); } @@ -2539,12 +2619,17 @@ ResponseType getSingleResponse(const auto & responses) return dynamic_cast(*responses[0].response); } -TEST_P(CoordinationTest, TestUncommittedStateBasicCrud) +TYPED_TEST(CoordinationTest, TestUncommittedStateBasicCrud) { using namespace DB; using namespace Coordination; - DB::KeeperMemoryStorage storage{500, "", keeper_context}; + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + Storage storage{500, "", this->keeper_context}; constexpr std::string_view path = "/test"; @@ -2656,12 +2741,17 @@ TEST_P(CoordinationTest, TestUncommittedStateBasicCrud) ASSERT_FALSE(get_committed_data()); } -TEST_P(CoordinationTest, TestListRequestTypes) +TYPED_TEST(CoordinationTest, TestListRequestTypes) { using namespace DB; using namespace Coordination; - KeeperMemoryStorage storage{500, "", keeper_context}; + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + Storage storage{500, "", this->keeper_context}; int32_t zxid = 0; @@ -2738,18 +2828,18 @@ TEST_P(CoordinationTest, TestListRequestTypes) } } -TEST_P(CoordinationTest, TestDurableState) +TYPED_TEST(CoordinationTest, TestDurableState) { ChangelogDirTest logs("./logs"); - setLogDirectory("./logs"); - setStateFileDirectory("."); + this->setLogDirectory("./logs"); + this->setStateFileDirectory("."); auto state = nuraft::cs_new(); std::optional state_manager; const auto reload_state_manager = [&] { - state_manager.emplace(1, "localhost", 9181, keeper_context); + state_manager.emplace(1, "localhost", 9181, this->keeper_context); state_manager->loadLogStore(1, 0); }; @@ -2812,10 +2902,15 @@ TEST_P(CoordinationTest, TestDurableState) } } -TEST_P(CoordinationTest, TestFeatureFlags) +TYPED_TEST(CoordinationTest, TestFeatureFlags) { using namespace Coordination; - KeeperMemoryStorage storage{500, "", keeper_context}; + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + Storage storage{500, "", this->keeper_context}; auto request = std::make_shared(); request->path = DB::keeper_api_feature_flags_path; auto responses = storage.processRequest(request, 0, std::nullopt, true, true); @@ -2827,14 +2922,19 @@ TEST_P(CoordinationTest, TestFeatureFlags) ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS)); } -TEST_P(CoordinationTest, TestSystemNodeModify) +TYPED_TEST(CoordinationTest, TestSystemNodeModify) { using namespace Coordination; int64_t zxid{0}; + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + // On INIT we abort when a system path is modified - keeper_context->setServerState(KeeperContext::Phase::RUNNING); - KeeperMemoryStorage storage{500, "", keeper_context}; + this->keeper_context->setServerState(KeeperContext::Phase::RUNNING); + Storage storage{500, "", this->keeper_context}; const auto assert_create = [&](const std::string_view path, const auto expected_code) { auto request = std::make_shared(); @@ -2859,11 +2959,11 @@ TEST_P(CoordinationTest, TestSystemNodeModify) assert_create("/keeper1/test", Error::ZOK); } -TEST_P(CoordinationTest, ChangelogTestMaxLogSize) +TYPED_TEST(CoordinationTest, ChangelogTestMaxLogSize) { - auto params = GetParam(); + ChangelogDirTest test("./logs"); - setLogDirectory("./logs"); + this->setLogDirectory("./logs"); uint64_t last_entry_index{0}; size_t i{0}; @@ -2871,9 +2971,9 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) SCOPED_TRACE("Small rotation interval, big size limit"); DB::KeeperLogStore changelog( DB::LogFileSettings{ - .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 20, .max_size = 50 * 1024 * 1024}, + .force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 20, .max_size = 50 * 1024 * 1024}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); for (; i < 100; ++i) @@ -2891,9 +2991,9 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) SCOPED_TRACE("Large rotation interval, small size limit"); DB::KeeperLogStore changelog( DB::LogFileSettings{ - .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}, + .force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100'000, .max_size = 4000}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); ASSERT_EQ(changelog.entry_at(last_entry_index)->get_term(), (i - 1 + 44) * 10); @@ -2913,20 +3013,25 @@ TEST_P(CoordinationTest, ChangelogTestMaxLogSize) SCOPED_TRACE("Final verify all logs"); DB::KeeperLogStore changelog( DB::LogFileSettings{ - .force_sync = true, .compress_logs = params.enable_compression, .rotate_interval = 100'000, .max_size = 4000}, + .force_sync = true, .compress_logs = this->enable_compression, .rotate_interval = 100'000, .max_size = 4000}, DB::FlushSettings(), - keeper_context); + this->keeper_context); changelog.init(1, 0); ASSERT_EQ(changelog.entry_at(last_entry_index)->get_term(), (i - 1 + 44) * 10); } } -TEST_P(CoordinationTest, TestCheckNotExistsRequest) +TYPED_TEST(CoordinationTest, TestCheckNotExistsRequest) { using namespace DB; using namespace Coordination; - KeeperMemoryStorage storage{500, "", keeper_context}; + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + Storage storage{500, "", this->keeper_context}; int32_t zxid = 0; @@ -2994,18 +3099,23 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest) } } -TEST_P(CoordinationTest, TestReapplyingDeltas) +TYPED_TEST(CoordinationTest, TestReapplyingDeltas) { using namespace DB; using namespace Coordination; + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + static constexpr int64_t initial_zxid = 100; const auto create_request = std::make_shared(); create_request->path = "/test/data"; create_request->is_sequential = true; - const auto process_create = [](KeeperMemoryStorage & storage, const auto & request, int64_t zxid) + const auto process_create = [](Storage & storage, const auto & request, int64_t zxid) { storage.preprocessRequest(request, 1, 0, zxid); auto responses = storage.processRequest(request, 1, zxid); @@ -3026,19 +3136,19 @@ TEST_P(CoordinationTest, TestReapplyingDeltas) process_create(storage, create_request, zxid); }; - KeeperMemoryStorage storage1{500, "", keeper_context}; + Storage storage1{500, "", this->keeper_context}; commit_initial_data(storage1); for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) storage1.preprocessRequest(create_request, 1, 0, zxid, /*check_acl=*/true, /*digest=*/std::nullopt, /*log_idx=*/zxid); /// create identical new storage - KeeperMemoryStorage storage2{500, "", keeper_context}; + Storage storage2{500, "", this->keeper_context}; commit_initial_data(storage2); storage1.applyUncommittedState(storage2, initial_zxid); - const auto commit_unprocessed = [&](KeeperMemoryStorage & storage) + const auto commit_unprocessed = [&](Storage & storage) { for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) { @@ -3051,7 +3161,7 @@ TEST_P(CoordinationTest, TestReapplyingDeltas) commit_unprocessed(storage1); commit_unprocessed(storage2); - const auto get_children = [&](KeeperMemoryStorage & storage) + const auto get_children = [&](Storage & storage) { const auto list_request = std::make_shared(); list_request->path = "/test"; @@ -3071,8 +3181,8 @@ TEST_P(CoordinationTest, TestReapplyingDeltas) ASSERT_TRUE(children1_set == children2_set); } -INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, - CoordinationTest, - ::testing::ValuesIn(std::initializer_list{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}})); +/// INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, +/// CoordinationTest, +/// ::testing::ValuesIn(std::initializer_list{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}})); #endif diff --git a/src/Coordination/tests/gtest_rocks_keeper.cpp b/src/Coordination/tests/gtest_rocks_keeper.cpp deleted file mode 100644 index d42eb66d683..00000000000 --- a/src/Coordination/tests/gtest_rocks_keeper.cpp +++ /dev/null @@ -1,1150 +0,0 @@ -#include -#include "config.h" - -#if USE_NURAFT and USE_ROCKSDB - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace fs = std::filesystem; -struct ChangelogDirTest -{ - std::string path; - bool drop; - explicit ChangelogDirTest(std::string path_, bool drop_ = true) : path(path_), drop(drop_) - { - EXPECT_FALSE(fs::exists(path)) << "Path " << path << " already exists, remove it to run test"; - fs::create_directory(path); - } - - ~ChangelogDirTest() - { - if (fs::exists(path) && drop) - fs::remove_all(path); - } -}; - -class RocksKeeperTest : public ::testing::Test -{ -protected: - DB::KeeperContextPtr keeper_context; - LoggerPtr log{getLogger("RocksKeeperTest")}; - - void SetUp() override - { - Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); - Poco::Logger::root().setChannel(channel); - Poco::Logger::root().setLevel("trace"); - - auto settings = std::make_shared(); - settings->use_rocksdb = true; - keeper_context = std::make_shared(true, settings); - keeper_context->setLocalLogsPreprocessed(); - keeper_context->setRocksDBOptions(nullptr); - } - - void setSnapshotDirectory(const std::string & path) - { - keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", path)); - } - - void setRocksDBDirectory(const std::string & path) - { - keeper_context->setRocksDBDisk(std::make_shared("RocksDisk", path)); - } -}; - -void addNode(DB::KeeperRocksStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner = 0) -{ - using Node = DB::KeeperRocksStorage::Node; - Node node{}; - node.setData(data); - node.setEphemeralOwner(ephemeral_owner); - storage.container.insertOrReplace(path, node); - storage.container.updateValue( - DB::parentNodePath(StringRef{path}), - [&](auto & parent) - { - parent.increaseNumChildren(); - }); -} - -namespace -{ -void waitDurableLogs(nuraft::log_store & log_store) -{ - while (log_store.last_durable_index() != log_store.next_slot() - 1) - std::this_thread::sleep_for(std::chrono::milliseconds(200)); -} -} - -TEST_F(RocksKeeperTest, TestStorageSnapshotSimple) -{ - ChangelogDirTest test("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setSnapshotDirectory("./snapshots"); - setRocksDBDirectory("./rocksdb"); - - DB::KeeperSnapshotManager manager(3, keeper_context); - - DB::KeeperRocksStorage storage(500, "", keeper_context); - addNode(storage, "/hello1", "world", 1); - addNode(storage, "/hello2", "somedata", 3); - storage.session_id_counter = 5; - storage.zxid = 2; - storage.ephemerals[3] = {"/hello2"}; - storage.ephemerals[1] = {"/hello1"}; - storage.getSessionID(130); - storage.getSessionID(130); - - DB::KeeperStorageSnapshot snapshot(&storage, 2); - - EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2); - EXPECT_EQ(snapshot.session_id, 7); - EXPECT_EQ(snapshot.snapshot_container_size, 6); - EXPECT_EQ(snapshot.session_and_timeout.size(), 2); - - auto buf = manager.serializeSnapshotToBuffer(snapshot); - manager.serializeSnapshotBufferToDisk(*buf, 2); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin.zstd")); - - - auto debuf = manager.deserializeSnapshotBufferFromDisk(2); - - auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); - - EXPECT_EQ(restored_storage->container.size(), 6); - EXPECT_EQ(restored_storage->container.find("/")->value.numChildren(), 3); - EXPECT_EQ(restored_storage->container.find("/hello1")->value.numChildren(), 0); - EXPECT_EQ(restored_storage->container.find("/hello2")->value.numChildren(), 0); - - EXPECT_EQ(restored_storage->container.find("/")->value.getData(), ""); - EXPECT_EQ(restored_storage->container.find("/hello1")->value.getData(), "world"); - EXPECT_EQ(restored_storage->container.find("/hello2")->value.getData(), "somedata"); - EXPECT_EQ(restored_storage->session_id_counter, 7); - EXPECT_EQ(restored_storage->zxid, 2); - EXPECT_EQ(restored_storage->ephemerals.size(), 2); - EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); - EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); - EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); -} - -TEST_F(RocksKeeperTest, TestStorageSnapshotMoreWrites) -{ - ChangelogDirTest test("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setSnapshotDirectory("./snapshots"); - setRocksDBDirectory("./rocksdb"); - - DB::KeeperSnapshotManager manager(3, keeper_context); - - DB::KeeperRocksStorage storage(500, "", keeper_context); - storage.getSessionID(130); - - for (size_t i = 0; i < 50; ++i) - { - addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); - } - - DB::KeeperStorageSnapshot snapshot(&storage, 50); - EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50); - EXPECT_EQ(snapshot.snapshot_container_size, 54); - - for (size_t i = 50; i < 100; ++i) - { - addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); - } - - EXPECT_EQ(storage.container.size(), 104); - - auto buf = manager.serializeSnapshotToBuffer(snapshot); - manager.serializeSnapshotBufferToDisk(*buf, 50); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin.zstd")); - - - auto debuf = manager.deserializeSnapshotBufferFromDisk(50); - auto [restored_storage, meta, _] = manager.deserializeSnapshotFromBuffer(debuf); - - EXPECT_EQ(restored_storage->container.size(), 54); - for (size_t i = 0; i < 50; ++i) - { - EXPECT_EQ(restored_storage->container.find("/hello_" + std::to_string(i))->value.getData(), "world_" + std::to_string(i)); - } -} - -TEST_F(RocksKeeperTest, TestStorageSnapshotManySnapshots) -{ - ChangelogDirTest test("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setSnapshotDirectory("./snapshots"); - setRocksDBDirectory("./rocksdb"); - - DB::KeeperSnapshotManager manager(3, keeper_context); - - DB::KeeperRocksStorage storage(500, "", keeper_context); - storage.getSessionID(130); - - for (size_t j = 1; j <= 5; ++j) - { - for (size_t i = (j - 1) * 50; i < j * 50; ++i) - { - addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); - } - - DB::KeeperStorageSnapshot snapshot(&storage, j * 50); - auto buf = manager.serializeSnapshotToBuffer(snapshot); - manager.serializeSnapshotBufferToDisk(*buf, j * 50); - EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin.zstd")); - } - - EXPECT_FALSE(fs::exists("./snapshots/snapshot_50.bin.zstd")); - EXPECT_FALSE(fs::exists("./snapshots/snapshot_100.bin.zstd")); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_150.bin.zstd")); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_200.bin.zstd")); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin.zstd")); - - - auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); - - EXPECT_EQ(restored_storage->container.size(), 254); - - for (size_t i = 0; i < 250; ++i) - { - EXPECT_EQ(restored_storage->container.find("/hello_" + std::to_string(i))->value.getData(), "world_" + std::to_string(i)); - } -} - -TEST_F(RocksKeeperTest, TestStorageSnapshotMode) -{ - ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - DB::KeeperSnapshotManager manager(3, keeper_context); - DB::KeeperRocksStorage storage(500, "", keeper_context); - for (size_t i = 0; i < 50; ++i) - { - addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); - } - - { - DB::KeeperStorageSnapshot snapshot(&storage, 50); - for (size_t i = 0; i < 50; ++i) - { - addNode(storage, "/hello_" + std::to_string(i), "wlrd_" + std::to_string(i)); - } - for (size_t i = 0; i < 50; ++i) - { - EXPECT_EQ(storage.container.find("/hello_" + std::to_string(i))->value.getData(), "wlrd_" + std::to_string(i)); - } - for (size_t i = 0; i < 50; ++i) - { - if (i % 2 == 0) - storage.container.erase("/hello_" + std::to_string(i)); - } - EXPECT_EQ(storage.container.size(), 29); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().first, 54); - EXPECT_EQ(storage.container.snapshotSizeWithVersion().second, 1); - auto buf = manager.serializeSnapshotToBuffer(snapshot); - manager.serializeSnapshotBufferToDisk(*buf, 50); - } - EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin.zstd")); - EXPECT_EQ(storage.container.size(), 29); - storage.clearGarbageAfterSnapshot(); - for (size_t i = 0; i < 50; ++i) - { - if (i % 2 != 0) - EXPECT_EQ(storage.container.find("/hello_" + std::to_string(i))->value.getData(), "wlrd_" + std::to_string(i)); - else - EXPECT_FALSE(storage.container.contains("/hello_" + std::to_string(i))); - } - - auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); - - for (size_t i = 0; i < 50; ++i) - { - EXPECT_EQ(restored_storage->container.find("/hello_" + std::to_string(i))->value.getData(), "world_" + std::to_string(i)); - } -} - -TEST_F(RocksKeeperTest, TestStorageSnapshotBroken) -{ - ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - DB::KeeperSnapshotManager manager(3, keeper_context); - DB::KeeperRocksStorage storage(500, "", keeper_context); - for (size_t i = 0; i < 50; ++i) - { - addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i)); - } - { - DB::KeeperStorageSnapshot snapshot(&storage, 50); - auto buf = manager.serializeSnapshotToBuffer(snapshot); - manager.serializeSnapshotBufferToDisk(*buf, 50); - } - EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin.zstd")); - - /// Let's corrupt file - DB::WriteBufferFromFile plain_buf( - "./snapshots/snapshot_50.bin.zstd", DB::DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); - plain_buf.truncate(34); - plain_buf.sync(); - - EXPECT_THROW(manager.restoreFromLatestSnapshot(), DB::Exception); -} - -nuraft::ptr getBufferFromZKRequest(int64_t session_id, int64_t zxid, const Coordination::ZooKeeperRequestPtr & request); - -nuraft::ptr -getLogEntryFromZKRequest(size_t term, int64_t session_id, int64_t zxid, const Coordination::ZooKeeperRequestPtr & request); - -static void testLogAndStateMachine( - DB::CoordinationSettingsPtr settings, - uint64_t total_logs, - bool enable_compression = true) -{ - using namespace Coordination; - using namespace DB; - - ChangelogDirTest snapshots("./snapshots"); - ChangelogDirTest logs("./logs"); - ChangelogDirTest rocks("./rocksdb"); - - auto get_keeper_context = [&] - { - auto local_keeper_context = std::make_shared(true, settings); - local_keeper_context->setSnapshotDisk(std::make_shared("SnapshotDisk", "./snapshots")); - local_keeper_context->setLogDisk(std::make_shared("LogDisk", "./logs")); - local_keeper_context->setRocksDBDisk(std::make_shared("RocksDisk", "./rocksdb")); - local_keeper_context->setRocksDBOptions(nullptr); - return local_keeper_context; - }; - - ResponsesQueue queue(std::numeric_limits::max()); - SnapshotsQueue snapshots_queue{1}; - - auto keeper_context = get_keeper_context(); - auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); - - state_machine->init(); - DB::KeeperLogStore changelog( - DB::LogFileSettings{ - .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, - DB::FlushSettings(), - keeper_context); - changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items); - - for (size_t i = 1; i < total_logs + 1; ++i) - { - std::shared_ptr request = std::make_shared(); - request->path = "/hello_" + std::to_string(i); - auto entry = getLogEntryFromZKRequest(0, 1, i, request); - changelog.append(entry); - changelog.end_of_append_batch(0, 0); - - waitDurableLogs(changelog); - - state_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); - state_machine->commit(i, changelog.entry_at(i)->get_buf()); - bool snapshot_created = false; - if (i % settings->snapshot_distance == 0) - { - nuraft::snapshot s(i, 0, std::make_shared()); - nuraft::async_result::handler_type when_done - = [&snapshot_created](bool & ret, nuraft::ptr & /*exception*/) - { - snapshot_created = ret; - }; - - state_machine->create_snapshot(s, when_done); - CreateSnapshotTask snapshot_task; - bool pop_result = snapshots_queue.pop(snapshot_task); - EXPECT_TRUE(pop_result); - - snapshot_task.create_snapshot(std::move(snapshot_task.snapshot), false); - } - - if (snapshot_created && changelog.size() > settings->reserved_log_items) - changelog.compact(i - settings->reserved_log_items); - } - - SnapshotsQueue snapshots_queue1{1}; - keeper_context = get_keeper_context(); - auto restore_machine = std::make_shared>(queue, snapshots_queue1, keeper_context, nullptr); - restore_machine->init(); - EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance); - - DB::KeeperLogStore restore_changelog( - DB::LogFileSettings{ - .force_sync = true, .compress_logs = enable_compression, .rotate_interval = settings->rotate_log_storage_interval}, - DB::FlushSettings(), - keeper_context); - restore_changelog.init(restore_machine->last_commit_index() + 1, settings->reserved_log_items); - - EXPECT_EQ(restore_changelog.size(), std::min(settings->reserved_log_items + total_logs % settings->snapshot_distance, total_logs)); - EXPECT_EQ(restore_changelog.next_slot(), total_logs + 1); - if (total_logs > settings->reserved_log_items + 1) - EXPECT_EQ( - restore_changelog.start_index(), total_logs - total_logs % settings->snapshot_distance - settings->reserved_log_items + 1); - else - EXPECT_EQ(restore_changelog.start_index(), 1); - - for (size_t i = restore_machine->last_commit_index() + 1; i < restore_changelog.next_slot(); ++i) - { - restore_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); - restore_machine->commit(i, changelog.entry_at(i)->get_buf()); - } - - auto & source_storage = state_machine->getStorageUnsafe(); - auto & restored_storage = restore_machine->getStorageUnsafe(); - - EXPECT_EQ(source_storage.container.size(), restored_storage.container.size()); - for (size_t i = 1; i < total_logs + 1; ++i) - { - auto path = "/hello_" + std::to_string(i); - EXPECT_EQ(source_storage.container.find(path)->value.getData(), restored_storage.container.find(path)->value.getData()); - } -} - -TEST_F(RocksKeeperTest, TestStateMachineAndLogStore) -{ - using namespace Coordination; - using namespace DB; - - { - CoordinationSettingsPtr settings = std::make_shared(); - settings->snapshot_distance = 10; - settings->reserved_log_items = 10; - settings->rotate_log_storage_interval = 10; - - testLogAndStateMachine(settings, 37); - } - { - CoordinationSettingsPtr settings = std::make_shared(); - settings->snapshot_distance = 10; - settings->reserved_log_items = 10; - settings->rotate_log_storage_interval = 10; - testLogAndStateMachine(settings, 11); - } - { - CoordinationSettingsPtr settings = std::make_shared(); - settings->snapshot_distance = 10; - settings->reserved_log_items = 10; - settings->rotate_log_storage_interval = 10; - testLogAndStateMachine(settings, 40); - } - { - CoordinationSettingsPtr settings = std::make_shared(); - settings->snapshot_distance = 10; - settings->reserved_log_items = 20; - settings->rotate_log_storage_interval = 30; - testLogAndStateMachine(settings, 40); - } - { - CoordinationSettingsPtr settings = std::make_shared(); - settings->snapshot_distance = 10; - settings->reserved_log_items = 0; - settings->rotate_log_storage_interval = 10; - testLogAndStateMachine(settings, 40); - } - { - CoordinationSettingsPtr settings = std::make_shared(); - settings->snapshot_distance = 1; - settings->reserved_log_items = 1; - settings->rotate_log_storage_interval = 32; - testLogAndStateMachine(settings, 32); - } - { - CoordinationSettingsPtr settings = std::make_shared(); - settings->snapshot_distance = 10; - settings->reserved_log_items = 7; - settings->rotate_log_storage_interval = 1; - testLogAndStateMachine(settings, 33); - } - { - CoordinationSettingsPtr settings = std::make_shared(); - settings->snapshot_distance = 37; - settings->reserved_log_items = 1000; - settings->rotate_log_storage_interval = 5000; - testLogAndStateMachine(settings, 33); - } - { - CoordinationSettingsPtr settings = std::make_shared(); - settings->snapshot_distance = 37; - settings->reserved_log_items = 1000; - settings->rotate_log_storage_interval = 5000; - testLogAndStateMachine(settings, 45); - } -} - -TEST_F(RocksKeeperTest, TestEphemeralNodeRemove) -{ - using namespace Coordination; - using namespace DB; - - ChangelogDirTest snapshots("./snapshots"); - setSnapshotDirectory("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - ResponsesQueue queue(std::numeric_limits::max()); - SnapshotsQueue snapshots_queue{1}; - - auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); - state_machine->init(); - - std::shared_ptr request_c = std::make_shared(); - request_c->path = "/hello"; - request_c->is_ephemeral = true; - auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c); - state_machine->pre_commit(1, entry_c->get_buf()); - state_machine->commit(1, entry_c->get_buf()); - const auto & storage = state_machine->getStorageUnsafe(); - - EXPECT_EQ(storage.ephemerals.size(), 1); - std::shared_ptr request_d = std::make_shared(); - request_d->path = "/hello"; - /// Delete from other session - auto entry_d = getLogEntryFromZKRequest(0, 2, state_machine->getNextZxid(), request_d); - state_machine->pre_commit(2, entry_d->get_buf()); - state_machine->commit(2, entry_d->get_buf()); - - EXPECT_EQ(storage.ephemerals.size(), 0); -} - -TEST_F(RocksKeeperTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitted) -{ - using namespace Coordination; - using namespace DB; - - ChangelogDirTest snapshots("./snapshots"); - setSnapshotDirectory("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - ResponsesQueue queue(std::numeric_limits::max()); - SnapshotsQueue snapshots_queue{1}; - - auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); - state_machine->init(); - - String user_auth_data = "test_user:test_password"; - String digest = KeeperRocksStorage::generateDigest(user_auth_data); - - std::shared_ptr auth_req = std::make_shared(); - auth_req->scheme = "digest"; - auth_req->data = user_auth_data; - - // Add auth data to the session - auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req); - state_machine->pre_commit(1, auth_entry->get_buf()); - - // Create a node with 'auth' scheme for ACL - String node_path = "/hello"; - std::shared_ptr create_req = std::make_shared(); - create_req->path = node_path; - // When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can - // create nodes with this ACL. - create_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}}; - auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); - state_machine->pre_commit(2, create_entry->get_buf()); - - const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; - ASSERT_TRUE(uncommitted_state.nodes.contains(node_path)); - - // commit log entries - state_machine->commit(1, auth_entry->get_buf()); - state_machine->commit(2, create_entry->get_buf()); - - auto node = uncommitted_state.getNode(node_path); - ASSERT_NE(node, nullptr); - auto acls = uncommitted_state.getACLs(node_path); - ASSERT_EQ(acls.size(), 1); - EXPECT_EQ(acls[0].scheme, "digest"); - EXPECT_EQ(acls[0].id, digest); - EXPECT_EQ(acls[0].permissions, 31); -} - -TEST_F(RocksKeeperTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted) -{ - using namespace Coordination; - using namespace DB; - - ChangelogDirTest snapshots("./snapshots"); - setSnapshotDirectory("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - ResponsesQueue queue(std::numeric_limits::max()); - SnapshotsQueue snapshots_queue{1}; - - auto state_machine = std::make_shared>(queue, snapshots_queue, keeper_context, nullptr); - state_machine->init(); - - String user_auth_data = "test_user:test_password"; - String digest = KeeperRocksStorage::generateDigest(user_auth_data); - - std::shared_ptr auth_req = std::make_shared(); - auth_req->scheme = "digest"; - auth_req->data = user_auth_data; - - // Add auth data to the session - auto auth_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), auth_req); - state_machine->pre_commit(1, auth_entry->get_buf()); - - // Create a node - String node_path = "/hello"; - std::shared_ptr create_req = std::make_shared(); - create_req->path = node_path; - auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req); - state_machine->pre_commit(2, create_entry->get_buf()); - - // Set ACL with 'auth' scheme for ACL - std::shared_ptr set_acl_req = std::make_shared(); - set_acl_req->path = node_path; - // When 'auth' scheme is used the creator must have been authenticated by the server (for example, using 'digest' scheme) before it can - // set this ACL. - set_acl_req->acls = {{.permissions = 31, .scheme = "auth", .id = ""}}; - auto set_acl_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), set_acl_req); - state_machine->pre_commit(3, set_acl_entry->get_buf()); - - // commit all entries - state_machine->commit(1, auth_entry->get_buf()); - state_machine->commit(2, create_entry->get_buf()); - state_machine->commit(3, set_acl_entry->get_buf()); - - const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state; - auto node = uncommitted_state.getNode(node_path); - - ASSERT_NE(node, nullptr); - auto acls = uncommitted_state.getACLs(node_path); - ASSERT_EQ(acls.size(), 1); - EXPECT_EQ(acls[0].scheme, "digest"); - EXPECT_EQ(acls[0].id, digest); - EXPECT_EQ(acls[0].permissions, 31); -} - -TEST_F(RocksKeeperTest, TestStorageSnapshotEqual) -{ - ChangelogDirTest test("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setSnapshotDirectory("./snapshots"); - setRocksDBDirectory("./rocksdb"); - - std::optional snapshot_hash; - for (size_t i = 0; i < 15; ++i) - { - DB::KeeperSnapshotManager manager(3, keeper_context); - - DB::KeeperRocksStorage storage(500, "", keeper_context); - addNode(storage, "/hello", ""); - for (size_t j = 0; j < 100; ++j) - { - addNode(storage, "/hello_" + std::to_string(j), "world", 1); - addNode(storage, "/hello/somepath_" + std::to_string(j), "somedata", 3); - } - - storage.session_id_counter = 5; - - storage.ephemerals[3] = {"/hello"}; - storage.ephemerals[1] = {"/hello/somepath"}; - - for (size_t j = 0; j < 3333; ++j) - storage.getSessionID(130 * j); - - DB::KeeperStorageSnapshot snapshot(&storage, storage.zxid); - - auto buf = manager.serializeSnapshotToBuffer(snapshot); - - auto new_hash = sipHash128(reinterpret_cast(buf->data()), buf->size()); - if (!snapshot_hash.has_value()) - { - snapshot_hash = new_hash; - } - else - { - EXPECT_EQ(*snapshot_hash, new_hash); - } - } -} - -TEST_F(RocksKeeperTest, TestStorageSnapshotDifferentCompressions) -{ - ChangelogDirTest test("./snapshots"); - setSnapshotDirectory("./snapshots"); - ChangelogDirTest rocks("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - DB::KeeperSnapshotManager manager(3, keeper_context); - - DB::KeeperRocksStorage storage(500, "", keeper_context); - addNode(storage, "/hello1", "world", 1); - addNode(storage, "/hello2", "somedata", 3); - storage.session_id_counter = 5; - storage.zxid = 2; - storage.ephemerals[3] = {"/hello2"}; - storage.ephemerals[1] = {"/hello1"}; - storage.getSessionID(130); - storage.getSessionID(130); - - DB::KeeperStorageSnapshot snapshot(&storage, 2); - - auto buf = manager.serializeSnapshotToBuffer(snapshot); - manager.serializeSnapshotBufferToDisk(*buf, 2); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin.zstd")); - - DB::KeeperSnapshotManager new_manager(3, keeper_context, false); - - auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); - - auto [restored_storage, snapshot_meta, _] = new_manager.deserializeSnapshotFromBuffer(debuf); - - EXPECT_EQ(restored_storage->container.size(), 6); - EXPECT_EQ(restored_storage->container.find("/")->value.numChildren(), 3); - EXPECT_EQ(restored_storage->container.find("/hello1")->value.numChildren(), 0); - EXPECT_EQ(restored_storage->container.find("/hello2")->value.numChildren(), 0); - - EXPECT_EQ(restored_storage->container.find("/")->value.getData(), ""); - EXPECT_EQ(restored_storage->container.find("/hello1")->value.getData(), "world"); - EXPECT_EQ(restored_storage->container.find("/hello2")->value.getData(), "somedata"); - EXPECT_EQ(restored_storage->session_id_counter, 7); - EXPECT_EQ(restored_storage->zxid, 2); - EXPECT_EQ(restored_storage->ephemerals.size(), 2); - EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); - EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); - EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); -} - -template -ResponseType getSingleResponse(const auto & responses) -{ - EXPECT_FALSE(responses.empty()); - return dynamic_cast(*responses[0].response); -} - -TEST_F(RocksKeeperTest, TestUncommittedStateBasicCrud) -{ - ChangelogDirTest test("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - using namespace DB; - using namespace Coordination; - - DB::KeeperRocksStorage storage{500, "", keeper_context}; - - constexpr std::string_view path = "/test"; - - const auto get_committed_data = [&]() -> std::optional - { - auto request = std::make_shared(); - request->path = path; - auto responses = storage.processRequest(request, 0, std::nullopt, true, true); - const auto & get_response = getSingleResponse(responses); - - if (get_response.error != Error::ZOK) - return std::nullopt; - - return get_response.data; - }; - - const auto preprocess_get = [&](int64_t zxid) - { - auto get_request = std::make_shared(); - get_request->path = path; - storage.preprocessRequest(get_request, 0, 0, zxid); - return get_request; - }; - - const auto create_request = std::make_shared(); - create_request->path = path; - create_request->data = "initial_data"; - storage.preprocessRequest(create_request, 0, 0, 1); - storage.preprocessRequest(create_request, 0, 0, 2); - - ASSERT_FALSE(get_committed_data()); - - const auto after_create_get = preprocess_get(3); - - ASSERT_FALSE(get_committed_data()); - - const auto set_request = std::make_shared(); - set_request->path = path; - set_request->data = "new_data"; - storage.preprocessRequest(set_request, 0, 0, 4); - - const auto after_set_get = preprocess_get(5); - - ASSERT_FALSE(get_committed_data()); - - const auto remove_request = std::make_shared(); - remove_request->path = path; - storage.preprocessRequest(remove_request, 0, 0, 6); - storage.preprocessRequest(remove_request, 0, 0, 7); - - const auto after_remove_get = preprocess_get(8); - - ASSERT_FALSE(get_committed_data()); - - { - const auto responses = storage.processRequest(create_request, 0, 1); - const auto & create_response = getSingleResponse(responses); - ASSERT_EQ(create_response.error, Error::ZOK); - } - - { - const auto responses = storage.processRequest(create_request, 0, 2); - const auto & create_response = getSingleResponse(responses); - ASSERT_EQ(create_response.error, Error::ZNODEEXISTS); - } - - { - const auto responses = storage.processRequest(after_create_get, 0, 3); - const auto & get_response = getSingleResponse(responses); - ASSERT_EQ(get_response.error, Error::ZOK); - ASSERT_EQ(get_response.data, "initial_data"); - } - - ASSERT_EQ(get_committed_data(), "initial_data"); - - { - const auto responses = storage.processRequest(set_request, 0, 4); - const auto & create_response = getSingleResponse(responses); - ASSERT_EQ(create_response.error, Error::ZOK); - } - - { - const auto responses = storage.processRequest(after_set_get, 0, 5); - const auto & get_response = getSingleResponse(responses); - ASSERT_EQ(get_response.error, Error::ZOK); - ASSERT_EQ(get_response.data, "new_data"); - } - - ASSERT_EQ(get_committed_data(), "new_data"); - - { - const auto responses = storage.processRequest(remove_request, 0, 6); - const auto & create_response = getSingleResponse(responses); - ASSERT_EQ(create_response.error, Error::ZOK); - } - - { - const auto responses = storage.processRequest(remove_request, 0, 7); - const auto & remove_response = getSingleResponse(responses); - ASSERT_EQ(remove_response.error, Error::ZNONODE); - } - - { - const auto responses = storage.processRequest(after_remove_get, 0, 8); - const auto & get_response = getSingleResponse(responses); - ASSERT_EQ(get_response.error, Error::ZNONODE); - } - - ASSERT_FALSE(get_committed_data()); -} - -TEST_F(RocksKeeperTest, TestListRequestTypes) -{ - ChangelogDirTest test("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - using namespace DB; - using namespace Coordination; - KeeperRocksStorage storage{500, "", keeper_context}; - - int32_t zxid = 0; - - static constexpr std::string_view test_path = "/list_request_type/node"; - - const auto create_path = [&](const auto & path, bool is_ephemeral, bool is_sequential = true) - { - const auto create_request = std::make_shared(); - int new_zxid = ++zxid; - create_request->path = path; - create_request->is_sequential = is_sequential; - create_request->is_ephemeral = is_ephemeral; - storage.preprocessRequest(create_request, 1, 0, new_zxid); - auto responses = storage.processRequest(create_request, 1, new_zxid); - - EXPECT_GE(responses.size(), 1); - EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path; - const auto & create_response = dynamic_cast(*responses[0].response); - return create_response.path_created; - }; - - create_path(parentNodePath(StringRef{test_path}).toString(), false, false); - - static constexpr size_t persistent_num = 5; - std::unordered_set expected_persistent_children; - for (size_t i = 0; i < persistent_num; ++i) - { - expected_persistent_children.insert(getBaseNodeName(create_path(test_path, false)).toString()); - } - ASSERT_EQ(expected_persistent_children.size(), persistent_num); - - static constexpr size_t ephemeral_num = 5; - std::unordered_set expected_ephemeral_children; - for (size_t i = 0; i < ephemeral_num; ++i) - { - expected_ephemeral_children.insert(getBaseNodeName(create_path(test_path, true)).toString()); - } - ASSERT_EQ(expected_ephemeral_children.size(), ephemeral_num); - - const auto get_children = [&](const auto list_request_type) - { - const auto list_request = std::make_shared(); - int new_zxid = ++zxid; - list_request->path = parentNodePath(StringRef{test_path}).toString(); - list_request->list_request_type = list_request_type; - storage.preprocessRequest(list_request, 1, 0, new_zxid); - auto responses = storage.processRequest(list_request, 1, new_zxid); - - EXPECT_GE(responses.size(), 1); - const auto & list_response = dynamic_cast(*responses[0].response); - EXPECT_EQ(list_response.error, Coordination::Error::ZOK); - return list_response.names; - }; - - const auto persistent_children = get_children(ListRequestType::PERSISTENT_ONLY); - EXPECT_EQ(persistent_children.size(), persistent_num); - for (const auto & child : persistent_children) - { - EXPECT_TRUE(expected_persistent_children.contains(child)) << "Missing persistent child " << child; - } - - const auto ephemeral_children = get_children(ListRequestType::EPHEMERAL_ONLY); - EXPECT_EQ(ephemeral_children.size(), ephemeral_num); - for (const auto & child : ephemeral_children) - { - EXPECT_TRUE(expected_ephemeral_children.contains(child)) << "Missing ephemeral child " << child; - } - - const auto all_children = get_children(ListRequestType::ALL); - EXPECT_EQ(all_children.size(), ephemeral_num + persistent_num); - for (const auto & child : all_children) - { - EXPECT_TRUE(expected_ephemeral_children.contains(child) || expected_persistent_children.contains(child)) - << "Missing child " << child; - } -} - -TEST_F(RocksKeeperTest, TestFeatureFlags) -{ - ChangelogDirTest test("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - using namespace Coordination; - KeeperMemoryStorage storage{500, "", keeper_context}; - auto request = std::make_shared(); - request->path = DB::keeper_api_feature_flags_path; - auto responses = storage.processRequest(request, 0, std::nullopt, true, true); - const auto & get_response = getSingleResponse(responses); - DB::KeeperFeatureFlags feature_flags; - feature_flags.setFeatureFlags(get_response.data); - ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::FILTERED_LIST)); - ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ)); - ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS)); -} - -TEST_F(RocksKeeperTest, TestSystemNodeModify) -{ - ChangelogDirTest test("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - using namespace Coordination; - int64_t zxid{0}; - - // On INIT we abort when a system path is modified - keeper_context->setServerState(KeeperContext::Phase::RUNNING); - KeeperRocksStorage storage{500, "", keeper_context}; - const auto assert_create = [&](const std::string_view path, const auto expected_code) - { - auto request = std::make_shared(); - request->path = path; - storage.preprocessRequest(request, 0, 0, zxid); - auto responses = storage.processRequest(request, 0, zxid); - ASSERT_FALSE(responses.empty()); - - const auto & response = responses[0]; - ASSERT_EQ(response.response->error, expected_code) << "Unexpected error for path " << path; - - ++zxid; - }; - - assert_create("/keeper", Error::ZBADARGUMENTS); - assert_create("/keeper/with_child", Error::ZBADARGUMENTS); - assert_create(DB::keeper_api_version_path, Error::ZBADARGUMENTS); - - assert_create("/keeper_map", Error::ZOK); - assert_create("/keeper1", Error::ZOK); - assert_create("/keepe", Error::ZOK); - assert_create("/keeper1/test", Error::ZOK); -} - -TEST_F(RocksKeeperTest, TestCheckNotExistsRequest) -{ - ChangelogDirTest test("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - using namespace DB; - using namespace Coordination; - - KeeperRocksStorage storage{500, "", keeper_context}; - - int32_t zxid = 0; - - const auto create_path = [&](const auto & path) - { - const auto create_request = std::make_shared(); - int new_zxid = ++zxid; - create_request->path = path; - storage.preprocessRequest(create_request, 1, 0, new_zxid); - auto responses = storage.processRequest(create_request, 1, new_zxid); - - EXPECT_GE(responses.size(), 1); - EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path; - }; - - const auto check_request = std::make_shared(); - check_request->path = "/test_node"; - check_request->not_exists = true; - - { - SCOPED_TRACE("CheckNotExists returns ZOK"); - int new_zxid = ++zxid; - storage.preprocessRequest(check_request, 1, 0, new_zxid); - auto responses = storage.processRequest(check_request, 1, new_zxid); - EXPECT_GE(responses.size(), 1); - auto error = responses[0].response->error; - EXPECT_EQ(error, Coordination::Error::ZOK) << "CheckNotExists returned invalid result: " << errorMessage(error); - } - - 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; - - { - SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS"); - int new_zxid = ++zxid; - storage.preprocessRequest(check_request, 1, 0, new_zxid); - auto responses = storage.processRequest(check_request, 1, new_zxid); - EXPECT_GE(responses.size(), 1); - auto error = responses[0].response->error; - EXPECT_EQ(error, Coordination::Error::ZNODEEXISTS) << "CheckNotExists returned invalid result: " << errorMessage(error); - } - - { - SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS for same version"); - int new_zxid = ++zxid; - check_request->version = node_version; - storage.preprocessRequest(check_request, 1, 0, new_zxid); - auto responses = storage.processRequest(check_request, 1, new_zxid); - EXPECT_GE(responses.size(), 1); - auto error = responses[0].response->error; - EXPECT_EQ(error, Coordination::Error::ZNODEEXISTS) << "CheckNotExists returned invalid result: " << errorMessage(error); - } - - { - SCOPED_TRACE("CheckNotExists returns ZOK for different version"); - int new_zxid = ++zxid; - check_request->version = node_version + 1; - storage.preprocessRequest(check_request, 1, 0, new_zxid); - auto responses = storage.processRequest(check_request, 1, new_zxid); - EXPECT_GE(responses.size(), 1); - auto error = responses[0].response->error; - EXPECT_EQ(error, Coordination::Error::ZOK) << "CheckNotExists returned invalid result: " << errorMessage(error); - } -} - -TEST_F(RocksKeeperTest, TestReapplyingDeltas) -{ - ChangelogDirTest test("./rocksdb"); - setRocksDBDirectory("./rocksdb"); - - using namespace DB; - using namespace Coordination; - - static constexpr int64_t initial_zxid = 100; - - const auto create_request = std::make_shared(); - create_request->path = "/test/data"; - create_request->is_sequential = true; - - const auto process_create = [](KeeperRocksStorage & storage, const auto & request, int64_t zxid) - { - storage.preprocessRequest(request, 1, 0, zxid); - auto responses = storage.processRequest(request, 1, zxid); - EXPECT_GE(responses.size(), 1); - EXPECT_EQ(responses[0].response->error, Error::ZOK); - }; - - const auto commit_initial_data = [&](auto & storage) - { - int64_t zxid = 1; - - const auto root_create = std::make_shared(); - root_create->path = "/test"; - process_create(storage, root_create, zxid); - ++zxid; - - for (; zxid <= initial_zxid; ++zxid) - process_create(storage, create_request, zxid); - }; - - KeeperRocksStorage storage1{500, "", keeper_context}; - commit_initial_data(storage1); - - for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) - storage1.preprocessRequest(create_request, 1, 0, zxid, /*check_acl=*/true, /*digest=*/std::nullopt, /*log_idx=*/zxid); - - /// create identical new storage - KeeperRocksStorage storage2{500, "", keeper_context}; - commit_initial_data(storage2); - - storage1.applyUncommittedState(storage2, initial_zxid); - - const auto commit_unprocessed = [&](KeeperRocksStorage & storage) - { - for (int64_t zxid = initial_zxid + 1; zxid < initial_zxid + 50; ++zxid) - { - auto responses = storage.processRequest(create_request, 1, zxid); - EXPECT_GE(responses.size(), 1); - EXPECT_EQ(responses[0].response->error, Error::ZOK); - } - }; - - commit_unprocessed(storage1); - commit_unprocessed(storage2); - - const auto get_children = [&](KeeperRocksStorage & storage) - { - const auto list_request = std::make_shared(); - list_request->path = "/test"; - auto responses = storage.processRequest(list_request, 1, std::nullopt, /*check_acl=*/true, /*is_local=*/true); - EXPECT_EQ(responses.size(), 1); - const auto * list_response = dynamic_cast(responses[0].response.get()); - EXPECT_TRUE(list_response); - return list_response->names; - }; - - auto children1 = get_children(storage1); - std::unordered_set children1_set(children1.begin(), children1.end()); - - auto children2 = get_children(storage2); - std::unordered_set children2_set(children2.begin(), children2.end()); - - ASSERT_TRUE(children1_set == children2_set); -} - -#endif From d89ed1fdcb34b23858e22325f19f9e0fc38e60bb Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 9 Jul 2024 15:22:20 +0000 Subject: [PATCH 28/79] disable WAL --- src/Coordination/RocksDBContainer.h | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index a4a236f332e..46dc755aad8 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -8,6 +8,7 @@ #include #include +#include #include #include #include @@ -195,7 +196,6 @@ public: rocksdb_dir, status.ToString()); } rocksdb_ptr = std::unique_ptr(db); - /// storage_ptr = storage_; initialized = true; } @@ -312,8 +312,10 @@ public: } else if (status.IsNotFound()) { - // storage->addDigest(value, key); - status = rocksdb_ptr->Put(rocksdb::WriteOptions(), encoded_key, value.getEncodedString()); + rocksdb::WriteOptions write_options; + write_options.disableWAL = true; + + status = rocksdb_ptr->Put(write_options, encoded_key, value.getEncodedString()); if (status.ok()) { counter++; @@ -336,7 +338,10 @@ public: else if (!status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during get. The error message is {}.", status.ToString()); - status = rocksdb_ptr->Put(rocksdb::WriteOptions(), encoded_key, value.getEncodedString()); + rocksdb::WriteOptions write_options; + write_options.disableWAL = true; + + status = rocksdb_ptr->Put(write_options, encoded_key, value.getEncodedString()); if (status.ok()) counter += increase_counter; else @@ -361,7 +366,11 @@ public: { /// storage->removeDigest(value, key); const std::string & encoded_key = getEncodedKey(key); - auto status = rocksdb_ptr->Delete(rocksdb::WriteOptions(), encoded_key); + + rocksdb::WriteOptions write_options; + write_options.disableWAL = true; + + auto status = rocksdb_ptr->Delete(write_options, encoded_key); if (status.IsNotFound()) return false; if (status.ok()) @@ -445,8 +454,6 @@ private: std::unique_ptr rocksdb_ptr; - /// Storage* storage_ptr; - const rocksdb::Snapshot * snapshot; bool snapshot_mode{false}; From fbaf6119eaad06c7ceb4eaa54cc346fbad20f50a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 9 Jul 2024 17:08:19 +0000 Subject: [PATCH 29/79] fix indexHint with fuzzer --- src/Functions/indexHint.h | 4 ++++ tests/queries/0_stateless/03204_index_hint_fuzzer.reference | 1 + tests/queries/0_stateless/03204_index_hint_fuzzer.sql | 1 + 3 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03204_index_hint_fuzzer.reference create mode 100644 tests/queries/0_stateless/03204_index_hint_fuzzer.sql diff --git a/src/Functions/indexHint.h b/src/Functions/indexHint.h index 3b71c7a5585..a71e1555c78 100644 --- a/src/Functions/indexHint.h +++ b/src/Functions/indexHint.h @@ -42,6 +42,10 @@ public: bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + + bool useDefaultImplementationForSparseColumns() const override { return false; } + bool isSuitableForConstantFolding() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } diff --git a/tests/queries/0_stateless/03204_index_hint_fuzzer.reference b/tests/queries/0_stateless/03204_index_hint_fuzzer.reference new file mode 100644 index 00000000000..f750ba35102 --- /dev/null +++ b/tests/queries/0_stateless/03204_index_hint_fuzzer.reference @@ -0,0 +1 @@ +(1,1) diff --git a/tests/queries/0_stateless/03204_index_hint_fuzzer.sql b/tests/queries/0_stateless/03204_index_hint_fuzzer.sql new file mode 100644 index 00000000000..5794f3eee53 --- /dev/null +++ b/tests/queries/0_stateless/03204_index_hint_fuzzer.sql @@ -0,0 +1 @@ +SELECT tuple(indexHint(toLowCardinality('aaa')), 1); From 0c843ae1463fd94026fb0367845e3c43bdfa2ab4 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 10 Jul 2024 13:01:15 +0000 Subject: [PATCH 30/79] disable rocksdb --- src/Coordination/CoordinationSettings.h | 2 +- src/Coordination/KeeperContext.cpp | 2 +- src/Coordination/KeeperServer.cpp | 2 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 5116a814871..35a23fc9e78 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -55,7 +55,7 @@ struct Settings; M(UInt64, min_request_size_for_cache, 50 * 1024, "Minimal size of the request to cache the deserialization result. Caching can have negative effect on latency for smaller requests, set to 0 to disable", 0) \ M(UInt64, raft_limits_reconnect_limit, 50, "If connection to a peer is silent longer than this limit * (multiplied by heartbeat interval), we re-establish the connection.", 0) \ M(Bool, async_replication, false, "Enable async replication. All write and read guarantees are preserved while better performance is achieved. Settings is disabled by default to not break backwards compatibility.", 0) \ - M(Bool, use_rocksdb, true, "Use rocksdb as backend storage", 0) \ + M(Bool, experimental_use_rocksdb, false, "Use rocksdb as backend storage", 0) \ M(UInt64, latest_logs_cache_size_threshold, 1 * 1024 * 1024 * 1024, "Maximum total size of in-memory cache of latest log entries.", 0) \ M(UInt64, commit_logs_cache_size_threshold, 500 * 1024 * 1024, "Maximum total size of in-memory cache of log entries needed next for commit.", 0) \ M(UInt64, disk_move_retries_wait_ms, 1000, "How long to wait between retries after a failure which happened while a file was being moved between disks.", 0) \ diff --git a/src/Coordination/KeeperContext.cpp b/src/Coordination/KeeperContext.cpp index b65bb02ca5d..60d042403e5 100644 --- a/src/Coordination/KeeperContext.cpp +++ b/src/Coordination/KeeperContext.cpp @@ -160,7 +160,7 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, initializeDisks(config); #if USE_ROCKSDB - if (config.getBool("keeper_server.coordination_settings.use_rocksdb", true)) + if (config.getBool("keeper_server.coordination_settings.experimental_use_rocksdb", false)) { rocksdb_options = std::make_shared(getRocksDBOptionsFromConfig(config)); digest_enabled = false; /// TODO: support digest diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 7352873107c..14b51d716de 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -134,7 +134,7 @@ KeeperServer::KeeperServer( #if USE_ROCKSDB const auto & coordination_settings = keeper_context->getCoordinationSettings(); - if (coordination_settings->use_rocksdb) + if (coordination_settings->experimental_use_rocksdb) { state_machine = nuraft::cs_new>( responses_queue_, diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 2f505873c65..360f848ffd1 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -85,7 +85,7 @@ public: Poco::Logger::root().setLevel("trace"); auto settings = std::make_shared(); - settings->use_rocksdb = true; + settings->experimental_use_rocksdb = true; keeper_context = std::make_shared(true, settings); keeper_context->setLocalLogsPreprocessed(); keeper_context->setRocksDBOptions(); From f81383b856293de861c9bb76e7eac3597ffac503 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 11 Jul 2024 12:33:41 +0000 Subject: [PATCH 31/79] tiny fix --- src/Coordination/KeeperServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 14b51d716de..fd1c5475911 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -140,7 +140,7 @@ KeeperServer::KeeperServer( responses_queue_, snapshots_queue_, keeper_context, - config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr, + config.getBool("keeper_server.upload_snapshot_on_exit", false) ? &snapshot_manager_s3 : nullptr, commit_callback, checkAndGetSuperdigest(configuration_and_settings_->super_digest)); LOG_WARNING(log, "Use RocksDB as Keeper backend storage."); From 35725b666f6afdf115aa977fadd15d2076cba020 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 Jul 2024 13:23:46 +0000 Subject: [PATCH 32/79] address comments --- src/Coordination/KeeperStateMachine.h | 2 -- src/Coordination/KeeperStorage.h | 7 ++++--- src/Coordination/RocksDBContainer.h | 11 ++--------- 3 files changed, 6 insertions(+), 14 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 4305e3b173d..dd2f239f551 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -114,8 +114,6 @@ protected: nuraft::ptr latest_snapshot_buf = nullptr; CoordinationSettingsPtr coordination_settings; - /// Main state machine logic - /// KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock); /// Save/Load and Serialize/Deserialize logic for snapshots. /// Put processed responses into this queue diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index ef8a8d1410e..4a9286d4835 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -19,12 +19,12 @@ namespace DB class KeeperContext; using KeeperContextPtr = std::shared_ptr; -/// struct KeeperStorageRequestProcessor; -/// using KeeperStorageRequestProcessorPtr = std::shared_ptr; using ResponseCallback = std::function; using ChildrenSet = absl::flat_hash_set; using SessionAndTimeout = std::unordered_map; +/// KeeperRocksNodeInfo is used in RocksDB keeper. +/// It is serialized directly as POD to RocksDB. struct KeeperRocksNodeInfo { int64_t czxid{0}; @@ -144,7 +144,8 @@ struct KeeperRocksNodeInfo void copyStats(const Coordination::Stat & stat); }; -struct KeeperRocksNode:KeeperRocksNodeInfo +/// KeeperRocksNode is the memory structure used by RocksDB +struct KeeperRocksNode : public KeeperRocksNodeInfo { #if USE_ROCKSDB friend struct RocksDBContainer; diff --git a/src/Coordination/RocksDBContainer.h b/src/Coordination/RocksDBContainer.h index 46dc755aad8..12b40bbb87e 100644 --- a/src/Coordination/RocksDBContainer.h +++ b/src/Coordination/RocksDBContainer.h @@ -196,6 +196,7 @@ public: rocksdb_dir, status.ToString()); } rocksdb_ptr = std::unique_ptr(db); + write_options.disableWAL = true; initialized = true; } @@ -312,9 +313,6 @@ public: } else if (status.IsNotFound()) { - rocksdb::WriteOptions write_options; - write_options.disableWAL = true; - status = rocksdb_ptr->Put(write_options, encoded_key, value.getEncodedString()); if (status.ok()) { @@ -338,9 +336,6 @@ public: else if (!status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "Got rocksdb error during get. The error message is {}.", status.ToString()); - rocksdb::WriteOptions write_options; - write_options.disableWAL = true; - status = rocksdb_ptr->Put(write_options, encoded_key, value.getEncodedString()); if (status.ok()) counter += increase_counter; @@ -367,9 +362,6 @@ public: /// storage->removeDigest(value, key); const std::string & encoded_key = getEncodedKey(key); - rocksdb::WriteOptions write_options; - write_options.disableWAL = true; - auto status = rocksdb_ptr->Delete(write_options, encoded_key); if (status.IsNotFound()) return false; @@ -453,6 +445,7 @@ private: String rocksdb_dir; std::unique_ptr rocksdb_ptr; + rocksdb::WriteOptions write_options; const rocksdb::Snapshot * snapshot; From 73ecbefd4123bae9ca9902111c28ace0cc05b994 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 07:52:33 +0200 Subject: [PATCH 33/79] Fix bad log message --- src/Common/ZooKeeper/ZooKeeper.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 56db9adb787..01bb508da95 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -13,14 +13,14 @@ #include #include #include -#include #include +#include #include #include #include -#include "Common/ZooKeeper/IKeeper.h" -#include +#include #include +#include #include #include @@ -114,7 +114,11 @@ void ZooKeeper::init(ZooKeeperArgs args_, std::unique_ptr /// availability_zones is empty on server startup or after config reloading /// We will keep the az info when starting new sessions availability_zones = args.availability_zones; - LOG_TEST(log, "Availability zones from config: [{}], client: {}", fmt::join(availability_zones, ", "), args.client_availability_zone); + + LOG_TEST(log, "Availability zones from config: [{}], client: {}", + fmt::join(collections::map(availability_zones, [](auto s){ return DB::quoteString(s); }), ", "), + DB::quoteString(args.client_availability_zone)); + if (args.availability_zone_autodetect) updateAvailabilityZones(); } From f1350b442c79854de25a06efca66fc888225f93b Mon Sep 17 00:00:00 2001 From: cangyin Date: Wed, 17 Jul 2024 17:30:27 +0800 Subject: [PATCH 34/79] Negative sign in prewhere optimization --- src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index e7b3585ecda..44ab12bde2a 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -233,7 +233,7 @@ static bool isConditionGood(const RPNBuilderTreeNode & condition, const NameSet else if (type == Field::Types::Float64) { const auto value = output_value.get(); - return value < threshold || threshold < value; + return value < -threshold || threshold < value; } return false; From e4816e5c2283dd6254e9d36991b60ad58c2c2dc0 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 17 Jul 2024 13:42:32 +0200 Subject: [PATCH 35/79] replace no-asan with no-azure-blob-storage tag in heavy tests --- .../03008_deduplication_insert_several_blocks_nonreplicated.sh | 2 +- .../03008_deduplication_insert_several_blocks_replicated.sh | 2 +- ...8_deduplication_mv_generates_several_blocks_nonreplicated.sh | 2 +- ...3008_deduplication_mv_generates_several_blocks_replicated.sh | 2 +- ...008_deduplication_several_mv_into_one_table_nonreplicated.sh | 2 +- .../03008_deduplication_several_mv_into_one_table_replicated.sh | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh index 0791c7566f9..f0a723501de 100755 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-asan +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh index 661dfa5f930..1b7f9be9510 100755 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-asan +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh index 10083506af4..e76b6eb77ce 100755 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-asan +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh index 0d5158d18cd..7e6cf5e95c9 100755 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-asan +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh index a9a135d6839..193e15d311c 100755 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-asan +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh index 49d556e70f9..3f4f0b17258 100755 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-asan +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8e013a1c5938e7cecba5804ac8e9f653a95e4ddc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Jul 2024 16:23:36 +0000 Subject: [PATCH 36/79] Disable getConstantResultForNonConstArguments for IS NULL with old analyzer. --- src/Functions/isNotNull.cpp | 10 +++++++- src/Functions/isNull.cpp | 15 ++++++++++-- src/Functions/isNullable.cpp | 15 ++++++++++-- ...constant_result_old_analyzer_bug.reference | 0 ..._null_constant_result_old_analyzer_bug.sql | 23 +++++++++++++++++++ 5 files changed, 58 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03206_is_null_constant_result_old_analyzer_bug.reference create mode 100644 tests/queries/0_stateless/03206_is_null_constant_result_old_analyzer_bug.sql diff --git a/src/Functions/isNotNull.cpp b/src/Functions/isNotNull.cpp index a10e7ebd40c..a48ace4243f 100644 --- a/src/Functions/isNotNull.cpp +++ b/src/Functions/isNotNull.cpp @@ -22,7 +22,9 @@ class FunctionIsNotNull : public IFunction public: static constexpr auto name = "isNotNull"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr context) { return std::make_shared(context->getSettingsRef().allow_experimental_analyzer); } + + explicit FunctionIsNotNull(bool use_analyzer_) : use_analyzer(use_analyzer_) {} std::string getName() const override { @@ -31,6 +33,10 @@ public: ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override { + /// (column IS NULL) triggers a bug in old analyzer when it is replaced to constant. + if (!use_analyzer) + return nullptr; + const ColumnWithTypeAndName & elem = arguments[0]; if (elem.type->onlyNull()) return result_type->createColumnConst(1, UInt8(0)); @@ -123,6 +129,8 @@ private: #endif vectorImpl(null_map, res); } + + bool use_analyzer; }; } diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index 95d659b103b..3317d73ba27 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include namespace DB @@ -21,11 +23,13 @@ class FunctionIsNull : public IFunction public: static constexpr auto name = "isNull"; - static FunctionPtr create(ContextPtr) + static FunctionPtr create(ContextPtr context) { - return std::make_shared(); + return std::make_shared(context->getSettingsRef().allow_experimental_analyzer); } + explicit FunctionIsNull(bool use_analyzer_) : use_analyzer(use_analyzer_) {} + std::string getName() const override { return name; @@ -33,6 +37,10 @@ public: ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override { + /// (column IS NULL) triggers a bug in old analyzer when it is replaced to constant. + if (!use_analyzer) + return nullptr; + const ColumnWithTypeAndName & elem = arguments[0]; if (elem.type->onlyNull()) return result_type->createColumnConst(1, UInt8(1)); @@ -95,6 +103,9 @@ public: return DataTypeUInt8().createColumnConst(elem.column->size(), 0u); } } + +private: + bool use_analyzer; }; } diff --git a/src/Functions/isNullable.cpp b/src/Functions/isNullable.cpp index b24ee4f5e73..d21ac9cf07c 100644 --- a/src/Functions/isNullable.cpp +++ b/src/Functions/isNullable.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB { @@ -14,11 +16,13 @@ class FunctionIsNullable : public IFunction { public: static constexpr auto name = "isNullable"; - static FunctionPtr create(ContextPtr) + static FunctionPtr create(ContextPtr context) { - return std::make_shared(); + return std::make_shared(context->getSettingsRef().allow_experimental_analyzer); } + explicit FunctionIsNullable(bool use_analyzer_) : use_analyzer(use_analyzer_) {} + String getName() const override { return name; @@ -26,6 +30,10 @@ public: ColumnPtr getConstantResultForNonConstArguments(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override { + /// isNullable(column) triggers a bug in old analyzer when it is replaced to constant. + if (!use_analyzer) + return nullptr; + const ColumnWithTypeAndName & elem = arguments[0]; if (elem.type->onlyNull() || canContainNull(*elem.type)) return result_type->createColumnConst(1, UInt8(1)); @@ -60,6 +68,9 @@ public: const auto & elem = arguments[0]; return ColumnUInt8::create(input_rows_count, isColumnNullable(*elem.column) || elem.type->isLowCardinalityNullable()); } + +private: + bool use_analyzer; }; } diff --git a/tests/queries/0_stateless/03206_is_null_constant_result_old_analyzer_bug.reference b/tests/queries/0_stateless/03206_is_null_constant_result_old_analyzer_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03206_is_null_constant_result_old_analyzer_bug.sql b/tests/queries/0_stateless/03206_is_null_constant_result_old_analyzer_bug.sql new file mode 100644 index 00000000000..39e9ebfcd77 --- /dev/null +++ b/tests/queries/0_stateless/03206_is_null_constant_result_old_analyzer_bug.sql @@ -0,0 +1,23 @@ +CREATE TABLE left (x UUID) ORDER BY tuple(); + +CREATE TABLE right (x UUID) ORDER BY tuple(); + +set allow_experimental_analyzer=0; + +SELECT left.x, (right.x IS NULL)::Boolean FROM left LEFT OUTER JOIN right ON left.x = right.x GROUP BY ALL; + +SELECT isNullable(number)::Boolean, now() FROM numbers(2) GROUP BY isNullable(number)::Boolean, now() FORMAT Null; + +SELECT isNull(number)::Boolean, now() FROM numbers(2) GROUP BY isNull(number)::Boolean, now() FORMAT Null; + +SELECT (number IS NULL)::Boolean, now() FROM numbers(2) GROUP BY (number IS NULL)::Boolean, now() FORMAT Null; + +set allow_experimental_analyzer=1; + +SELECT left.x, (right.x IS NULL)::Boolean FROM left LEFT OUTER JOIN right ON left.x = right.x GROUP BY ALL; + +SELECT isNullable(number)::Boolean, now() FROM numbers(2) GROUP BY isNullable(number)::Boolean, now() FORMAT Null; + +SELECT isNull(number)::Boolean, now() FROM numbers(2) GROUP BY isNull(number)::Boolean, now() FORMAT Null; + +SELECT (number IS NULL)::Boolean, now() FROM numbers(2) GROUP BY (number IS NULL)::Boolean, now() FORMAT Null; From a3d02291b42ad0de78534bf9ae64a2f0dc3a3826 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Jul 2024 18:24:13 +0200 Subject: [PATCH 37/79] Increase backoff a little for inserts --- tests/config/users.d/insert_keeper_retries.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml index f01157d40ac..0cf41473616 100644 --- a/tests/config/users.d/insert_keeper_retries.xml +++ b/tests/config/users.d/insert_keeper_retries.xml @@ -4,7 +4,7 @@ 100 0.01 1 - 1 + 10 From 8bc537b490c5fc8ac499227e0b1cbb4ccb6a5af2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Jul 2024 17:26:51 +0000 Subject: [PATCH 38/79] Allo scalar subquery in fisrt argiment of IN with new analyzer. --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 11 +++++++++++ .../0_stateless/02370_analyzer_in_function.reference | 3 +++ .../0_stateless/02370_analyzer_in_function.sql | 4 ++++ 3 files changed, 18 insertions(+) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 63788560959..5b31d727e37 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -2919,6 +2919,17 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi resolveExpressionNode(in_second_argument, scope, false /*allow_lambda_expression*/, true /*allow_table_expression*/); } + + /// Edge case when the first argument of IN is scalar subquery. + auto & in_first_argument = function_in_arguments_nodes[0]; + auto first_argument_type = in_first_argument->getNodeType(); + if (first_argument_type == QueryTreeNodeType::QUERY || first_argument_type == QueryTreeNodeType::UNION) + { + IdentifierResolveScope subquery_scope(in_first_argument, &scope /*parent_scope*/); + subquery_scope.subquery_depth = scope.subquery_depth + 1; + + evaluateScalarSubqueryIfNeeded(in_first_argument, subquery_scope); + } } /// Initialize function argument columns diff --git a/tests/queries/0_stateless/02370_analyzer_in_function.reference b/tests/queries/0_stateless/02370_analyzer_in_function.reference index fda174c0b7c..49a080c2616 100644 --- a/tests/queries/0_stateless/02370_analyzer_in_function.reference +++ b/tests/queries/0_stateless/02370_analyzer_in_function.reference @@ -12,3 +12,6 @@ 1 1 0 +1 +1 +1 diff --git a/tests/queries/0_stateless/02370_analyzer_in_function.sql b/tests/queries/0_stateless/02370_analyzer_in_function.sql index 7287c94deda..a6e4400e101 100644 --- a/tests/queries/0_stateless/02370_analyzer_in_function.sql +++ b/tests/queries/0_stateless/02370_analyzer_in_function.sql @@ -21,3 +21,7 @@ SELECT (1, 2) IN 1; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT (1, 2) IN [1]; -- { serverError INCORRECT_ELEMENT_OF_SET } SELECT (1, 2) IN (((1, 2), (1, 2)), ((1, 2), (1, 2))); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT (1, 2) IN [((1, 2), (1, 2)), ((1, 2), (1, 2))]; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +select (select 1) in (1); +select in(untuple(((1), (1)))); +select in(untuple(((select 1), (1)))); From b4a3cba519b397f91ec38e806dcc5597b846a1c5 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 Jul 2024 20:09:14 +0200 Subject: [PATCH 39/79] Fix possible PARAMETER_OUT_OF_BOUND error during reading variant subcolumn --- .../SerializationVariantElement.cpp | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationVariantElement.cpp b/src/DataTypes/Serializations/SerializationVariantElement.cpp index 8ceab17cba4..03b5d9584e0 100644 --- a/src/DataTypes/Serializations/SerializationVariantElement.cpp +++ b/src/DataTypes/Serializations/SerializationVariantElement.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } struct SerializationVariantElement::DeserializeBinaryBulkStateVariantElement : public ISerialization::DeserializeBinaryBulkState @@ -188,13 +189,6 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( assert_cast(*variant_element_state->variant->assumeMutable()).nestedRemoveNullable(); } - /// If nothing to deserialize, just insert defaults. - if (variant_limit == 0) - { - mutable_column->insertManyDefaults(num_new_discriminators); - return; - } - addVariantToPath(settings.path); nested_serialization->deserializeBinaryBulkWithMultipleStreams(variant_element_state->variant, *variant_limit, settings, variant_element_state->variant_element_state, cache); removeVariantFromPath(settings.path); @@ -209,6 +203,17 @@ void SerializationVariantElement::deserializeBinaryBulkWithMultipleStreams( return; } + /// If there was nothing to deserialize or nothing was actually deserialized when variant_limit > 0, just insert defaults. + /// The second case means that we don't have a stream for such sub-column. It may happen during ALTER MODIFY column with Variant extension. + if (variant_limit == 0 || variant_element_state->variant->empty()) + { + mutable_column->insertManyDefaults(num_new_discriminators); + return; + } + + if (variant_element_state->variant->size() < *variant_limit) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of deserialized variant column less than the limit: {} < {}", variant_element_state->variant->size(), *variant_limit); + size_t variant_offset = variant_element_state->variant->size() - *variant_limit; /// If we have only our discriminator in range, insert the whole range to result column. From 787a9137df53d6c46f0b6f47ba21b23f49f8e749 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jul 2024 03:08:34 +0200 Subject: [PATCH 40/79] Fix bad test `02210_processors_profile_log` --- tests/queries/0_stateless/02210_processors_profile_log.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02210_processors_profile_log.sql b/tests/queries/0_stateless/02210_processors_profile_log.sql index 59edbb71457..75e5bcbb585 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log.sql +++ b/tests/queries/0_stateless/02210_processors_profile_log.sql @@ -14,10 +14,12 @@ SELECT multiIf( -- ExpressionTransform executes sleep(), -- so IProcessor::work() will spend 1 sec. - name = 'ExpressionTransform', elapsed_us>=1e6 ? 1 : elapsed_us, + -- We use two different timers to measure time: CLOCK_MONOTONIC for sleep and CLOCK_MONOTONIC_COARSE for profiling + -- that's why we cannot compare directly with 1,000,000 microseconds - let's compare with 900,000 microseconds. + name = 'ExpressionTransform', elapsed_us >= 0.9e6 ? 1 : elapsed_us, -- SourceFromSingleChunk, that feed data to ExpressionTransform, -- will feed first block and then wait in PortFull. - name = 'SourceFromSingleChunk', output_wait_elapsed_us>=1e6 ? 1 : output_wait_elapsed_us, + name = 'SourceFromSingleChunk', output_wait_elapsed_us >= 0.9e6 ? 1 : output_wait_elapsed_us, -- NullSource/LazyOutputFormatLazyOutputFormat are the outputs -- so they cannot starts to execute before sleep(1) will be executed. input_wait_elapsed_us>=1e6 ? 1 : input_wait_elapsed_us) From f16a05cf2d31c1306e89e2fa3e990233a5d34288 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 10:51:53 +0000 Subject: [PATCH 41/79] Fix a test. --- tests/queries/0_stateless/02892_orc_filter_pushdown.reference | 2 +- tests/queries/0_stateless/02892_orc_filter_pushdown.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.reference b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference index 903d42bf492..0be6d13adb3 100644 --- a/tests/queries/0_stateless/02892_orc_filter_pushdown.reference +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.reference @@ -205,7 +205,7 @@ select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative 600 419700 select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null < -500); 596 -1099 -501 -select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null is null); +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null is null) settings allow_experimental_analyzer=1; 0 0 select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null is null); 0 0 0 diff --git a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql index e3736de6a17..29055eb911a 100644 --- a/tests/queries/0_stateless/02892_orc_filter_pushdown.sql +++ b/tests/queries/0_stateless/02892_orc_filter_pushdown.sql @@ -206,7 +206,7 @@ select count(), min(nEgAtIvE_oR_nUlL), max(nEgAtIvE_oR_nUlL) from file('02892.or select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null < -500); select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null < -500); -select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null is null); +select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null is null) settings allow_experimental_analyzer=1; select count(), min(negative_or_null), max(negative_or_null) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where (negative_or_null is null); select count(), sum(number) from file('02892.orc', ORC, 'number UInt64, negative_or_null Int64') where indexHint(negative_or_null in (0, -1, -10, -100, -1000)); From 81fb9b18f623bc85b1044242ab908d50731ae624 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 18 Jul 2024 17:41:36 +0200 Subject: [PATCH 42/79] Update 02125_many_mutations.sh --- tests/queries/0_stateless/02125_many_mutations.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02125_many_mutations.sh b/tests/queries/0_stateless/02125_many_mutations.sh index 474fc8ef199..239f30811d4 100755 --- a/tests/queries/0_stateless/02125_many_mutations.sh +++ b/tests/queries/0_stateless/02125_many_mutations.sh @@ -48,6 +48,7 @@ $CLICKHOUSE_CLIENT --multiquery -q " select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done; system start merges many_mutations; optimize table many_mutations final SETTINGS optimize_throw_if_noop = 1; +alter table many_mutations update y = y + 1 where 1 settings mutations_sync=2; select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done; select x, y from many_mutations order by x; truncate table many_mutations; From f71e91b19f579580346dc430e9603c6dfa8102e5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 18 Jul 2024 17:42:25 +0200 Subject: [PATCH 43/79] Update 02125_many_mutations.reference --- tests/queries/0_stateless/02125_many_mutations.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02125_many_mutations.reference b/tests/queries/0_stateless/02125_many_mutations.reference index c98d8221c7f..a94b8b78480 100644 --- a/tests/queries/0_stateless/02125_many_mutations.reference +++ b/tests/queries/0_stateless/02125_many_mutations.reference @@ -2,5 +2,5 @@ 1 1 20000 0 -0 20000 -1 20001 +0 20001 +1 20002 From 36275fdacc34206931f69087fe77539e25bbbedd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 18 Jul 2024 17:45:52 +0200 Subject: [PATCH 44/79] Update 02125_many_mutations_2.sh --- tests/queries/0_stateless/02125_many_mutations_2.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02125_many_mutations_2.sh b/tests/queries/0_stateless/02125_many_mutations_2.sh index b7585991738..958f50287b2 100755 --- a/tests/queries/0_stateless/02125_many_mutations_2.sh +++ b/tests/queries/0_stateless/02125_many_mutations_2.sh @@ -52,6 +52,7 @@ $CLICKHOUSE_CLIENT --multiquery -q " select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done; system start merges many_mutations; optimize table many_mutations final SETTINGS optimize_throw_if_noop = 1; +alter table many_mutations update y = y + 1 where 1 settings mutations_sync=2; system flush logs; select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done; select count() from many_mutations; From f50538c596050974c7afef5a87e10c97c2b1656b Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 18 Jul 2024 19:31:49 +0200 Subject: [PATCH 45/79] add 03008_deduplication_rundom_setttings.sh --- ...ion_insert_several_blocks_nonreplicated.sh | 2 +- ...cation_insert_several_blocks_replicated.sh | 2 +- ..._generates_several_blocks_nonreplicated.sh | 2 +- ..._mv_generates_several_blocks_replicated.sh | 2 +- ...8_deduplication_rundom_setttings.reference | 3 + .../03008_deduplication_rundom_setttings.sh | 78 +++++++++++++++++++ ...several_mv_into_one_table_nonreplicated.sh | 2 +- ...on_several_mv_into_one_table_replicated.sh | 2 +- 8 files changed, 87 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03008_deduplication_rundom_setttings.reference create mode 100755 tests/queries/0_stateless/03008_deduplication_rundom_setttings.sh diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh index f0a723501de..ef4866cb69e 100755 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage, no-s3-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh index 1b7f9be9510..dd4b1f7cec0 100755 --- a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage, no-s3-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh index e76b6eb77ce..33386c76edb 100755 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage, no-s3-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh index 7e6cf5e95c9..b66ef83abf2 100755 --- a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage, no-s3-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_rundom_setttings.reference b/tests/queries/0_stateless/03008_deduplication_rundom_setttings.reference new file mode 100644 index 00000000000..8e7e24a7dd2 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_rundom_setttings.reference @@ -0,0 +1,3 @@ +insert_several_blocks_into_table OK +mv_generates_several_blocks OK +several_mv_into_one_table OK diff --git a/tests/queries/0_stateless/03008_deduplication_rundom_setttings.sh b/tests/queries/0_stateless/03008_deduplication_rundom_setttings.sh new file mode 100755 index 00000000000..20513c35c71 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_rundom_setttings.sh @@ -0,0 +1,78 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="ReplicatedMergeTree" + +engine_options=("ReplicatedMergeTree" "MergeTree") +engine=${engine_options[ $RANDOM % ${#engine_options[@]} ]} + +insert_method_option=("InsertSelect" "InsertValues") +insert_method=${insert_method_option[ $RANDOM % ${#insert_method_option[@]} ]} + +use_insert_token_option=("True" "False") +use_insert_token=${use_insert_token_option[ $RANDOM % ${#use_insert_token_option[@]} ]} + +single_thread_options=("True" "False") +single_thread=${single_thread_options[ $RANDOM % ${#single_thread_options[@]} ]} + +deduplicate_src_table_options=("True" "False") +deduplicate_src_table=${deduplicate_src_table_options[ $RANDOM % ${#deduplicate_src_table_options[@]} ]} + +deduplicate_dst_table_options=("True" "False") +deduplicate_dst_table=${deduplicate_dst_table_options[ $RANDOM % ${#deduplicate_dst_table_options[@]} ]} + +insert_unique_blocks_options=("True" "False") +insert_unique_blocks=${insert_unique_blocks_options[ $RANDOM % ${#insert_unique_blocks_options[@]} ]} + + +THIS_RUN="Test case:" +THIS_RUN+=" insert_method=$insert_method" +THIS_RUN+=" engine=$ENGINE" +THIS_RUN+=" use_insert_token=$use_insert_token" +THIS_RUN+=" single_thread=$single_thread" +THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" +THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" +THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + +$CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) +" 1>/dev/null 2>&1 && echo 'insert_several_blocks_into_table OK' || echo "FAIL: insert_several_blocks_into_table ${THIS_RUN}" + +$CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) +" 1>/dev/null 2>&1 && echo 'mv_generates_several_blocks OK' || echo "FAIL: mv_generates_several_blocks ${THIS_RUN}" + +$CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) +" 1>/dev/null 2>&1 && echo 'several_mv_into_one_table OK' || echo "FAIL: several_mv_into_one_table ${THIS_RUN}" + diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh index 193e15d311c..f9e1838f491 100755 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage, no-s3-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh index 3f4f0b17258..698e70d4064 100755 --- a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage +# Tags: long, no-fasttest, no-parallel, no-azure-blob-storage, no-s3-storage CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From a7220ffff4a96683e0a9fb648efbe43dd66c19c0 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 18 Jul 2024 19:35:47 +0200 Subject: [PATCH 46/79] fix test name --- ...s.reference => 03008_deduplication_random_setttings.reference} | 0 ...undom_setttings.sh => 03008_deduplication_random_setttings.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03008_deduplication_rundom_setttings.reference => 03008_deduplication_random_setttings.reference} (100%) rename tests/queries/0_stateless/{03008_deduplication_rundom_setttings.sh => 03008_deduplication_random_setttings.sh} (100%) diff --git a/tests/queries/0_stateless/03008_deduplication_rundom_setttings.reference b/tests/queries/0_stateless/03008_deduplication_random_setttings.reference similarity index 100% rename from tests/queries/0_stateless/03008_deduplication_rundom_setttings.reference rename to tests/queries/0_stateless/03008_deduplication_random_setttings.reference diff --git a/tests/queries/0_stateless/03008_deduplication_rundom_setttings.sh b/tests/queries/0_stateless/03008_deduplication_random_setttings.sh similarity index 100% rename from tests/queries/0_stateless/03008_deduplication_rundom_setttings.sh rename to tests/queries/0_stateless/03008_deduplication_random_setttings.sh From 36468c528debe37bb6023a37d7ed9b93e7b56836 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 18 Jul 2024 19:55:09 +0200 Subject: [PATCH 47/79] relax condition in test, remove unused counters --- src/Common/ProfileEvents.cpp | 2 -- src/Coordination/KeeperConstants.cpp | 2 -- src/IO/ReadBufferFromS3.cpp | 2 -- .../02789_reading_from_s3_with_connection_pool.sh | 5 +---- 4 files changed, 1 insertion(+), 10 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 871ba7cab8b..2b090136e2e 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -442,8 +442,6 @@ The server successfully detected this situation and will download merged part fr M(ReadBufferFromS3InitMicroseconds, "Time spent initializing connection to S3.") \ M(ReadBufferFromS3Bytes, "Bytes read from S3.") \ M(ReadBufferFromS3RequestsErrors, "Number of exceptions while reading from S3.") \ - M(ReadBufferFromS3ResetSessions, "Number of HTTP sessions that were reset in ReadBufferFromS3.") \ - M(ReadBufferFromS3PreservedSessions, "Number of HTTP sessions that were preserved in ReadBufferFromS3.") \ \ M(WriteBufferFromS3Microseconds, "Time spent on writing to S3.") \ M(WriteBufferFromS3Bytes, "Bytes written to S3.") \ diff --git a/src/Coordination/KeeperConstants.cpp b/src/Coordination/KeeperConstants.cpp index 7589e3393be..daa6a4bcb3f 100644 --- a/src/Coordination/KeeperConstants.cpp +++ b/src/Coordination/KeeperConstants.cpp @@ -183,8 +183,6 @@ M(ReadBufferFromS3InitMicroseconds) \ M(ReadBufferFromS3Bytes) \ M(ReadBufferFromS3RequestsErrors) \ - M(ReadBufferFromS3ResetSessions) \ - M(ReadBufferFromS3PreservedSessions) \ \ M(WriteBufferFromS3Microseconds) \ M(WriteBufferFromS3Bytes) \ diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 9e001232e65..bf97e959ee0 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -25,8 +25,6 @@ namespace ProfileEvents extern const Event ReadBufferFromS3InitMicroseconds; extern const Event ReadBufferFromS3Bytes; extern const Event ReadBufferFromS3RequestsErrors; - extern const Event ReadBufferFromS3ResetSessions; - extern const Event ReadBufferFromS3PreservedSessions; extern const Event ReadBufferSeekCancelConnection; extern const Event S3GetObject; extern const Event DiskS3GetObject; diff --git a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh index e4a1de9a2ec..751b2798243 100755 --- a/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh +++ b/tests/queries/0_stateless/02789_reading_from_s3_with_connection_pool.sh @@ -18,10 +18,7 @@ query="SELECT a, b FROM test_s3" query_id=$(${CLICKHOUSE_CLIENT} --query "select queryID() from ($query) limit 1" 2>&1) ${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS" ${CLICKHOUSE_CLIENT} -nm --query " -WITH - ProfileEvents['DiskConnectionsReset'] AS reset, - ProfileEvents['DiskConnectionsPreserved'] AS preserved -SELECT preserved > reset +SELECT ProfileEvents['DiskConnectionsPreserved'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() From c73dd41c58e8b59ef97373f64429a07045a3923a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jul 2024 19:55:36 +0200 Subject: [PATCH 48/79] Remove bad test `host_resolver_fail_count` --- .../test_host_resolver_fail_count/__init__.py | 0 .../configs/config.d/cluster.xml | 12 -- .../configs/config.d/s3.xml | 21 --- .../test_case.py | 126 ------------------ 4 files changed, 159 deletions(-) delete mode 100644 tests/integration/test_host_resolver_fail_count/__init__.py delete mode 100644 tests/integration/test_host_resolver_fail_count/configs/config.d/cluster.xml delete mode 100644 tests/integration/test_host_resolver_fail_count/configs/config.d/s3.xml delete mode 100644 tests/integration/test_host_resolver_fail_count/test_case.py diff --git a/tests/integration/test_host_resolver_fail_count/__init__.py b/tests/integration/test_host_resolver_fail_count/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_host_resolver_fail_count/configs/config.d/cluster.xml b/tests/integration/test_host_resolver_fail_count/configs/config.d/cluster.xml deleted file mode 100644 index bde62b82719..00000000000 --- a/tests/integration/test_host_resolver_fail_count/configs/config.d/cluster.xml +++ /dev/null @@ -1,12 +0,0 @@ - - - - 5 - 5 - 5 - 5 - 5 - 5 - - - \ No newline at end of file diff --git a/tests/integration/test_host_resolver_fail_count/configs/config.d/s3.xml b/tests/integration/test_host_resolver_fail_count/configs/config.d/s3.xml deleted file mode 100644 index 94ac83b32ac..00000000000 --- a/tests/integration/test_host_resolver_fail_count/configs/config.d/s3.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - - - - - -
- s3 -
-
-
-
-
-
diff --git a/tests/integration/test_host_resolver_fail_count/test_case.py b/tests/integration/test_host_resolver_fail_count/test_case.py deleted file mode 100644 index d25681f0781..00000000000 --- a/tests/integration/test_host_resolver_fail_count/test_case.py +++ /dev/null @@ -1,126 +0,0 @@ -"""Test Interserver responses on configured IP.""" - -import pytest -import time -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance( - "node", - main_configs=["configs/config.d/cluster.xml", "configs/config.d/s3.xml"], - with_minio=True, -) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -# The same value as in ClickHouse, this can't be confugured via config now -DEFAULT_RESOLVE_TIME_HISTORY_SECONDS = 2 * 60 - - -def test_host_resolver(start_cluster): - minio_ip = cluster.get_instance_ip("minio1") - - # drop DNS cache - node.set_hosts( - [ - (minio_ip, "minio1"), - (node.ip_address, "minio1"), # no answer on 9001 port on this IP - ] - ) - - node.query("SYSTEM DROP DNS CACHE") - node.query("SYSTEM DROP CONNECTIONS CACHE") - - node.query( - """ - CREATE TABLE test (key UInt32, value UInt32) - Engine=MergeTree() - ORDER BY key PARTITION BY key - SETTINGS storage_policy='s3' - """ - ) - - initial_fails = "0\n" - k = 0 - limit = 100 - while initial_fails == "0\n": - node.query( - f""" - INSERT INTO test VALUES (0,{k}) - """ - ) - # HostResolver chooses IP randomly, so on single call can choose worked ID - initial_fails = node.query( - "SELECT value FROM system.events WHERE event LIKE 'AddressesMarkedAsFailed'" - ) - k += 1 - if k >= limit: - # Dead IP was not choosen for 100 iteration. - # This is not expected, but not an error actually. - # And test should be stopped. - return - - # initial_fails can be more than 1 if clickhouse does something in several parallel threads - - for j in range(10): - for i in range(10): - node.query( - f""" - INSERT INTO test VALUES ({i+1},{j+1}) - """ - ) - fails = node.query( - "SELECT value FROM system.events WHERE event LIKE 'AddressesMarkedAsFailed'" - ) - assert fails == initial_fails - - # Check that clickhouse tries to recheck IP after 2 minutes - time.sleep(DEFAULT_RESOLVE_TIME_HISTORY_SECONDS) - - intermediate_fails = initial_fails - limit = k + 100 - while intermediate_fails == initial_fails: - node.query( - f""" - INSERT INTO test VALUES (101,{k}) - """ - ) - intermediate_fails = node.query( - "SELECT value FROM system.events WHERE event LIKE 'AddressesMarkedAsFailed'" - ) - k += 1 - if k >= limit: - # Dead IP was not choosen for 100 iteration. - # This is not expected, but not an error actually. - # And test should be stopped. - return - - # After another 2 minutes shoudl not be new fails, next retry after 4 minutes - time.sleep(DEFAULT_RESOLVE_TIME_HISTORY_SECONDS) - - initial_fails = intermediate_fails - limit = k + 100 - while intermediate_fails == initial_fails: - node.query( - f""" - INSERT INTO test VALUES (102,{k}) - """ - ) - intermediate_fails = node.query( - "SELECT value FROM system.events WHERE event LIKE 'AddressesMarkedAsFailed'" - ) - k += 1 - if k >= limit: - break - - assert k == limit From 3ef2cbdec5b960a873ee4fd70d9788248d4324a1 Mon Sep 17 00:00:00 2001 From: Yong Wang Date: Thu, 18 Jul 2024 11:21:52 -0700 Subject: [PATCH 49/79] openssl-dynamic-build-ppc64le : fix openssl dynamic build for ppc64le --- contrib/openssl-cmake/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/openssl-cmake/CMakeLists.txt b/contrib/openssl-cmake/CMakeLists.txt index 85de0340996..a9e4a3df698 100644 --- a/contrib/openssl-cmake/CMakeLists.txt +++ b/contrib/openssl-cmake/CMakeLists.txt @@ -1298,7 +1298,6 @@ elseif(ARCH_PPC64LE) ${OPENSSL_SOURCE_DIR}/crypto/camellia/camellia.c ${OPENSSL_SOURCE_DIR}/crypto/camellia/cmll_cbc.c ${OPENSSL_SOURCE_DIR}/crypto/chacha/chacha_enc.c - ${OPENSSL_SOURCE_DIR}/crypto/mem_clr.c ${OPENSSL_SOURCE_DIR}/crypto/rc4/rc4_enc.c ${OPENSSL_SOURCE_DIR}/crypto/rc4/rc4_skey.c ${OPENSSL_SOURCE_DIR}/crypto/sha/keccak1600.c From 465e4ad73f60a3843426c88301b9d3c1376fc851 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 17 Jul 2024 17:19:56 +0200 Subject: [PATCH 50/79] CI: Remove aws lambda packages from oss --- tests/ci/autoscale_runners_lambda/app.py | 235 ---------- .../build_and_deploy_archive.sh | 1 - .../ci/autoscale_runners_lambda/lambda_shared | 1 - .../autoscale_runners_lambda/requirements.txt | 1 - .../test_autoscale.py | 196 --------- tests/ci/build_check.py | 5 +- .../cancel_and_rerun_workflow_lambda/app.py | 376 ---------------- .../build_and_deploy_archive.sh | 1 - .../lambda_shared | 1 - .../requirements.txt | 1 - tests/ci/cherry_pick.py | 2 +- tests/ci/ci_config.py | 3 + tests/ci/ci_definitions.py | 47 ++ tests/ci/ci_runners_metrics_lambda/app.py | 164 ------- .../build_and_deploy_archive.sh | 1 - .../ci_runners_metrics_lambda/lambda_shared | 1 - .../requirements.txt | 2 - tests/ci/ci_utils.py | 115 ++++- tests/ci/clean_lost_instances_lambda/app.py | 336 --------------- .../build_and_deploy_archive.sh | 1 - .../clean_lost_instances_lambda/lambda_shared | 1 - .../requirements.txt | 2 - tests/ci/lambda_shared_package/.gitignore | 2 - tests/ci/lambda_shared_package/__init__.py | 0 .../lambda_shared/__init__.py | 237 ---------- .../lambda_shared_package/lambda_shared/pr.py | 168 -------- .../lambda_shared/token.py | 95 ---- tests/ci/lambda_shared_package/pyproject.toml | 24 -- tests/ci/lambda_shared_package/setup.cfg | 8 - tests/ci/pr_info.py | 2 +- tests/ci/release.py | 2 +- tests/ci/run_check.py | 50 ++- tests/ci/runner_token_rotation_lambda/app.py | 93 ---- .../build_and_deploy_archive.sh | 1 - .../lambda_shared | 1 - .../requirements.txt | 1 - tests/ci/slack_bot_ci_lambda/app.py | 323 -------------- .../build_and_deploy_archive.sh | 1 - tests/ci/slack_bot_ci_lambda/requirements.txt | 1 - tests/ci/team_keys_lambda/app.py | 136 ------ .../build_and_deploy_archive.sh | 76 ---- tests/ci/team_keys_lambda/lambda_shared | 1 - tests/ci/team_keys_lambda/requirements.txt | 1 - tests/ci/terminate_runner_lambda/app.py | 278 ------------ .../build_and_deploy_archive.sh | 1 - .../ci/terminate_runner_lambda/lambda_shared | 1 - .../terminate_runner_lambda/requirements.txt | 1 - tests/ci/workflow_approve_rerun_lambda/app.py | 404 ------------------ .../build_and_deploy_archive.sh | 1 - .../lambda_shared | 1 - .../requirements.txt | 1 - tests/ci/workflow_jobs_lambda/app.py | 202 --------- .../build_and_deploy_archive.sh | 1 - tests/ci/workflow_jobs_lambda/lambda_shared | 1 - .../ci/workflow_jobs_lambda/requirements.txt | 1 - 55 files changed, 194 insertions(+), 3415 deletions(-) delete mode 100644 tests/ci/autoscale_runners_lambda/app.py delete mode 120000 tests/ci/autoscale_runners_lambda/build_and_deploy_archive.sh delete mode 120000 tests/ci/autoscale_runners_lambda/lambda_shared delete mode 100644 tests/ci/autoscale_runners_lambda/requirements.txt delete mode 100644 tests/ci/autoscale_runners_lambda/test_autoscale.py delete mode 100644 tests/ci/cancel_and_rerun_workflow_lambda/app.py delete mode 120000 tests/ci/cancel_and_rerun_workflow_lambda/build_and_deploy_archive.sh delete mode 120000 tests/ci/cancel_and_rerun_workflow_lambda/lambda_shared delete mode 100644 tests/ci/cancel_and_rerun_workflow_lambda/requirements.txt delete mode 100644 tests/ci/ci_runners_metrics_lambda/app.py delete mode 120000 tests/ci/ci_runners_metrics_lambda/build_and_deploy_archive.sh delete mode 120000 tests/ci/ci_runners_metrics_lambda/lambda_shared delete mode 100644 tests/ci/ci_runners_metrics_lambda/requirements.txt delete mode 100644 tests/ci/clean_lost_instances_lambda/app.py delete mode 120000 tests/ci/clean_lost_instances_lambda/build_and_deploy_archive.sh delete mode 120000 tests/ci/clean_lost_instances_lambda/lambda_shared delete mode 100644 tests/ci/clean_lost_instances_lambda/requirements.txt delete mode 100644 tests/ci/lambda_shared_package/.gitignore delete mode 100644 tests/ci/lambda_shared_package/__init__.py delete mode 100644 tests/ci/lambda_shared_package/lambda_shared/__init__.py delete mode 100644 tests/ci/lambda_shared_package/lambda_shared/pr.py delete mode 100644 tests/ci/lambda_shared_package/lambda_shared/token.py delete mode 100644 tests/ci/lambda_shared_package/pyproject.toml delete mode 100644 tests/ci/lambda_shared_package/setup.cfg delete mode 100644 tests/ci/runner_token_rotation_lambda/app.py delete mode 120000 tests/ci/runner_token_rotation_lambda/build_and_deploy_archive.sh delete mode 120000 tests/ci/runner_token_rotation_lambda/lambda_shared delete mode 100644 tests/ci/runner_token_rotation_lambda/requirements.txt delete mode 100755 tests/ci/slack_bot_ci_lambda/app.py delete mode 120000 tests/ci/slack_bot_ci_lambda/build_and_deploy_archive.sh delete mode 100644 tests/ci/slack_bot_ci_lambda/requirements.txt delete mode 100644 tests/ci/team_keys_lambda/app.py delete mode 100644 tests/ci/team_keys_lambda/build_and_deploy_archive.sh delete mode 120000 tests/ci/team_keys_lambda/lambda_shared delete mode 100644 tests/ci/team_keys_lambda/requirements.txt delete mode 100644 tests/ci/terminate_runner_lambda/app.py delete mode 120000 tests/ci/terminate_runner_lambda/build_and_deploy_archive.sh delete mode 120000 tests/ci/terminate_runner_lambda/lambda_shared delete mode 100644 tests/ci/terminate_runner_lambda/requirements.txt delete mode 100644 tests/ci/workflow_approve_rerun_lambda/app.py delete mode 120000 tests/ci/workflow_approve_rerun_lambda/build_and_deploy_archive.sh delete mode 120000 tests/ci/workflow_approve_rerun_lambda/lambda_shared delete mode 100644 tests/ci/workflow_approve_rerun_lambda/requirements.txt delete mode 100644 tests/ci/workflow_jobs_lambda/app.py delete mode 120000 tests/ci/workflow_jobs_lambda/build_and_deploy_archive.sh delete mode 120000 tests/ci/workflow_jobs_lambda/lambda_shared delete mode 100644 tests/ci/workflow_jobs_lambda/requirements.txt diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py deleted file mode 100644 index c790a5081dd..00000000000 --- a/tests/ci/autoscale_runners_lambda/app.py +++ /dev/null @@ -1,235 +0,0 @@ -#!/usr/bin/env python3 - -"""The lambda to decrease/increase ASG desired capacity based on current queue""" - -import logging -from dataclasses import dataclass -from pprint import pformat -from typing import Any, List, Literal, Optional, Tuple - -import boto3 # type: ignore -from lambda_shared import ( - RUNNER_TYPE_LABELS, - CHException, - ClickHouseHelper, - get_parameter_from_ssm, -) - -### Update comment on the change ### -# 4 HOUR - is a balance to get the most precise values -# - Our longest possible running check is around 5h on the worst scenario -# - The long queue won't be wiped out and replaced, so the measurmenet is fine -# - If the data is spoiled by something, we are from the bills perspective -# Changed it to 3 HOUR: in average we have 1h tasks, but p90 is around 2h. -# With 4h we have too much wasted computing time in case of issues with DB -QUEUE_QUERY = f"""SELECT - last_status AS status, - toUInt32(count()) AS length, - labels -FROM -( - SELECT - arraySort(groupArray(status))[-1] AS last_status, - labels, - id, - html_url - FROM default.workflow_jobs - WHERE has(labels, 'self-hosted') - AND hasAny({RUNNER_TYPE_LABELS}, labels) - AND started_at > now() - INTERVAL 3 HOUR - GROUP BY ALL - HAVING last_status IN ('in_progress', 'queued') -) -GROUP BY ALL -ORDER BY labels, last_status""" - - -@dataclass -class Queue: - status: Literal["in_progress", "queued"] - lentgh: int - label: str - - -def get_scales(runner_type: str) -> Tuple[int, int]: - "returns the multipliers for scaling down and up ASG by types" - # Scaling down is quicker on the lack of running jobs than scaling up on - # queue - - # The ASG should deflate almost instantly - scale_down = 1 - # the style checkers have so many noise, so it scales up too quickly - # The 5 was too quick, there are complainings regarding too slow with - # 10. I am trying 7 now. - # 7 still looks a bit slow, so I try 6 - # Let's have it the same as the other ASG - # - # All type of style-checkers should be added very quickly to not block the workflows - # UPDATE THE COMMENT ON CHANGES - scale_up = 3 - if "style" in runner_type: - scale_up = 1 - return scale_down, scale_up - - -CH_CLIENT = None # type: Optional[ClickHouseHelper] - - -def set_capacity( - runner_type: str, queues: List[Queue], client: Any, dry_run: bool = True -) -> None: - assert len(queues) in (1, 2) - assert all(q.label == runner_type for q in queues) - as_groups = client.describe_auto_scaling_groups( - Filters=[ - {"Name": "tag-key", "Values": ["github:runner-type"]}, - {"Name": "tag-value", "Values": [runner_type]}, - ] - )["AutoScalingGroups"] - assert len(as_groups) == 1 - asg = as_groups[0] - running = 0 - queued = 0 - for q in queues: - if q.status == "in_progress": - running = q.lentgh - continue - if q.status == "queued": - queued = q.lentgh - continue - raise ValueError("Queue status is not in ['in_progress', 'queued']") - - # scale_down, scale_up = get_scales(runner_type) - _, scale_up = get_scales(runner_type) - # With lyfecycle hooks some instances are actually free because some of - # them are in 'Terminating:Wait' state - effective_capacity = max( - asg["DesiredCapacity"], - len([ins for ins in asg["Instances"] if ins["HealthStatus"] == "Healthy"]), - ) - - # How much nodes are free (positive) or need to be added (negative) - capacity_reserve = effective_capacity - running - queued - stop = False - if capacity_reserve <= 0: - # This part is about scaling up - capacity_deficit = -capacity_reserve - # It looks that we are still OK, since no queued jobs exist - stop = stop or queued == 0 - # Are we already at the capacity limits - stop = stop or asg["MaxSize"] <= asg["DesiredCapacity"] - # Let's calculate a new desired capacity - # (capacity_deficit + scale_up - 1) // scale_up : will increase min by 1 - # if there is any capacity_deficit - new_capacity = ( - asg["DesiredCapacity"] + (capacity_deficit + scale_up - 1) // scale_up - ) - new_capacity = max(new_capacity, asg["MinSize"]) - new_capacity = min(new_capacity, asg["MaxSize"]) - # Finally, should the capacity be even changed - stop = stop or asg["DesiredCapacity"] == new_capacity - if stop: - logging.info( - "Do not increase ASG %s capacity, current capacity=%s, effective " - "capacity=%s, maximum capacity=%s, running jobs=%s, queue size=%s", - asg["AutoScalingGroupName"], - asg["DesiredCapacity"], - effective_capacity, - asg["MaxSize"], - running, - queued, - ) - return - - logging.info( - "The ASG %s capacity will be increased to %s, current capacity=%s, " - "effective capacity=%s, maximum capacity=%s, running jobs=%s, queue size=%s", - asg["AutoScalingGroupName"], - new_capacity, - asg["DesiredCapacity"], - effective_capacity, - asg["MaxSize"], - running, - queued, - ) - if not dry_run: - client.set_desired_capacity( - AutoScalingGroupName=asg["AutoScalingGroupName"], - DesiredCapacity=new_capacity, - ) - return - - # FIXME: try decreasing capacity from runners that finished their jobs and have no job assigned - # IMPORTANT: Runner init script must be of version that supports ASG decrease - # # Now we will calculate if we need to scale down - # stop = stop or asg["DesiredCapacity"] == asg["MinSize"] - # new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) - # new_capacity = max(new_capacity, asg["MinSize"]) - # new_capacity = min(new_capacity, asg["MaxSize"]) - # stop = stop or asg["DesiredCapacity"] == new_capacity - # if stop: - # logging.info( - # "Do not decrease ASG %s capacity, current capacity=%s, effective " - # "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", - # asg["AutoScalingGroupName"], - # asg["DesiredCapacity"], - # effective_capacity, - # asg["MinSize"], - # running, - # queued, - # ) - # return - # - # logging.info( - # "The ASG %s capacity will be decreased to %s, current capacity=%s, effective " - # "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", - # asg["AutoScalingGroupName"], - # new_capacity, - # asg["DesiredCapacity"], - # effective_capacity, - # asg["MinSize"], - # running, - # queued, - # ) - # if not dry_run: - # client.set_desired_capacity( - # AutoScalingGroupName=asg["AutoScalingGroupName"], - # DesiredCapacity=new_capacity, - # ) - - -def main(dry_run: bool = True) -> None: - logging.getLogger().setLevel(logging.INFO) - asg_client = boto3.client("autoscaling") - try: - global CH_CLIENT - CH_CLIENT = CH_CLIENT or ClickHouseHelper( - get_parameter_from_ssm("clickhouse-test-stat-url"), "play" - ) - queues = CH_CLIENT.select_json_each_row("default", QUEUE_QUERY) - except CHException as ex: - logging.exception( - "Got an exception on insert, tryuing to update the client " - "credentials and repeat", - exc_info=ex, - ) - CH_CLIENT = ClickHouseHelper( - get_parameter_from_ssm("clickhouse-test-stat-url"), "play" - ) - queues = CH_CLIENT.select_json_each_row("default", QUEUE_QUERY) - - logging.info("Received queue data:\n%s", pformat(queues, width=120)) - for runner_type in RUNNER_TYPE_LABELS: - runner_queues = [ - Queue(queue["status"], queue["length"], runner_type) - for queue in queues - if runner_type in queue["labels"] - ] - runner_queues = runner_queues or [Queue("in_progress", 0, runner_type)] - set_capacity(runner_type, runner_queues, asg_client, dry_run) - - -def handler(event: dict, context: Any) -> None: - _ = event - _ = context - return main(False) diff --git a/tests/ci/autoscale_runners_lambda/build_and_deploy_archive.sh b/tests/ci/autoscale_runners_lambda/build_and_deploy_archive.sh deleted file mode 120000 index 96ba3fa024e..00000000000 --- a/tests/ci/autoscale_runners_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1 +0,0 @@ -../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/autoscale_runners_lambda/lambda_shared b/tests/ci/autoscale_runners_lambda/lambda_shared deleted file mode 120000 index ba86e090f6c..00000000000 --- a/tests/ci/autoscale_runners_lambda/lambda_shared +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/autoscale_runners_lambda/requirements.txt b/tests/ci/autoscale_runners_lambda/requirements.txt deleted file mode 100644 index 098e04a9798..00000000000 --- a/tests/ci/autoscale_runners_lambda/requirements.txt +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package diff --git a/tests/ci/autoscale_runners_lambda/test_autoscale.py b/tests/ci/autoscale_runners_lambda/test_autoscale.py deleted file mode 100644 index d1a1f9b358f..00000000000 --- a/tests/ci/autoscale_runners_lambda/test_autoscale.py +++ /dev/null @@ -1,196 +0,0 @@ -#!/usr/bin/env python - -import unittest -from dataclasses import dataclass -from typing import Any, List - -from app import Queue, set_capacity - - -@dataclass -class TestCase: - name: str - min_size: int - desired_capacity: int - max_size: int - queues: List[Queue] - expected_capacity: int - - -class TestSetCapacity(unittest.TestCase): - class FakeClient: - def __init__(self): - self._expected_data = {} # type: dict - self._expected_capacity = -1 - - @property - def expected_data(self) -> dict: - """a one-time property""" - data, self._expected_data = self._expected_data, {} - return data - - @expected_data.setter - def expected_data(self, value: dict) -> None: - self._expected_data = value - - @property - def expected_capacity(self) -> int: - """a one-time property""" - capacity, self._expected_capacity = self._expected_capacity, -1 - return capacity - - def describe_auto_scaling_groups(self, **kwargs: Any) -> dict: - _ = kwargs - return self.expected_data - - def set_desired_capacity(self, **kwargs: Any) -> None: - self._expected_capacity = kwargs["DesiredCapacity"] - - def data_helper( - self, name: str, min_size: int, desired_capacity: int, max_size: int - ) -> None: - self.expected_data = { - "AutoScalingGroups": [ - { - "AutoScalingGroupName": name, - "DesiredCapacity": desired_capacity, - "MinSize": min_size, - "MaxSize": max_size, - "Instances": [], # necessary for ins["HealthStatus"] check - } - ] - } - - def setUp(self): - self.client = self.FakeClient() - - def test_normal_cases(self): - test_cases = ( - # Do not change capacity - TestCase("noqueue", 1, 13, 20, [Queue("in_progress", 155, "noqueue")], -1), - TestCase("reserve", 1, 13, 20, [Queue("queued", 13, "reserve")], -1), - # Increase capacity - TestCase( - "increase-always", - 1, - 13, - 20, - [Queue("queued", 14, "increase-always")], - 14, - ), - TestCase("increase-1", 1, 13, 20, [Queue("queued", 23, "increase-1")], 17), - TestCase( - "style-checker", 1, 13, 20, [Queue("queued", 19, "style-checker")], 19 - ), - TestCase("increase-2", 1, 13, 20, [Queue("queued", 18, "increase-2")], 15), - TestCase("increase-3", 1, 13, 20, [Queue("queued", 183, "increase-3")], 20), - TestCase( - "increase-w/o reserve", - 1, - 13, - 20, - [ - Queue("in_progress", 11, "increase-w/o reserve"), - Queue("queued", 12, "increase-w/o reserve"), - ], - 17, - ), - TestCase("lower-min", 10, 5, 20, [Queue("queued", 5, "lower-min")], 10), - # Decrease capacity - # FIXME: Tests changed for lambda that can only scale up - # TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5), - TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], -1), - # TestCase( - # "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5 - # ), - TestCase( - "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], -1 - ), - # TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 17), - TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], -1), - # TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 3), - TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], -1), - # TestCase( - # "style-checker", - # 1, - # 13, - # 20, - # [Queue("in_progress", 5, "style-checker")], - # 5, - # ), - TestCase( - "style-checker", - 1, - 13, - 20, - [Queue("in_progress", 5, "style-checker")], - -1, - ), - ) - for t in test_cases: - self.client.data_helper(t.name, t.min_size, t.desired_capacity, t.max_size) - set_capacity(t.name, t.queues, self.client, False) - self.assertEqual(t.expected_capacity, self.client.expected_capacity, t.name) - - def test_effective_capacity(self): - """Normal cases test increasing w/o considering - effective_capacity much lower than DesiredCapacity""" - test_cases = ( - TestCase( - "desired-overwritten", - 1, - 20, # DesiredCapacity, overwritten by effective_capacity - 50, - [ - Queue("in_progress", 30, "desired-overwritten"), - Queue("queued", 60, "desired-overwritten"), - ], - 40, - ), - ) - for t in test_cases: - self.client.data_helper(t.name, t.min_size, t.desired_capacity, t.max_size) - # we test that effective_capacity is 30 (a half of 60) - data_with_instances = self.client.expected_data - data_with_instances["AutoScalingGroups"][0]["Instances"] = [ - {"HealthStatus": "Healthy" if i % 2 else "Unhealthy"} for i in range(60) - ] - self.client.expected_data = data_with_instances - set_capacity(t.name, t.queues, self.client, False) - self.assertEqual(t.expected_capacity, self.client.expected_capacity, t.name) - - def test_exceptions(self): - test_cases = ( - ( - TestCase( - "different names", - 1, - 1, - 1, - [Queue("queued", 5, "another name")], - -1, - ), - AssertionError, - ), - (TestCase("wrong queue len", 1, 1, 1, [], -1), AssertionError), - ( - TestCase( - "wrong queue", 1, 1, 1, [Queue("wrong", 1, "wrong queue")], -1 # type: ignore - ), - ValueError, - ), - ) - for t, error in test_cases: - with self.assertRaises(error): - self.client.data_helper( - t.name, t.min_size, t.desired_capacity, t.max_size - ) - set_capacity(t.name, t.queues, self.client, False) - - with self.assertRaises(AssertionError): - self.client.expected_data = {"AutoScalingGroups": [1, 2]} - set_capacity( - "wrong number of ASGs", - [Queue("queued", 1, "wrong number of ASGs")], - self.client, - ) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 39f34ed9ccf..77d91c8400b 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -12,7 +12,6 @@ import docker_images_helper from ci_config import CI from env_helper import REPO_COPY, S3_BUILDS_BUCKET, TEMP_PATH from git_helper import Git -from lambda_shared_package.lambda_shared.pr import Labels from pr_info import PRInfo from report import FAILURE, SUCCESS, JobReport, StatusType from stopwatch import Stopwatch @@ -108,7 +107,9 @@ def build_clickhouse( def is_release_pr(pr_info: PRInfo) -> bool: - return Labels.RELEASE in pr_info.labels or Labels.RELEASE_LTS in pr_info.labels + return ( + CI.Labels.RELEASE in pr_info.labels or CI.Labels.RELEASE_LTS in pr_info.labels + ) def get_release_or_pr(pr_info: PRInfo, version: ClickHouseVersion) -> Tuple[str, str]: diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py deleted file mode 100644 index 578ade5c8a0..00000000000 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ /dev/null @@ -1,376 +0,0 @@ -#!/usr/bin/env python3 - -import json -import time -from base64 import b64decode -from collections import namedtuple -from queue import Queue -from threading import Thread -from typing import Any, Dict, List, Optional - -import requests -from lambda_shared.pr import Labels -from lambda_shared.token import get_cached_access_token - -NEED_RERUN_OR_CANCELL_WORKFLOWS = { - "BackportPR", - "DocsCheck", - "MasterCI", - "PullRequestCI", -} - -MAX_RETRY = 5 - -DEBUG_INFO = {} # type: Dict[str, Any] - - -class Worker(Thread): - def __init__( - self, request_queue: Queue, token: str, ignore_exception: bool = False - ): - Thread.__init__(self) - self.queue = request_queue - self.token = token - self.ignore_exception = ignore_exception - self.response = {} # type: Dict - - def run(self): - m = self.queue.get() - try: - self.response = _exec_get_with_retry(m, self.token) - except Exception as e: - if not self.ignore_exception: - raise - print(f"Exception occured, still continue: {e}") - self.queue.task_done() - - -def _exec_get_with_retry(url: str, token: str) -> dict: - headers = {"Authorization": f"token {token}"} - e = Exception() - for i in range(MAX_RETRY): - try: - response = requests.get(url, headers=headers, timeout=30) - response.raise_for_status() - return response.json() # type: ignore - except Exception as ex: - print("Got exception executing request", ex) - e = ex - time.sleep(i + 1) - - raise requests.HTTPError("Cannot execute GET request with retries") from e - - -WorkflowDescription = namedtuple( - "WorkflowDescription", - [ - "url", - "run_id", - "name", - "head_sha", - "status", - "rerun_url", - "cancel_url", - "conclusion", - ], -) - - -def get_workflows_description_for_pull_request( - pull_request_event: dict, token: str -) -> List[WorkflowDescription]: - head_repo = pull_request_event["head"]["repo"]["full_name"] - head_branch = pull_request_event["head"]["ref"] - print("PR", pull_request_event["number"], "has head ref", head_branch) - - workflows_data = [] - repo_url = pull_request_event["base"]["repo"]["url"] - request_url = f"{repo_url}/actions/runs?per_page=100" - # Get all workflows for the current branch - for i in range(1, 11): - workflows = _exec_get_with_retry( - f"{request_url}&event=pull_request&branch={head_branch}&page={i}", token - ) - if not workflows["workflow_runs"]: - break - workflows_data += workflows["workflow_runs"] - if i == 10: - print("Too many workflows found") - - if not workflows_data: - print("No workflows found by filter") - return [] - - print(f"Total workflows for the branch {head_branch} found: {len(workflows_data)}") - - DEBUG_INFO["workflows"] = [] - workflow_descriptions = [] - for workflow in workflows_data: - # Some time workflow["head_repository"]["full_name"] is None - if workflow["head_repository"] is None: - continue - DEBUG_INFO["workflows"].append( - { - "full_name": workflow["head_repository"]["full_name"], - "name": workflow["name"], - "branch": workflow["head_branch"], - } - ) - # unfortunately we cannot filter workflows from forks in request to API - # so doing it manually - if ( - workflow["head_repository"]["full_name"] == head_repo - and workflow["name"] in NEED_RERUN_OR_CANCELL_WORKFLOWS - ): - workflow_descriptions.append( - WorkflowDescription( - url=workflow["url"], - run_id=workflow["id"], - name=workflow["name"], - head_sha=workflow["head_sha"], - status=workflow["status"], - rerun_url=workflow["rerun_url"], - cancel_url=workflow["cancel_url"], - conclusion=workflow["conclusion"], - ) - ) - - return workflow_descriptions - - -def get_workflow_description_fallback( - pull_request_event: dict, token: str -) -> List[WorkflowDescription]: - head_repo = pull_request_event["head"]["repo"]["full_name"] - head_branch = pull_request_event["head"]["ref"] - print("Get last 500 workflows from API to search related there") - # Fallback for a case of an already deleted branch and no workflows received - repo_url = pull_request_event["base"]["repo"]["url"] - request_url = f"{repo_url}/actions/runs?per_page=100" - q = Queue() # type: Queue - workers = [] - workflows_data = [] - i = 1 - for i in range(1, 6): - q.put(f"{request_url}&page={i}") - worker = Worker(q, token, True) - worker.start() - workers.append(worker) - - for worker in workers: - worker.join() - if not worker.response: - # We ignore get errors, so response can be empty - continue - # Prefilter workflows - workflows_data += [ - wf - for wf in worker.response["workflow_runs"] - if wf["head_repository"] is not None - and wf["head_repository"]["full_name"] == head_repo - and wf["head_branch"] == head_branch - and wf["name"] in NEED_RERUN_OR_CANCELL_WORKFLOWS - ] - - print(f"Total workflows in last 500 actions matches: {len(workflows_data)}") - - DEBUG_INFO["workflows"] = [ - { - "full_name": wf["head_repository"]["full_name"], - "name": wf["name"], - "branch": wf["head_branch"], - } - for wf in workflows_data - ] - - workflow_descriptions = [ - WorkflowDescription( - url=wf["url"], - run_id=wf["id"], - name=wf["name"], - head_sha=wf["head_sha"], - status=wf["status"], - rerun_url=wf["rerun_url"], - cancel_url=wf["cancel_url"], - conclusion=wf["conclusion"], - ) - for wf in workflows_data - ] - - return workflow_descriptions - - -def get_workflow_description(workflow_url: str, token: str) -> WorkflowDescription: - workflow = _exec_get_with_retry(workflow_url, token) - return WorkflowDescription( - url=workflow["url"], - run_id=workflow["id"], - name=workflow["name"], - head_sha=workflow["head_sha"], - status=workflow["status"], - rerun_url=workflow["rerun_url"], - cancel_url=workflow["cancel_url"], - conclusion=workflow["conclusion"], - ) - - -def _exec_post_with_retry(url: str, token: str, json: Optional[Any] = None) -> Any: - headers = {"Authorization": f"token {token}"} - e = Exception() - for i in range(MAX_RETRY): - try: - response = requests.post(url, headers=headers, json=json, timeout=30) - response.raise_for_status() - return response.json() - except Exception as ex: - print("Got exception executing request", ex) - e = ex - time.sleep(i + 1) - - raise requests.HTTPError("Cannot execute POST request with retry") from e - - -def exec_workflow_url(urls_to_post, token): - for url in urls_to_post: - print("Post for workflow workflow using url", url) - _exec_post_with_retry(url, token) - print("Workflow post finished") - - -def main(event): - token = get_cached_access_token() - DEBUG_INFO["event"] = event - if event["isBase64Encoded"]: - event_data = json.loads(b64decode(event["body"])) - else: - event_data = json.loads(event["body"]) - - print("Got event for PR", event_data["number"]) - action = event_data["action"] - print("Got action", event_data["action"]) - pull_request = event_data["pull_request"] - label = "" - if action == "labeled": - label = event_data["label"]["name"] - print("Added label:", label) - - print("PR has labels", {label["name"] for label in pull_request["labels"]}) - if action == "opened" or ( - action == "labeled" and pull_request["created_at"] == pull_request["updated_at"] - ): - print("Freshly opened PR, nothing to do") - return - - if action == "closed" or label == Labels.DO_NOT_TEST: - print("PR merged/closed or manually labeled 'do not test', will kill workflows") - workflow_descriptions = get_workflows_description_for_pull_request( - pull_request, token - ) - workflow_descriptions = ( - workflow_descriptions - or get_workflow_description_fallback(pull_request, token) - ) - urls_to_cancel = [] - for workflow_description in workflow_descriptions: - if ( - workflow_description.status != "completed" - and workflow_description.conclusion != "cancelled" - ): - urls_to_cancel.append(workflow_description.cancel_url) - print(f"Found {len(urls_to_cancel)} workflows to cancel") - exec_workflow_url(urls_to_cancel, token) - return - - if label == Labels.CAN_BE_TESTED: - print("PR marked with can be tested label, rerun workflow") - workflow_descriptions = get_workflows_description_for_pull_request( - pull_request, token - ) - workflow_descriptions = ( - workflow_descriptions - or get_workflow_description_fallback(pull_request, token) - ) - if not workflow_descriptions: - print("Not found any workflows") - return - - workflow_descriptions.sort(key=lambda x: x.run_id) # type: ignore - most_recent_workflow = workflow_descriptions[-1] - print("Latest workflow", most_recent_workflow) - if ( - most_recent_workflow.status != "completed" - and most_recent_workflow.conclusion != "cancelled" - ): - print("Latest workflow is not completed, cancelling") - exec_workflow_url([most_recent_workflow.cancel_url], token) - print("Cancelled") - - for _ in range(45): - # If the number of retries is changed: tune the lambda limits accordingly - latest_workflow_desc = get_workflow_description( - most_recent_workflow.url, token - ) - print("Checking latest workflow", latest_workflow_desc) - if latest_workflow_desc.status in ("completed", "cancelled"): - print("Finally latest workflow done, going to rerun") - exec_workflow_url([most_recent_workflow.rerun_url], token) - print("Rerun finished, exiting") - break - print("Still have strange status") - time.sleep(3) - return - - if action == "edited": - print("PR is edited - do nothing") - # error, _ = check_pr_description( - # pull_request["body"], pull_request["base"]["repo"]["full_name"] - # ) - # if error: - # print( - # f"The PR's body is wrong, is going to comment it. The error is: {error}" - # ) - # post_json = { - # "body": "This is an automatic comment. The PR descriptions does not " - # f"match the [template]({pull_request['base']['repo']['html_url']}/" - # "blob/master/.github/PULL_REQUEST_TEMPLATE.md?plain=1).\n\n" - # f"Please, edit it accordingly.\n\nThe error is: {error}" - # } - # _exec_post_with_retry(pull_request["comments_url"], token, json=post_json) - return - - if action == "synchronize": - print("PR is synchronized, going to stop old actions") - workflow_descriptions = get_workflows_description_for_pull_request( - pull_request, token - ) - workflow_descriptions = ( - workflow_descriptions - or get_workflow_description_fallback(pull_request, token) - ) - urls_to_cancel = [] - for workflow_description in workflow_descriptions: - if ( - workflow_description.status != "completed" - and workflow_description.conclusion != "cancelled" - and workflow_description.head_sha != pull_request["head"]["sha"] - ): - urls_to_cancel.append(workflow_description.cancel_url) - print(f"Found {len(urls_to_cancel)} workflows to cancel") - exec_workflow_url(urls_to_cancel, token) - return - - print("Nothing to do") - - -def handler(event, _): - try: - main(event) - - return { - "statusCode": 200, - "headers": {"Content-Type": "application/json"}, - "body": '{"status": "OK"}', - } - finally: - for name, value in DEBUG_INFO.items(): - print(f"Value of {name}: ", value) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/build_and_deploy_archive.sh b/tests/ci/cancel_and_rerun_workflow_lambda/build_and_deploy_archive.sh deleted file mode 120000 index 96ba3fa024e..00000000000 --- a/tests/ci/cancel_and_rerun_workflow_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1 +0,0 @@ -../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/lambda_shared b/tests/ci/cancel_and_rerun_workflow_lambda/lambda_shared deleted file mode 120000 index ba86e090f6c..00000000000 --- a/tests/ci/cancel_and_rerun_workflow_lambda/lambda_shared +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/requirements.txt b/tests/ci/cancel_and_rerun_workflow_lambda/requirements.txt deleted file mode 100644 index 4cb3fba0f7b..00000000000 --- a/tests/ci/cancel_and_rerun_workflow_lambda/requirements.txt +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package[token] diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 459be12ada0..a7fc6d02853 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -38,7 +38,7 @@ from env_helper import TEMP_PATH from get_robot_token import get_best_robot_token from git_helper import GIT_PREFIX, git_runner, is_shallow from github_helper import GitHub, PullRequest, PullRequests, Repository -from lambda_shared_package.lambda_shared.pr import Labels +from ci_config import Labels from ssh import SSHKey diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index f5e39f343b2..98c7e99a495 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -32,6 +32,9 @@ class CI: from ci_definitions import MQ_JOBS as MQ_JOBS from ci_definitions import WorkflowStages as WorkflowStages from ci_definitions import Runners as Runners + from ci_definitions import Labels as Labels + from ci_definitions import TRUSTED_CONTRIBUTORS as TRUSTED_CONTRIBUTORS + from ci_utils import CATEGORY_TO_LABEL as CATEGORY_TO_LABEL # Jobs that run for doc related updates _DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index acd9b7fa904..d41a621bc2e 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -7,6 +7,53 @@ from ci_utils import WithIter from integration_test_images import IMAGES +class Labels: + PR_BUGFIX = "pr-bugfix" + PR_CRITICAL_BUGFIX = "pr-critical-bugfix" + CAN_BE_TESTED = "can be tested" + DO_NOT_TEST = "do not test" + MUST_BACKPORT = "pr-must-backport" + MUST_BACKPORT_CLOUD = "pr-must-backport-cloud" + JEPSEN_TEST = "jepsen-test" + SKIP_MERGEABLE_CHECK = "skip mergeable check" + PR_BACKPORT = "pr-backport" + PR_BACKPORTS_CREATED = "pr-backports-created" + PR_BACKPORTS_CREATED_CLOUD = "pr-backports-created-cloud" + PR_CHERRYPICK = "pr-cherrypick" + PR_CI = "pr-ci" + PR_FEATURE = "pr-feature" + PR_SYNCED_TO_CLOUD = "pr-synced-to-cloud" + PR_SYNC_UPSTREAM = "pr-sync-upstream" + RELEASE = "release" + RELEASE_LTS = "release-lts" + SUBMODULE_CHANGED = "submodule changed" + + # automatic backport for critical bug fixes + AUTO_BACKPORT = {"pr-critical-bugfix"} + + +TRUSTED_CONTRIBUTORS = { + e.lower() + for e in [ + "amosbird", + "azat", # SEMRush + "bharatnc", # Many contributions. + "cwurm", # ClickHouse, Inc + "den-crane", # Documentation contributor + "ildus", # adjust, ex-pgpro + "nvartolomei", # Seasoned contributor, CloudFlare + "taiyang-li", + "ucasFL", # Amos Bird's friend + "thomoco", # ClickHouse, Inc + "tonickkozlov", # Cloudflare + "tylerhannan", # ClickHouse, Inc + "tsolodov", # ClickHouse, Inc + "justindeguzman", # ClickHouse, Inc + "XuJia0210", # ClickHouse, Inc + ] +} + + class WorkflowStages(metaclass=WithIter): """ Stages of GitHUb actions workflow diff --git a/tests/ci/ci_runners_metrics_lambda/app.py b/tests/ci/ci_runners_metrics_lambda/app.py deleted file mode 100644 index 47161215a97..00000000000 --- a/tests/ci/ci_runners_metrics_lambda/app.py +++ /dev/null @@ -1,164 +0,0 @@ -#!/usr/bin/env python3 -""" -Lambda function to: - - calculate number of running runners - - cleaning dead runners from GitHub - - terminating stale lost runners in EC2 -""" - -import argparse -import sys -from typing import Dict - -import boto3 # type: ignore -from lambda_shared import RUNNER_TYPE_LABELS, RunnerDescriptions, list_runners -from lambda_shared.token import ( - get_access_token_by_key_app, - get_cached_access_token, - get_key_and_app_from_aws, -) - -UNIVERSAL_LABEL = "universal" - - -def handler(event, context): - _ = event - _ = context - main(get_cached_access_token(), True) - - -def group_runners_by_tag( - listed_runners: RunnerDescriptions, -) -> Dict[str, RunnerDescriptions]: - result = {} # type: Dict[str, RunnerDescriptions] - - def add_to_result(tag, runner): - if tag not in result: - result[tag] = [] - result[tag].append(runner) - - for runner in listed_runners: - if UNIVERSAL_LABEL in runner.tags: - # Do not proceed other labels if UNIVERSAL_LABEL is included - add_to_result(UNIVERSAL_LABEL, runner) - continue - - for tag in runner.tags: - if tag in RUNNER_TYPE_LABELS: - add_to_result(tag, runner) - break - else: - add_to_result("unlabeled", runner) - return result - - -def push_metrics_to_cloudwatch( - listed_runners: RunnerDescriptions, group_name: str -) -> None: - client = boto3.client("cloudwatch") - namespace = "RunnersMetrics" - metrics_data = [] - busy_runners = sum( - 1 for runner in listed_runners if runner.busy and not runner.offline - ) - dimensions = [{"Name": "group", "Value": group_name}] - metrics_data.append( - { - "MetricName": "BusyRunners", - "Value": busy_runners, - "Unit": "Count", - "Dimensions": dimensions, - } - ) - total_active_runners = sum(1 for runner in listed_runners if not runner.offline) - metrics_data.append( - { - "MetricName": "ActiveRunners", - "Value": total_active_runners, - "Unit": "Count", - "Dimensions": dimensions, - } - ) - total_runners = len(listed_runners) - metrics_data.append( - { - "MetricName": "TotalRunners", - "Value": total_runners, - "Unit": "Count", - "Dimensions": dimensions, - } - ) - if total_active_runners == 0: - busy_ratio = 100.0 - else: - busy_ratio = busy_runners / total_active_runners * 100 - - metrics_data.append( - { - "MetricName": "BusyRunnersRatio", - "Value": busy_ratio, - "Unit": "Percent", - "Dimensions": dimensions, - } - ) - - client.put_metric_data(Namespace=namespace, MetricData=metrics_data) - - -def main( - access_token: str, - push_to_cloudwatch: bool, -) -> None: - gh_runners = list_runners(access_token) - grouped_runners = group_runners_by_tag(gh_runners) - for group, group_runners in grouped_runners.items(): - if push_to_cloudwatch: - print(f"Pushing metrics for group '{group}'") - push_metrics_to_cloudwatch(group_runners, group) - else: - print(group, f"({len(group_runners)})") - for runner in group_runners: - print("\t", runner) - - -if __name__ == "__main__": - parser = argparse.ArgumentParser(description="Get list of runners and their states") - parser.add_argument( - "-p", "--private-key-path", help="Path to file with private key" - ) - parser.add_argument("-k", "--private-key", help="Private key") - parser.add_argument( - "-a", "--app-id", type=int, help="GitHub application ID", required=True - ) - parser.add_argument( - "--push-to-cloudwatch", - action="store_true", - help="Push metrics for active and busy runners to cloudwatch", - ) - - args = parser.parse_args() - - if not args.private_key_path and not args.private_key: - print( - "Either --private-key-path or --private-key must be specified", - file=sys.stderr, - ) - - if args.private_key_path and args.private_key: - print( - "Either --private-key-path or --private-key must be specified", - file=sys.stderr, - ) - - if args.private_key: - private_key = args.private_key - elif args.private_key_path: - with open(args.private_key_path, "r", encoding="utf-8") as key_file: - private_key = key_file.read() - else: - print("Attempt to get key and id from AWS secret manager") - private_key, args.app_id = get_key_and_app_from_aws() - - token = get_access_token_by_key_app(private_key, args.app_id) - - main(token, args.push_to_cloudwatch) diff --git a/tests/ci/ci_runners_metrics_lambda/build_and_deploy_archive.sh b/tests/ci/ci_runners_metrics_lambda/build_and_deploy_archive.sh deleted file mode 120000 index 96ba3fa024e..00000000000 --- a/tests/ci/ci_runners_metrics_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1 +0,0 @@ -../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/ci_runners_metrics_lambda/lambda_shared b/tests/ci/ci_runners_metrics_lambda/lambda_shared deleted file mode 120000 index ba86e090f6c..00000000000 --- a/tests/ci/ci_runners_metrics_lambda/lambda_shared +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/ci_runners_metrics_lambda/requirements.txt b/tests/ci/ci_runners_metrics_lambda/requirements.txt deleted file mode 100644 index e2b16067a93..00000000000 --- a/tests/ci/ci_runners_metrics_lambda/requirements.txt +++ /dev/null @@ -1,2 +0,0 @@ -../lambda_shared_package -../lambda_shared_package[token] diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 25875e55df6..49f0447b5ca 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -3,7 +3,42 @@ import re import subprocess from contextlib import contextmanager from pathlib import Path -from typing import Any, Iterator, List, Union, Optional +from typing import Any, Iterator, List, Union, Optional, Tuple + + +LABEL_CATEGORIES = { + "pr-backward-incompatible": ["Backward Incompatible Change"], + "pr-bugfix": [ + "Bug Fix", + "Bug Fix (user-visible misbehavior in an official stable release)", + "Bug Fix (user-visible misbehaviour in official stable or prestable release)", + "Bug Fix (user-visible misbehavior in official stable or prestable release)", + ], + "pr-critical-bugfix": ["Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)"], + "pr-build": [ + "Build/Testing/Packaging Improvement", + "Build Improvement", + "Build/Testing Improvement", + "Build", + "Packaging Improvement", + ], + "pr-documentation": [ + "Documentation (changelog entry is not required)", + "Documentation", + ], + "pr-feature": ["New Feature"], + "pr-improvement": ["Improvement"], + "pr-not-for-changelog": [ + "Not for changelog (changelog entry is not required)", + "Not for changelog", + ], + "pr-performance": ["Performance Improvement"], + "pr-ci": ["CI Fix or Improvement (changelog entry is not required)"], +} + +CATEGORY_TO_LABEL = { + c: lb for lb, categories in LABEL_CATEGORIES.items() for c in categories +} class WithIter(type): @@ -109,3 +144,81 @@ class Utils: @staticmethod def clear_dmesg(): Shell.run("sudo dmesg --clear ||:") + + @staticmethod + def check_pr_description(pr_body: str, repo_name: str) -> Tuple[str, str]: + """The function checks the body to being properly formatted according to + .github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty, + then there is an error.""" + lines = list(map(lambda x: x.strip(), pr_body.split("\n") if pr_body else [])) + lines = [re.sub(r"\s+", " ", line) for line in lines] + + # Check if body contains "Reverts ClickHouse/ClickHouse#36337" + if [ + True for line in lines if re.match(rf"\AReverts {repo_name}#[\d]+\Z", line) + ]: + return "", LABEL_CATEGORIES["pr-not-for-changelog"][0] + + category = "" + entry = "" + description_error = "" + + i = 0 + while i < len(lines): + if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): + i += 1 + if i >= len(lines): + break + # Can have one empty line between header and the category + # itself. Filter it out. + if not lines[i]: + i += 1 + if i >= len(lines): + break + category = re.sub(r"^[-*\s]*", "", lines[i]) + i += 1 + + # Should not have more than one category. Require empty line + # after the first found category. + if i >= len(lines): + break + if lines[i]: + second_category = re.sub(r"^[-*\s]*", "", lines[i]) + description_error = ( + "More than one changelog category specified: " + f"'{category}', '{second_category}'" + ) + return description_error, category + + elif re.match( + r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] + ): + i += 1 + # Can have one empty line between header and the entry itself. + # Filter it out. + if i < len(lines) and not lines[i]: + i += 1 + # All following lines until empty one are the changelog entry. + entry_lines = [] + while i < len(lines) and lines[i]: + entry_lines.append(lines[i]) + i += 1 + entry = " ".join(entry_lines) + # Don't accept changelog entries like '...'. + entry = re.sub(r"[#>*_.\- ]", "", entry) + # Don't accept changelog entries like 'Close #12345'. + entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry) + else: + i += 1 + + if not category: + description_error = "Changelog category is empty" + # Filter out the PR categories that are not for changelog. + elif "(changelog entry is not required)" in category: + pass # to not check the rest of the conditions + elif category not in CATEGORY_TO_LABEL: + description_error, category = f"Category '{category}' is not valid", "" + elif not entry: + description_error = f"Changelog entry required for category '{category}'" + + return description_error, category diff --git a/tests/ci/clean_lost_instances_lambda/app.py b/tests/ci/clean_lost_instances_lambda/app.py deleted file mode 100644 index 4accc14f7ae..00000000000 --- a/tests/ci/clean_lost_instances_lambda/app.py +++ /dev/null @@ -1,336 +0,0 @@ -#!/usr/bin/env python3 -""" -Lambda function to: - - calculate number of running runners - - cleaning dead runners from GitHub - - terminating stale lost runners in EC2 -""" - -import argparse -import sys -from dataclasses import dataclass -from datetime import datetime -from typing import Dict, List - -import boto3 # type: ignore -import requests -from botocore.exceptions import ClientError # type: ignore -from lambda_shared import ( - RUNNER_TYPE_LABELS, - RunnerDescription, - RunnerDescriptions, - list_runners, -) -from lambda_shared.token import ( - get_access_token_by_key_app, - get_cached_access_token, - get_key_and_app_from_aws, -) - -UNIVERSAL_LABEL = "universal" - - -@dataclass -class LostInstance: - counter: int - seen: datetime - - def set_offline(self) -> None: - now = datetime.now() - if now.timestamp() <= self.seen.timestamp() + 120: - # the instance is offline for more than 2 minutes, so we increase - # the counter - self.counter += 1 - else: - self.counter = 1 - self.seen = now - - @property - def recently_offline(self) -> bool: - """Returns True if the instance has been seen less than 5 minutes ago""" - return datetime.now().timestamp() <= self.seen.timestamp() + 300 - - @property - def stable_offline(self) -> bool: - return self.counter >= 3 - - -LOST_INSTANCES = {} # type: Dict["str", LostInstance] - - -def get_dead_runners_in_ec2(runners: RunnerDescriptions) -> RunnerDescriptions: - """Returns instances that are offline/dead in EC2, or not found in EC2""" - ids = { - runner.name: runner - for runner in runners - # Only `i-deadbead123` are valid names for an instance ID - if runner.name.startswith("i-") and runner.offline and not runner.busy - } - if not ids: - return [] - - # Delete all offline runners with wrong name - result_to_delete = [ - runner - for runner in runners - if not ids.get(runner.name) and runner.offline and not runner.busy - ] - - client = boto3.client("ec2") - - i = 0 - inc = 100 - - print("Checking ids: ", " ".join(ids.keys())) - instances_statuses = [] - while i < len(ids.keys()): - try: - instances_statuses.append( - client.describe_instance_status( - InstanceIds=list(ids.keys())[i : i + inc] - ) - ) - # It applied only if all ids exist in EC2 - i += inc - except ClientError as e: - # The list of non-existent instances is in the message: - # The instance IDs 'i-069b1c256c06cf4e3, i-0f26430432b044035, - # i-0faa2ff44edbc147e, i-0eccf2514585045ec, i-0ee4ee53e0daa7d4a, - # i-07928f15acd473bad, i-0eaddda81298f9a85' do not exist - message = e.response["Error"]["Message"] - if message.startswith("The instance IDs '") and message.endswith( - "' do not exist" - ): - non_existent = message[18:-14].split(", ") - for n in non_existent: - result_to_delete.append(ids.pop(n)) - else: - raise - - found_instances = set([]) - print("Response", instances_statuses) - for instances_status in instances_statuses: - for instance_status in instances_status["InstanceStatuses"]: - if instance_status["InstanceState"]["Name"] in ("pending", "running"): - found_instances.add(instance_status["InstanceId"]) - - print("Found instances", found_instances) - for runner in result_to_delete: - print("Instance", runner.name, "is not alive, going to remove it") - for instance_id, runner in ids.items(): - if instance_id not in found_instances: - print("Instance", instance_id, "is not found in EC2, going to remove it") - result_to_delete.append(runner) - return result_to_delete - - -def handler(event, context): - _ = event - _ = context - main(get_cached_access_token(), True) - - -def delete_runner(access_token: str, runner: RunnerDescription) -> bool: - headers = { - "Authorization": f"token {access_token}", - "Accept": "application/vnd.github.v3+json", - } - - response = requests.delete( - f"https://api.github.com/orgs/ClickHouse/actions/runners/{runner.id}", - headers=headers, - timeout=30, - ) - response.raise_for_status() - print(f"Response code deleting {runner.name} is {response.status_code}") - return bool(response.status_code == 204) - - -def get_lost_ec2_instances(runners: RunnerDescriptions) -> List[str]: - global LOST_INSTANCES - now = datetime.now() - client = boto3.client("ec2") - reservations = client.describe_instances( - Filters=[ - {"Name": "tag-key", "Values": ["github:runner-type"]}, - {"Name": "instance-state-name", "Values": ["pending", "running"]}, - ], - )["Reservations"] - # flatten the reservation into instances - instances = [ - instance - for reservation in reservations - for instance in reservation["Instances"] - ] - offline_runner_names = { - runner.name for runner in runners if runner.offline and not runner.busy - } - runner_names = {runner.name for runner in runners} - - def offline_instance(iid: str) -> None: - if iid in LOST_INSTANCES: - LOST_INSTANCES[iid].set_offline() - return - LOST_INSTANCES[iid] = LostInstance(1, now) - - for instance in instances: - # Do not consider instances started 20 minutes ago as problematic - if now.timestamp() - instance["LaunchTime"].timestamp() < 1200: - continue - - runner_type = [ - tag["Value"] - for tag in instance["Tags"] - if tag["Key"] == "github:runner-type" - ][0] - # If there's no necessary labels in runner type it's fine - if not (UNIVERSAL_LABEL in runner_type or runner_type in RUNNER_TYPE_LABELS): - continue - - if instance["InstanceId"] in offline_runner_names: - offline_instance(instance["InstanceId"]) - continue - - if ( - instance["State"]["Name"] == "running" - and not instance["InstanceId"] in runner_names - ): - offline_instance(instance["InstanceId"]) - - instance_ids = [instance["InstanceId"] for instance in instances] - # clean out long unseen instances - LOST_INSTANCES = { - instance_id: stats - for instance_id, stats in LOST_INSTANCES.items() - if stats.recently_offline and instance_id in instance_ids - } - print("The remained LOST_INSTANCES: ", LOST_INSTANCES) - - return [ - instance_id - for instance_id, stats in LOST_INSTANCES.items() - if stats.stable_offline - ] - - -def continue_lifecycle_hooks(delete_offline_runners: bool) -> None: - """The function to trigger CONTINUE for instances' lifectycle hooks""" - client = boto3.client("ec2") - reservations = client.describe_instances( - Filters=[ - {"Name": "tag-key", "Values": ["github:runner-type"]}, - {"Name": "instance-state-name", "Values": ["shutting-down", "terminated"]}, - ], - )["Reservations"] - # flatten the reservation into instances - terminated_instances = [ - instance["InstanceId"] - for reservation in reservations - for instance in reservation["Instances"] - ] - - asg_client = boto3.client("autoscaling") - as_groups = asg_client.describe_auto_scaling_groups( - Filters=[{"Name": "tag-key", "Values": ["github:runner-type"]}] - )["AutoScalingGroups"] - for asg in as_groups: - lifecycle_hooks = [ - lch - for lch in asg_client.describe_lifecycle_hooks( - AutoScalingGroupName=asg["AutoScalingGroupName"] - )["LifecycleHooks"] - if lch["LifecycleTransition"] == "autoscaling:EC2_INSTANCE_TERMINATING" - ] - if not lifecycle_hooks: - continue - for instance in asg["Instances"]: - continue_instance = False - if instance["LifecycleState"] == "Terminating:Wait": - if instance["HealthStatus"] == "Unhealthy": - print(f"The instance {instance['InstanceId']} is Unhealthy") - continue_instance = True - elif ( - instance["HealthStatus"] == "Healthy" - and instance["InstanceId"] in terminated_instances - ): - print( - f"The instance {instance['InstanceId']} is already terminated" - ) - continue_instance = True - if continue_instance: - if delete_offline_runners: - for lch in lifecycle_hooks: - print(f"Continue lifecycle hook {lch['LifecycleHookName']}") - asg_client.complete_lifecycle_action( - LifecycleHookName=lch["LifecycleHookName"], - AutoScalingGroupName=asg["AutoScalingGroupName"], - LifecycleActionResult="CONTINUE", - InstanceId=instance["InstanceId"], - ) - - -def main( - access_token: str, - delete_offline_runners: bool, -) -> None: - gh_runners = list_runners(access_token) - - dead_runners = get_dead_runners_in_ec2(gh_runners) - print("Runners in GH API to terminate: ", [runner.name for runner in dead_runners]) - if delete_offline_runners and dead_runners: - print("Going to delete offline runners") - for runner in dead_runners: - print("Deleting runner", runner) - delete_runner(access_token, runner) - elif dead_runners: - print("Would delete dead runners: ", dead_runners) - - lost_instances = get_lost_ec2_instances(gh_runners) - print("Instances to terminate: ", lost_instances) - if delete_offline_runners: - if lost_instances: - print("Going to terminate lost instances") - boto3.client("ec2").terminate_instances(InstanceIds=lost_instances) - - continue_lifecycle_hooks(delete_offline_runners) - - -if __name__ == "__main__": - parser = argparse.ArgumentParser(description="Get list of runners and their states") - parser.add_argument( - "-p", "--private-key-path", help="Path to file with private key" - ) - parser.add_argument("-k", "--private-key", help="Private key") - parser.add_argument( - "-a", "--app-id", type=int, help="GitHub application ID", required=True - ) - parser.add_argument( - "--delete-offline", action="store_true", help="Remove offline runners" - ) - - args = parser.parse_args() - - if not args.private_key_path and not args.private_key: - print( - "Either --private-key-path or --private-key must be specified", - file=sys.stderr, - ) - - if args.private_key_path and args.private_key: - print( - "Either --private-key-path or --private-key must be specified", - file=sys.stderr, - ) - - if args.private_key: - private_key = args.private_key - elif args.private_key_path: - with open(args.private_key_path, "r", encoding="utf-8") as key_file: - private_key = key_file.read() - else: - print("Attempt to get key and id from AWS secret manager") - private_key, args.app_id = get_key_and_app_from_aws() - - token = get_access_token_by_key_app(private_key, args.app_id) - - main(token, args.delete_offline) diff --git a/tests/ci/clean_lost_instances_lambda/build_and_deploy_archive.sh b/tests/ci/clean_lost_instances_lambda/build_and_deploy_archive.sh deleted file mode 120000 index 96ba3fa024e..00000000000 --- a/tests/ci/clean_lost_instances_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1 +0,0 @@ -../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/clean_lost_instances_lambda/lambda_shared b/tests/ci/clean_lost_instances_lambda/lambda_shared deleted file mode 120000 index ba86e090f6c..00000000000 --- a/tests/ci/clean_lost_instances_lambda/lambda_shared +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/clean_lost_instances_lambda/requirements.txt b/tests/ci/clean_lost_instances_lambda/requirements.txt deleted file mode 100644 index e2b16067a93..00000000000 --- a/tests/ci/clean_lost_instances_lambda/requirements.txt +++ /dev/null @@ -1,2 +0,0 @@ -../lambda_shared_package -../lambda_shared_package[token] diff --git a/tests/ci/lambda_shared_package/.gitignore b/tests/ci/lambda_shared_package/.gitignore deleted file mode 100644 index 59d52651e06..00000000000 --- a/tests/ci/lambda_shared_package/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -build -*.egg-info diff --git a/tests/ci/lambda_shared_package/__init__.py b/tests/ci/lambda_shared_package/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/ci/lambda_shared_package/lambda_shared/__init__.py b/tests/ci/lambda_shared_package/lambda_shared/__init__.py deleted file mode 100644 index 8b53f9dcb23..00000000000 --- a/tests/ci/lambda_shared_package/lambda_shared/__init__.py +++ /dev/null @@ -1,237 +0,0 @@ -"""The shared code and types for all our CI lambdas -It exists as __init__.py and lambda_shared/__init__.py to work both in local and venv""" - -import json -import logging -import time -from collections import namedtuple -from typing import Any, Dict, Iterable, List, Optional - -import boto3 # type: ignore -import requests - -RUNNER_TYPE_LABELS = [ - "builder", - "func-tester", - "func-tester-aarch64", - "fuzzer-unit-tester", - "limited-tester", - "stress-tester", - "style-checker", - "style-checker-aarch64", - # private runners - "private-builder", - "private-clickpipes", - "private-func-tester", - "private-fuzzer-unit-tester", - "private-stress-tester", - "private-style-checker", -] - - -### VENDORING -def get_parameter_from_ssm( - name: str, decrypt: bool = True, client: Optional[Any] = None -) -> str: - if not client: - client = boto3.client("ssm", region_name="us-east-1") - return client.get_parameter(Name=name, WithDecryption=decrypt)[ # type: ignore - "Parameter" - ]["Value"] - - -class CHException(Exception): - pass - - -class InsertException(CHException): - pass - - -class ClickHouseHelper: - def __init__( - self, - url: str, - user: Optional[str] = None, - password: Optional[str] = None, - ): - self.url = url - self.auth = {} - if user: - self.auth["X-ClickHouse-User"] = user - if password: - self.auth["X-ClickHouse-Key"] = password - - @staticmethod - def _insert_json_str_info_impl( - url: str, auth: Dict[str, str], db: str, table: str, json_str: str - ) -> None: - params = { - "database": db, - "query": f"INSERT INTO {table} FORMAT JSONEachRow", - "date_time_input_format": "best_effort", - "send_logs_level": "warning", - } - - for i in range(5): - try: - response = requests.post( - url, params=params, data=json_str, headers=auth - ) - except Exception as e: - error = f"Received exception while sending data to {url} on {i} attempt: {e}" - logging.warning(error) - continue - - logging.info("Response content '%s'", response.content) - - if response.ok: - break - - error = ( - "Cannot insert data into clickhouse at try " - + str(i) - + ": HTTP code " - + str(response.status_code) - + ": '" - + str(response.text) - + "'" - ) - - if response.status_code >= 500: - # A retriable error - time.sleep(1) - continue - - logging.info( - "Request headers '%s', body '%s'", - response.request.headers, - response.request.body, - ) - - raise InsertException(error) - else: - raise InsertException(error) - - def _insert_json_str_info(self, db: str, table: str, json_str: str) -> None: - self._insert_json_str_info_impl(self.url, self.auth, db, table, json_str) - - def insert_event_into( - self, db: str, table: str, event: object, safe: bool = True - ) -> None: - event_str = json.dumps(event) - try: - self._insert_json_str_info(db, table, event_str) - except InsertException as e: - logging.error( - "Exception happened during inserting data into clickhouse: %s", e - ) - if not safe: - raise - - def insert_events_into( - self, db: str, table: str, events: Iterable[object], safe: bool = True - ) -> None: - jsons = [] - for event in events: - jsons.append(json.dumps(event)) - - try: - self._insert_json_str_info(db, table, ",".join(jsons)) - except InsertException as e: - logging.error( - "Exception happened during inserting data into clickhouse: %s", e - ) - if not safe: - raise - - def _select_and_get_json_each_row(self, db: str, query: str) -> str: - params = { - "database": db, - "query": query, - "default_format": "JSONEachRow", - } - for i in range(5): - response = None - try: - response = requests.get(self.url, params=params, headers=self.auth) - response.raise_for_status() - return response.text # type: ignore - except Exception as ex: - logging.warning("Cannot fetch data with exception %s", str(ex)) - if response: - logging.warning("Reponse text %s", response.text) - time.sleep(0.1 * i) - - raise CHException("Cannot fetch data from clickhouse") - - def select_json_each_row(self, db: str, query: str) -> List[dict]: - text = self._select_and_get_json_each_row(db, query) - result = [] - for line in text.split("\n"): - if line: - result.append(json.loads(line)) - return result - - -### Runners - -RunnerDescription = namedtuple( - "RunnerDescription", ["id", "name", "tags", "offline", "busy"] -) -RunnerDescriptions = List[RunnerDescription] - - -def list_runners(access_token: str) -> RunnerDescriptions: - headers = { - "Authorization": f"token {access_token}", - "Accept": "application/vnd.github.v3+json", - } - per_page = 100 - response = requests.get( - f"https://api.github.com/orgs/ClickHouse/actions/runners?per_page={per_page}", - headers=headers, - ) - response.raise_for_status() - data = response.json() - total_runners = data["total_count"] - print("Expected total runners", total_runners) - runners = data["runners"] - - # round to 0 for 0, 1 for 1..100, but to 2 for 101..200 - total_pages = (total_runners - 1) // per_page + 1 - - print("Total pages", total_pages) - for i in range(2, total_pages + 1): - response = requests.get( - "https://api.github.com/orgs/ClickHouse/actions/runners" - f"?page={i}&per_page={per_page}", - headers=headers, - ) - response.raise_for_status() - data = response.json() - runners += data["runners"] - - print("Total runners", len(runners)) - result = [] - for runner in runners: - tags = [tag["name"] for tag in runner["labels"]] - desc = RunnerDescription( - id=runner["id"], - name=runner["name"], - tags=tags, - offline=runner["status"] == "offline", - busy=runner["busy"], - ) - result.append(desc) - - return result - - -def cached_value_is_valid(updated_at: float, ttl: float) -> bool: - "a common function to identify if cachable value is still valid" - if updated_at == 0: - return False - if time.time() - ttl < updated_at: - return True - return False diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py deleted file mode 100644 index 95130fc2a0f..00000000000 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ /dev/null @@ -1,168 +0,0 @@ -#!/usr/bin/env python - -import re -from typing import Tuple - -# Individual trusted contributors who are not in any trusted organization. -# Can be changed in runtime: we will append users that we learned to be in -# a trusted org, to save GitHub API calls. -TRUSTED_CONTRIBUTORS = { - e.lower() - for e in [ - "amosbird", - "azat", # SEMRush - "bharatnc", # Many contributions. - "cwurm", # ClickHouse, Inc - "den-crane", # Documentation contributor - "ildus", # adjust, ex-pgpro - "nvartolomei", # Seasoned contributor, CloudFlare - "taiyang-li", - "ucasFL", # Amos Bird's friend - "thomoco", # ClickHouse, Inc - "tonickkozlov", # Cloudflare - "tylerhannan", # ClickHouse, Inc - "tsolodov", # ClickHouse, Inc - "justindeguzman", # ClickHouse, Inc - "XuJia0210", # ClickHouse, Inc - ] -} - - -class Labels: - PR_BUGFIX = "pr-bugfix" - PR_CRITICAL_BUGFIX = "pr-critical-bugfix" - CAN_BE_TESTED = "can be tested" - DO_NOT_TEST = "do not test" - MUST_BACKPORT = "pr-must-backport" - MUST_BACKPORT_CLOUD = "pr-must-backport-cloud" - JEPSEN_TEST = "jepsen-test" - SKIP_MERGEABLE_CHECK = "skip mergeable check" - PR_BACKPORT = "pr-backport" - PR_BACKPORTS_CREATED = "pr-backports-created" - PR_BACKPORTS_CREATED_CLOUD = "pr-backports-created-cloud" - PR_CHERRYPICK = "pr-cherrypick" - PR_CI = "pr-ci" - PR_FEATURE = "pr-feature" - PR_SYNCED_TO_CLOUD = "pr-synced-to-cloud" - PR_SYNC_UPSTREAM = "pr-sync-upstream" - RELEASE = "release" - RELEASE_LTS = "release-lts" - SUBMODULE_CHANGED = "submodule changed" - - # automatic backport for critical bug fixes - AUTO_BACKPORT = {"pr-critical-bugfix"} - - -# Descriptions are used in .github/PULL_REQUEST_TEMPLATE.md, keep comments there -# updated accordingly -# The following lists are append only, try to avoid editing them -# They still could be cleaned out after the decent time though. -LABEL_CATEGORIES = { - "pr-backward-incompatible": ["Backward Incompatible Change"], - "pr-bugfix": [ - "Bug Fix", - "Bug Fix (user-visible misbehavior in an official stable release)", - "Bug Fix (user-visible misbehaviour in official stable or prestable release)", - "Bug Fix (user-visible misbehavior in official stable or prestable release)", - ], - "pr-critical-bugfix": ["Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)"], - "pr-build": [ - "Build/Testing/Packaging Improvement", - "Build Improvement", - "Build/Testing Improvement", - "Build", - "Packaging Improvement", - ], - "pr-documentation": [ - "Documentation (changelog entry is not required)", - "Documentation", - ], - "pr-feature": ["New Feature"], - "pr-improvement": ["Improvement"], - "pr-not-for-changelog": [ - "Not for changelog (changelog entry is not required)", - "Not for changelog", - ], - "pr-performance": ["Performance Improvement"], - "pr-ci": ["CI Fix or Improvement (changelog entry is not required)"], -} - -CATEGORY_TO_LABEL = { - c: lb for lb, categories in LABEL_CATEGORIES.items() for c in categories -} - - -def check_pr_description(pr_body: str, repo_name: str) -> Tuple[str, str]: - """The function checks the body to being properly formatted according to - .github/PULL_REQUEST_TEMPLATE.md, if the first returned string is not empty, - then there is an error.""" - lines = list(map(lambda x: x.strip(), pr_body.split("\n") if pr_body else [])) - lines = [re.sub(r"\s+", " ", line) for line in lines] - - # Check if body contains "Reverts ClickHouse/ClickHouse#36337" - if [True for line in lines if re.match(rf"\AReverts {repo_name}#[\d]+\Z", line)]: - return "", LABEL_CATEGORIES["pr-not-for-changelog"][0] - - category = "" - entry = "" - description_error = "" - - i = 0 - while i < len(lines): - if re.match(r"(?i)^[#>*_ ]*change\s*log\s*category", lines[i]): - i += 1 - if i >= len(lines): - break - # Can have one empty line between header and the category - # itself. Filter it out. - if not lines[i]: - i += 1 - if i >= len(lines): - break - category = re.sub(r"^[-*\s]*", "", lines[i]) - i += 1 - - # Should not have more than one category. Require empty line - # after the first found category. - if i >= len(lines): - break - if lines[i]: - second_category = re.sub(r"^[-*\s]*", "", lines[i]) - description_error = ( - "More than one changelog category specified: " - f"'{category}', '{second_category}'" - ) - return description_error, category - - elif re.match( - r"(?i)^[#>*_ ]*(short\s*description|change\s*log\s*entry)", lines[i] - ): - i += 1 - # Can have one empty line between header and the entry itself. - # Filter it out. - if i < len(lines) and not lines[i]: - i += 1 - # All following lines until empty one are the changelog entry. - entry_lines = [] - while i < len(lines) and lines[i]: - entry_lines.append(lines[i]) - i += 1 - entry = " ".join(entry_lines) - # Don't accept changelog entries like '...'. - entry = re.sub(r"[#>*_.\- ]", "", entry) - # Don't accept changelog entries like 'Close #12345'. - entry = re.sub(r"^[\w\-\s]{0,10}#?\d{5,6}\.?$", "", entry) - else: - i += 1 - - if not category: - description_error = "Changelog category is empty" - # Filter out the PR categories that are not for changelog. - elif "(changelog entry is not required)" in category: - pass # to not check the rest of the conditions - elif category not in CATEGORY_TO_LABEL: - description_error, category = f"Category '{category}' is not valid", "" - elif not entry: - description_error = f"Changelog entry required for category '{category}'" - - return description_error, category diff --git a/tests/ci/lambda_shared_package/lambda_shared/token.py b/tests/ci/lambda_shared_package/lambda_shared/token.py deleted file mode 100644 index 3fb8f10c0e2..00000000000 --- a/tests/ci/lambda_shared_package/lambda_shared/token.py +++ /dev/null @@ -1,95 +0,0 @@ -"""Module to get the token for GitHub""" - -from dataclasses import dataclass -import json -import time -from typing import Tuple - -import boto3 # type: ignore -import jwt -import requests - -from . import cached_value_is_valid - - -def get_key_and_app_from_aws() -> Tuple[str, int]: - secret_name = "clickhouse_github_secret_key" - session = boto3.session.Session() - client = session.client( - service_name="secretsmanager", - ) - get_secret_value_response = client.get_secret_value(SecretId=secret_name) - data = json.loads(get_secret_value_response["SecretString"]) - return data["clickhouse-app-key"], int(data["clickhouse-app-id"]) - - -def get_installation_id(jwt_token: str) -> int: - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.get("https://api.github.com/app/installations", headers=headers) - response.raise_for_status() - data = response.json() - for installation in data: - if installation["account"]["login"] == "ClickHouse": - installation_id = installation["id"] - - return installation_id # type: ignore - - -def get_access_token_by_jwt(jwt_token: str, installation_id: int) -> str: - headers = { - "Authorization": f"Bearer {jwt_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.post( - f"https://api.github.com/app/installations/{installation_id}/access_tokens", - headers=headers, - ) - response.raise_for_status() - data = response.json() - return data["token"] # type: ignore - - -def get_token_from_aws() -> str: - private_key, app_id = get_key_and_app_from_aws() - return get_access_token_by_key_app(private_key, app_id) - - -def get_access_token_by_key_app(private_key: str, app_id: int) -> str: - payload = { - "iat": int(time.time()) - 60, - "exp": int(time.time()) + (10 * 60), - "iss": app_id, - } - - # FIXME: apparently should be switched to this so that mypy is happy - # jwt_instance = JWT() - # encoded_jwt = jwt_instance.encode(payload, private_key, algorithm="RS256") - encoded_jwt = jwt.encode(payload, private_key, algorithm="RS256") # type: ignore - installation_id = get_installation_id(encoded_jwt) - return get_access_token_by_jwt(encoded_jwt, installation_id) - - -@dataclass -class CachedToken: - time: float - value: str - updating: bool = False - - -_cached_token = CachedToken(0, "") - - -def get_cached_access_token() -> str: - if time.time() - 550 < _cached_token.time or _cached_token.updating: - return _cached_token.value - # Indicate that the value is updating now, so the cached value can be - # used. The first setting and close-to-ttl are not counted as update - _cached_token.updating = cached_value_is_valid(_cached_token.time, 590) - private_key, app_id = get_key_and_app_from_aws() - _cached_token.time = time.time() - _cached_token.value = get_access_token_by_key_app(private_key, app_id) - _cached_token.updating = False - return _cached_token.value diff --git a/tests/ci/lambda_shared_package/pyproject.toml b/tests/ci/lambda_shared_package/pyproject.toml deleted file mode 100644 index dff36b89fbb..00000000000 --- a/tests/ci/lambda_shared_package/pyproject.toml +++ /dev/null @@ -1,24 +0,0 @@ -[build-system] -requires = ["setuptools"] -build-backend = "setuptools.build_meta" - -[project] -name = "lambda_shared" -version = "0.0.1" -dependencies = [ - "requests", - "urllib3 < 2" -] - -[project.optional-dependencies] -token = [ - "PyJWT", - "cryptography", -] -dev = [ - "boto3", - "lambda_shared[token]", -] - -[tool.distutils.bdist_wheel] -universal = true diff --git a/tests/ci/lambda_shared_package/setup.cfg b/tests/ci/lambda_shared_package/setup.cfg deleted file mode 100644 index 744280ae41b..00000000000 --- a/tests/ci/lambda_shared_package/setup.cfg +++ /dev/null @@ -1,8 +0,0 @@ -### This file exists for clear builds in docker ### -# without it the `build` directory wouldn't be # -# updated on the fly and will require manual clean # -[build] -build_base = /tmp/lambda_shared - -[egg_info] -egg_base = /tmp/ diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index a411fc4e8f6..9f3b5a586cc 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -15,7 +15,7 @@ from env_helper import ( GITHUB_SERVER_URL, GITHUB_UPSTREAM_REPOSITORY, ) -from lambda_shared_package.lambda_shared.pr import Labels +from ci_config import Labels from get_robot_token import get_best_robot_token from github_helper import GitHub diff --git a/tests/ci/release.py b/tests/ci/release.py index 7ab1149310f..2de20d00a00 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -25,7 +25,7 @@ from contextlib import contextmanager from typing import Any, Final, Iterator, List, Optional, Tuple from git_helper import Git, commit, release_branch -from lambda_shared_package.lambda_shared.pr import Labels +from ci_config import Labels from report import SUCCESS from version_helper import ( FILE_WITH_VERSION_PATH, diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 00942352dde..0ad01e3accd 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -15,26 +15,22 @@ from commit_status_helper import ( ) from env_helper import GITHUB_REPOSITORY, GITHUB_SERVER_URL from get_robot_token import get_best_robot_token -from lambda_shared_package.lambda_shared.pr import ( - CATEGORY_TO_LABEL, - TRUSTED_CONTRIBUTORS, - Labels, - check_pr_description, -) +from ci_config import CI +from ci_utils import Utils from pr_info import PRInfo from report import FAILURE, PENDING, SUCCESS, StatusType -from ci_config import CI + TRUSTED_ORG_IDS = { 54801242, # clickhouse } -OK_SKIP_LABELS = {Labels.RELEASE, Labels.PR_BACKPORT, Labels.PR_CHERRYPICK} +OK_SKIP_LABELS = {CI.Labels.RELEASE, CI.Labels.PR_BACKPORT, CI.Labels.PR_CHERRYPICK} PR_CHECK = "PR Check" def pr_is_by_trusted_user(pr_user_login, pr_user_orgs): - if pr_user_login.lower() in TRUSTED_CONTRIBUTORS: + if pr_user_login.lower() in CI.TRUSTED_CONTRIBUTORS: logging.info("User '%s' is trusted", pr_user_login) return True @@ -63,13 +59,13 @@ def should_run_ci_for_pr(pr_info: PRInfo) -> Tuple[bool, str]: if OK_SKIP_LABELS.intersection(pr_info.labels): return True, "Don't try new checks for release/backports/cherry-picks" - if Labels.CAN_BE_TESTED not in pr_info.labels and not pr_is_by_trusted_user( + if CI.Labels.CAN_BE_TESTED not in pr_info.labels and not pr_is_by_trusted_user( pr_info.user_login, pr_info.user_orgs ): logging.info( "PRs by untrusted users need the '%s' label - " "please contact a member of the core team", - Labels.CAN_BE_TESTED, + CI.Labels.CAN_BE_TESTED, ) return False, "Needs 'can be tested' label" @@ -96,30 +92,32 @@ def main(): commit = get_commit(gh, pr_info.sha) status = SUCCESS # type: StatusType - description_error, category = check_pr_description(pr_info.body, GITHUB_REPOSITORY) + description_error, category = Utils.check_pr_description( + pr_info.body, GITHUB_REPOSITORY + ) pr_labels_to_add = [] pr_labels_to_remove = [] if ( - category in CATEGORY_TO_LABEL - and CATEGORY_TO_LABEL[category] not in pr_info.labels + category in CI.CATEGORY_TO_LABEL + and CI.CATEGORY_TO_LABEL[category] not in pr_info.labels ): - pr_labels_to_add.append(CATEGORY_TO_LABEL[category]) + pr_labels_to_add.append(CI.CATEGORY_TO_LABEL[category]) for label in pr_info.labels: if ( - label in CATEGORY_TO_LABEL.values() - and category in CATEGORY_TO_LABEL - and label != CATEGORY_TO_LABEL[category] + label in CI.CATEGORY_TO_LABEL.values() + and category in CI.CATEGORY_TO_LABEL + and label != CI.CATEGORY_TO_LABEL[category] ): pr_labels_to_remove.append(label) if pr_info.has_changes_in_submodules(): - pr_labels_to_add.append(Labels.SUBMODULE_CHANGED) - elif Labels.SUBMODULE_CHANGED in pr_info.labels: - pr_labels_to_remove.append(Labels.SUBMODULE_CHANGED) + pr_labels_to_add.append(CI.Labels.SUBMODULE_CHANGED) + elif CI.Labels.SUBMODULE_CHANGED in pr_info.labels: + pr_labels_to_remove.append(CI.Labels.SUBMODULE_CHANGED) - if any(label in Labels.AUTO_BACKPORT for label in pr_labels_to_add): - backport_labels = [Labels.MUST_BACKPORT, Labels.MUST_BACKPORT_CLOUD] + if any(label in CI.Labels.AUTO_BACKPORT for label in pr_labels_to_add): + backport_labels = [CI.Labels.MUST_BACKPORT, CI.Labels.MUST_BACKPORT_CLOUD] pr_labels_to_add += [ label for label in backport_labels if label not in pr_info.labels ] @@ -164,15 +162,15 @@ def main(): # 2. Then we check if the documentation is not created to fail the Mergeable check if ( - Labels.PR_FEATURE in pr_info.labels + CI.Labels.PR_FEATURE in pr_info.labels and not pr_info.has_changes_in_documentation() ): print( - f"::error ::The '{Labels.PR_FEATURE}' in the labels, " + f"::error ::The '{CI.Labels.PR_FEATURE}' in the labels, " "but there's no changed documentation" ) status = FAILURE - description = f"expect adding docs for {Labels.PR_FEATURE}" + description = f"expect adding docs for {CI.Labels.PR_FEATURE}" # 3. But we allow the workflow to continue # 4. And post only a single commit status on a failure diff --git a/tests/ci/runner_token_rotation_lambda/app.py b/tests/ci/runner_token_rotation_lambda/app.py deleted file mode 100644 index 445704648a9..00000000000 --- a/tests/ci/runner_token_rotation_lambda/app.py +++ /dev/null @@ -1,93 +0,0 @@ -#!/usr/bin/env python3 - -import argparse -import sys - -import boto3 # type: ignore -import requests -from lambda_shared.token import get_access_token_by_key_app, get_cached_access_token - - -def get_runner_registration_token(access_token): - headers = { - "Authorization": f"token {access_token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.post( - "https://api.github.com/orgs/ClickHouse/actions/runners/registration-token", - headers=headers, - timeout=30, - ) - response.raise_for_status() - data = response.json() - return data["token"] - - -def main(access_token, push_to_ssm, ssm_parameter_name): - runner_registration_token = get_runner_registration_token(access_token) - - if push_to_ssm: - print("Trying to put params into ssm manager") - client = boto3.client("ssm") - client.put_parameter( - Name=ssm_parameter_name, - Value=runner_registration_token, - Type="SecureString", - Overwrite=True, - ) - else: - print( - "Not push token to AWS Parameter Store, just print:", - runner_registration_token, - ) - - -def handler(event, context): - _, _ = event, context - main(get_cached_access_token(), True, "github_runner_registration_token") - - -if __name__ == "__main__": - parser = argparse.ArgumentParser( - description="Get new token from github to add runners" - ) - parser.add_argument( - "-p", "--private-key-path", help="Path to file with private key" - ) - parser.add_argument("-k", "--private-key", help="Private key") - parser.add_argument( - "-a", "--app-id", type=int, help="GitHub application ID", required=True - ) - parser.add_argument( - "--push-to-ssm", - action="store_true", - help="Store received token in parameter store", - ) - parser.add_argument( - "--ssm-parameter-name", - default="github_runner_registration_token", - help="AWS paramater store parameter name", - ) - - args = parser.parse_args() - - if not args.private_key_path and not args.private_key: - print( - "Either --private-key-path or --private-key must be specified", - file=sys.stderr, - ) - - if args.private_key_path and args.private_key: - print( - "Either --private-key-path or --private-key must be specified", - file=sys.stderr, - ) - - if args.private_key: - private_key = args.private_key - else: - with open(args.private_key_path, "r", encoding="utf-8") as key_file: - private_key = key_file.read() - - token = get_access_token_by_key_app(private_key, args.app_id) - main(token, args.push_to_ssm, args.ssm_parameter_name) diff --git a/tests/ci/runner_token_rotation_lambda/build_and_deploy_archive.sh b/tests/ci/runner_token_rotation_lambda/build_and_deploy_archive.sh deleted file mode 120000 index 96ba3fa024e..00000000000 --- a/tests/ci/runner_token_rotation_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1 +0,0 @@ -../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/runner_token_rotation_lambda/lambda_shared b/tests/ci/runner_token_rotation_lambda/lambda_shared deleted file mode 120000 index ba86e090f6c..00000000000 --- a/tests/ci/runner_token_rotation_lambda/lambda_shared +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/runner_token_rotation_lambda/requirements.txt b/tests/ci/runner_token_rotation_lambda/requirements.txt deleted file mode 100644 index 4cb3fba0f7b..00000000000 --- a/tests/ci/runner_token_rotation_lambda/requirements.txt +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package[token] diff --git a/tests/ci/slack_bot_ci_lambda/app.py b/tests/ci/slack_bot_ci_lambda/app.py deleted file mode 100755 index 94b71724b1c..00000000000 --- a/tests/ci/slack_bot_ci_lambda/app.py +++ /dev/null @@ -1,323 +0,0 @@ -#!/usr/bin/env python3 - -""" -A trivial stateless slack bot that notifies about new broken tests in ClickHouse CI. -It checks what happened to our CI during the last check_period hours (1 hour) and - notifies us in slack if necessary. -This script should be executed once each check_period hours (1 hour). -It will post duplicate messages if you run it more often; it will lose some messages - if you run it less often. - -You can run it locally with no arguments, it will work in a dry-run mode. - Or you can set your own SLACK_URL_DEFAULT. -Feel free to add more checks, more details to messages, or better heuristics. - -It's deployed to slack-bot-ci-lambda in CI/CD account - -See also: https://aretestsgreenyet.com/ -""" - -import base64 -import json -import os -import random - -import requests - -DRY_RUN_MARK = "" - -MAX_FAILURES_DEFAULT = 30 -SLACK_URL_DEFAULT = DRY_RUN_MARK - -FLAKY_ALERT_PROBABILITY = 0.50 -REPORT_NO_FAILURES_PROBABILITY = 0.99 - -MAX_TESTS_TO_REPORT = 4 - -# Slack has a stupid limitation on message size, it splits long messages into multiple, -# ones breaking formatting -MESSAGE_LENGTH_LIMIT = 4000 - -# Find tests that failed in master during the last check_period * 24 hours, -# but did not fail during the last 2 weeks. Assuming these tests were broken recently. -# Counts number of failures in check_period and check_period * 24 time windows -# to distinguish rare flaky tests from completely broken tests -NEW_BROKEN_TESTS_QUERY = """ -WITH - 1 AS check_period, - check_period * 24 AS extended_check_period, - now() as now -SELECT - test_name, - any(report_url), - countIf((check_start_time + check_duration_ms / 1000) < now - INTERVAL check_period HOUR) AS count_prev_periods, - countIf((check_start_time + check_duration_ms / 1000) >= now - INTERVAL check_period HOUR) AS count -FROM checks -WHERE 1 - AND check_start_time BETWEEN now - INTERVAL 1 WEEK AND now - AND (check_start_time + check_duration_ms / 1000) >= now - INTERVAL extended_check_period HOUR - AND pull_request_number = 0 - AND test_status LIKE 'F%' - AND check_status != 'success' - AND test_name NOT IN ( - SELECT test_name FROM checks WHERE 1 - AND check_start_time >= now - INTERVAL 1 MONTH - AND (check_start_time + check_duration_ms / 1000) BETWEEN now - INTERVAL 2 WEEK AND now - INTERVAL extended_check_period HOUR - AND pull_request_number = 0 - AND check_status != 'success' - AND test_status LIKE 'F%') - AND test_context_raw NOT LIKE '%CannotSendRequest%' and test_context_raw NOT LIKE '%Server does not respond to health check%' -GROUP BY test_name -ORDER BY (count_prev_periods + count) DESC -""" - -# Returns total number of failed checks during the last 24 hours -# and previous value of that metric (check_period hours ago) -COUNT_FAILURES_QUERY = """ -WITH - 1 AS check_period, - '%' AS check_name_pattern, - now() as now -SELECT - countIf((check_start_time + check_duration_ms / 1000) >= now - INTERVAL 24 HOUR) AS new_val, - countIf((check_start_time + check_duration_ms / 1000) <= now - INTERVAL check_period HOUR) AS prev_val -FROM checks -WHERE 1 - AND check_start_time >= now - INTERVAL 1 WEEK - AND (check_start_time + check_duration_ms / 1000) >= now - INTERVAL 24 + check_period HOUR - AND pull_request_number = 0 - AND test_status LIKE 'F%' - AND check_status != 'success' - AND check_name ILIKE check_name_pattern -""" - -# Returns percentage of failed checks (once per day, at noon) -FAILED_CHECKS_PERCENTAGE_QUERY = """ -SELECT if(toHour(now('Europe/Amsterdam')) = 12, v, 0) -FROM -( - SELECT - countDistinctIf((commit_sha, check_name), (test_status LIKE 'F%') AND (check_status != 'success')) - / countDistinct((commit_sha, check_name)) AS v - FROM checks - WHERE 1 - AND (pull_request_number = 0) - AND (test_status != 'SKIPPED') - AND (check_start_time > (now() - toIntervalDay(1))) -) -""" - -# It shows all recent failures of the specified test (helps to find when it started) -ALL_RECENT_FAILURES_QUERY = """ -WITH - '{}' AS name_substr, - 90 AS interval_days, - ('Stateless tests (asan)', 'Stateless tests (address)', 'Stateless tests (address, actions)', 'Integration tests (asan) [1/3]', 'Stateless tests (tsan) [1/3]') AS backport_and_release_specific_checks -SELECT - toStartOfDay(check_start_time) AS d, - count(), - groupUniqArray(pull_request_number) AS prs, - any(report_url) -FROM checks -WHERE ((now() - toIntervalDay(interval_days)) <= check_start_time) AND (pull_request_number NOT IN ( - SELECT pull_request_number AS prn - FROM checks - WHERE (prn != 0) AND ((now() - toIntervalDay(interval_days)) <= check_start_time) AND (check_name IN (backport_and_release_specific_checks)) -)) AND (position(test_name, name_substr) > 0) AND (test_status IN ('FAIL', 'ERROR', 'FLAKY')) -GROUP BY d -ORDER BY d DESC -""" - -SLACK_MESSAGE_JSON = {"type": "mrkdwn", "text": None} - - -def get_play_url(query): - return ( - "https://play.clickhouse.com/play?user=play#" - + base64.b64encode(query.encode()).decode() - ) - - -def run_clickhouse_query(query): - url = "https://play.clickhouse.com/?user=play&query=" + requests.compat.quote(query) - res = requests.get(url, timeout=30) - if res.status_code != 200: - print("Failed to execute query: ", res.status_code, res.content) - res.raise_for_status() - - lines = res.text.strip().splitlines() - return [x.split("\t") for x in lines] - - -def split_broken_and_flaky_tests(failed_tests): - if not failed_tests: - return None - - broken_tests = [] - flaky_tests = [] - for name, report, count_prev_str, count_str in failed_tests: - count_prev, count = int(count_prev_str), int(count_str) - if (count_prev < 2 <= count) or (count_prev == count == 1): - # It failed 2 times or more within extended time window, it's definitely broken. - # 2 <= count means that it was not reported as broken on previous runs - broken_tests.append([name, report]) - elif 0 < count and count_prev == 0: - # It failed only once, can be a rare flaky test - flaky_tests.append([name, report]) - - return broken_tests, flaky_tests - - -def format_failed_tests_list(failed_tests, failure_type): - if len(failed_tests) == 1: - res = f"There is a new {failure_type} test:\n" - else: - res = f"There are {len(failed_tests)} new {failure_type} tests:\n" - - for name, report in failed_tests[:MAX_TESTS_TO_REPORT]: - cidb_url = get_play_url(ALL_RECENT_FAILURES_QUERY.format(name)) - res += f"- *{name}* - <{report}|Report> - <{cidb_url}|CI DB> \n" - - if MAX_TESTS_TO_REPORT < len(failed_tests): - res += ( - f"- and {len(failed_tests) - MAX_TESTS_TO_REPORT} other " - "tests... :this-is-fine-fire:" - ) - - return res - - -def get_new_broken_tests_message(failed_tests): - if not failed_tests: - return None - - broken_tests, flaky_tests = split_broken_and_flaky_tests(failed_tests) - if len(broken_tests) == 0 and len(flaky_tests) == 0: - return None - - msg = "" - if len(broken_tests) > 0: - msg += format_failed_tests_list(broken_tests, "*BROKEN*") - elif random.random() > FLAKY_ALERT_PROBABILITY: - looks_like_fuzzer = [x[0].count(" ") > 2 for x in flaky_tests] - if not any(looks_like_fuzzer): - print("Will not report flaky tests to avoid noise: ", flaky_tests) - return None - - if len(flaky_tests) > 0: - if len(msg) > 0: - msg += "\n" - msg += format_failed_tests_list(flaky_tests, "flaky") - - return msg - - -def get_too_many_failures_message_impl(failures_count): - MAX_FAILURES = int(os.environ.get("MAX_FAILURES", MAX_FAILURES_DEFAULT)) - curr_failures = int(failures_count[0][0]) - prev_failures = int(failures_count[0][1]) - if curr_failures == 0 and prev_failures != 0: - if random.random() < REPORT_NO_FAILURES_PROBABILITY: - return None - return "Wow, there are *no failures* at all... 0_o" - return_none = ( - curr_failures < MAX_FAILURES - or curr_failures < prev_failures - or (curr_failures - prev_failures) / prev_failures < 0.2 - ) - if return_none: - return None - if prev_failures < MAX_FAILURES: - return f":alert: *CI is broken: there are {curr_failures} failures during the last 24 hours*" - return "CI is broken and it's getting worse: there are {curr_failures} failures during the last 24 hours" - - -def get_too_many_failures_message(failures_count): - msg = get_too_many_failures_message_impl(failures_count) - if msg: - msg += "\nSee https://aretestsgreenyet.com/" - return msg - - -def get_failed_checks_percentage_message(percentage): - p = float(percentage[0][0]) * 100 - - # Always report more than 1% of failed checks - # For <= 1%: higher percentage of failures == higher probability - if p <= random.random(): - return None - - msg = ":alert: " if p > 1 else "Only " if p < 0.5 else "" - msg += f"*{p:.2f}%* of all checks in master have failed yesterday" - return msg - - -def split_slack_message(long_message): - lines = long_message.split("\n") - messages = [] - curr_msg = "" - for line in lines: - if len(curr_msg) + len(line) < MESSAGE_LENGTH_LIMIT: - curr_msg += "\n" - curr_msg += line - else: - messages.append(curr_msg) - curr_msg = line - messages.append(curr_msg) - return messages - - -def send_to_slack_impl(message): - SLACK_URL = os.environ.get("SLACK_URL", SLACK_URL_DEFAULT) - if SLACK_URL == DRY_RUN_MARK: - return - - payload = SLACK_MESSAGE_JSON.copy() - payload["text"] = message - res = requests.post(SLACK_URL, json.dumps(payload), timeout=30) - if res.status_code != 200: - print("Failed to send a message to Slack: ", res.status_code, res.content) - res.raise_for_status() - - -def send_to_slack(message): - messages = split_slack_message(message) - for msg in messages: - send_to_slack_impl(msg) - - -def query_and_alert_if_needed(query, get_message_func): - query_res = run_clickhouse_query(query) - print("Got result {} for query {}", query_res, query) - msg = get_message_func(query_res) - if msg is None: - return - - msg += f"\nCI DB query: <{get_play_url(query)}|link>" - print("Sending message to slack:", msg) - send_to_slack(msg) - - -def check_and_alert(): - query_and_alert_if_needed(NEW_BROKEN_TESTS_QUERY, get_new_broken_tests_message) - query_and_alert_if_needed(COUNT_FAILURES_QUERY, get_too_many_failures_message) - query_and_alert_if_needed( - FAILED_CHECKS_PERCENTAGE_QUERY, get_failed_checks_percentage_message - ) - - -def handler(event, context): - _, _ = event, context - try: - check_and_alert() - return {"statusCode": 200, "body": "OK"} - except Exception as e: - send_to_slack( - "I failed, please help me " - f"(see ClickHouse/ClickHouse/tests/ci/slack_bot_ci_lambda/app.py): {e}" - ) - return {"statusCode": 200, "body": "FAIL"} - - -if __name__ == "__main__": - check_and_alert() diff --git a/tests/ci/slack_bot_ci_lambda/build_and_deploy_archive.sh b/tests/ci/slack_bot_ci_lambda/build_and_deploy_archive.sh deleted file mode 120000 index 96ba3fa024e..00000000000 --- a/tests/ci/slack_bot_ci_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1 +0,0 @@ -../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/slack_bot_ci_lambda/requirements.txt b/tests/ci/slack_bot_ci_lambda/requirements.txt deleted file mode 100644 index 098e04a9798..00000000000 --- a/tests/ci/slack_bot_ci_lambda/requirements.txt +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package diff --git a/tests/ci/team_keys_lambda/app.py b/tests/ci/team_keys_lambda/app.py deleted file mode 100644 index c056808a9b4..00000000000 --- a/tests/ci/team_keys_lambda/app.py +++ /dev/null @@ -1,136 +0,0 @@ -#!/usr/bin/env python3 - -import argparse -import json -from datetime import datetime -from queue import Queue -from threading import Thread - -import boto3 # type: ignore -import requests - - -class Keys(set): - def __init__(self, *args, **kwargs): - super().__init__(*args, **kwargs) - self.updated_at = 0.0 - - def update_now(self): - self.updated_at = datetime.now().timestamp() - - -keys = Keys() - - -class Worker(Thread): - def __init__(self, request_queue): - Thread.__init__(self) - self.queue = request_queue - self.results = set() - - def run(self): - while True: - m = self.queue.get() - if m == "": - break - response = requests.get(f"https://github.com/{m}.keys", timeout=30) - self.results.add(f"# {m}\n{response.text}\n") - self.queue.task_done() - - -def get_org_team_members(token: str, org: str, team_slug: str) -> set: - headers = { - "Authorization": f"token {token}", - "Accept": "application/vnd.github.v3+json", - } - response = requests.get( - f"https://api.github.com/orgs/{org}/teams/{team_slug}/members", - headers=headers, - timeout=30, - ) - response.raise_for_status() - data = response.json() - return set(m["login"] for m in data) - - -def get_cached_members_keys(members: set) -> Keys: - if (datetime.now().timestamp() - 3600) <= keys.updated_at: - return keys - - q = Queue() # type: Queue - workers = [] - for m in members: - q.put(m) - # Create workers and add to the queue - worker = Worker(q) - worker.start() - workers.append(worker) - - # Workers keep working till they receive an empty string - for _ in workers: - q.put("") - - # Join workers to wait till they finished - for worker in workers: - worker.join() - - keys.clear() - for worker in workers: - keys.update(worker.results) - keys.update_now() - return keys - - -def get_token_from_aws() -> str: - # We need a separate token, since the clickhouse-ci app does not have - # access to the organization members' endpoint - secret_name = "clickhouse_robot_token" - session = boto3.session.Session() - client = session.client( - service_name="secretsmanager", - ) - get_secret_value_response = client.get_secret_value(SecretId=secret_name) - data = json.loads(get_secret_value_response["SecretString"]) - return data["clickhouse_robot_token"] # type: ignore - - -def main(token: str, org: str, team_slug: str) -> str: - members = get_org_team_members(token, org, team_slug) - keys = get_cached_members_keys(members) - - return "".join(sorted(keys)) - - -def handler(event, context): - _ = context - _ = event - if keys.updated_at < (datetime.now().timestamp() - 3600): - token = get_token_from_aws() - body = main(token, "ClickHouse", "core") - else: - body = "".join(sorted(keys)) - - result = { - "statusCode": 200, - "headers": { - "Content-Type": "text/html", - }, - "body": body, - } - return result - - -if __name__ == "__main__": - parser = argparse.ArgumentParser( - description="Get the public SSH keys for members of given org and team" - ) - parser.add_argument("--token", required=True, help="Github PAT") - parser.add_argument( - "--organization", help="GitHub organization name", default="ClickHouse" - ) - parser.add_argument("--team", help="GitHub team name", default="core") - - args = parser.parse_args() - output = main(args.token, args.organization, args.team) - - print(f"# Just showing off the keys:\n{output}") diff --git a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh deleted file mode 100644 index b72bce4a677..00000000000 --- a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1,76 +0,0 @@ -#!/usr/bin/env bash -set -xeo pipefail - -WORKDIR=$(dirname "$0") -WORKDIR=$(readlink -f "${WORKDIR}") -DIR_NAME=$(basename "$WORKDIR") -cd "$WORKDIR" - -# Do not deploy the lambda to AWS -DRY_RUN=${DRY_RUN:-} -# Python runtime to install dependencies -PY_VERSION=${PY_VERSION:-3.10} -PY_EXEC="python${PY_VERSION}" -# Image to build the lambda zip package -DOCKER_IMAGE="public.ecr.aws/lambda/python:${PY_VERSION}" -# Rename the_lambda_name directory to the-lambda-name lambda in AWS -LAMBDA_NAME=${DIR_NAME//_/-} -# The name of directory with lambda code -PACKAGE=lambda-package - -# Do not rebuild and deploy the archive if it's newer than sources -if [ -e "$PACKAGE.zip" ] && [ -z "$FORCE" ]; then - REBUILD="" - for src in app.py build_and_deploy_archive.sh requirements.txt lambda_shared/*; do - if [ "$src" -nt "$PACKAGE.zip" ]; then - REBUILD=1 - fi - done - [ -n "$REBUILD" ] || exit 0 -fi - -docker_cmd=( - docker run -i --net=host --rm --user="${UID}" -e HOME=/tmp --entrypoint=/bin/bash - --volume="${WORKDIR}/..:/ci" --workdir="/ci/${DIR_NAME}" "${DOCKER_IMAGE}" -) -rm -rf "$PACKAGE" "$PACKAGE".zip -mkdir "$PACKAGE" -cp app.py "$PACKAGE" -if [ -f requirements.txt ]; then - VENV=lambda-venv - rm -rf "$VENV" - "${docker_cmd[@]}" -ex < dict: - """return cached instances description with updating it once per five minutes""" - if time.time() - 250 < cached_instances.time or cached_instances.updating: - return cached_instances.value - cached_instances.updating = cached_value_is_valid(cached_instances.time, 300) - ec2_client = boto3.client("ec2") - instances_response = ec2_client.describe_instances( - Filters=[{"Name": "instance-state-name", "Values": ["running"]}] - ) - cached_instances.time = time.time() - cached_instances.value = { - instance["InstanceId"]: instance - for reservation in instances_response["Reservations"] - for instance in reservation["Instances"] - } - cached_instances.updating = False - return cached_instances.value - - -@dataclass -class CachedRunners: - time: float - value: RunnerDescriptions - updating: bool = False - - -cached_runners = CachedRunners(0, []) - - -def get_cached_runners(access_token: str) -> RunnerDescriptions: - """From time to time request to GH api costs up to 3 seconds, and - it's a disaster from the termination lambda perspective""" - if time.time() - 5 < cached_runners.time or cached_instances.updating: - return cached_runners.value - cached_runners.updating = cached_value_is_valid(cached_runners.time, 15) - cached_runners.value = list_runners(access_token) - cached_runners.time = time.time() - cached_runners.updating = False - return cached_runners.value - - -def how_many_instances_to_kill(event_data: dict) -> Dict[str, int]: - data_array = event_data["CapacityToTerminate"] - to_kill_by_zone = {} # type: Dict[str, int] - for av_zone in data_array: - zone_name = av_zone["AvailabilityZone"] - to_kill = av_zone["Capacity"] - if zone_name not in to_kill_by_zone: - to_kill_by_zone[zone_name] = 0 - - to_kill_by_zone[zone_name] += to_kill - - return to_kill_by_zone - - -def get_candidates_to_be_killed(event_data: dict) -> Dict[str, List[str]]: - data_array = event_data["Instances"] - instances_by_zone = {} # type: Dict[str, List[str]] - for instance in data_array: - zone_name = instance["AvailabilityZone"] - instance_id = instance["InstanceId"] # type: str - if zone_name not in instances_by_zone: - instances_by_zone[zone_name] = [] - instances_by_zone[zone_name].append(instance_id) - - return instances_by_zone - - -def main(access_token: str, event: dict) -> Dict[str, List[str]]: - start = time.time() - print("Got event", json.dumps(event, sort_keys=True).replace("\n", "")) - to_kill_by_zone = how_many_instances_to_kill(event) - instances_by_zone = get_candidates_to_be_killed(event) - # Getting ASG and instances' descriptions from the API - # We don't kill instances that alive for less than 10 minutes, since they - # could be not in the GH active runners yet - print(f"Check other hosts from the same ASG {event['AutoScalingGroupName']}") - asg_client = boto3.client("autoscaling") - as_groups_response = asg_client.describe_auto_scaling_groups( - AutoScalingGroupNames=[event["AutoScalingGroupName"]] - ) - assert len(as_groups_response["AutoScalingGroups"]) == 1 - asg = as_groups_response["AutoScalingGroups"][0] - asg_instance_ids = [instance["InstanceId"] for instance in asg["Instances"]] - instance_descriptions = get_cached_instances() - # The instances launched less than 10 minutes ago - immune_ids = [ - instance["InstanceId"] - for instance in instance_descriptions.values() - if start - instance["LaunchTime"].timestamp() < 600 - ] - # if the ASG's instance ID not in instance_descriptions, it's most probably - # is not cached yet, so we must mark it as immuned - immune_ids.extend( - iid for iid in asg_instance_ids if iid not in instance_descriptions - ) - print("Time spent on the requests to AWS: ", time.time() - start) - - runners = get_cached_runners(access_token) - runner_ids = set(runner.name for runner in runners) - # We used to delete potential hosts to terminate from GitHub runners pool, - # but the documentation states: - # --- Returning an instance first in the response data does not guarantee its termination - # so they will be cleaned out by ci_runners_metrics_lambda eventually - - instances_to_kill = [] - total_to_kill = 0 - for zone, num_to_kill in to_kill_by_zone.items(): - candidates = instances_by_zone[zone] - total_to_kill += num_to_kill - if num_to_kill > len(candidates): - raise RuntimeError( - f"Required to kill {num_to_kill}, but have only {len(candidates)}" - f" candidates in AV {zone}" - ) - - delete_for_av = [] # type: RunnerDescriptions - for candidate in candidates: - if candidate in immune_ids: - print( - f"Candidate {candidate} started less than 10 minutes ago, won't touch a child" - ) - break - if candidate not in runner_ids: - print( - f"Candidate {candidate} was not in runners list, simply delete it" - ) - instances_to_kill.append(candidate) - break - if len(delete_for_av) + len(instances_to_kill) == num_to_kill: - break - if candidate in instances_to_kill: - continue - - for runner in runners: - if runner.name == candidate: - if not runner.busy: - print( - f"Runner {runner.name} is not busy and can be deleted from AV {zone}" - ) - delete_for_av.append(runner) - else: - print(f"Runner {runner.name} is busy, not going to delete it") - break - - if len(delete_for_av) < num_to_kill: - print( - f"Checked all candidates for av {zone}, get to delete " - f"{len(delete_for_av)}, but still cannot get required {num_to_kill}" - ) - - instances_to_kill += [runner.name for runner in delete_for_av] - - if len(instances_to_kill) < total_to_kill: - for instance in asg_instance_ids: - if instance in immune_ids: - continue - for runner in runners: - if runner.name == instance and not runner.busy: - print(f"Runner {runner.name} is not busy and can be deleted") - instances_to_kill.append(runner.name) - - if total_to_kill <= len(instances_to_kill): - print("Got enough instances to kill") - break - - response = {"InstanceIDs": instances_to_kill} - print("Got instances to kill: ", response) - print("Time spent on the request: ", time.time() - start) - return response - - -def handler(event: dict, context: Any) -> Dict[str, List[str]]: - _ = context - return main(get_cached_access_token(), event) - - -if __name__ == "__main__": - parser = argparse.ArgumentParser(description="Get list of runners and their states") - parser.add_argument( - "-p", "--private-key-path", help="Path to file with private key" - ) - parser.add_argument("-k", "--private-key", help="Private key") - parser.add_argument( - "-a", "--app-id", type=int, help="GitHub application ID", required=True - ) - - args = parser.parse_args() - - if not args.private_key_path and not args.private_key: - print( - "Either --private-key-path or --private-key must be specified", - file=sys.stderr, - ) - - if args.private_key_path and args.private_key: - print( - "Either --private-key-path or --private-key must be specified", - file=sys.stderr, - ) - - if args.private_key: - private_key = args.private_key - else: - with open(args.private_key_path, "r", encoding="utf-8") as key_file: - private_key = key_file.read() - - token = get_access_token_by_key_app(private_key, args.app_id) - - sample_event = { - "AutoScalingGroupARN": "arn:aws:autoscaling:us-east-1::autoScalingGroup:d4738357-2d40-4038-ae7e-b00ae0227003:autoScalingGroupName/my-asg", - "AutoScalingGroupName": "my-asg", - "CapacityToTerminate": [ - { - "AvailabilityZone": "us-east-1b", - "Capacity": 1, - "InstanceMarketOption": "OnDemand", - }, - { - "AvailabilityZone": "us-east-1c", - "Capacity": 2, - "InstanceMarketOption": "OnDemand", - }, - ], - "Instances": [ - { - "AvailabilityZone": "us-east-1b", - "InstanceId": "i-08d0b3c1a137e02a5", - "InstanceType": "t2.nano", - "InstanceMarketOption": "OnDemand", - }, - { - "AvailabilityZone": "us-east-1c", - "InstanceId": "ip-172-31-45-253.eu-west-1.compute.internal", - "InstanceType": "t2.nano", - "InstanceMarketOption": "OnDemand", - }, - { - "AvailabilityZone": "us-east-1c", - "InstanceId": "ip-172-31-27-227.eu-west-1.compute.internal", - "InstanceType": "t2.nano", - "InstanceMarketOption": "OnDemand", - }, - { - "AvailabilityZone": "us-east-1c", - "InstanceId": "ip-172-31-45-253.eu-west-1.compute.internal", - "InstanceType": "t2.nano", - "InstanceMarketOption": "OnDemand", - }, - ], - "Cause": "SCALE_IN", - } - - main(token, sample_event) diff --git a/tests/ci/terminate_runner_lambda/build_and_deploy_archive.sh b/tests/ci/terminate_runner_lambda/build_and_deploy_archive.sh deleted file mode 120000 index 96ba3fa024e..00000000000 --- a/tests/ci/terminate_runner_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1 +0,0 @@ -../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/terminate_runner_lambda/lambda_shared b/tests/ci/terminate_runner_lambda/lambda_shared deleted file mode 120000 index ba86e090f6c..00000000000 --- a/tests/ci/terminate_runner_lambda/lambda_shared +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/terminate_runner_lambda/requirements.txt b/tests/ci/terminate_runner_lambda/requirements.txt deleted file mode 100644 index 4cb3fba0f7b..00000000000 --- a/tests/ci/terminate_runner_lambda/requirements.txt +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package[token] diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py deleted file mode 100644 index 8ed47a54ab1..00000000000 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ /dev/null @@ -1,404 +0,0 @@ -#!/usr/bin/env python3 - -import fnmatch -import json -import time -from collections import namedtuple -from urllib.parse import quote - -import requests -from lambda_shared.pr import TRUSTED_CONTRIBUTORS -from lambda_shared.token import get_cached_access_token - -SUSPICIOUS_CHANGED_FILES_NUMBER = 200 - -SUSPICIOUS_PATTERNS = [ - ".github/*", - "docker/*", - "docs/tools/*", - "packages/*", - "tests/ci/*", -] - -# Number of retries for API calls. -MAX_RETRY = 5 - -# Number of times a check can re-run as a whole. -# It is needed, because we are using AWS "spot" instances, that are terminated often -MAX_WORKFLOW_RERUN = 30 - -WorkflowDescription = namedtuple( - "WorkflowDescription", - [ - "name", - "action", - "run_id", - "event", - "workflow_id", - "conclusion", - "status", - "api_url", - "fork_owner_login", - "fork_branch", - "rerun_url", - "jobs_url", - "attempt", - "repo_url", - "url", - ], -) - -# See https://api.github.com/orgs/{name} -TRUSTED_ORG_IDS = { - 54801242, # clickhouse -} - -# See https://api.github.com/repos/ClickHouse/ClickHouse/actions/workflows -# Use ID to not inject a malicious workflow -TRUSTED_WORKFLOW_IDS = { - 14586616, # Cancel workflows, always trusted -} - -NEED_RERUN_WORKFLOWS = { - "BackportPR", - "DocsCheck", - "MasterCI", - "NightlyBuilds", - "PublishedReleaseCI", - "PullRequestCI", - "ReleaseBranchCI", -} - - -def is_trusted_contributor(pr_user_login, pr_user_orgs): - if pr_user_login.lower() in TRUSTED_CONTRIBUTORS: - print(f"User '{pr_user_login}' is trusted") - return True - - print(f"User '{pr_user_login}' is not trusted") - - for org_id in pr_user_orgs: - if org_id in TRUSTED_ORG_IDS: - print( - f"Org '{org_id}' is trusted; will mark user {pr_user_login} as trusted" - ) - return True - print(f"Org '{org_id}' is not trusted") - - return False - - -def _exec_get_with_retry(url, token): - headers = {"Authorization": f"token {token}"} - e = Exception() - for i in range(MAX_RETRY): - try: - response = requests.get(url, headers=headers, timeout=30) - response.raise_for_status() - return response.json() - except Exception as ex: - print("Got exception executing request", ex) - e = ex - time.sleep(i + 1) - - raise requests.HTTPError("Cannot execute GET request with retries") from e - - -def _exec_post_with_retry(url, token, data=None): - headers = {"Authorization": f"token {token}"} - e = Exception() - for i in range(MAX_RETRY): - try: - if data: - response = requests.post(url, headers=headers, json=data, timeout=30) - else: - response = requests.post(url, headers=headers, timeout=30) - if response.status_code == 403: - data = response.json() - if ( - "message" in data - and data["message"] - == "This workflow run is not waiting for approval" - ): - print("Workflow doesn't need approval") - return data - response.raise_for_status() - return response.json() - except Exception as ex: - print("Got exception executing request", ex) - e = ex - time.sleep(i + 1) - - raise requests.HTTPError("Cannot execute POST request with retry") from e - - -def _get_pull_requests_from(repo_url, owner, branch, token): - url = f"{repo_url}/pulls?head={quote(owner)}:{quote(branch)}" - return _exec_get_with_retry(url, token) - - -def get_workflow_description_from_event(event): - action = event["action"] - run_id = event["workflow_run"]["id"] - event_type = event["workflow_run"]["event"] - fork_owner = event["workflow_run"]["head_repository"]["owner"]["login"] - fork_branch = event["workflow_run"]["head_branch"] - name = event["workflow_run"]["name"] - workflow_id = event["workflow_run"]["workflow_id"] - conclusion = event["workflow_run"]["conclusion"] - attempt = event["workflow_run"]["run_attempt"] - status = event["workflow_run"]["status"] - jobs_url = event["workflow_run"]["jobs_url"] - rerun_url = event["workflow_run"]["rerun_url"] - url = event["workflow_run"]["html_url"] - api_url = event["workflow_run"]["url"] - repo_url = event["repository"]["url"] - return WorkflowDescription( - name=name, - action=action, - run_id=run_id, - event=event_type, - fork_owner_login=fork_owner, - fork_branch=fork_branch, - workflow_id=workflow_id, - conclusion=conclusion, - attempt=attempt, - status=status, - jobs_url=jobs_url, - rerun_url=rerun_url, - url=url, - repo_url=repo_url, - api_url=api_url, - ) - - -def get_pr_author_and_orgs(pull_request, token): - author = pull_request["user"]["login"] - orgs = _exec_get_with_retry(pull_request["user"]["organizations_url"], token) - return author, [org["id"] for org in orgs] - - -def get_changed_files_for_pull_request(pull_request, token): - url = pull_request["url"] - - changed_files = set([]) - for i in range(1, 31): - print("Requesting changed files page", i) - data = _exec_get_with_retry(f"{url}/files?page={i}&per_page=100", token) - print(f"Got {len(data)} changed files") - if len(data) == 0: - print("No more changed files") - break - - for change in data: - # print("Adding changed file", change['filename']) - changed_files.add(change["filename"]) - - if len(changed_files) >= SUSPICIOUS_CHANGED_FILES_NUMBER: - print( - f"More than {len(changed_files)} changed files. " - "Will stop fetching new files." - ) - break - - return changed_files - - -def check_suspicious_changed_files(changed_files): - if len(changed_files) >= SUSPICIOUS_CHANGED_FILES_NUMBER: - print(f"Too many files changed {len(changed_files)}, need manual approve") - return True - - for path in changed_files: - for pattern in SUSPICIOUS_PATTERNS: - if fnmatch.fnmatch(path, pattern): - print( - f"File {path} match suspicious pattern {pattern}, " - "will not approve automatically" - ) - return True - - print("No changed files match suspicious patterns, run could be approved") - return False - - -def approve_run(workflow_description: WorkflowDescription, token: str) -> None: - print("Approving run") - url = f"{workflow_description.api_url}/approve" - _exec_post_with_retry(url, token) - - -def label_manual_approve(pull_request, token): - url = f"{pull_request['issue_url']}/labels" - data = {"labels": ["manual approve"]} - - _exec_post_with_retry(url, token, data) - - -def get_workflow_jobs(workflow_description, token): - jobs_url = ( - workflow_description.api_url + f"/attempts/{workflow_description.attempt}/jobs" - ) - jobs = [] - i = 1 - while True: - got_jobs = _exec_get_with_retry(jobs_url + f"?page={i}", token) - if len(got_jobs["jobs"]) == 0: - break - - jobs += got_jobs["jobs"] - i += 1 - - return jobs - - -def check_need_to_rerun(workflow_description, token): - if workflow_description.attempt >= MAX_WORKFLOW_RERUN: - print( - "Not going to rerun workflow because it's already tried more than two times" - ) - return False - print("Going to check jobs") - - jobs = get_workflow_jobs(workflow_description, token) - print("Got jobs", len(jobs)) - for job in jobs: - print(f"Job {job['name']} has a conclusion '{job['conclusion']}'") - if job["conclusion"] not in ("success", "skipped"): - print("Job", job["name"], "failed, checking steps") - for step in job["steps"]: - # always the last job - if step["name"] == "Complete job": - print("Found Complete job step for job", job["name"]) - break - else: - print( - "Checked all steps and doesn't found Complete job, going to rerun" - ) - return True - - return False - - -def rerun_workflow(workflow_description, token): - print("Going to rerun workflow") - try: - _exec_post_with_retry(f"{workflow_description.rerun_url}-failed-jobs", token) - except Exception: - _exec_post_with_retry(workflow_description.rerun_url, token) - - -def check_workflow_completed( - event_data: dict, workflow_description: WorkflowDescription, token: str -) -> bool: - if workflow_description.action == "completed": - attempt = 0 - # Nice and reliable GH API sends from time to time such events, e.g: - # action='completed', conclusion=None, status='in_progress', - # So let's try receiving a real workflow data - while workflow_description.conclusion is None and attempt < MAX_RETRY: - progressive_sleep = 3 * sum(i + 1 for i in range(attempt)) - time.sleep(progressive_sleep) - event_data["workflow_run"] = _exec_get_with_retry( - workflow_description.api_url, token - ) - workflow_description = get_workflow_description_from_event(event_data) - attempt += 1 - - if workflow_description.conclusion != "failure": - print( - "Workflow finished with status " - f"{workflow_description.conclusion}, exiting" - ) - return True - - print( - "Workflow", - workflow_description.url, - "completed and failed, let's check for rerun", - ) - - if workflow_description.name not in NEED_RERUN_WORKFLOWS: - print( - "Workflow", - workflow_description.name, - "not in list of rerunable workflows", - ) - return True - - if check_need_to_rerun(workflow_description, token): - rerun_workflow(workflow_description, token) - return True - - return False - - -def main(event): - token = get_cached_access_token() - event_data = json.loads(event["body"]) - print("The body received:", event["body"]) - workflow_description = get_workflow_description_from_event(event_data) - - print("Got workflow description", workflow_description) - if check_workflow_completed(event_data, workflow_description, token): - return - - if workflow_description.action != "requested": - print("Exiting, event action is", workflow_description.action) - return - - if workflow_description.workflow_id in TRUSTED_WORKFLOW_IDS: - print("Workflow in trusted list, approving run") - approve_run(workflow_description, token) - return - - pull_requests = _get_pull_requests_from( - workflow_description.repo_url, - workflow_description.fork_owner_login, - workflow_description.fork_branch, - token, - ) - - print("Got pull requests for workflow", len(pull_requests)) - if len(pull_requests) != 1: - print(f"Can't continue with non-uniq PRs: {pull_requests}") - return - - pull_request = pull_requests[0] - print("Pull request for workflow number", pull_request["number"]) - - author, author_orgs = get_pr_author_and_orgs(pull_request, token) - if is_trusted_contributor(author, author_orgs): - print("Contributor is trusted, approving run") - approve_run(workflow_description, token) - return - - labels = {label["name"] for label in pull_request["labels"]} - if "can be tested" not in labels: - print("Label 'can be tested' is required for untrusted users") - return - - changed_files = get_changed_files_for_pull_request(pull_request, token) - print(f"Totally have {len(changed_files)} changed files in PR:", changed_files) - if check_suspicious_changed_files(changed_files): - print(f"Pull Request {pull_request['number']} has suspicious changes") - if "manual approve" not in labels: - print("Label the PR as needed for manuall approve") - label_manual_approve(pull_request, token) - else: - print(f"Pull Request {pull_request['number']} has no suspicious changes") - approve_run(workflow_description, token) - - -def handler(event, _): - try: - main(event) - - return { - "statusCode": 200, - "headers": {"Content-Type": "application/json"}, - "body": '{"status": "OK"}', - } - except Exception: - print("Received event: ", event) - raise diff --git a/tests/ci/workflow_approve_rerun_lambda/build_and_deploy_archive.sh b/tests/ci/workflow_approve_rerun_lambda/build_and_deploy_archive.sh deleted file mode 120000 index 96ba3fa024e..00000000000 --- a/tests/ci/workflow_approve_rerun_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1 +0,0 @@ -../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/workflow_approve_rerun_lambda/lambda_shared b/tests/ci/workflow_approve_rerun_lambda/lambda_shared deleted file mode 120000 index ba86e090f6c..00000000000 --- a/tests/ci/workflow_approve_rerun_lambda/lambda_shared +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/workflow_approve_rerun_lambda/requirements.txt b/tests/ci/workflow_approve_rerun_lambda/requirements.txt deleted file mode 100644 index 4cb3fba0f7b..00000000000 --- a/tests/ci/workflow_approve_rerun_lambda/requirements.txt +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package[token] diff --git a/tests/ci/workflow_jobs_lambda/app.py b/tests/ci/workflow_jobs_lambda/app.py deleted file mode 100644 index 4210ca22da9..00000000000 --- a/tests/ci/workflow_jobs_lambda/app.py +++ /dev/null @@ -1,202 +0,0 @@ -#!/usr/bin/env python - -""" -Lambda gets the workflow_job events, see -https://docs.github.com/en/developers/webhooks-and-events/webhooks/webhook-events-and-payloads#workflow_job - -Then it either posts it as is to the play.clickhouse.com, or anonymizes the sensitive -fields for private repositories -""" - -import json -import logging -from base64 import b64decode -from dataclasses import dataclass -from typing import Any, List, Optional - -from lambda_shared import ClickHouseHelper, InsertException, get_parameter_from_ssm - -logging.getLogger().setLevel(logging.INFO) - - -@dataclass -class WorkflowJob: - id: int - run_id: int - workflow_name: str - head_branch: str - run_url: str - run_attempt: int - node_id: str - head_sha: str - url: str - html_url: str - status: str - conclusion: str - started_at: str - completed_at: str - name: str - steps: int # just number of steps, we don't keep steps - check_run_url: str - labels: List[str] - runner_id: int - runner_name: str - runner_group_id: int - runner_group_name: str - repository: str - - def anonimyze_url(self, url: str) -> str: - return url.replace(self.repository, "ANONYMIZED_REPO") - - def anonimyze(self): - anm = "ANONYMIZED" - self.workflow_name = anm - self.head_branch = anm - self.run_url = self.anonimyze_url(self.run_url) - self.node_id = anm - self.url = self.anonimyze_url(self.url) - self.html_url = self.anonimyze_url(self.html_url) - self.name = anm - self.check_run_url = self.anonimyze_url(self.check_run_url) - self.repository = anm - - def as_dict(self) -> dict: - return self.__dict__ - - -CH_CLIENT = None # type: Optional[ClickHouseHelper] - - -def send_event_workflow_job(workflow_job: WorkflowJob) -> None: - # # SHOW CREATE TABLE default.workflow_jobs - # CREATE TABLE default.workflow_jobs UUID 'c0351924-8ccd-47a6-9db0-e28a9eee2fdf' - # ( - # `id` UInt64, - # `run_id` UInt64, - # `workflow_name` LowCardinality(String), - # `head_branch` LowCardinality(String), - # `run_url` String, - # `run_attempt` UInt16, - # `node_id` String, - # `head_sha` String, - # `url` String, - # `html_url` String, - # `status` Enum8('waiting' = 1, 'queued' = 2, 'in_progress' = 3, 'completed' = 4), - # `conclusion` LowCardinality(String), - # `started_at` DateTime, - # `completed_at` DateTime, - # `name` LowCardinality(String), - # `steps` UInt16, - # `check_run_url` String, - # `labels` Array(LowCardinality(String)), - # `runner_id` UInt64, - # `runner_name` String, - # `runner_group_id` UInt64, - # `runner_group_name` LowCardinality(String), - # `repository` LowCardinality(String), - # `updated_at` DateTime DEFAULT now() - # ) - # ENGINE = ReplicatedMergeTree('/clickhouse/tables/c0351924-8ccd-47a6-9db0-e28a9eee2fdf/{shard}', '{replica}') - # PARTITION BY toStartOfMonth(started_at) - # ORDER BY (id, updated_at) - # SETTINGS index_granularity = 8192 - global CH_CLIENT - CH_CLIENT = CH_CLIENT or ClickHouseHelper( - get_parameter_from_ssm("clickhouse-test-stat-url"), - get_parameter_from_ssm("clickhouse-test-stat-login"), - get_parameter_from_ssm("clickhouse-test-stat-password"), - ) - try: - CH_CLIENT.insert_event_into( - "default", "workflow_jobs", workflow_job.as_dict(), False - ) - except InsertException as ex: - logging.exception( - "Got an exception on insert, tryuing to update the client " - "credentials and repeat", - exc_info=ex, - ) - CH_CLIENT = ClickHouseHelper( - get_parameter_from_ssm("clickhouse-test-stat-url"), - get_parameter_from_ssm("clickhouse-test-stat-login"), - get_parameter_from_ssm("clickhouse-test-stat-password"), - ) - CH_CLIENT.insert_event_into( - "default", "workflow_jobs", workflow_job.as_dict(), False - ) - - -def killed_job(wf_job: dict) -> bool: - """a hack to identify the killed runner if "Complete job" is omit""" - if ( - wf_job.get("status", "") != "completed" - or wf_job.get("conclusion", "") != "failure" - ): - # The task either success or in progress - return False - return not any( - step["name"] == "Complete job" and step["conclusion"] is not None - for step in wf_job["steps"] - ) - - -def handler(event: dict, context: Any) -> dict: - if event["isBase64Encoded"]: - event_data = json.loads(b64decode(event["body"])) - else: - event_data = json.loads(event["body"]) - - logging.info("Got the next raw event from the github hook: %s", event_data) - repo = event_data["repository"] - try: - wf_job = event_data["workflow_job"] - except KeyError: - logging.error("The event does not contain valid workflow_jobs data") - logging.error("The event data: %s", event) - logging.error("The context data: %s", context) - - if killed_job(wf_job): - # for killed job we record 0 - steps = 0 - else: - # We record only finished steps - steps = sum(1 for st in wf_job["steps"] if st["conclusion"] is not None) - - workflow_job = WorkflowJob( - wf_job["id"], - wf_job["run_id"], - wf_job["workflow_name"] or "", # nullable - wf_job["head_branch"], - wf_job["run_url"], - wf_job["run_attempt"], - wf_job["node_id"], - wf_job["head_sha"], - wf_job["url"], - wf_job["html_url"], - wf_job["status"], - wf_job["conclusion"] or "", # nullable - wf_job["started_at"], - wf_job["completed_at"] or "1970-01-01T00:00:00", # nullable date - wf_job["name"], - steps, - wf_job["check_run_url"], - wf_job["labels"], - wf_job["runner_id"] or 0, # nullable - wf_job["runner_name"] or "", # nullable - wf_job["runner_group_id"] or 0, # nullable - wf_job["runner_group_name"] or "", # nullable - repo["full_name"], - ) - logging.info( - "Got the next event (private_repo=%s): %s", repo["private"], workflow_job - ) - if repo["private"]: - workflow_job.anonimyze() - - send_event_workflow_job(workflow_job) - - return { - "statusCode": 200, - "headers": {"Content-Type": "application/json"}, - "body": '{"status": "OK"}', - } diff --git a/tests/ci/workflow_jobs_lambda/build_and_deploy_archive.sh b/tests/ci/workflow_jobs_lambda/build_and_deploy_archive.sh deleted file mode 120000 index 96ba3fa024e..00000000000 --- a/tests/ci/workflow_jobs_lambda/build_and_deploy_archive.sh +++ /dev/null @@ -1 +0,0 @@ -../team_keys_lambda/build_and_deploy_archive.sh \ No newline at end of file diff --git a/tests/ci/workflow_jobs_lambda/lambda_shared b/tests/ci/workflow_jobs_lambda/lambda_shared deleted file mode 120000 index ba86e090f6c..00000000000 --- a/tests/ci/workflow_jobs_lambda/lambda_shared +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package/lambda_shared \ No newline at end of file diff --git a/tests/ci/workflow_jobs_lambda/requirements.txt b/tests/ci/workflow_jobs_lambda/requirements.txt deleted file mode 100644 index 098e04a9798..00000000000 --- a/tests/ci/workflow_jobs_lambda/requirements.txt +++ /dev/null @@ -1 +0,0 @@ -../lambda_shared_package From 626e1f70cb6f46407a11771cce67c879a9f1a0e5 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Sun, 14 Jul 2024 19:33:19 +0200 Subject: [PATCH 51/79] Stateful tests: decrease tests timeout --- docker/test/stateful/run.sh | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 35ffeee5438..857385f4715 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -4,6 +4,9 @@ source /setup_export_logs.sh set -e -x +MAX_RUN_TIME=${MAX_RUN_TIME:-3600} +MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 3600 : MAX_RUN_TIME)) + # Choose random timezone for this test run TZ="$(rg -v '#' /usr/share/zoneinfo/zone.tab | awk '{print $3}' | shuf | head -n1)" echo "Choosen random timezone $TZ" @@ -242,7 +245,22 @@ function run_tests() } export -f run_tests -timeout "$MAX_RUN_TIME" bash -c run_tests ||: + +function timeout_with_logging() { + local exit_code=0 + + timeout -s TERM --preserve-status "${@}" || exit_code="${?}" + + if [[ "${exit_code}" -eq "124" ]] + then + echo "The command 'timeout ${*}' has been killed by timeout" + fi + + return $exit_code +} + +TIMEOUT=$((MAX_RUN_TIME - 700)) +timeout_with_logging "$TIMEOUT" bash -c run_tests ||: echo "Files in current directory" ls -la ./ From 17fc423062fea64c392d48028eb32957e185269a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Mon, 15 Jul 2024 11:58:56 +0200 Subject: [PATCH 52/79] Stateless tests: add `no-parallel` tag to high-load tests --- .../00840_long_concurrent_select_and_drop_deadlock.sh | 2 +- .../queries/0_stateless/00965_shard_unresolvable_addresses.sql | 2 +- tests/queries/0_stateless/01004_rename_deadlock.sh | 2 +- tests/queries/0_stateless/01005_rwr_shard_deadlock.sh | 2 +- tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh | 2 +- ...azy_database_concurrent_recreate_reattach_and_show_tables.sh | 2 +- .../01035_concurrent_move_partition_from_table_zookeeper.sh | 2 +- .../0_stateless/01079_parallel_alter_modify_zookeeper_long.sh | 2 +- .../0_stateless/01111_create_drop_replicated_db_stress.sh | 2 +- tests/queries/0_stateless/01412_cache_dictionary_race.sh | 2 +- .../0_stateless/01454_storagememory_data_race_challenge.sh | 2 +- tests/queries/0_stateless/01475_read_subcolumns.sql | 2 +- .../queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh | 2 +- tests/queries/0_stateless/01533_multiple_nested.sql | 2 +- tests/queries/0_stateless/01778_mmap_cache_infra.sql | 1 + .../01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh | 2 +- tests/queries/0_stateless/02417_load_marks_async.sh | 1 + .../0_stateless/03148_async_queries_in_query_log_errors.sh | 1 + .../0_stateless/03164_s3_settings_for_queries_and_merges.sql | 2 +- tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql | 2 +- 20 files changed, 20 insertions(+), 17 deletions(-) diff --git a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh index ae728c8d10d..f0bc52ee356 100755 --- a/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh +++ b/tests/queries/0_stateless/00840_long_concurrent_select_and_drop_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock, no-debug +# Tags: deadlock, no-debug, no-parallel # NOTE: database = $CLICKHOUSE_DATABASE is unwanted diff --git a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql index 79600c6f67e..16b62c37d80 100644 --- a/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql +++ b/tests/queries/0_stateless/00965_shard_unresolvable_addresses.sql @@ -1,4 +1,4 @@ --- Tags: shard +-- Tags: shard, no-parallel SET prefer_localhost_replica = 1; diff --git a/tests/queries/0_stateless/01004_rename_deadlock.sh b/tests/queries/0_stateless/01004_rename_deadlock.sh index f0adf136e94..60d67e1744a 100755 --- a/tests/queries/0_stateless/01004_rename_deadlock.sh +++ b/tests/queries/0_stateless/01004_rename_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock +# Tags: deadlock, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01005_rwr_shard_deadlock.sh b/tests/queries/0_stateless/01005_rwr_shard_deadlock.sh index ef352606b69..51e8cdbc8e8 100755 --- a/tests/queries/0_stateless/01005_rwr_shard_deadlock.sh +++ b/tests/queries/0_stateless/01005_rwr_shard_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock, shard +# Tags: deadlock, shard, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh b/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh index 9f4b2241732..d602ff6df3a 100755 --- a/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh +++ b/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock +# Tags: deadlock, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh b/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh index 3046fcbcd73..e4b3a31b13f 100755 --- a/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh +++ b/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.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 diff --git a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh index 0d57bb25543..59a1e3cb980 100755 --- a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-fasttest +# Tags: zookeeper, no-fasttest, long, no-parallel set -e diff --git a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh index 399c9e488a4..382e3279711 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper, no-fasttest +# Tags: long, zookeeper, no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh index 90128d7a8ad..619b6e91d11 100755 --- a/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh +++ b/tests/queries/0_stateless/01111_create_drop_replicated_db_stress.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, zookeeper +# Tags: race, zookeeper, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01412_cache_dictionary_race.sh b/tests/queries/0_stateless/01412_cache_dictionary_race.sh index 9aa39652021..36295ca01ea 100755 --- a/tests/queries/0_stateless/01412_cache_dictionary_race.sh +++ b/tests/queries/0_stateless/01412_cache_dictionary_race.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race +# Tags: race, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh b/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh index 2bfd350ec51..fa9238041b1 100755 --- a/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh +++ b/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race +# Tags: race, no-parallel set -e diff --git a/tests/queries/0_stateless/01475_read_subcolumns.sql b/tests/queries/0_stateless/01475_read_subcolumns.sql index d6eec2f84a1..d387230d9e5 100644 --- a/tests/queries/0_stateless/01475_read_subcolumns.sql +++ b/tests/queries/0_stateless/01475_read_subcolumns.sql @@ -1,4 +1,4 @@ --- Tags: no-object-storage, no-random-settings +-- Tags: no-object-storage, no-random-settings, no-parallel SET use_uncompressed_cache = 0; diff --git a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh index b8efee89b4a..5a146968a3b 100755 --- a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh +++ b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock +# Tags: deadlock, no-parallel set -e diff --git a/tests/queries/0_stateless/01533_multiple_nested.sql b/tests/queries/0_stateless/01533_multiple_nested.sql index 80e9fc7e2fb..4756bd4e919 100644 --- a/tests/queries/0_stateless/01533_multiple_nested.sql +++ b/tests/queries/0_stateless/01533_multiple_nested.sql @@ -1,4 +1,4 @@ --- Tags: no-object-storage, no-random-merge-tree-settings +-- Tags: no-object-storage, no-random-merge-tree-settings, no-parallel -- no-s3 because read FileOpen metric DROP TABLE IF EXISTS nested; diff --git a/tests/queries/0_stateless/01778_mmap_cache_infra.sql b/tests/queries/0_stateless/01778_mmap_cache_infra.sql index 29a84c5507b..50fdb6ffbba 100644 --- a/tests/queries/0_stateless/01778_mmap_cache_infra.sql +++ b/tests/queries/0_stateless/01778_mmap_cache_infra.sql @@ -1,3 +1,4 @@ +-- Tags: no-parallel -- We check the existence of queries and metrics and don't check the results (a smoke test). SYSTEM DROP MMAP CACHE; diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh index edffc0a3807..0dfde4997a7 100755 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, zookeeper +# Tags: long, zookeeper, no-parallel CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=error diff --git a/tests/queries/0_stateless/02417_load_marks_async.sh b/tests/queries/0_stateless/02417_load_marks_async.sh index 72b35a565df..950656e7ab6 100755 --- a/tests/queries/0_stateless/02417_load_marks_async.sh +++ b/tests/queries/0_stateless/02417_load_marks_async.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.sh b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.sh index 9c290133bf9..2b4b96a9cbf 100755 --- a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.sh +++ b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql index 652b27b8a67..ac2070fbd76 100644 --- a/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql +++ b/tests/queries/0_stateless/03164_s3_settings_for_queries_and_merges.sql @@ -1,4 +1,4 @@ --- Tags: no-random-settings, no-fasttest +-- Tags: no-random-settings, no-fasttest, no-parallel SET allow_prefetched_read_pool_for_remote_filesystem=0; SET allow_prefetched_read_pool_for_local_filesystem=0; diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql index 5d473064c68..bbe701f022b 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -1,4 +1,4 @@ --- Tags: no-random-settings, no-object-storage +-- Tags: no-random-settings, no-object-storage, no-parallel -- Tag no-object-storage: this test relies on the number of opened files in MergeTree that can differ in object storages SET allow_experimental_dynamic_type = 1; From bd10951db8d6f9abc0d1e74aa122adf7ed7f0843 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 16 Jul 2024 10:32:10 +0200 Subject: [PATCH 53/79] Stateless tests: tune MAX_RUN_TIME --- docker/test/stateless/run.sh | 6 ------ tests/ci/functional_test_check.py | 4 ++-- 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 2b535f8dd23..54bc2f184b1 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -183,9 +183,6 @@ if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then --keeper_server.tcp_port 29181 --keeper_server.server_id 3 \ --prometheus.port 29988 \ --macros.shard s2 # It doesn't work :( - - MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) - MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) fi if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then @@ -210,9 +207,6 @@ if [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then --keeper_server.tcp_port 19181 --keeper_server.server_id 2 \ --prometheus.port 19988 \ --macros.replica r2 # It doesn't work :( - - MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) - MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) fi # Wait for the server to start, but not for too long. diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 74dd4d8fbd7..66db082677f 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -112,8 +112,8 @@ def get_run_command( ] if flaky_check: - envs.append("-e NUM_TRIES=100") - envs.append("-e MAX_RUN_TIME=1800") + envs.append("-e NUM_TRIES=50") + envs.append("-e MAX_RUN_TIME=2800") envs += [f"-e {e}" for e in additional_envs] From 466c9f55c143b22f9fd9697420be1079b0adee8c Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Tue, 16 Jul 2024 11:37:16 +0200 Subject: [PATCH 54/79] Stateless tests: fix test `01053_ssd_dictionary.sh` --- tests/queries/0_stateless/01053_ssd_dictionary.sh | 4 ++-- tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01053_ssd_dictionary.sh b/tests/queries/0_stateless/01053_ssd_dictionary.sh index 00e5719a9a9..fdd40940ab5 100755 --- a/tests/queries/0_stateless/01053_ssd_dictionary.sh +++ b/tests/queries/0_stateless/01053_ssd_dictionary.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash # Tags: no-parallel, no-fasttest -# Tag no-fasttest: this test mistakenly requires acces to /var/lib/clickhouse -- can't run this locally, disabled +# Tag no-fasttest: this test mistakenly requires access to /var/lib/clickhouse -- can't run this locally, disabled CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -n --query=" DROP DATABASE IF EXISTS 01053_db; - CREATE DATABASE 01053_db Engine = Ordinary; + CREATE DATABASE 01053_db; DROP TABLE IF EXISTS 01053_db.table_for_dict; diff --git a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh index 9a80820dd58..60a65b9a253 100755 --- a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh +++ b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sh @@ -6,9 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -n --query=" - set allow_deprecated_database_ordinary=1; DROP DATABASE IF EXISTS 01280_db; - CREATE DATABASE 01280_db Engine = Ordinary; + CREATE DATABASE 01280_db; DROP TABLE IF EXISTS 01280_db.table_for_dict; CREATE TABLE 01280_db.table_for_dict ( From 7c1ed1b445822fa91d6554497fd4b6d0f76ad8cb Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Thu, 18 Jul 2024 19:48:12 +0200 Subject: [PATCH 55/79] Stateless tests: check parallelization --- docker/test/stateless/run.sh | 79 +----------------------------------- tests/clickhouse-test | 74 ++++----------------------------- 2 files changed, 10 insertions(+), 143 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 54bc2f184b1..0647ed02839 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -12,12 +12,6 @@ MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 7200 : MAX_RUN_TIME)) USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0} USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0} -RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0 - -if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then - RUN_SEQUENTIAL_TESTS_IN_PARALLEL=0 -fi - # Choose random timezone for this test run. # # NOTE: that clickhouse-test will randomize session_timezone by itself as well @@ -101,53 +95,6 @@ if [ "$NUM_TRIES" -gt "1" ]; then mkdir -p /var/run/clickhouse-server fi -# Run a CH instance to execute sequential tests on it in parallel with all other tests. -if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then - mkdir -p /var/run/clickhouse-server3 /etc/clickhouse-server3 /var/lib/clickhouse3 - cp -r -L /etc/clickhouse-server/* /etc/clickhouse-server3/ - - sudo chown clickhouse:clickhouse /var/run/clickhouse-server3 /var/lib/clickhouse3 /etc/clickhouse-server3/ - sudo chown -R clickhouse:clickhouse /etc/clickhouse-server3/* - - function replace(){ - sudo find /etc/clickhouse-server3/ -type f -name '*.xml' -exec sed -i "$1" {} \; - } - - replace "s|9000|19000|g" - replace "s|9440|19440|g" - replace "s|9988|19988|g" - replace "s|9234|19234|g" - replace "s|9181|19181|g" - replace "s|8443|18443|g" - replace "s|9000|19000|g" - replace "s|9181|19181|g" - replace "s|9440|19440|g" - replace "s|9010|19010|g" - replace "s|9004|19004|g" - replace "s|9005|19005|g" - replace "s|9009|19009|g" - replace "s|8123|18123|g" - replace "s|/var/lib/clickhouse/|/var/lib/clickhouse3/|g" - replace "s|/etc/clickhouse-server/|/etc/clickhouse-server3/|g" - # distributed cache - replace "s|10001|10003|g" - replace "s|10002|10004|g" - - sudo -E -u clickhouse /usr/bin/clickhouse server --daemon --config /etc/clickhouse-server3/config.xml \ - --pid-file /var/run/clickhouse-server3/clickhouse-server.pid \ - -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ - --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ - --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ - --prometheus.port 19988 --keeper_server.raft_configuration.server.port 19234 --keeper_server.tcp_port 19181 \ - --mysql_port 19004 --postgresql_port 19005 - - for _ in {1..100} - do - clickhouse-client --port 19000 --query "SELECT 1" && break - sleep 1 - done -fi - # simplest way to forward env variables to server sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server/config.xml --daemon --pid-file /var/run/clickhouse-server/clickhouse-server.pid @@ -217,7 +164,6 @@ do done setup_logs_replication - attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 function fn_exists() { @@ -278,11 +224,7 @@ function run_tests() else # All other configurations are OK. ADDITIONAL_OPTIONS+=('--jobs') - ADDITIONAL_OPTIONS+=('5') - fi - - if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then - ADDITIONAL_OPTIONS+=('--run-sequential-tests-in-parallel') + ADDITIONAL_OPTIONS+=('7') fi if [[ -n "$RUN_BY_HASH_NUM" ]] && [[ -n "$RUN_BY_HASH_TOTAL" ]]; then @@ -367,9 +309,6 @@ done # Because it's the simplest way to read it when server has crashed. sudo clickhouse stop ||: -if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then - sudo clickhouse stop --pid-path /var/run/clickhouse-server3 ||: -fi if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||: @@ -387,12 +326,6 @@ rg -Fa "" /var/log/clickhouse-server/clickhouse-server.log ||: rg -A50 -Fa "============" /var/log/clickhouse-server/stderr.log ||: zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst & -if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then - rg -Fa "" /var/log/clickhouse-server3/clickhouse-server.log ||: - rg -A50 -Fa "============" /var/log/clickhouse-server3/stderr.log ||: - zstd --threads=0 < /var/log/clickhouse-server3/clickhouse-server.log > /test_output/clickhouse-server3.log.zst & -fi - data_path_config="--path=/var/lib/clickhouse/" if [[ -n "$USE_S3_STORAGE_FOR_MERGE_TREE" ]] && [[ "$USE_S3_STORAGE_FOR_MERGE_TREE" -eq 1 ]]; then # We need s3 storage configuration (but it's more likely that clickhouse-local will fail for some reason) @@ -413,10 +346,6 @@ if [ $failed_to_save_logs -ne 0 ]; then do clickhouse-local "$data_path_config" --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||: - if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then - clickhouse-local --path /var/lib/clickhouse3/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.3.tsv.zst ||: - fi - if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||: clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||: @@ -458,12 +387,6 @@ rm -rf /var/lib/clickhouse/data/system/*/ tar -chf /test_output/store.tar /var/lib/clickhouse/store ||: tar -chf /test_output/metadata.tar /var/lib/clickhouse/metadata/*.sql ||: -if [[ "$RUN_SEQUENTIAL_TESTS_IN_PARALLEL" -eq 1 ]]; then - rm -rf /var/lib/clickhouse3/data/system/*/ - tar -chf /test_output/store.tar /var/lib/clickhouse3/store ||: - tar -chf /test_output/metadata.tar /var/lib/clickhouse3/metadata/*.sql ||: -fi - if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then rg -Fa "" /var/log/clickhouse-server/clickhouse-server1.log ||: diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 60160e71e81..a8a6ce68b8f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2187,7 +2187,7 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool description = "" test_case_name = removesuffix(test_case.name, ".gen", ".sql") + ": " - if is_concurrent or args.run_sequential_tests_in_parallel: + if is_concurrent: description = f"{test_case_name:72}" else: sys.stdout.flush() @@ -2447,35 +2447,6 @@ def extract_key(key: str) -> str: )[1] -def override_envs(*args_, **kwargs): - global args - args.client += " --port 19000" - args.http_port = 18123 - args.https_port = 18443 - - updated_env = { - "CLICKHOUSE_CONFIG": "/etc/clickhouse-server3/config.xml", - "CLICKHOUSE_CONFIG_DIR": "/etc/clickhouse-server3", - "CLICKHOUSE_CONFIG_GREP": "/etc/clickhouse-server3/preprocessed/config.xml", - "CLICKHOUSE_USER_FILES": "/var/lib/clickhouse3/user_files", - "CLICKHOUSE_SCHEMA_FILES": "/var/lib/clickhouse3/format_schemas", - "CLICKHOUSE_PATH": "/var/lib/clickhouse3", - "CLICKHOUSE_PORT_TCP": "19000", - "CLICKHOUSE_PORT_TCP_SECURE": "19440", - "CLICKHOUSE_PORT_TCP_WITH_PROXY": "19010", - "CLICKHOUSE_PORT_HTTP": "18123", - "CLICKHOUSE_PORT_HTTPS": "18443", - "CLICKHOUSE_PORT_INTERSERVER": "19009", - "CLICKHOUSE_PORT_KEEPER": "19181", - "CLICKHOUSE_PORT_PROMTHEUS_PORT": "19988", - "CLICKHOUSE_PORT_MYSQL": "19004", - "CLICKHOUSE_PORT_POSTGRESQL": "19005", - } - os.environ.update(updated_env) - - run_tests_array(*args_, **kwargs) - - def run_tests_process(*args, **kwargs): return run_tests_array(*args, **kwargs) @@ -2519,24 +2490,6 @@ def do_run_tests(jobs, test_suite: TestSuite): processes.append(process) process.start() - if args.run_sequential_tests_in_parallel: - # Run parallel tests and sequential tests at the same time - # Sequential tests will use different ClickHouse instance - # In this process we can safely override values in `args` and `os.environ` - process = multiprocessing.Process( - target=override_envs, - args=( - ( - test_suite.sequential_tests, - len(test_suite.sequential_tests), - test_suite, - False, - ), - ), - ) - processes.append(process) - process.start() - while processes: sys.stdout.flush() # Periodically check the server for hangs @@ -2568,15 +2521,15 @@ def do_run_tests(jobs, test_suite: TestSuite): sleep(5) - if not args.run_sequential_tests_in_parallel: - run_tests_array( - ( - test_suite.sequential_tests, - len(test_suite.sequential_tests), - test_suite, - False, - ) + run_tests_array( + ( + test_suite.sequential_tests, + len(test_suite.sequential_tests), + test_suite, + False, ) + ) + return len(test_suite.sequential_tests) + len(test_suite.parallel_tests) else: num_tests = len(test_suite.all_tests) @@ -3419,15 +3372,6 @@ def parse_args(): help="Path to file for fatal logs from client", ) - parser.add_argument( - "--run-sequential-tests-in-parallel", - action="store_true", - default=False, - help="If `true`, tests with the tag `no-parallel` will run on a " - "separate ClickHouse instance in parallel with other tests. " - "This is used in CI to make test jobs run faster.", - ) - return parser.parse_args() From c021f9dcd363d01d46108a9084b90c9463c9116a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jul 2024 22:56:15 +0200 Subject: [PATCH 56/79] Fix SHOW MERGES --- src/Interpreters/InterpreterShowTablesQuery.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 51038aaca46..78580f2ec02 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -121,9 +121,18 @@ String InterpreterShowTablesQuery::getRewrittenQuery() if (query.merges) { WriteBufferFromOwnString rewritten_query; - rewritten_query << "SELECT table, database, round((elapsed * (1 / merges.progress)) - merges.elapsed, 2) AS estimate_complete, round(elapsed,2) elapsed, " - "round(progress*100, 2) AS progress, is_mutation, formatReadableSize(total_size_bytes_compressed) AS size_compressed, " - "formatReadableSize(memory_usage) AS memory_usage FROM system.merges"; + rewritten_query << R"( + SELECT + table, + database, + merges.progress > 0 ? round(merges.elapsed * (1 - merges.progress) / merges.progress, 2) : NULL AS estimate_complete, + round(elapsed, 2) AS elapsed, + round(progress * 100, 2) AS progress, + is_mutation, + formatReadableSize(total_size_bytes_compressed) AS size_compressed, + formatReadableSize(memory_usage) AS memory_usage + FROM system.merges + )"; if (!query.like.empty()) { From 09141163b4444fca79f0ab05f3c012bd69ecf1c3 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 17 Jul 2024 18:19:54 +0200 Subject: [PATCH 57/79] remove lambda unit tests --- .github/workflows/backport_branches.yml | 4 ---- .github/workflows/master.yml | 4 ---- .github/workflows/merge_queue.yml | 4 ---- .github/workflows/pull_request.yml | 4 ---- .github/workflows/release_branches.yml | 4 ---- tests/ci/.gitignore | 3 --- tests/ci/create_release.py | 5 ++--- 7 files changed, 2 insertions(+), 26 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 60bd79560eb..9645d0e46de 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -36,10 +36,6 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" echo "Testing the main ci directory" python3 -m unittest discover -s . -p 'test_*.py' - for dir in *_lambda/; do - echo "Testing $dir" - python3 -m unittest discover -s "$dir" -p 'test_*.py' - done - name: PrepareRunConfig id: runconfig run: | diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index d27b1987532..09acef5eb8b 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -33,10 +33,6 @@ jobs: # cd "$GITHUB_WORKSPACE/tests/ci" # echo "Testing the main ci directory" # python3 -m unittest discover -s . -p 'test_*.py' -# for dir in *_lambda/; do -# echo "Testing $dir" -# python3 -m unittest discover -s "$dir" -p 'test_*.py' -# done - name: PrepareRunConfig id: runconfig run: | diff --git a/.github/workflows/merge_queue.yml b/.github/workflows/merge_queue.yml index c08c3fb30ac..31a65ac3d15 100644 --- a/.github/workflows/merge_queue.yml +++ b/.github/workflows/merge_queue.yml @@ -30,10 +30,6 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" echo "Testing the main ci directory" python3 -m unittest discover -s . -p 'test_*.py' - for dir in *_lambda/; do - echo "Testing $dir" - python3 -m unittest discover -s "$dir" -p 'test_*.py' - done - name: PrepareRunConfig id: runconfig run: | diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 04ce4d29ce9..5124e4dba2c 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -48,10 +48,6 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" echo "Testing the main ci directory" python3 -m unittest discover -s . -p 'test_*.py' - for dir in *_lambda/; do - echo "Testing $dir" - python3 -m unittest discover -s "$dir" -p 'test_*.py' - done - name: PrepareRunConfig id: runconfig run: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index e1a97b91016..6a18999d74e 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -33,10 +33,6 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" echo "Testing the main ci directory" python3 -m unittest discover -s . -p 'test_*.py' - for dir in *_lambda/; do - echo "Testing $dir" - python3 -m unittest discover -s "$dir" -p 'test_*.py' - done - name: PrepareRunConfig id: runconfig run: | diff --git a/tests/ci/.gitignore b/tests/ci/.gitignore index bc407a1f961..b5e58611040 100644 --- a/tests/ci/.gitignore +++ b/tests/ci/.gitignore @@ -1,4 +1 @@ -*_lambda/lambda-venv -*_lambda/lambda-package -*_lambda/lambda-package.zip gh_cache \ No newline at end of file diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index e5723e00e2a..277134c3991 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -13,7 +13,6 @@ from git_helper import Git, GIT_PREFIX from ssh import SSHAgent from env_helper import GITHUB_REPOSITORY, S3_BUILDS_BUCKET from s3_helper import S3Helper -from autoscale_runners_lambda.lambda_shared.pr import Labels from ci_utils import Shell from version_helper import ( FILE_WITH_VERSION_PATH, @@ -220,9 +219,9 @@ class ReleaseInfo: ) with checkout(self.release_branch): with checkout_new(new_release_branch): - pr_labels = f"--label {Labels.RELEASE}" + pr_labels = f"--label {CI.Labels.RELEASE}" if stable_release_type == VersionType.LTS: - pr_labels += f" --label {Labels.RELEASE_LTS}" + pr_labels += f" --label {CI.Labels.RELEASE_LTS}" cmd_push_branch = ( f"{GIT_PREFIX} push --set-upstream origin {new_release_branch}" ) From b29518537061bda2d73a0a98e7b2d1fcdda3cedf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jul 2024 00:11:29 +0200 Subject: [PATCH 58/79] Fix bad test 03036_join_filter_push_down_equivalent_sets --- .../03036_join_filter_push_down_equivalent_sets.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql index 3893ec41c0e..d94a68aa4d8 100644 --- a/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql +++ b/tests/queries/0_stateless/03036_join_filter_push_down_equivalent_sets.sql @@ -7,13 +7,13 @@ CREATE TABLE test_table_1 ( id UInt64, value String -) ENGINE=MergeTree ORDER BY id; +) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; CREATE TABLE test_table_2 ( id UInt64, value String -) ENGINE=MergeTree ORDER BY id; +) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; INSERT INTO test_table_1 SELECT number, number FROM numbers(10); INSERT INTO test_table_2 SELECT number, number FROM numbers(10); From f6380dac0cbf08f913069606beeea433fb7f8857 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jul 2024 04:03:28 +0200 Subject: [PATCH 59/79] Fix tidy. --- src/Databases/IDatabase.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index ce5a52b1b0f..f94326d220e 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include From b2adabd59eedad6805b7f8a250cb4a787d0e0998 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 19 Jul 2024 10:27:15 +0200 Subject: [PATCH 60/79] fix pylint --- utils/check-style/check-pylint | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/utils/check-style/check-pylint b/utils/check-style/check-pylint index 8cfbc68ac96..919a1661e45 100755 --- a/utils/check-style/check-pylint +++ b/utils/check-style/check-pylint @@ -12,5 +12,4 @@ function xargs-pylint { # exclude ci unittest scripts from check: test_* find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | grep -v "/test_" | xargs-pylint 50 -# Beware, there lambdas are checked. All of them contain `app`, and it causes brain-cucumber-zalgo -find "$ROOT_PATH/tests/ci" -mindepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 1 + From 5cca0f1ab712e9814ce9e68844e98c241ce2fba1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 19 Jul 2024 09:30:26 +0000 Subject: [PATCH 61/79] Add test test_storage_mysql/test.py::test_joins, close #65989 --- tests/integration/test_storage_mysql/test.py | 57 ++++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index b131d8fe51c..5948954ff5f 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -908,6 +908,63 @@ def test_mysql_point(started_cluster): conn.close() +def test_joins(started_cluster): + conn = get_mysql_conn(started_cluster, cluster.mysql8_ip) + drop_mysql_table(conn, "test_joins_mysql_users") + with conn.cursor() as cursor: + cursor.execute( + "CREATE TABLE clickhouse.test_joins_mysql_users (id INT NOT NULL, name varchar(50) NOT NULL, created TIMESTAMP, PRIMARY KEY (`id`)) ENGINE=InnoDB;" + ) + cursor.execute( + f"INSERT INTO clickhouse.test_joins_mysql_users VALUES (469722, 'user@example.com', '2019-08-30 07:55:01')" + ) + + drop_mysql_table(conn, "test_joins_mysql_tickets") + with conn.cursor() as cursor: + cursor.execute( + "CREATE TABLE clickhouse.test_joins_mysql_tickets (id INT NOT NULL, subject varchar(50), created TIMESTAMP, creator INT NOT NULL, PRIMARY KEY (`id`)) ENGINE=InnoDB;" + ) + cursor.execute( + f"INSERT INTO clickhouse.test_joins_mysql_tickets VALUES (281607, 'Feedback', '2024-06-25 12:09:41', 469722)" + ) + + conn.commit() + + node1.query( + """ + CREATE TABLE test_joins_table_users + ( + `id` Int32, + `Name` String, + `Created` Nullable(DateTime) + ) + ENGINE = MySQL('mysql80:3306', 'clickhouse', 'test_joins_mysql_users', 'root', 'clickhouse'); + """ + ) + + node1.query( + """ + CREATE TABLE test_joins_table_tickets + ( + `id` Int32, + `Subject` Nullable(String), + `Created` Nullable(DateTime), + `Creator` Int32 + ) + ENGINE = MySQL('mysql80:3306', 'clickhouse', 'test_joins_mysql_tickets', 'root', 'clickhouse'); + """ + ) + + node1.query( + """ + SELECT test_joins_table_tickets.id, Subject, test_joins_table_tickets.Created, Name + FROM test_joins_table_tickets + LEFT JOIN test_joins_table_users ON test_joins_table_tickets.Creator = test_joins_table_users.id + WHERE test_joins_table_tickets.Created = '2024-06-25 12:09:41' + """ + ) == "281607\tFeedback\t2024-06-25 12:09:41\tuser@example.com\n" + + if __name__ == "__main__": with contextmanager(started_cluster)() as cluster: for name, instance in list(cluster.instances.items()): From 983d2b474b3d3adba2d7c7f8824f0e45eb8ea68e Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 19 Jul 2024 12:44:31 +0200 Subject: [PATCH 62/79] CI: Next fix for builds in CI workflow --- tests/ci/ci_config.py | 18 ++++-------------- tests/ci/ci_definitions.py | 2 -- tests/ci/ci_settings.py | 10 ++++------ tests/ci/test_ci_options.py | 6 +++--- 4 files changed, 11 insertions(+), 25 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 98c7e99a495..90797b36d8e 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -48,24 +48,14 @@ class CI: JobNames.INTEGRATION_TEST_ARM, ] ), - Tags.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), + Tags.CI_SET_REQUIRED: LabelConfig( + run_jobs=REQUIRED_CHECKS + + [build for build in BuildNames if build != BuildNames.FUZZERS] + ), Tags.CI_SET_BUILDS: LabelConfig( run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK] + [build for build in BuildNames if build != BuildNames.FUZZERS] ), - Tags.CI_SET_NON_REQUIRED: LabelConfig( - run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS] - ), - Tags.CI_SET_OLD_ANALYZER: LabelConfig( - run_jobs=[ - JobNames.STYLE_CHECK, - JobNames.FAST_TEST, - BuildNames.PACKAGE_RELEASE, - BuildNames.PACKAGE_ASAN, - JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE, - JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER, - ] - ), Tags.CI_SET_SYNC: LabelConfig( run_jobs=[ BuildNames.PACKAGE_ASAN, diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index d41a621bc2e..20d29d2337e 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -102,8 +102,6 @@ class Tags(metaclass=WithIter): CI_SET_ARM = "ci_set_arm" CI_SET_REQUIRED = "ci_set_required" CI_SET_BUILDS = "ci_set_builds" - CI_SET_NON_REQUIRED = "ci_set_non_required" - CI_SET_OLD_ANALYZER = "ci_set_old_analyzer" libFuzzer = "libFuzzer" diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index 54323ef868f..c29c5777dba 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -160,11 +160,8 @@ class CiSettings: else: return False - if CI.is_build_job(job): - print(f"Build job [{job}] - always run") - return True - - if self.exclude_keywords: + # do not exclude builds + if self.exclude_keywords and not CI.is_build_job(job): for keyword in self.exclude_keywords: if keyword in normalize_string(job): print(f"Job [{job}] matches Exclude keyword [{keyword}] - deny") @@ -172,7 +169,8 @@ class CiSettings: to_deny = False if self.include_keywords: - if job == CI.JobNames.STYLE_CHECK: + # do not exclude builds + if job == CI.JobNames.STYLE_CHECK or CI.is_build_job(job): # never exclude Style Check by include keywords return True for keyword in self.include_keywords: diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index f71320abf2c..534885dda2b 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -8,8 +8,8 @@ from ci_config import CI _TEST_BODY_1 = """ #### Run only: -- [x] Non required -- [ ] Integration tests (arm64) +- [ ] Some Set +- [x] Integration tests (arm64) - [x] Integration tests - [x] Integration tests - [ ] Integration tests @@ -149,7 +149,7 @@ class TestCIOptions(unittest.TestCase): self.assertFalse(ci_options.no_ci_cache) self.assertTrue(ci_options.no_merge_commit) self.assertTrue(ci_options.woolen_wolfdog) - self.assertEqual(ci_options.ci_sets, ["ci_set_non_required"]) + self.assertEqual(ci_options.ci_sets, ["ci_set_arm"]) self.assertCountEqual(ci_options.include_keywords, ["foo", "foo_bar"]) self.assertCountEqual(ci_options.exclude_keywords, ["foo", "foo_bar"]) From 74033e6fbcd63c741447cc5501dd27292e2d79f7 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 19 Jul 2024 13:14:19 +0200 Subject: [PATCH 63/79] fix var names --- .../03008_deduplication_random_setttings.sh | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/03008_deduplication_random_setttings.sh b/tests/queries/0_stateless/03008_deduplication_random_setttings.sh index 20513c35c71..e9f59138177 100755 --- a/tests/queries/0_stateless/03008_deduplication_random_setttings.sh +++ b/tests/queries/0_stateless/03008_deduplication_random_setttings.sh @@ -4,8 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -ENGINE="ReplicatedMergeTree" - engine_options=("ReplicatedMergeTree" "MergeTree") engine=${engine_options[ $RANDOM % ${#engine_options[@]} ]} @@ -30,7 +28,7 @@ insert_unique_blocks=${insert_unique_blocks_options[ $RANDOM % ${#insert_unique_ THIS_RUN="Test case:" THIS_RUN+=" insert_method=$insert_method" -THIS_RUN+=" engine=$ENGINE" +THIS_RUN+=" engine=$engine" THIS_RUN+=" use_insert_token=$use_insert_token" THIS_RUN+=" single_thread=$single_thread" THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" @@ -40,7 +38,7 @@ THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ --insert-method $insert_method \ - --table-engine $ENGINE \ + --table-engine $engine \ --use-insert-token $use_insert_token \ --single-thread $single_thread \ --deduplicate-src-table $deduplicate_src_table \ @@ -53,7 +51,7 @@ $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ --insert-method $insert_method \ - --table-engine $ENGINE \ + --table-engine $engine \ --use-insert-token $use_insert_token \ --single-thread $single_thread \ --deduplicate-src-table $deduplicate_src_table \ @@ -66,7 +64,7 @@ $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ --insert-method $insert_method \ - --table-engine $ENGINE \ + --table-engine $engine \ --use-insert-token $use_insert_token \ --single-thread $single_thread \ --deduplicate-src-table $deduplicate_src_table \ From 09cf5b1188d8b7a7485647ad62c936a7b28d41c7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 17 Jul 2024 09:31:31 +0800 Subject: [PATCH 64/79] column_length is not updated in ColumnTuple::insertManyFrom --- src/Columns/ColumnTuple.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 9b822d7f570..9310d92e2ea 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -201,6 +201,7 @@ bool ColumnTuple::tryInsert(const Field & x) return false; } } + ++column_length; return true; } @@ -236,6 +237,7 @@ void ColumnTuple::doInsertManyFrom(const IColumn & src, size_t position, size_t for (size_t i = 0; i < tuple_size; ++i) columns[i]->insertManyFrom(*src_tuple.columns[i], position, length); + column_length += length; } void ColumnTuple::insertDefault() From 9164c989368adcdc4b6fe9f138706beef57706f1 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 17 Jul 2024 09:51:24 +0800 Subject: [PATCH 65/79] add test and merge #66623 --- src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp | 4 +--- tests/queries/0_stateless/03205_column_type_check.reference | 3 +++ tests/queries/0_stateless/03205_column_type_check.sql | 2 ++ 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp index ff40b1fcc2d..a7f56999d73 100644 --- a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp @@ -69,9 +69,7 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre bool allow_where = true; if (const auto * join_node = join_tree->as()) { - if (join_node->getStrictness() != JoinStrictness::All) - allow_where = false; - else if (join_node->getKind() == JoinKind::Left) + if (join_node->getKind() == JoinKind::Left) allow_where = join_node->getLeftTableExpression()->isEqual(*table_expression); else if (join_node->getKind() == JoinKind::Right) allow_where = join_node->getRightTableExpression()->isEqual(*table_expression); diff --git a/tests/queries/0_stateless/03205_column_type_check.reference b/tests/queries/0_stateless/03205_column_type_check.reference index 3b6c93a0610..466cabedd2e 100644 --- a/tests/queries/0_stateless/03205_column_type_check.reference +++ b/tests/queries/0_stateless/03205_column_type_check.reference @@ -1,2 +1,5 @@ 1 nan 1048575 2 1 1 1 1 1 +() 1 nan 1048575 2 + +() 1 nan 1048575 2 diff --git a/tests/queries/0_stateless/03205_column_type_check.sql b/tests/queries/0_stateless/03205_column_type_check.sql index ab122821eb0..332061d6be8 100644 --- a/tests/queries/0_stateless/03205_column_type_check.sql +++ b/tests/queries/0_stateless/03205_column_type_check.sql @@ -5,3 +5,5 @@ SET join_algorithm='hash'; SET allow_experimental_join_condition=1; SELECT * FROM ( SELECT 1 AS a, toLowCardinality(1), 1) AS t1 CROSS JOIN (SELECT toLowCardinality(1 AS a), 1 AS b) AS t2; + +SELECT * FROM (SELECT tuple(), 1 GROUP BY greatCircleAngle(toNullable(1048575), 257, toInt128(-9223372036854775808), materialize(1048576)) WITH TOTALS) AS t, (SELECT greatCircleAngle(toUInt256(1048575), 257, toNullable(-9223372036854775808), 1048576), 1048575, 2) AS u From eb51dc8980a4fa29e7836132158023fbba39db7f Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Mon, 15 Jul 2024 12:08:18 +0000 Subject: [PATCH 66/79] Don't start GWP allocations until init is finished --- programs/keeper/Keeper.cpp | 8 ++++++++ programs/server/Server.cpp | 1 + src/Client/ClientBase.cpp | 9 +++++++++ src/Common/Allocator.cpp | 4 ++-- src/Common/GWPAsan.cpp | 7 +++++++ src/Common/GWPAsan.h | 18 ++++++++++++++++++ src/Common/memory.h | 6 +++--- 7 files changed, 48 insertions(+), 5 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 44c2daa33ad..8cf1a4d1999 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -52,6 +52,10 @@ # include #endif +#if USE_GWP_ASAN +# include +#endif + #include #include @@ -639,6 +643,10 @@ try tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); } +#if USE_GWP_ASAN + GWPAsan::initFinished(); +#endif + LOG_INFO(log, "Ready for connections."); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 0b695c3dde6..16888015f8b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2213,6 +2213,7 @@ try CannotAllocateThreadFaultInjector::setFaultProbability(server_settings.cannot_allocate_thread_fault_injection_probability); #if USE_GWP_ASAN + GWPAsan::initFinished(); GWPAsan::setForceSampleProbability(server_settings.gwp_asan_force_sample_probability); #endif diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 13dce05cabc..7af199131b6 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -81,6 +81,10 @@ #include #include "config.h" +#if USE_GWP_ASAN +# include +#endif + namespace fs = std::filesystem; using namespace std::literals; @@ -3264,6 +3268,11 @@ void ClientBase::init(int argc, char ** argv) fatal_log = createLogger("ClientBase", fatal_channel_ptr.get(), Poco::Message::PRIO_FATAL); signal_listener = std::make_unique(nullptr, fatal_log); signal_listener_thread.start(*signal_listener); + +#if USE_GWP_ASAN + GWPAsan::initFinished(); +#endif + } } diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index bcc5d187047..7f2241ab4c0 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -68,7 +68,7 @@ void * allocNoTrack(size_t size, size_t alignment) { void * buf; #if USE_GWP_ASAN - if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::shouldSample())) { if (void * ptr = GWPAsan::GuardedAlloc.allocate(size, alignment)) { @@ -185,7 +185,7 @@ void * Allocator::realloc(void * buf, size_t old_size, } #if USE_GWP_ASAN - if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::shouldSample())) { auto trace_alloc = CurrentMemoryTracker::alloc(new_size); if (void * ptr = GWPAsan::GuardedAlloc.allocate(new_size, alignment)) diff --git a/src/Common/GWPAsan.cpp b/src/Common/GWPAsan.cpp index 48fbd07ec34..de6991191ea 100644 --- a/src/Common/GWPAsan.cpp +++ b/src/Common/GWPAsan.cpp @@ -217,6 +217,13 @@ void printReport([[maybe_unused]] uintptr_t fault_address) reinterpret_cast(trace.data()), 0, trace_length, [&](const auto line) { LOG_FATAL(logger, fmt::runtime(line)); }); } +std::atomic init_finished = false; + +void initFinished() +{ + init_finished.store(true, std::memory_order_relaxed); +} + std::atomic force_sample_probability = 0.0; void setForceSampleProbability(double value) diff --git a/src/Common/GWPAsan.h b/src/Common/GWPAsan.h index b3215c6157e..846c3417db4 100644 --- a/src/Common/GWPAsan.h +++ b/src/Common/GWPAsan.h @@ -19,12 +19,30 @@ bool isGWPAsanError(uintptr_t fault_address); void printReport(uintptr_t fault_address); +extern std::atomic init_finished; + +void initFinished(); + extern std::atomic force_sample_probability; void setForceSampleProbability(double value); +/** + * We'd like to postpone sampling allocations under the startup is finished. There are mainly + * two reasons for that: + * + * - To avoid complex issues with initialization order + * - Don't waste MaxSimultaneousAllocations on global objects as it's not useful +*/ +inline bool shouldSample() +{ + return init_finished.load(std::memory_order_relaxed) && GuardedAlloc.shouldSample(); +} + inline bool shouldForceSample() { + if (!init_finished.load(std::memory_order_relaxed)) + return false; std::bernoulli_distribution dist(force_sample_probability.load(std::memory_order_relaxed)); return dist(thread_local_rng); } diff --git a/src/Common/memory.h b/src/Common/memory.h index dbef069b408..d673f954fb2 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -37,7 +37,7 @@ requires DB::OptionalArgument inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) { #if USE_GWP_ASAN - if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::shouldSample())) { if constexpr (sizeof...(TAlign) == 1) { @@ -83,7 +83,7 @@ inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) inline ALWAYS_INLINE void * newNoExcept(std::size_t size) noexcept { #if USE_GWP_ASAN - if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::shouldSample())) { if (void * ptr = GWPAsan::GuardedAlloc.allocate(size)) { @@ -102,7 +102,7 @@ inline ALWAYS_INLINE void * newNoExcept(std::size_t size) noexcept inline ALWAYS_INLINE void * newNoExcept(std::size_t size, std::align_val_t align) noexcept { #if USE_GWP_ASAN - if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) + if (unlikely(GWPAsan::shouldSample())) { if (void * ptr = GWPAsan::GuardedAlloc.allocate(size, alignToSizeT(align))) { From 0ef2c7d486669d8c604e332594a02dfc1c38278a Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Thu, 18 Jul 2024 11:53:55 +0000 Subject: [PATCH 67/79] Disable parallel run for 02440_mutations_finalization test --- tests/queries/0_stateless/02440_mutations_finalization.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02440_mutations_finalization.sql b/tests/queries/0_stateless/02440_mutations_finalization.sql index c522d8ab9df..92ed6a327ad 100644 --- a/tests/queries/0_stateless/02440_mutations_finalization.sql +++ b/tests/queries/0_stateless/02440_mutations_finalization.sql @@ -1,3 +1,4 @@ +-- Tags: no-parallel create table mut (n int) engine=ReplicatedMergeTree('/test/02440/{database}/mut', '1') order by tuple(); set insert_keeper_fault_injection_probability=0; From e91c4b2e057323fc2eba16317dbcb210eedd0cab Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 19 Jul 2024 16:13:36 +0200 Subject: [PATCH 68/79] Fix flakiness of async insert tests due to adaptative timeout --- .../queries/0_stateless/02726_async_insert_flush_queue.sql | 3 ++- .../0_stateless/02884_async_insert_skip_settings.reference | 4 ++-- .../0_stateless/02884_async_insert_skip_settings.sql | 7 ++++--- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql index 1ae24e4f3da..e393d96fc40 100644 --- a/tests/queries/0_stateless/02726_async_insert_flush_queue.sql +++ b/tests/queries/0_stateless/02726_async_insert_flush_queue.sql @@ -6,7 +6,8 @@ CREATE TABLE t_async_inserts_flush (a UInt64) ENGINE = Memory; SET async_insert = 1; SET wait_for_async_insert = 0; -SET async_insert_busy_timeout_min_ms = 1000000; +-- Disable adaptive timeout to prevent immediate push of the first message (if the queue last push was old) +SET async_insert_use_adaptive_busy_timeout=0; SET async_insert_busy_timeout_max_ms = 10000000; INSERT INTO t_async_inserts_flush VALUES (1) (2); diff --git a/tests/queries/0_stateless/02884_async_insert_skip_settings.reference b/tests/queries/0_stateless/02884_async_insert_skip_settings.reference index 318966ce93b..957be52c59b 100644 --- a/tests/queries/0_stateless/02884_async_insert_skip_settings.reference +++ b/tests/queries/0_stateless/02884_async_insert_skip_settings.reference @@ -1,5 +1,5 @@ -4 +pending to flush 4 1 1 2 -1 +flush queries 1 diff --git a/tests/queries/0_stateless/02884_async_insert_skip_settings.sql b/tests/queries/0_stateless/02884_async_insert_skip_settings.sql index 9bc689fb4ec..df715af873a 100644 --- a/tests/queries/0_stateless/02884_async_insert_skip_settings.sql +++ b/tests/queries/0_stateless/02884_async_insert_skip_settings.sql @@ -9,7 +9,8 @@ ORDER BY id; SET async_insert = 1; SET async_insert_deduplicate = 1; SET wait_for_async_insert = 0; -SET async_insert_busy_timeout_min_ms = 100000; +-- Disable adaptive timeout to prevent immediate push of the first message (if the queue last push was old) +SET async_insert_use_adaptive_busy_timeout=0; SET async_insert_busy_timeout_max_ms = 1000000; SET insert_deduplication_token = '1'; @@ -30,7 +31,7 @@ INSERT INTO t_async_insert_skip_settings VALUES (2); SYSTEM FLUSH LOGS; -SELECT length(entries.bytes) FROM system.asynchronous_inserts +SELECT 'pending to flush', length(entries.bytes) FROM system.asynchronous_inserts WHERE database = currentDatabase() AND table = 't_async_insert_skip_settings' ORDER BY first_update; @@ -40,7 +41,7 @@ SELECT * FROM t_async_insert_skip_settings ORDER BY id; SYSTEM FLUSH LOGS; -SELECT uniqExact(flush_query_id) FROM system.asynchronous_insert_log +SELECT 'flush queries', uniqExact(flush_query_id) FROM system.asynchronous_insert_log WHERE database = currentDatabase() AND table = 't_async_insert_skip_settings'; DROP TABLE t_async_insert_skip_settings SYNC; From 7d379388d24fba77de9eab9d99e69e7dc23763c2 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Fri, 19 Jul 2024 17:12:34 +0200 Subject: [PATCH 69/79] Turn off randomization of harmful setting --- tests/clickhouse-test | 5 ++++- tests/queries/0_stateless/replication.lib | 8 ++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 60160e71e81..f166f8287d5 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -821,7 +821,10 @@ class SettingsRandomizer: get_localzone(), ] ), - "prefer_warmed_unmerged_parts_seconds": lambda: random.randint(0, 10), + # This setting affect part names and their content which can be read from tables in tests. + # We have a lot of tests which relies on part names, so it's very unsafe to enable randomization + # of this setting + # "prefer_warmed_unmerged_parts_seconds": lambda: random.randint(0, 10), "use_page_cache_for_disks_without_file_cache": lambda: random.random() < 0.7, "page_cache_inject_eviction": lambda: random.random() < 0.5, "merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability": lambda: round( diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 2e21f351d2a..1a86cd9f8db 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -79,7 +79,9 @@ function check_replication_consistency() fi # Touch all data to check that it's readable (and trigger PartCheckThread if needed) - while ! $CLICKHOUSE_CLIENT -q "SELECT * FROM merge(currentDatabase(), '$table_name_prefix') FORMAT Null" 2>/dev/null; do + # it's important to disable prefer warmed unmerged parts because + # otherwise it can read non-syncrhonized state of replicas + while ! $CLICKHOUSE_CLIENT --prefer_warmed_unmerged_parts_seconds=0 -q "SELECT * FROM merge(currentDatabase(), '$table_name_prefix') FORMAT Null" 2>/dev/null; do sleep 1; num_tries=$((num_tries+1)) if [ $num_tries -eq 250 ]; then @@ -102,7 +104,9 @@ function check_replication_consistency() try_sync_replicas "$table_name_prefix" "$time_left" || exit 1 - res=$($CLICKHOUSE_CLIENT -q \ + # it's important to disable prefer warmed unmerged parts because + # otherwise it can read non-syncrhonized state of replicas + res=$($CLICKHOUSE_CLIENT --prefer_warmed_unmerged_parts_seconds=0 -q \ "SELECT if((countDistinct(data) as c) == 0, 1, c) FROM From 6aa754decf9ce156c8e912e9f4e3a0b5e0ece9a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 19 Jul 2024 17:38:04 +0200 Subject: [PATCH 70/79] Fix docs for default value on from_env --- docs/en/operations/configuration-files.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index 57fea3cca3a..4a818410ebe 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -124,7 +124,7 @@ which is equal to #### Default values for from_env and from_zk attributes -It's possible to set the default value and substitute it only if the environment variable or zookeeper node is set using `replace="1"`. +It's possible to set the default value and substitute it only if the environment variable or zookeeper node is set using `replace="1"` (must be declared before from_env). With previous example, but `MAX_QUERY_SIZE` is unset: @@ -132,7 +132,7 @@ With previous example, but `MAX_QUERY_SIZE` is unset: - 150000 + 150000 From e57a10189c5e2444beae08f6e55bab9959cde063 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 19 Jul 2024 19:09:51 +0200 Subject: [PATCH 71/79] Update ci_config.py --- tests/ci/ci_config.py | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 98c7e99a495..d18faaf8f1b 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -317,42 +317,42 @@ class CI: random_bucket="parrepl_with_sanitizer", ), JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2 + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4 ), JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_TSAN], num_batches=2 + required_builds=[BuildNames.PACKAGE_TSAN], num_batches=5 ), JobNames.STATELESS_TEST_MSAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_MSAN], num_batches=3 + required_builds=[BuildNames.PACKAGE_MSAN], num_batches=6 ), JobNames.STATELESS_TEST_UBSAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_UBSAN], num_batches=1 + required_builds=[BuildNames.PACKAGE_UBSAN], num_batches=2 ), JobNames.STATELESS_TEST_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2 + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=5 ), JobNames.STATELESS_TEST_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), JobNames.STATELESS_TEST_RELEASE_COVERAGE: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE], num_batches=5 + required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE], num_batches=6 ), JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=3 + required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 ), JobNames.STATELESS_TEST_S3_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2 + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=6 ), JobNames.STATELESS_TEST_AZURE_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2, release_only=True + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4, release_only=True ), JobNames.STATELESS_TEST_S3_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], - num_batches=3, + num_batches=5, ), JobNames.STRESS_TEST_DEBUG: CommonJobConfigs.STRESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_DEBUG], From e1a659f83b96e33b9482157f0a18dd3c3efb3926 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jul 2024 19:24:22 +0200 Subject: [PATCH 72/79] Remove obsolete code from CMakeLists --- src/CMakeLists.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d985595154c..dac20aaca48 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -419,9 +419,6 @@ dbms_target_link_libraries ( boost::circular_buffer boost::heap) -target_include_directories(clickhouse_common_io PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/Core/include") # uses some includes from core -dbms_target_include_directories(PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/Core/include") - target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::miniselect ch_contrib::pdqsort) From 00e14bde80c343919bc237bdcdeb15f22f153eab Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 19 Jul 2024 19:52:30 +0200 Subject: [PATCH 73/79] Update ci_config.py --- tests/ci/ci_config.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index d18faaf8f1b..00b545f9f33 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -317,19 +317,19 @@ class CI: random_bucket="parrepl_with_sanitizer", ), JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4 + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=2 ), JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_TSAN], num_batches=5 + required_builds=[BuildNames.PACKAGE_TSAN], num_batches=4 ), JobNames.STATELESS_TEST_MSAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_MSAN], num_batches=6 + required_builds=[BuildNames.PACKAGE_MSAN], num_batches=4 ), JobNames.STATELESS_TEST_UBSAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_UBSAN], num_batches=2 ), JobNames.STATELESS_TEST_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=5 + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2 ), JobNames.STATELESS_TEST_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -345,14 +345,14 @@ class CI: required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 ), JobNames.STATELESS_TEST_S3_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=6 + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=2 ), JobNames.STATELESS_TEST_AZURE_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4, release_only=True + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=3, release_only=True ), JobNames.STATELESS_TEST_S3_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_TSAN], - num_batches=5, + num_batches=4, ), JobNames.STRESS_TEST_DEBUG: CommonJobConfigs.STRESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_DEBUG], From fa2659a87cde713011b2a2452269397216c54d6f Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 19 Jul 2024 20:10:08 +0200 Subject: [PATCH 74/79] Increase CI timeout for stateless tests --- docker/test/stateless/run.sh | 4 ++-- tests/ci/ci_definitions.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 0647ed02839..264dcd09da3 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -6,8 +6,8 @@ source /setup_export_logs.sh # fail on errors, verbose and export all env variables set -e -x -a -MAX_RUN_TIME=${MAX_RUN_TIME:-7200} -MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 7200 : MAX_RUN_TIME)) +MAX_RUN_TIME=${MAX_RUN_TIME:-9000} +MAX_RUN_TIME=$((MAX_RUN_TIME == 0 ? 9000 : MAX_RUN_TIME)) USE_DATABASE_REPLICATED=${USE_DATABASE_REPLICATED:=0} USE_SHARED_CATALOG=${USE_SHARED_CATALOG:=0} diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index d41a621bc2e..8addc0de077 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -432,7 +432,7 @@ class CommonJobConfigs: ), run_command='functional_test_check.py "$CHECK_NAME"', runner_type=Runners.FUNC_TESTER, - timeout=7200, + timeout=9000, ) STATEFUL_TEST = JobConfig( job_name_keyword="stateful", From 1e160189f0652f476b3efc0ea1ebeb0a1eb5e162 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 19 Jul 2024 20:10:41 +0200 Subject: [PATCH 75/79] Increase parallel jobs for stateless tests --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 264dcd09da3..670a2420ddf 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -224,7 +224,7 @@ function run_tests() else # All other configurations are OK. ADDITIONAL_OPTIONS+=('--jobs') - ADDITIONAL_OPTIONS+=('7') + ADDITIONAL_OPTIONS+=('8') fi if [[ -n "$RUN_BY_HASH_NUM" ]] && [[ -n "$RUN_BY_HASH_TOTAL" ]]; then From 7fd779cc6f6b271366a27b2cbcfe6bad8cb15913 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 19 Jul 2024 20:41:52 +0200 Subject: [PATCH 76/79] Stateless tests: split parallel tests more evenly --- tests/clickhouse-test | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a8a6ce68b8f..c543e181c4f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2168,7 +2168,10 @@ def run_tests_array(all_tests_with_params: Tuple[List[str], int, TestSuite, bool while True: if all_tests: - case = all_tests.pop(0) + try: + case = all_tests.pop(0) + except IndexError: + break else: break @@ -2474,18 +2477,16 @@ def do_run_tests(jobs, test_suite: TestSuite): # of failures will be nearly the same for all tests from the group. random.shuffle(test_suite.parallel_tests) - batch_size = max(1, (len(test_suite.parallel_tests) // jobs) + 1) - parallel_tests_array = [] - for job in range(jobs): - range_ = job * batch_size, job * batch_size + batch_size - batch = test_suite.parallel_tests[range_[0] : range_[1]] - parallel_tests_array.append((batch, batch_size, test_suite, True)) + batch_size = len(test_suite.parallel_tests) // jobs + manager = multiprocessing.Manager() + parallel_tests = manager.list() + parallel_tests.extend(test_suite.parallel_tests) processes = [] - - for test_batch in parallel_tests_array: + for job in range(jobs): process = multiprocessing.Process( - target=run_tests_process, args=(test_batch,) + target=run_tests_process, + args=((parallel_tests, batch_size, test_suite, True),), ) processes.append(process) process.start() From 565ee9ad66ba1198fdbec9ad6c1b19b76e9cf185 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 19 Jul 2024 21:05:21 +0200 Subject: [PATCH 77/79] Update 02724_limit_num_mutations.sh --- tests/queries/0_stateless/02724_limit_num_mutations.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02724_limit_num_mutations.sh b/tests/queries/0_stateless/02724_limit_num_mutations.sh index a9d69b2ed48..60888db0e2e 100755 --- a/tests/queries/0_stateless/02724_limit_num_mutations.sh +++ b/tests/queries/0_stateless/02724_limit_num_mutations.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 14a0c7e300e1f451ab9d46fc6cfd8c1e5f811946 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 19 Jul 2024 21:13:51 +0200 Subject: [PATCH 78/79] Stateless tests: fix stylecheck --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c543e181c4f..6edd3159c7f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2483,7 +2483,7 @@ def do_run_tests(jobs, test_suite: TestSuite): parallel_tests.extend(test_suite.parallel_tests) processes = [] - for job in range(jobs): + for _ in range(jobs): process = multiprocessing.Process( target=run_tests_process, args=((parallel_tests, batch_size, test_suite, True),), From 42ba3229b5484ebe29f4191cd5672433ba1f6588 Mon Sep 17 00:00:00 2001 From: Rich Raposa Date: Fri, 19 Jul 2024 16:33:15 -0600 Subject: [PATCH 79/79] Update azureBlobStorageCluster.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This has been driving me crazy for a while 😄 The table functions are listed alphabetically except for this one - so it's a trivial fix. --- .../en/sql-reference/table-functions/azureBlobStorageCluster.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md b/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md index 20dfd35d5db..ca3959d6ed1 100644 --- a/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md +++ b/docs/en/sql-reference/table-functions/azureBlobStorageCluster.md @@ -1,6 +1,6 @@ --- slug: /en/sql-reference/table-functions/azureBlobStorageCluster -sidebar_position: 55 +sidebar_position: 15 sidebar_label: azureBlobStorageCluster title: "azureBlobStorageCluster Table Function" ---