From 3bccce2eae52f5f746600c31f83d89f71af80a67 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sat, 11 Nov 2023 19:37:30 +0100 Subject: [PATCH 001/145] 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 002/145] 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 003/145] 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 004/145] 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 005/145] 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 006/145] 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 007/145] 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 008/145] 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 009/145] 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 010/145] 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 011/145] 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 012/145] 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: Fri, 8 Mar 2024 21:37:30 +0100 Subject: [PATCH 013/145] add signal handlers in client and local --- src/Client/ClientBase.cpp | 17 +- src/Client/ClientBase.h | 6 + src/Common/SignalHandlers.cpp | 592 ++++++++++++++++++++++++++++++++ src/Common/SignalHandlers.h | 121 +++++++ src/Daemon/BaseDaemon.cpp | 611 +--------------------------------- src/Daemon/BaseDaemon.h | 2 - 6 files changed, 744 insertions(+), 605 deletions(-) create mode 100644 src/Common/SignalHandlers.cpp create mode 100644 src/Common/SignalHandlers.h diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 48962880b8f..6ee76576515 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -300,7 +301,13 @@ public: }; -ClientBase::~ClientBase() = default; +ClientBase::~ClientBase() +{ + writeSignalIDtoSignalPipe(SignalListener::StopThread); + signal_listener_thread.join(); + HandledSignals::instance().reset(); +} + ClientBase::ClientBase() = default; @@ -3007,6 +3014,14 @@ void ClientBase::init(int argc, char ** argv) } has_log_comment = config().has("log_comment"); + + /// Print stacktrace in case of crash + HandledSignals::instance().setupCommonDeadlySignalHandlers(); + + fatal_channel_ptr = new Poco::ConsoleChannel; + fatal_log = createLogger("ClientBase", fatal_channel_ptr.get(), Poco::Message::PRIO_TRACE); + signal_listener = std::make_unique(nullptr, fatal_log); + signal_listener_thread.start(*signal_listener); } } diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index dd08e7c059b..d7e93be9435 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -192,6 +193,11 @@ protected: SharedContextHolder shared_context; ContextMutablePtr global_context; + LoggerPtr fatal_log; + Poco::AutoPtr fatal_channel_ptr; + Poco::Thread signal_listener_thread; + std::unique_ptr signal_listener; + bool is_interactive = false; /// Use either interactive line editing interface or batch mode. bool is_multiquery = false; bool delayed_interactive = false; diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp new file mode 100644 index 00000000000..9270320d536 --- /dev/null +++ b/src/Common/SignalHandlers.cpp @@ -0,0 +1,592 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ +extern const int CANNOT_SET_SIGNAL_HANDLER; +extern const int CANNOT_SEND_SIGNAL; +} +} + +using namespace DB; + + +void call_default_signal_handler(int sig) +{ + if (SIG_ERR == signal(sig, SIG_DFL)) + throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); + + if (0 != raise(sig)) + throw ErrnoException(ErrorCodes::CANNOT_SEND_SIGNAL, "Cannot send signal"); +} + + +void writeSignalIDtoSignalPipe(int sig) +{ + auto saved_errno = errno; /// We must restore previous value of errno in signal handler. + + char buf[signal_pipe_buf_size]; + auto & signal_pipe = HandledSignals::instance().signal_pipe; + WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf); + writeBinary(sig, out); + out.next(); + + errno = saved_errno; +} + +void closeLogsSignalHandler(int sig, siginfo_t *, void *) +{ + DENY_ALLOCATIONS_IN_SCOPE; + writeSignalIDtoSignalPipe(sig); +} + +void terminateRequestedSignalHandler(int sig, siginfo_t *, void *) +{ + DENY_ALLOCATIONS_IN_SCOPE; + writeSignalIDtoSignalPipe(sig); +} + + +void signalHandler(int sig, siginfo_t * info, void * context) +{ + DENY_ALLOCATIONS_IN_SCOPE; + auto saved_errno = errno; /// We must restore previous value of errno in signal handler. + + char buf[signal_pipe_buf_size]; + auto & signal_pipe = HandledSignals::instance().signal_pipe; + WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf); + + const ucontext_t * signal_context = reinterpret_cast(context); + const StackTrace stack_trace(*signal_context); + + writeBinary(sig, out); + writePODBinary(*info, out); + writePODBinary(signal_context, out); + writePODBinary(stack_trace, out); + writeVectorBinary(Exception::enable_job_stack_trace ? Exception::thread_frame_pointers : std::vector{}, out); + writeBinary(static_cast(getThreadId()), out); + writePODBinary(current_thread, out); + + out.next(); + + if (sig != SIGTSTP) /// This signal is used for debugging. + { + /// The time that is usually enough for separate thread to print info into log. + /// Under MSan full stack unwinding with DWARF info about inline functions takes 101 seconds in one case. + for (size_t i = 0; i < 300; ++i) + { + /// We will synchronize with the thread printing the messages with an atomic variable to finish earlier. + if (HandledSignals::instance().fatal_error_printed.test()) + break; + + /// This coarse method of synchronization is perfectly ok for fatal signals. + sleepForSeconds(1); + } + + /// Wait for all logs flush operations + sleepForSeconds(3); + call_default_signal_handler(sig); + } + + errno = saved_errno; +} + + +[[noreturn]] void terminate_handler() +{ + static thread_local bool terminating = false; + if (terminating) + abort(); + + terminating = true; + + std::string log_message; + + if (std::current_exception()) + log_message = "Terminate called for uncaught exception:\n" + getCurrentExceptionMessage(true); + else + log_message = "Terminate called without an active exception"; + + /// POSIX.1 says that write(2)s of less than PIPE_BUF bytes must be atomic - man 7 pipe + /// And the buffer should not be too small because our exception messages can be large. + static constexpr size_t buf_size = PIPE_BUF; + + if (log_message.size() > buf_size - 16) + log_message.resize(buf_size - 16); + + char buf[buf_size]; + auto & signal_pipe = HandledSignals::instance().signal_pipe; + WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], buf_size, buf); + + writeBinary(static_cast(SignalListener::StdTerminate), out); + writeBinary(static_cast(getThreadId()), out); + writeBinary(log_message, out); + out.next(); + + abort(); +} + +#if defined(SANITIZER) +static DISABLE_SANITIZER_INSTRUMENTATION void sanitizerDeathCallback() +{ + DENY_ALLOCATIONS_IN_SCOPE; + /// Also need to send data via pipe. Otherwise it may lead to deadlocks or failures in printing diagnostic info. + + char buf[signal_pipe_buf_size]; + auto & signal_pipe = HandledSignals::instance().signal_pipe; + WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf); + + const StackTrace stack_trace; + + int sig = SignalListener::SanitizerTrap; + writeBinary(sig, out); + writePODBinary(stack_trace, out); + writeBinary(UInt32(getThreadId()), out); + writePODBinary(current_thread, out); + + out.next(); + + /// The time that is usually enough for separate thread to print info into log. + sleepForSeconds(20); +} +#endif + + +void HandledSignals::addSignalHandler(const std::vector & signals, signal_function handler, bool register_signal) +{ + struct sigaction sa; + memset(&sa, 0, sizeof(sa)); + sa.sa_sigaction = handler; + sa.sa_flags = SA_SIGINFO; + +#if defined(OS_DARWIN) + sigemptyset(&sa.sa_mask); + for (auto signal : signals) + sigaddset(&sa.sa_mask, signal); +#else + if (sigemptyset(&sa.sa_mask)) + throw Poco::Exception("Cannot set signal handler."); + + for (auto signal : signals) + if (sigaddset(&sa.sa_mask, signal)) + throw Poco::Exception("Cannot set signal handler."); +#endif + + for (auto signal : signals) + if (sigaction(signal, &sa, nullptr)) + throw Poco::Exception("Cannot set signal handler."); + + if (register_signal) + std::copy(signals.begin(), signals.end(), std::back_inserter(handled_signals)); +} + +void blockSignals(const std::vector & signals) +{ + sigset_t sig_set; + +#if defined(OS_DARWIN) + sigemptyset(&sig_set); + for (auto signal : signals) + sigaddset(&sig_set, signal); +#else + if (sigemptyset(&sig_set)) + throw Poco::Exception("Cannot block signal."); + + for (auto signal : signals) + if (sigaddset(&sig_set, signal)) + throw Poco::Exception("Cannot block signal."); +#endif + + if (pthread_sigmask(SIG_BLOCK, &sig_set, nullptr)) + throw Poco::Exception("Cannot block signal."); +} + + +void SignalListener::run() +{ + static_assert(PIPE_BUF >= 512); + static_assert(signal_pipe_buf_size <= PIPE_BUF, "Only write of PIPE_BUF to pipe is atomic and the minimal known PIPE_BUF across supported platforms is 512"); + char buf[signal_pipe_buf_size]; + auto & signal_pipe = HandledSignals::instance().signal_pipe; + ReadBufferFromFileDescriptor in(signal_pipe.fds_rw[0], signal_pipe_buf_size, buf); + + while (!in.eof()) + { + int sig = 0; + readBinary(sig, in); + // We may log some specific signals afterwards, with different log + // levels and more info, but for completeness we log all signals + // here at trace level. + // Don't use strsignal here, because it's not thread-safe. + LOG_TRACE(log, "Received signal {}", sig); + + if (sig == StopThread) + { + LOG_INFO(log, "Stop SignalListener thread"); + break; + } + else if (sig == SIGHUP) + { + LOG_DEBUG(log, "Received signal to close logs."); + BaseDaemon::instance().closeLogs(BaseDaemon::instance().logger()); + LOG_INFO(log, "Opened new log file after received signal."); + } + else if (sig == StdTerminate) + { + UInt32 thread_num; + std::string message; + + readBinary(thread_num, in); + readBinary(message, in); + + onTerminate(message, thread_num); + } + else if (sig == SIGINT || + sig == SIGQUIT || + sig == SIGTERM) + { + if (daemon) + daemon->handleSignal(sig); + } + else + { + siginfo_t info{}; + ucontext_t * context{}; + StackTrace stack_trace(NoCapture{}); + std::vector thread_frame_pointers; + UInt32 thread_num{}; + ThreadStatus * thread_ptr{}; + + if (sig != SanitizerTrap) + { + readPODBinary(info, in); + readPODBinary(context, in); + } + + readPODBinary(stack_trace, in); + readVectorBinary(thread_frame_pointers, in); + readBinary(thread_num, in); + readPODBinary(thread_ptr, in); + + /// This allows to receive more signals if failure happens inside onFault function. + /// Example: segfault while symbolizing stack trace. + try + { + std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); }).detach(); + } + catch (...) + { + /// Likely cannot allocate thread + onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); + } + } + } +} + +void SignalListener::onTerminate(std::string_view message, UInt32 thread_num) const +{ + size_t pos = message.find('\n'); + + LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) {}", + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", thread_num, message.substr(0, pos)); + + /// Print trace from std::terminate exception line-by-line to make it easy for grep. + while (pos != std::string_view::npos) + { + ++pos; + size_t next_pos = message.find('\n', pos); + size_t size = next_pos; + if (next_pos != std::string_view::npos) + size = next_pos - pos; + + LOG_FATAL(log, fmt::runtime(message.substr(pos, size))); + pos = next_pos; + } +} + +void SignalListener::onFault( + int sig, + const siginfo_t & info, + ucontext_t * context, + const StackTrace & stack_trace, + const std::vector & thread_frame_pointers, + UInt32 thread_num, + DB::ThreadStatus * thread_ptr) const +{ + ThreadStatus thread_status; + + /// First log those fields that are safe to access and that should not cause new fault. + /// That way we will have some duplicated info in the log but we don't loose important info + /// in case of double fault. + + LOG_FATAL(log, "########## Short fault info ############"); + LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}", + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", + thread_num, sig); + + std::string signal_description = "Unknown signal"; + + /// Some of these are not really signals, but our own indications on failure reason. + if (sig == StdTerminate) + signal_description = "std::terminate"; + else if (sig == SanitizerTrap) + signal_description = "sanitizer trap"; + else if (sig >= 0) + signal_description = strsignal(sig); // NOLINT(concurrency-mt-unsafe) // it is not thread-safe but ok in this context + + LOG_FATAL(log, "Signal description: {}", signal_description); + + String error_message; + + if (sig != SanitizerTrap) + error_message = signalToErrorMessage(sig, info, *context); + else + error_message = "Sanitizer trap."; + + LOG_FATAL(log, fmt::runtime(error_message)); + + String bare_stacktrace_str; + if (stack_trace.getSize()) + { + /// Write bare stack trace (addresses) just in case if we will fail to print symbolized stack trace. + /// NOTE: This still require memory allocations and mutex lock inside logger. + /// BTW we can also print it to stderr using write syscalls. + + WriteBufferFromOwnString bare_stacktrace; + writeString("Stack trace:", bare_stacktrace); + for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i) + { + writeChar(' ', bare_stacktrace); + writePointerHex(stack_trace.getFramePointers()[i], bare_stacktrace); + } + + LOG_FATAL(log, fmt::runtime(bare_stacktrace.str())); + bare_stacktrace_str = bare_stacktrace.str(); + } + + /// Now try to access potentially unsafe data in thread_ptr. + + String query_id; + String query; + + /// Send logs from this thread to client if possible. + /// It will allow client to see failure messages directly. + if (thread_ptr) + { + query_id = thread_ptr->getQueryId(); + query = thread_ptr->getQueryForLog(); + + if (auto logs_queue = thread_ptr->getInternalTextLogsQueue()) + { + CurrentThread::attachInternalTextLogsQueue(logs_queue, LogsLevel::trace); + } + } + + LOG_FATAL(log, "########################################"); + + if (query_id.empty()) + { + LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (no query) Received signal {} ({})", + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", + thread_num, signal_description, sig); + } + else + { + LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (query_id: {}) (query: {}) Received signal {} ({})", + VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", daemon ? daemon->git_hash : "", + thread_num, query_id, query, signal_description, sig); + } + + LOG_FATAL(log, fmt::runtime(error_message)); + + if (!bare_stacktrace_str.empty()) + { + LOG_FATAL(log, fmt::runtime(bare_stacktrace_str)); + } + + /// Write symbolized stack trace line by line for better grep-ability. + stack_trace.toStringEveryLine([&](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); + + /// In case it's a scheduled job write all previous jobs origins call stacks + std::for_each(thread_frame_pointers.rbegin(), thread_frame_pointers.rend(), + [this](const StackTrace::FramePointers & frame_pointers) + { + if (size_t size = std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()) + { + LOG_FATAL(log, "========================================"); + WriteBufferFromOwnString bare_stacktrace; + writeString("Job's origin stack trace:", bare_stacktrace); + std::for_each_n(frame_pointers.begin(), size, + [&bare_stacktrace](const void * ptr) + { + writeChar(' ', bare_stacktrace); + writePointerHex(ptr, bare_stacktrace); + } + ); + + LOG_FATAL(log, fmt::runtime(bare_stacktrace.str())); + + StackTrace::toStringEveryLine(const_cast(frame_pointers.data()), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); + } + } + ); + + +#if defined(OS_LINUX) + /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. + /// Please keep the below log messages in-sync with the ones in programs/server/Server.cpp + + if (daemon && daemon->stored_binary_hash.empty()) + { + LOG_FATAL(log, "Integrity check of the executable skipped because the reference checksum could not be read."); + } + else if (daemon) + { + String calculated_binary_hash = getHashOfLoadedBinaryHex(); + if (calculated_binary_hash == daemon->stored_binary_hash) + { + LOG_FATAL(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash); + } + else + { + LOG_FATAL( + log, + "Calculated checksum of the executable ({0}) does not correspond" + " to the reference checksum stored in the executable ({1})." + " This may indicate one of the following:" + " - the executable was changed just after startup;" + " - the executable was corrupted on disk due to faulty hardware;" + " - the loaded executable was corrupted in memory due to faulty hardware;" + " - the file was intentionally modified;" + " - a logical error in the code.", + calculated_binary_hash, + daemon->stored_binary_hash); + } + } +#endif + + /// Write crash to system.crash_log table if available. + if (collectCrashLog) + collectCrashLog(sig, thread_num, query_id, stack_trace); + +#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD + Context::getGlobalContextInstance()->handleCrash(); +#endif + + /// Send crash report to developers (if configured) + if (sig != SanitizerTrap) + { + if (daemon) + SentryWriter::onFault(sig, error_message, stack_trace); + + /// Advice the user to send it manually. + if (std::string_view(VERSION_OFFICIAL).contains("official build")) + { + const auto & date_lut = DateLUT::instance(); + + /// Approximate support period, upper bound. + if (time(nullptr) - date_lut.makeDate(2000 + VERSION_MAJOR, VERSION_MINOR, 1) < (365 + 30) * 86400) + { + LOG_FATAL(log, "Report this error to https://github.com/ClickHouse/ClickHouse/issues"); + } + else + { + LOG_FATAL(log, "ClickHouse version {} is old and should be upgraded to the latest version.", VERSION_STRING); + } + } + else + { + LOG_FATAL(log, "This ClickHouse version is not official and should be upgraded to the official build."); + } + } + + /// ClickHouse Keeper does not link to some part of Settings. +#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD + /// List changed settings. + if (!query_id.empty()) + { + ContextPtr query_context = thread_ptr->getQueryContext(); + if (query_context) + { + String changed_settings = query_context->getSettingsRef().toString(); + + if (changed_settings.empty()) + LOG_FATAL(log, "No settings were changed"); + else + LOG_FATAL(log, "Changed settings: {}", changed_settings); + } + } +#endif + + /// When everything is done, we will try to send these error messages to client. + if (thread_ptr) + thread_ptr->onFatalError(); + + HandledSignals::instance().fatal_error_printed.test_and_set(); +} + + +HandledSignals::HandledSignals() +{ + signal_pipe.setNonBlockingWrite(); + signal_pipe.tryIncreaseSize(1 << 20); +} + +void HandledSignals::reset() +{ + /// Reset signals to SIG_DFL to avoid trying to write to the signal_pipe that will be closed after. + for (int sig : handled_signals) + { + if (SIG_ERR == signal(sig, SIG_DFL)) + { + try + { + throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); + } + catch (ErrnoException &) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + } + + signal_pipe.close(); +} + +HandledSignals::~HandledSignals() +{ + reset(); +}; + +HandledSignals & HandledSignals::instance() +{ + static HandledSignals res; + return res; +} + +void HandledSignals::setupCommonDeadlySignalHandlers() +{ + std::set_terminate(terminate_handler); + + /// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime. + addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, true); + addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, true); + +#if defined(SANITIZER) + __sanitizer_set_death_callback(sanitizerDeathCallback); +#endif +} + diff --git a/src/Common/SignalHandlers.h b/src/Common/SignalHandlers.h new file mode 100644 index 00000000000..b7af9dc2943 --- /dev/null +++ b/src/Common/SignalHandlers.h @@ -0,0 +1,121 @@ +#pragma once +#include + +#include +#include +#include +#include +#include +#include + +class BaseDaemon; + +/** Reset signal handler to the default and send signal to itself. + * It's called from user signal handler to write core dump. + */ +void call_default_signal_handler(int sig); + +const size_t signal_pipe_buf_size = + sizeof(int) + + sizeof(siginfo_t) + + sizeof(ucontext_t*) + + sizeof(StackTrace) + + sizeof(UInt64) + + sizeof(UInt32) + + sizeof(void*); + +using signal_function = void(int, siginfo_t*, void*); + +void writeSignalIDtoSignalPipe(int sig); + +/** Signal handler for HUP */ +void closeLogsSignalHandler(int sig, siginfo_t *, void *); + +void terminateRequestedSignalHandler(int sig, siginfo_t *, void *); + + +/** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log. + */ +void signalHandler(int sig, siginfo_t * info, void * context); + + +/** To use with std::set_terminate. + * Collects slightly more info than __gnu_cxx::__verbose_terminate_handler, + * and send it to pipe. Other thread will read this info from pipe and asynchronously write it to log. + * Look at libstdc++-v3/libsupc++/vterminate.cc for example. + */ +[[noreturn]] void terminate_handler(); + +#if defined(SANITIZER) +extern "C" void __sanitizer_set_death_callback(void (*)()); + +/// Sanitizers may not expect some function calls from death callback. +/// Let's try to disable instrumentation to avoid possible issues. +/// However, this callback may call other functions that are still instrumented. +/// We can try [[clang::always_inline]] attribute for statements in future (available in clang-15) +/// See https://github.com/google/sanitizers/issues/1543 and https://github.com/google/sanitizers/issues/1549. +DISABLE_SANITIZER_INSTRUMENTATION void sanitizerDeathCallback(); +#endif + + +/// Avoid link time dependency on DB/Interpreters - will use this function only when linked. +__attribute__((__weak__)) void collectCrashLog( + Int32 signal, UInt64 thread_id, const String & query_id, const StackTrace & stack_trace); + + +void blockSignals(const std::vector & signals); + + +/** The thread that read info about signal or std::terminate from pipe. + * On HUP, close log files (for new files to be opened later). + * On information about std::terminate, write it to log. + * On other signals, write info to log. + */ +class SignalListener : public Poco::Runnable +{ +public: + static constexpr int StdTerminate = -1; + static constexpr int StopThread = -2; + static constexpr int SanitizerTrap = -3; + + explicit SignalListener(BaseDaemon * daemon_, LoggerPtr log_) + : daemon(daemon_), log(log_) + { + } + + void run() override; + +private: + BaseDaemon * daemon; + LoggerPtr log; + + void onTerminate(std::string_view message, UInt32 thread_num) const; + + void onFault( + int sig, + const siginfo_t & info, + ucontext_t * context, + const StackTrace & stack_trace, + const std::vector & thread_frame_pointers, + UInt32 thread_num, + DB::ThreadStatus * thread_ptr) const; +}; + +struct HandledSignals +{ + std::vector handled_signals; + DB::PipeFDs signal_pipe; + std::atomic_flag fatal_error_printed; + + HandledSignals(); + ~HandledSignals(); + + void setupCommonDeadlySignalHandlers(); + + void addSignalHandler(const std::vector & signals, signal_function handler, bool register_signal); + + void reset(); + + static HandledSignals & instance(); +}; + diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 289a41bb75e..15ab92464a0 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -16,7 +16,6 @@ #endif #include #include -#include #include #include @@ -33,6 +32,7 @@ #include #include +#include #include #include #include @@ -50,7 +50,6 @@ #include #include #include -#include #include #include #include @@ -75,112 +74,12 @@ namespace DB { namespace ErrorCodes { - extern const int CANNOT_SET_SIGNAL_HANDLER; - extern const int CANNOT_SEND_SIGNAL; extern const int SYSTEM_ERROR; } } using namespace DB; -PipeFDs signal_pipe; - - -/** Reset signal handler to the default and send signal to itself. - * It's called from user signal handler to write core dump. - */ -static void call_default_signal_handler(int sig) -{ - if (SIG_ERR == signal(sig, SIG_DFL)) - throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); - - if (0 != raise(sig)) - throw ErrnoException(ErrorCodes::CANNOT_SEND_SIGNAL, "Cannot send signal"); -} - -static const size_t signal_pipe_buf_size = - sizeof(int) - + sizeof(siginfo_t) - + sizeof(ucontext_t*) - + sizeof(StackTrace) - + sizeof(UInt64) - + sizeof(UInt32) - + sizeof(void*); - -using signal_function = void(int, siginfo_t*, void*); - -static void writeSignalIDtoSignalPipe(int sig) -{ - auto saved_errno = errno; /// We must restore previous value of errno in signal handler. - - char buf[signal_pipe_buf_size]; - WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf); - writeBinary(sig, out); - out.next(); - - errno = saved_errno; -} - -/** Signal handler for HUP */ -static void closeLogsSignalHandler(int sig, siginfo_t *, void *) -{ - DENY_ALLOCATIONS_IN_SCOPE; - writeSignalIDtoSignalPipe(sig); -} - -static void terminateRequestedSignalHandler(int sig, siginfo_t *, void *) -{ - DENY_ALLOCATIONS_IN_SCOPE; - writeSignalIDtoSignalPipe(sig); -} - - -static std::atomic_flag fatal_error_printed; - -/** Handler for "fault" or diagnostic signals. Send data about fault to separate thread to write into log. - */ -static void signalHandler(int sig, siginfo_t * info, void * context) -{ - DENY_ALLOCATIONS_IN_SCOPE; - auto saved_errno = errno; /// We must restore previous value of errno in signal handler. - - char buf[signal_pipe_buf_size]; - WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf); - - const ucontext_t * signal_context = reinterpret_cast(context); - const StackTrace stack_trace(*signal_context); - - writeBinary(sig, out); - writePODBinary(*info, out); - writePODBinary(signal_context, out); - writePODBinary(stack_trace, out); - writeVectorBinary(Exception::enable_job_stack_trace ? Exception::thread_frame_pointers : std::vector{}, out); - writeBinary(static_cast(getThreadId()), out); - writePODBinary(current_thread, out); - - out.next(); - - if (sig != SIGTSTP) /// This signal is used for debugging. - { - /// The time that is usually enough for separate thread to print info into log. - /// Under MSan full stack unwinding with DWARF info about inline functions takes 101 seconds in one case. - for (size_t i = 0; i < 300; ++i) - { - /// We will synchronize with the thread printing the messages with an atomic variable to finish earlier. - if (fatal_error_printed.test()) - break; - - /// This coarse method of synchronization is perfectly ok for fatal signals. - sleepForSeconds(1); - } - - /// Wait for all logs flush operations - sleepForSeconds(3); - call_default_signal_handler(sig); - } - - errno = saved_errno; -} static bool getenvBool(const char * name) { @@ -192,426 +91,6 @@ static bool getenvBool(const char * name) } -/// Avoid link time dependency on DB/Interpreters - will use this function only when linked. -__attribute__((__weak__)) void collectCrashLog( - Int32 signal, UInt64 thread_id, const String & query_id, const StackTrace & stack_trace); - - -/** The thread that read info about signal or std::terminate from pipe. - * On HUP, close log files (for new files to be opened later). - * On information about std::terminate, write it to log. - * On other signals, write info to log. - */ -class SignalListener : public Poco::Runnable -{ -public: - static constexpr int StdTerminate = -1; - static constexpr int StopThread = -2; - static constexpr int SanitizerTrap = -3; - - explicit SignalListener(BaseDaemon & daemon_) - : log(getLogger("BaseDaemon")) - , daemon(daemon_) - { - } - - void run() override - { - static_assert(PIPE_BUF >= 512); - static_assert(signal_pipe_buf_size <= PIPE_BUF, "Only write of PIPE_BUF to pipe is atomic and the minimal known PIPE_BUF across supported platforms is 512"); - char buf[signal_pipe_buf_size]; - ReadBufferFromFileDescriptor in(signal_pipe.fds_rw[0], signal_pipe_buf_size, buf); - - while (!in.eof()) - { - int sig = 0; - readBinary(sig, in); - // We may log some specific signals afterwards, with different log - // levels and more info, but for completeness we log all signals - // here at trace level. - // Don't use strsignal here, because it's not thread-safe. - LOG_TRACE(log, "Received signal {}", sig); - - if (sig == StopThread) - { - LOG_INFO(log, "Stop SignalListener thread"); - break; - } - else if (sig == SIGHUP) - { - LOG_DEBUG(log, "Received signal to close logs."); - BaseDaemon::instance().closeLogs(BaseDaemon::instance().logger()); - LOG_INFO(log, "Opened new log file after received signal."); - } - else if (sig == StdTerminate) - { - UInt32 thread_num; - std::string message; - - readBinary(thread_num, in); - readBinary(message, in); - - onTerminate(message, thread_num); - } - else if (sig == SIGINT || - sig == SIGQUIT || - sig == SIGTERM) - { - daemon.handleSignal(sig); - } - else - { - siginfo_t info{}; - ucontext_t * context{}; - StackTrace stack_trace(NoCapture{}); - std::vector thread_frame_pointers; - UInt32 thread_num{}; - ThreadStatus * thread_ptr{}; - - if (sig != SanitizerTrap) - { - readPODBinary(info, in); - readPODBinary(context, in); - } - - readPODBinary(stack_trace, in); - readVectorBinary(thread_frame_pointers, in); - readBinary(thread_num, in); - readPODBinary(thread_ptr, in); - - /// This allows to receive more signals if failure happens inside onFault function. - /// Example: segfault while symbolizing stack trace. - try - { - std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); }).detach(); - } - catch (...) - { - /// Likely cannot allocate thread - onFault(sig, info, context, stack_trace, thread_frame_pointers, thread_num, thread_ptr); - } - } - } - } - -private: - LoggerPtr log; - BaseDaemon & daemon; - - void onTerminate(std::string_view message, UInt32 thread_num) const - { - size_t pos = message.find('\n'); - - LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) {}", - VERSION_STRING, VERSION_OFFICIAL, daemon.build_id, daemon.git_hash, thread_num, message.substr(0, pos)); - - /// Print trace from std::terminate exception line-by-line to make it easy for grep. - while (pos != std::string_view::npos) - { - ++pos; - size_t next_pos = message.find('\n', pos); - size_t size = next_pos; - if (next_pos != std::string_view::npos) - size = next_pos - pos; - - LOG_FATAL(log, fmt::runtime(message.substr(pos, size))); - pos = next_pos; - } - } - - void onFault( - int sig, - const siginfo_t & info, - ucontext_t * context, - const StackTrace & stack_trace, - const std::vector & thread_frame_pointers, - UInt32 thread_num, - ThreadStatus * thread_ptr) const - { - ThreadStatus thread_status; - - /// First log those fields that are safe to access and that should not cause new fault. - /// That way we will have some duplicated info in the log but we don't loose important info - /// in case of double fault. - - LOG_FATAL(log, "########## Short fault info ############"); - LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}", - VERSION_STRING, VERSION_OFFICIAL, daemon.build_id, daemon.git_hash, - thread_num, sig); - - std::string signal_description = "Unknown signal"; - - /// Some of these are not really signals, but our own indications on failure reason. - if (sig == StdTerminate) - signal_description = "std::terminate"; - else if (sig == SanitizerTrap) - signal_description = "sanitizer trap"; - else if (sig >= 0) - signal_description = strsignal(sig); // NOLINT(concurrency-mt-unsafe) // it is not thread-safe but ok in this context - - LOG_FATAL(log, "Signal description: {}", signal_description); - - String error_message; - - if (sig != SanitizerTrap) - error_message = signalToErrorMessage(sig, info, *context); - else - error_message = "Sanitizer trap."; - - LOG_FATAL(log, fmt::runtime(error_message)); - - String bare_stacktrace_str; - if (stack_trace.getSize()) - { - /// Write bare stack trace (addresses) just in case if we will fail to print symbolized stack trace. - /// NOTE: This still require memory allocations and mutex lock inside logger. - /// BTW we can also print it to stderr using write syscalls. - - WriteBufferFromOwnString bare_stacktrace; - writeString("Stack trace:", bare_stacktrace); - for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i) - { - writeChar(' ', bare_stacktrace); - writePointerHex(stack_trace.getFramePointers()[i], bare_stacktrace); - } - - LOG_FATAL(log, fmt::runtime(bare_stacktrace.str())); - bare_stacktrace_str = bare_stacktrace.str(); - } - - /// Now try to access potentially unsafe data in thread_ptr. - - String query_id; - String query; - - /// Send logs from this thread to client if possible. - /// It will allow client to see failure messages directly. - if (thread_ptr) - { - query_id = thread_ptr->getQueryId(); - query = thread_ptr->getQueryForLog(); - - if (auto logs_queue = thread_ptr->getInternalTextLogsQueue()) - { - CurrentThread::attachInternalTextLogsQueue(logs_queue, LogsLevel::trace); - } - } - - LOG_FATAL(log, "########################################"); - - if (query_id.empty()) - { - LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (no query) Received signal {} ({})", - VERSION_STRING, VERSION_OFFICIAL, daemon.build_id, daemon.git_hash, - thread_num, signal_description, sig); - } - else - { - LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) (query_id: {}) (query: {}) Received signal {} ({})", - VERSION_STRING, VERSION_OFFICIAL, daemon.build_id, daemon.git_hash, - thread_num, query_id, query, signal_description, sig); - } - - LOG_FATAL(log, fmt::runtime(error_message)); - - if (!bare_stacktrace_str.empty()) - { - LOG_FATAL(log, fmt::runtime(bare_stacktrace_str)); - } - - /// Write symbolized stack trace line by line for better grep-ability. - stack_trace.toStringEveryLine([&](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); - - /// In case it's a scheduled job write all previous jobs origins call stacks - std::for_each(thread_frame_pointers.rbegin(), thread_frame_pointers.rend(), - [this](const StackTrace::FramePointers & frame_pointers) - { - if (size_t size = std::ranges::find(frame_pointers, nullptr) - frame_pointers.begin()) - { - LOG_FATAL(log, "========================================"); - WriteBufferFromOwnString bare_stacktrace; - writeString("Job's origin stack trace:", bare_stacktrace); - std::for_each_n(frame_pointers.begin(), size, - [&bare_stacktrace](const void * ptr) - { - writeChar(' ', bare_stacktrace); - writePointerHex(ptr, bare_stacktrace); - } - ); - - LOG_FATAL(log, fmt::runtime(bare_stacktrace.str())); - - StackTrace::toStringEveryLine(const_cast(frame_pointers.data()), 0, size, [this](std::string_view s) { LOG_FATAL(log, fmt::runtime(s)); }); - } - } - ); - - -#if defined(OS_LINUX) - /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. - /// Please keep the below log messages in-sync with the ones in programs/server/Server.cpp - - if (daemon.stored_binary_hash.empty()) - { - LOG_FATAL(log, "Integrity check of the executable skipped because the reference checksum could not be read."); - } - else - { - String calculated_binary_hash = getHashOfLoadedBinaryHex(); - if (calculated_binary_hash == daemon.stored_binary_hash) - { - LOG_FATAL(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash); - } - else - { - LOG_FATAL( - log, - "Calculated checksum of the executable ({0}) does not correspond" - " to the reference checksum stored in the executable ({1})." - " This may indicate one of the following:" - " - the executable was changed just after startup;" - " - the executable was corrupted on disk due to faulty hardware;" - " - the loaded executable was corrupted in memory due to faulty hardware;" - " - the file was intentionally modified;" - " - a logical error in the code.", - calculated_binary_hash, - daemon.stored_binary_hash); - } - } -#endif - - /// Write crash to system.crash_log table if available. - if (collectCrashLog) - collectCrashLog(sig, thread_num, query_id, stack_trace); - -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD - Context::getGlobalContextInstance()->handleCrash(); -#endif - - /// Send crash report to developers (if configured) - if (sig != SanitizerTrap) - { - SentryWriter::onFault(sig, error_message, stack_trace); - - /// Advice the user to send it manually. - if (std::string_view(VERSION_OFFICIAL).contains("official build")) - { - const auto & date_lut = DateLUT::instance(); - - /// Approximate support period, upper bound. - if (time(nullptr) - date_lut.makeDate(2000 + VERSION_MAJOR, VERSION_MINOR, 1) < (365 + 30) * 86400) - { - LOG_FATAL(log, "Report this error to https://github.com/ClickHouse/ClickHouse/issues"); - } - else - { - LOG_FATAL(log, "ClickHouse version {} is old and should be upgraded to the latest version.", VERSION_STRING); - } - } - else - { - LOG_FATAL(log, "This ClickHouse version is not official and should be upgraded to the official build."); - } - } - - /// ClickHouse Keeper does not link to some part of Settings. -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD - /// List changed settings. - if (!query_id.empty()) - { - ContextPtr query_context = thread_ptr->getQueryContext(); - if (query_context) - { - String changed_settings = query_context->getSettingsRef().toString(); - - if (changed_settings.empty()) - LOG_FATAL(log, "No settings were changed"); - else - LOG_FATAL(log, "Changed settings: {}", changed_settings); - } - } -#endif - - /// When everything is done, we will try to send these error messages to client. - if (thread_ptr) - thread_ptr->onFatalError(); - - fatal_error_printed.test_and_set(); - } -}; - - -#if defined(SANITIZER) -extern "C" void __sanitizer_set_death_callback(void (*)()); - -/// Sanitizers may not expect some function calls from death callback. -/// Let's try to disable instrumentation to avoid possible issues. -/// However, this callback may call other functions that are still instrumented. -/// We can try [[clang::always_inline]] attribute for statements in future (available in clang-15) -/// See https://github.com/google/sanitizers/issues/1543 and https://github.com/google/sanitizers/issues/1549. -static DISABLE_SANITIZER_INSTRUMENTATION void sanitizerDeathCallback() -{ - DENY_ALLOCATIONS_IN_SCOPE; - /// Also need to send data via pipe. Otherwise it may lead to deadlocks or failures in printing diagnostic info. - - char buf[signal_pipe_buf_size]; - WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf); - - const StackTrace stack_trace; - - int sig = SignalListener::SanitizerTrap; - writeBinary(sig, out); - writePODBinary(stack_trace, out); - writeBinary(UInt32(getThreadId()), out); - writePODBinary(current_thread, out); - - out.next(); - - /// The time that is usually enough for separate thread to print info into log. - sleepForSeconds(20); -} -#endif - - -/** To use with std::set_terminate. - * Collects slightly more info than __gnu_cxx::__verbose_terminate_handler, - * and send it to pipe. Other thread will read this info from pipe and asynchronously write it to log. - * Look at libstdc++-v3/libsupc++/vterminate.cc for example. - */ -[[noreturn]] static void terminate_handler() -{ - static thread_local bool terminating = false; - if (terminating) - abort(); - - terminating = true; - - std::string log_message; - - if (std::current_exception()) - log_message = "Terminate called for uncaught exception:\n" + getCurrentExceptionMessage(true); - else - log_message = "Terminate called without an active exception"; - - /// POSIX.1 says that write(2)s of less than PIPE_BUF bytes must be atomic - man 7 pipe - /// And the buffer should not be too small because our exception messages can be large. - static constexpr size_t buf_size = PIPE_BUF; - - if (log_message.size() > buf_size - 16) - log_message.resize(buf_size - 16); - - char buf[buf_size]; - WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], buf_size, buf); - - writeBinary(static_cast(SignalListener::StdTerminate), out); - writeBinary(static_cast(getThreadId()), out); - writeBinary(log_message, out); - out.next(); - - abort(); -} - - static std::string createDirectory(const std::string & file) { fs::path path = fs::path(file).parent_path(); @@ -664,21 +143,7 @@ BaseDaemon::~BaseDaemon() { writeSignalIDtoSignalPipe(SignalListener::StopThread); signal_listener_thread.join(); - /// Reset signals to SIG_DFL to avoid trying to write to the signal_pipe that will be closed after. - for (int sig : handled_signals) - if (SIG_ERR == signal(sig, SIG_DFL)) - { - try - { - throw ErrnoException(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER, "Cannot set signal handler"); - } - catch (ErrnoException &) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - signal_pipe.close(); + HandledSignals::instance().reset(); } @@ -715,6 +180,8 @@ void BaseDaemon::closeFDs() #else fs::path proc_path{"/proc/self/fd"}; #endif + + const auto & signal_pipe = HandledSignals::instance().signal_pipe; if (fs::is_directory(proc_path)) /// Hooray, proc exists { /// in /proc/self/fd directory filenames are numeric file descriptors. @@ -935,84 +402,24 @@ void BaseDaemon::initialize(Application & self) } -static void addSignalHandler(const std::vector & signals, signal_function handler, std::vector * out_handled_signals) -{ - struct sigaction sa; - memset(&sa, 0, sizeof(sa)); - sa.sa_sigaction = handler; - sa.sa_flags = SA_SIGINFO; - -#if defined(OS_DARWIN) - sigemptyset(&sa.sa_mask); - for (auto signal : signals) - sigaddset(&sa.sa_mask, signal); -#else - if (sigemptyset(&sa.sa_mask)) - throw Poco::Exception("Cannot set signal handler."); - - for (auto signal : signals) - if (sigaddset(&sa.sa_mask, signal)) - throw Poco::Exception("Cannot set signal handler."); -#endif - - for (auto signal : signals) - if (sigaction(signal, &sa, nullptr)) - throw Poco::Exception("Cannot set signal handler."); - - if (out_handled_signals) - std::copy(signals.begin(), signals.end(), std::back_inserter(*out_handled_signals)); -} - - -static void blockSignals(const std::vector & signals) -{ - sigset_t sig_set; - -#if defined(OS_DARWIN) - sigemptyset(&sig_set); - for (auto signal : signals) - sigaddset(&sig_set, signal); -#else - if (sigemptyset(&sig_set)) - throw Poco::Exception("Cannot block signal."); - - for (auto signal : signals) - if (sigaddset(&sig_set, signal)) - throw Poco::Exception("Cannot block signal."); -#endif - - if (pthread_sigmask(SIG_BLOCK, &sig_set, nullptr)) - throw Poco::Exception("Cannot block signal."); -} - extern const char * GIT_HASH; void BaseDaemon::initializeTerminationAndSignalProcessing() { SentryWriter::initialize(config()); - std::set_terminate(terminate_handler); /// We want to avoid SIGPIPE when working with sockets and pipes, and just handle return value/errno instead. blockSignals({SIGPIPE}); /// Setup signal handlers. - /// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime. - addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, &handled_signals); - addSignalHandler({SIGHUP}, closeLogsSignalHandler, &handled_signals); - addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, &handled_signals); - -#if defined(SANITIZER) - __sanitizer_set_death_callback(sanitizerDeathCallback); -#endif + HandledSignals::instance().setupCommonDeadlySignalHandlers(); + HandledSignals::instance().addSignalHandler({SIGHUP}, closeLogsSignalHandler, true); /// Set up Poco ErrorHandler for Poco Threads. static KillingErrorHandler killing_error_handler; Poco::ErrorHandler::set(&killing_error_handler); - signal_pipe.setNonBlockingWrite(); - signal_pipe.tryIncreaseSize(1 << 20); - - signal_listener = std::make_unique(*this); + signal_listener = std::make_unique(this, getLogger("BaseDaemon")); signal_listener_thread.start(*signal_listener); #if defined(__ELF__) && !defined(OS_FREEBSD) @@ -1218,7 +625,7 @@ void BaseDaemon::setupWatchdog() /// Forward signals to the child process. if (forward_signals) { - addSignalHandler( + HandledSignals::instance().addSignalHandler( {SIGHUP, SIGINT, SIGQUIT, SIGTERM}, [](int sig, siginfo_t *, void *) { @@ -1234,7 +641,7 @@ void BaseDaemon::setupWatchdog() (void)res; } }, - nullptr); + false); } else { diff --git a/src/Daemon/BaseDaemon.h b/src/Daemon/BaseDaemon.h index 952cf61d8e0..0c262de328e 100644 --- a/src/Daemon/BaseDaemon.h +++ b/src/Daemon/BaseDaemon.h @@ -168,8 +168,6 @@ protected: String git_hash; String stored_binary_hash; - std::vector handled_signals; - bool should_setup_watchdog = false; char * argv0 = nullptr; }; From 159334845e0c844ca445de5f346898038ae04c03 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Mar 2024 14:41:29 +0000 Subject: [PATCH 014/145] Revert "Revert "Fix wacky primary key sorting in `SHOW INDEX`"" This reverts commit 655f9cb1953c371d7b9a9ba6a58fb4a4fc3ca40e. --- .../InterpreterShowIndexesQuery.cpp | 4 ++-- .../0_stateless/02724_show_indexes.reference | 24 +++++++++---------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index e8005ead91e..5be72dc8ce6 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -37,7 +37,7 @@ FROM ( name AS table, 1 AS non_unique, 'PRIMARY' AS key_name, - row_number() over (order by column_name) AS seq_in_index, + row_number() over (order by null) AS seq_in_index, arrayJoin(splitByString(', ', primary_key)) AS column_name, 'A' AS collation, 0 AS cardinality, @@ -75,7 +75,7 @@ FROM ( database = '{0}' AND table = '{1}')) {2} -ORDER BY index_type, expression, column_name, seq_in_index;)", database, table, where_expression); +ORDER BY index_type, expression, seq_in_index;)", database, table, where_expression); /// Sorting is strictly speaking not necessary but 1. it is convenient for users, 2. SQL currently does not allow to /// sort the output of SHOW INDEXES otherwise (SELECT * FROM (SHOW INDEXES ...) ORDER BY ...) is rejected) and 3. some diff --git a/tests/queries/0_stateless/02724_show_indexes.reference b/tests/queries/0_stateless/02724_show_indexes.reference index e41f2521f5c..ac0461fc506 100644 --- a/tests/queries/0_stateless/02724_show_indexes.reference +++ b/tests/queries/0_stateless/02724_show_indexes.reference @@ -2,33 +2,33 @@ tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- EXTENDED tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- WHERE --- Check with weird table names @@ -40,8 +40,8 @@ NULL 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES tbl 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b tbl 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d tbl 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES -tbl 1 PRIMARY 2 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tbl 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- Equally named table in other database tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b From 89e46807dd88f3fddbbdc095eec88745a09b60ad Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Mar 2024 14:58:05 +0000 Subject: [PATCH 015/145] Resolve flaky DatabaseRepliated tests --- src/Interpreters/InterpreterShowIndexesQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 5be72dc8ce6..c6b5d5ac727 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -37,7 +37,7 @@ FROM ( name AS table, 1 AS non_unique, 'PRIMARY' AS key_name, - row_number() over (order by null) AS seq_in_index, + row_number() over (order by database) AS seq_in_index, arrayJoin(splitByString(', ', primary_key)) AS column_name, 'A' AS collation, 0 AS cardinality, From 60a865c9c98bba7f1b18b358b502aaf9cd359ca5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 11 Mar 2024 19:21:32 +0100 Subject: [PATCH 016/145] Debugging --- src/Common/SignalHandlers.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 9270320d536..13679a0b611 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -231,11 +231,11 @@ void SignalListener::run() // levels and more info, but for completeness we log all signals // here at trace level. // Don't use strsignal here, because it's not thread-safe. - LOG_TRACE(log, "Received signal {}", sig); + // LOG_TRACE(log, "Received signal {}", sig); if (sig == StopThread) { - LOG_INFO(log, "Stop SignalListener thread"); + // LOG_INFO(log, "Stop SignalListener thread"); break; } else if (sig == SIGHUP) From 6b13fc93712631b61180582520ba4f66777628c3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 11 Mar 2024 18:24:55 +0000 Subject: [PATCH 017/145] Revert "Debugging" This reverts commit 60a865c9c98bba7f1b18b358b502aaf9cd359ca5. --- src/Common/SignalHandlers.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 13679a0b611..9270320d536 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -231,11 +231,11 @@ void SignalListener::run() // levels and more info, but for completeness we log all signals // here at trace level. // Don't use strsignal here, because it's not thread-safe. - // LOG_TRACE(log, "Received signal {}", sig); + LOG_TRACE(log, "Received signal {}", sig); if (sig == StopThread) { - // LOG_INFO(log, "Stop SignalListener thread"); + LOG_INFO(log, "Stop SignalListener thread"); break; } else if (sig == SIGHUP) From ff4ae13d9ac1c7991c3de8e55d44faa9ed223093 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 11 Mar 2024 18:26:23 +0000 Subject: [PATCH 018/145] Print only fatals --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6ee76576515..5368384b785 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -3019,7 +3019,7 @@ void ClientBase::init(int argc, char ** argv) HandledSignals::instance().setupCommonDeadlySignalHandlers(); fatal_channel_ptr = new Poco::ConsoleChannel; - fatal_log = createLogger("ClientBase", fatal_channel_ptr.get(), Poco::Message::PRIO_TRACE); + 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); } From 9dbd380b5040a76a2e7a8ee316689e77b6081f27 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 11 Mar 2024 22:40:19 +0000 Subject: [PATCH 019/145] Dont handle SIGTERM and others for client --- src/Client/ClientBase.cpp | 3 +++ src/Common/SignalHandlers.cpp | 10 ++++++++-- src/Common/SignalHandlers.h | 3 ++- src/Daemon/BaseDaemon.cpp | 2 ++ 4 files changed, 15 insertions(+), 3 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5368384b785..9e0d5262245 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -3016,7 +3016,10 @@ void ClientBase::init(int argc, char ** argv) has_log_comment = config().has("log_comment"); /// Print stacktrace in case of crash + HandledSignals::instance().setupTerminateHandler(); HandledSignals::instance().setupCommonDeadlySignalHandlers(); + /// We don't setup signal handlers for SIGINT, SIGQUIT, SIGTERM because we don't + /// have an option for client to shutdown gracefully. fatal_channel_ptr = new Poco::ConsoleChannel; fatal_log = createLogger("ClientBase", fatal_channel_ptr.get(), Poco::Message::PRIO_FATAL); diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 9270320d536..8414fc54adc 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -577,16 +577,22 @@ HandledSignals & HandledSignals::instance() return res; } -void HandledSignals::setupCommonDeadlySignalHandlers() +void HandledSignals::setupTerminateHandler() { std::set_terminate(terminate_handler); +} +void HandledSignals::setupCommonDeadlySignalHandlers() +{ /// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime. addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, true); - addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, true); #if defined(SANITIZER) __sanitizer_set_death_callback(sanitizerDeathCallback); #endif } +void HandledSignals::setupCommonTerminateRequestSignalHandlers() +{ + addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, true); +} diff --git a/src/Common/SignalHandlers.h b/src/Common/SignalHandlers.h index b7af9dc2943..8bfeadd612e 100644 --- a/src/Common/SignalHandlers.h +++ b/src/Common/SignalHandlers.h @@ -110,7 +110,9 @@ struct HandledSignals HandledSignals(); ~HandledSignals(); + void setupTerminateHandler(); void setupCommonDeadlySignalHandlers(); + void setupCommonTerminateRequestSignalHandlers(); void addSignalHandler(const std::vector & signals, signal_function handler, bool register_signal); @@ -118,4 +120,3 @@ struct HandledSignals static HandledSignals & instance(); }; - diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 15ab92464a0..e807ea2706b 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -412,7 +412,9 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() blockSignals({SIGPIPE}); /// Setup signal handlers. + HandledSignals::instance().setupTerminateHandler(); HandledSignals::instance().setupCommonDeadlySignalHandlers(); + HandledSignals::instance().setupCommonTerminateRequestSignalHandlers(); HandledSignals::instance().addSignalHandler({SIGHUP}, closeLogsSignalHandler, true); /// Set up Poco ErrorHandler for Poco Threads. From 09860cf7fd59ed67c4ce9ea3333c2ffaf6e2f46a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 12 Mar 2024 02:55:23 +0100 Subject: [PATCH 020/145] Blind fix for build --- programs/keeper/CMakeLists.txt | 1 + src/Common/SignalHandlers.cpp | 9 +++++++++ src/Common/SignalHandlers.h | 12 ------------ 3 files changed, 10 insertions(+), 12 deletions(-) diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 70e0f229fd4..5752eadc2a9 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -98,6 +98,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/NamedCollections/NamedCollections.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/NamedCollections/NamedCollectionConfiguration.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/Jemalloc.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/SignalHandlers.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/IKeeper.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/TestKeeper.cpp diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 8414fc54adc..21fd5152e6a 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -12,6 +12,8 @@ #include #include +#pragma clang diagnostic ignored "-Wreserved-identifier" + namespace DB { namespace ErrorCodes @@ -140,6 +142,13 @@ void signalHandler(int sig, siginfo_t * info, void * context) } #if defined(SANITIZER) +extern "C" void __sanitizer_set_death_callback(void (*)()); + +/// Sanitizers may not expect some function calls from death callback. +/// Let's try to disable instrumentation to avoid possible issues. +/// However, this callback may call other functions that are still instrumented. +/// We can try [[clang::always_inline]] attribute for statements in future (available in clang-15) +/// See https://github.com/google/sanitizers/issues/1543 and https://github.com/google/sanitizers/issues/1549. static DISABLE_SANITIZER_INSTRUMENTATION void sanitizerDeathCallback() { DENY_ALLOCATIONS_IN_SCOPE; diff --git a/src/Common/SignalHandlers.h b/src/Common/SignalHandlers.h index 8bfeadd612e..e7519f7aee2 100644 --- a/src/Common/SignalHandlers.h +++ b/src/Common/SignalHandlers.h @@ -46,18 +46,6 @@ void signalHandler(int sig, siginfo_t * info, void * context); */ [[noreturn]] void terminate_handler(); -#if defined(SANITIZER) -extern "C" void __sanitizer_set_death_callback(void (*)()); - -/// Sanitizers may not expect some function calls from death callback. -/// Let's try to disable instrumentation to avoid possible issues. -/// However, this callback may call other functions that are still instrumented. -/// We can try [[clang::always_inline]] attribute for statements in future (available in clang-15) -/// See https://github.com/google/sanitizers/issues/1543 and https://github.com/google/sanitizers/issues/1549. -DISABLE_SANITIZER_INSTRUMENTATION void sanitizerDeathCallback(); -#endif - - /// Avoid link time dependency on DB/Interpreters - will use this function only when linked. __attribute__((__weak__)) void collectCrashLog( Int32 signal, UInt64 thread_id, const String & query_id, const StackTrace & stack_trace); From 1e4b27e923dfcda94b2d4b019fdb8e58ed5ccf33 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 12 Mar 2024 14:51:30 +0100 Subject: [PATCH 021/145] Bump From b319d62620657ab9c85b9543b0f0cac41fcbccd3 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 18 Mar 2024 01:07:03 +0100 Subject: [PATCH 022/145] 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 023/145] 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 024/145] 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 025/145] 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 026/145] 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 027/145] 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 028/145] 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 029/145] 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 030/145] 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 031/145] 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 032/145] 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 033/145] 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 034/145] 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 035/145] 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 036/145] 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 1d4e26fa6a94799fa8ff708488d17dfeeab991e3 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Wed, 15 May 2024 13:45:18 -0700 Subject: [PATCH 037/145] Add back missing ProtobufList docs --- docs/en/interfaces/formats.md | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index a137eb2bdf2..a20eff0223d 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -67,6 +67,7 @@ The supported formats are: | [Prometheus](#prometheus) | ✗ | ✔ | | [Protobuf](#protobuf) | ✔ | ✔ | | [ProtobufSingle](#protobufsingle) | ✔ | ✔ | +| [ProtobufList](#protobuflist) | ✔ | ✔ | | [Avro](#data-format-avro) | ✔ | ✔ | | [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ | | [Parquet](#data-format-parquet) | ✔ | ✔ | @@ -1945,6 +1946,35 @@ SYSTEM DROP FORMAT SCHEMA CACHE FOR Protobuf Same as [Protobuf](#protobuf) but for storing/parsing single Protobuf message without length delimiters. +## ProtobufList {#protobuflist} + +Similar to Protobuf but rows are represented as a sequence of sub-messages contained in a message with fixed name "Envelope". + +Usage example: + +``` sql +SELECT * FROM test.table FORMAT ProtobufList SETTINGS format_schema = 'schemafile:MessageType' +``` + +``` bash +cat protobuflist_messages.bin | clickhouse-client --query "INSERT INTO test.table FORMAT ProtobufList SETTINGS format_schema='schemafile:MessageType'" +``` + +where the file `schemafile.proto` looks like this: + +``` capnp +syntax = "proto3"; +message Envelope { + message MessageType { + string name = 1; + string surname = 2; + uint32 birthDate = 3; + repeated string phoneNumbers = 4; + }; + MessageType row = 1; +}; +``` + ## Avro {#data-format-avro} [Apache Avro](https://avro.apache.org/) is a row-oriented data serialization framework developed within Apache’s Hadoop project. From 891f449fbdcc759053238971d979dda244c1af09 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Tue, 2 Jul 2024 14:47:18 +0200 Subject: [PATCH 038/145] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 9d5ae17b156..bf2b3794d1e 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -759,6 +759,7 @@ PromQL Promql Promtail Protobuf +ProtobufList ProtobufSingle ProxySQL Punycode @@ -2201,6 +2202,7 @@ proleptic prometheus proto protobuf +protobuflist protobufsingle proxied pseudorandom From accb994dbef7eb0a1e6c5eaf5f9bbeae70f1b28a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 4 Jul 2024 18:43:45 +0200 Subject: [PATCH 039/145] init --- .../AggregateFunctionGroupArrayIntersect.cpp | 44 ++++++++++--------- .../03198_group_array_intersect.reference | 4 ++ .../03198_group_array_intersect.sql | 10 +++++ 3 files changed, 38 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/03198_group_array_intersect.reference create mode 100644 tests/queries/0_stateless/03198_group_array_intersect.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index 903adf5c547..0da5ade1c79 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -111,14 +111,12 @@ public: return; UInt64 version = this->data(place).version++; - if (version == 0) + if (version == 1) { for (auto & rhs_elem : rhs_set) set.insert(rhs_elem.getValue()); - return; } - - if (!set.empty()) + else if (!set.empty()) { auto create_new_set = [](auto & lhs_val, auto & rhs_val) { @@ -150,8 +148,18 @@ public: void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - readVarUInt(this->data(place).version, buf); - this->data(place).value.read(buf); + auto & set = this->data(place).value; + auto & version = this->data(place).version; + size_t size; + readVarUInt(version, buf); + readVarUInt(size, buf); + set.reserve(size); + for (size_t i = 0; i < size; ++i) + { + int key; + readIntBinary(key, buf); + set.insert(key); + } } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override @@ -270,30 +278,28 @@ public: return; UInt64 version = this->data(place).version++; - if (version == 0) + if (version == 1) { bool inserted; State::Set::LookupResult it; for (auto & rhs_elem : rhs_value) - { set.emplace(ArenaKeyHolder{rhs_elem.getValue(), *arena}, it, inserted); - } } else if (!set.empty()) { - auto create_new_map = [](auto & lhs_val, auto & rhs_val) + auto create_matched_set = [](auto & lhs_val, auto & rhs_val) { - typename State::Set new_map; - for (auto & lhs_elem : lhs_val) + typename State::Set new_set; + for (const auto & lhs_elem : lhs_val) { - auto val = rhs_val.find(lhs_elem.getValue()); - if (val != nullptr) - new_map.insert(lhs_elem.getValue()); + auto is_match = rhs_val.find(lhs_elem.getValue()); + if (is_match != nullptr) + new_set.insert(lhs_elem.getValue()); } - return new_map; + return new_set; }; - auto new_map = rhs_value.size() < set.size() ? create_new_map(rhs_value, set) : create_new_map(set, rhs_value); - set = std::move(new_map); + auto matched_set = rhs_value.size() < set.size() ? create_matched_set(rhs_value, set) : create_matched_set(set, rhs_value); + set = std::move(matched_set); } } @@ -316,11 +322,9 @@ public: readVarUInt(version, buf); readVarUInt(size, buf); set.reserve(size); - UInt64 elem_version; for (size_t i = 0; i < size; ++i) { auto key = readStringBinaryInto(*arena, buf); - readVarUInt(elem_version, buf); set.insert(key); } } diff --git a/tests/queries/0_stateless/03198_group_array_intersect.reference b/tests/queries/0_stateless/03198_group_array_intersect.reference new file mode 100644 index 00000000000..f6b2def2f55 --- /dev/null +++ b/tests/queries/0_stateless/03198_group_array_intersect.reference @@ -0,0 +1,4 @@ +['6\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','2\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','3\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] +['2\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] +['3\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] +['5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] diff --git a/tests/queries/0_stateless/03198_group_array_intersect.sql b/tests/queries/0_stateless/03198_group_array_intersect.sql new file mode 100644 index 00000000000..44422fd0ba6 --- /dev/null +++ b/tests/queries/0_stateless/03198_group_array_intersect.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS test_numbers__fuzz_29; +CREATE TABLE test_numbers__fuzz_29 (`a` Array(Nullable(FixedString(19)))) ENGINE = MergeTree ORDER BY a SETTINGS allow_nullable_key=1; + +INSERT INTO test_numbers__fuzz_29 VALUES ([1,2,3,4,5,6]); +INSERT INTO test_numbers__fuzz_29 VALUES ([1,2,4,5]); +INSERT INTO test_numbers__fuzz_29 VALUES ([1,4,3,0,5,5,5]); + +SELECT groupArrayIntersect(*) FROM test_numbers__fuzz_29 GROUP BY a WITH ROLLUP; + +DROP TABLE test_numbers__fuzz_29; From 4f510c588a34473531101c5b2e2fc9e26d18c1c9 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 5 Jul 2024 20:49:47 +0200 Subject: [PATCH 040/145] quick workaround --- .../AggregateFunctionGroupArrayIntersect.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index 0da5ade1c79..4f45d532179 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -111,7 +111,7 @@ public: return; UInt64 version = this->data(place).version++; - if (version == 1) + if (version == 0) { for (auto & rhs_elem : rhs_set) set.insert(rhs_elem.getValue()); @@ -278,14 +278,14 @@ public: return; UInt64 version = this->data(place).version++; - if (version == 1) + if (version == 1 && set.empty()) { bool inserted; State::Set::LookupResult it; for (auto & rhs_elem : rhs_value) set.emplace(ArenaKeyHolder{rhs_elem.getValue(), *arena}, it, inserted); } - else if (!set.empty()) + else { auto create_matched_set = [](auto & lhs_val, auto & rhs_val) { From d89ed1fdcb34b23858e22325f19f9e0fc38e60bb Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 9 Jul 2024 15:22:20 +0000 Subject: [PATCH 041/145] 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 042/145] 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 27738747e0da63684ed1d831fd823e9966923409 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 9 Jul 2024 23:59:53 +0200 Subject: [PATCH 043/145] save fatal logs to file --- src/Client/ClientBase.cpp | 12 +++++++++++- src/Client/ClientBase.h | 6 +++++- tests/clickhouse-test | 23 +++++++++++++++++++++++ 3 files changed, 39 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index cda80fbdc03..b72fa0edf1e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -3068,6 +3068,8 @@ void ClientBase::init(int argc, char ** argv) ("max_memory_usage_in_client", po::value(), "Set memory limit in client/local server") ("fuzzer-args", po::value(), "Command line arguments for the LLVM's libFuzzer driver. Only relevant if the application is compiled with libFuzzer.") + + ("client_log_file", po::value(), "Path to a file for logging fatal errors in client") ; addOptions(options_description); @@ -3231,7 +3233,15 @@ void ClientBase::init(int argc, char ** argv) /// We don't setup signal handlers for SIGINT, SIGQUIT, SIGTERM because we don't /// have an option for client to shutdown gracefully. - fatal_channel_ptr = new Poco::ConsoleChannel; + fatal_channel_ptr = new Poco::SplitterChannel; + fatal_console_channel_ptr = new Poco::ConsoleChannel; + fatal_channel_ptr->addChannel(fatal_console_channel_ptr); + if (options.count("client_log_file")) + { + fatal_file_channel_ptr = new Poco::SimpleFileChannel(options["client_log_file"].as()); + fatal_channel_ptr->addChannel(fatal_file_channel_ptr); + } + 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); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index ffc385a2e98..31039ed1e53 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include #include #include @@ -213,7 +215,9 @@ protected: ContextMutablePtr global_context; LoggerPtr fatal_log; - Poco::AutoPtr fatal_channel_ptr; + Poco::AutoPtr fatal_channel_ptr; + Poco::AutoPtr fatal_console_channel_ptr; + Poco::AutoPtr fatal_file_channel_ptr; Poco::Thread signal_listener_thread; std::unique_ptr signal_listener; diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 8e7002af889..91cecaadf7a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1519,6 +1519,15 @@ class TestCase: start_time = args.testcase_start_time database = args.testcase_database + if args.client_log: + log_opt = " --client_log_file=" + args.client_log + client_options += log_opt + os.environ["CLICKHOUSE_CLIENT_OPT"] = ( + os.environ["CLICKHOUSE_CLIENT_OPT"] + if "CLICKHOUSE_CLIENT_OPT" in os.environ + else "" + ) + log_opt + # This is for .sh tests os.environ["CLICKHOUSE_LOG_COMMENT"] = args.testcase_basename @@ -2832,6 +2841,15 @@ def main(args): else: print(colored("\nNo queries hung.", args, "green", attrs=["bold"])) + if args.client_log: + if os.path.exists(args.client_log): + with open(args.client_log, "rb") as stream: + content = stream.read().decode() + if len(content): + print("Has fatal logs from client:\n") + print(content) + os.remove(args.client_log) + if len(restarted_tests) > 0: print("\nSome tests were restarted:\n") @@ -3233,6 +3251,11 @@ def parse_args(): default=False, help="Replace ordinary MergeTree engine with SharedMergeTree", ) + parser.add_argument( + "--client-log", + default="./client.fatal.log", + help="Path to file for fatal logs from client", + ) return parser.parse_args() From 0c843ae1463fd94026fb0367845e3c43bdfa2ab4 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 10 Jul 2024 13:01:15 +0000 Subject: [PATCH 044/145] 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 045/145] 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 92b46616005500a2674cdf4719a5476f341c9289 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 11 Jul 2024 18:58:53 +0200 Subject: [PATCH 046/145] fix --- .../AggregateFunctionGroupArrayIntersect.cpp | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index 4f45d532179..20acda213da 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -115,8 +115,10 @@ public: { for (auto & rhs_elem : rhs_set) set.insert(rhs_elem.getValue()); + return; } - else if (!set.empty()) + + if (!set.empty()) { auto create_new_set = [](auto & lhs_val, auto & rhs_val) { @@ -278,28 +280,30 @@ public: return; UInt64 version = this->data(place).version++; - if (version == 1 && set.empty()) + if (version == 0) { bool inserted; State::Set::LookupResult it; for (auto & rhs_elem : rhs_value) - set.emplace(ArenaKeyHolder{rhs_elem.getValue(), *arena}, it, inserted); - } - else - { - auto create_matched_set = [](auto & lhs_val, auto & rhs_val) { - typename State::Set new_set; - for (const auto & lhs_elem : lhs_val) + set.emplace(ArenaKeyHolder{rhs_elem.getValue(), *arena}, it, inserted); + } + } + else if (!set.empty()) + { + auto create_new_map = [](auto & lhs_val, auto & rhs_val) + { + typename State::Set new_map; + for (auto & lhs_elem : lhs_val) { - auto is_match = rhs_val.find(lhs_elem.getValue()); - if (is_match != nullptr) - new_set.insert(lhs_elem.getValue()); + auto val = rhs_val.find(lhs_elem.getValue()); + if (val != nullptr) + new_map.insert(lhs_elem.getValue()); } - return new_set; + return new_map; }; - auto matched_set = rhs_value.size() < set.size() ? create_matched_set(rhs_value, set) : create_matched_set(set, rhs_value); - set = std::move(matched_set); + auto new_map = create_new_map(set, rhs_value); + set = std::move(new_map); } } From 67b7cd40d42f306fe3b8ed997bded7678c9c4e78 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 11 Jul 2024 21:28:56 +0200 Subject: [PATCH 047/145] Update 03198_group_array_intersect.sql --- tests/queries/0_stateless/03198_group_array_intersect.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03198_group_array_intersect.sql b/tests/queries/0_stateless/03198_group_array_intersect.sql index 44422fd0ba6..06ab0fbd819 100644 --- a/tests/queries/0_stateless/03198_group_array_intersect.sql +++ b/tests/queries/0_stateless/03198_group_array_intersect.sql @@ -1,4 +1,5 @@ DROP TABLE IF EXISTS test_numbers__fuzz_29; +SET max_threads=1, max_insert_threads=1; CREATE TABLE test_numbers__fuzz_29 (`a` Array(Nullable(FixedString(19)))) ENGINE = MergeTree ORDER BY a SETTINGS allow_nullable_key=1; INSERT INTO test_numbers__fuzz_29 VALUES ([1,2,3,4,5,6]); From ea64527ea1e88160b69f9d4d552421af07fc0d32 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 12 Jul 2024 19:18:16 +0200 Subject: [PATCH 048/145] suppress broken test --- src/Client/ClientBase.cpp | 6 +++--- tests/clickhouse-test | 2 +- tests/queries/0_stateless/02895_npy_format.sh | 15 +++++++++------ 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b72fa0edf1e..b29eebd4fde 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -3069,7 +3069,7 @@ void ClientBase::init(int argc, char ** argv) ("fuzzer-args", po::value(), "Command line arguments for the LLVM's libFuzzer driver. Only relevant if the application is compiled with libFuzzer.") - ("client_log_file", po::value(), "Path to a file for logging fatal errors in client") + ("client_logs_file", po::value(), "Path to a file for writing client logs. Currently we only have fatal logs (when the client crashes)") ; addOptions(options_description); @@ -3236,9 +3236,9 @@ void ClientBase::init(int argc, char ** argv) fatal_channel_ptr = new Poco::SplitterChannel; fatal_console_channel_ptr = new Poco::ConsoleChannel; fatal_channel_ptr->addChannel(fatal_console_channel_ptr); - if (options.count("client_log_file")) + if (options.count("client_logs_file")) { - fatal_file_channel_ptr = new Poco::SimpleFileChannel(options["client_log_file"].as()); + fatal_file_channel_ptr = new Poco::SimpleFileChannel(options["client_logs_file"].as()); fatal_channel_ptr->addChannel(fatal_file_channel_ptr); } diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 1fcc9d111dd..cca0e2950b0 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1526,7 +1526,7 @@ class TestCase: database = args.testcase_database if args.client_log: - log_opt = " --client_log_file=" + args.client_log + log_opt = " --client_logs_file=" + args.client_log + " " client_options += log_opt os.environ["CLICKHOUSE_CLIENT_OPT"] = ( os.environ["CLICKHOUSE_CLIENT_OPT"] diff --git a/tests/queries/0_stateless/02895_npy_format.sh b/tests/queries/0_stateless/02895_npy_format.sh index 9d05303a091..a655f275179 100755 --- a/tests/queries/0_stateless/02895_npy_format.sh +++ b/tests/queries/0_stateless/02895_npy_format.sh @@ -51,13 +51,16 @@ $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_str.npy', Npy $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/two_dim.npy', Npy, 'value Array(Int8)')" $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/three_dim.npy', Npy, 'value Array(Array(Int8))')" -$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Array(Float32)')" 2>&1 | grep -c "BAD_ARGUMENTS" -$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value UUID')" 2>&1 | grep -c "BAD_ARGUMENTS" -$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Tuple(UInt8)')" 2>&1 | grep -c "BAD_ARGUMENTS" +# The test is wrong: https://github.com/ClickHouse/ClickHouse/issues/66458 +echo -e "0\n0\n0" +#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Array(Float32)')" 2>&1 | grep -c "BAD_ARGUMENTS" +#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value UUID')" 2>&1 | grep -c "BAD_ARGUMENTS" +#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Tuple(UInt8)')" 2>&1 | grep -c "BAD_ARGUMENTS" -$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Int8')" 2>&1 | grep -c "BAD_ARGUMENTS" -$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_str.npy', Npy, 'value Int8')" 2>&1 | grep -c "BAD_ARGUMENTS" -$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_unicode.npy', Npy, 'value Float32')" 2>&1 | grep -c "BAD_ARGUMENTS" +echo -e "0\n0\n0" +#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_float.npy', Npy, 'value Int8')" 2>&1 | grep -c "BAD_ARGUMENTS" +#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_str.npy', Npy, 'value Int8')" 2>&1 | grep -c "BAD_ARGUMENTS" +#$CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/one_dim_unicode.npy', Npy, 'value Float32')" 2>&1 | grep -c "BAD_ARGUMENTS" $CLICKHOUSE_LOCAL -q "select * from file('$CURDIR/data_npy/complex.npy')" 2>&1 | grep -c "BAD_ARGUMENTS" From 6f08e785144b1d6794f78b8bc9d0d5b21a0f160d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 13 Jul 2024 22:27:16 +0200 Subject: [PATCH 049/145] Update 03198_group_array_intersect.sql --- tests/queries/0_stateless/03198_group_array_intersect.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03198_group_array_intersect.sql b/tests/queries/0_stateless/03198_group_array_intersect.sql index 06ab0fbd819..ab7ba0b0d70 100644 --- a/tests/queries/0_stateless/03198_group_array_intersect.sql +++ b/tests/queries/0_stateless/03198_group_array_intersect.sql @@ -6,6 +6,6 @@ INSERT INTO test_numbers__fuzz_29 VALUES ([1,2,3,4,5,6]); INSERT INTO test_numbers__fuzz_29 VALUES ([1,2,4,5]); INSERT INTO test_numbers__fuzz_29 VALUES ([1,4,3,0,5,5,5]); -SELECT groupArrayIntersect(*) FROM test_numbers__fuzz_29 GROUP BY a WITH ROLLUP; +SELECT arraySort(groupArrayIntersect(*)) FROM test_numbers__fuzz_29 GROUP BY a WITH ROLLUP ORDER BY ALL; DROP TABLE test_numbers__fuzz_29; From e1aac19630e1e1dcdcc9e7636b3912c01e5a38ab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 13 Jul 2024 22:43:30 +0200 Subject: [PATCH 050/145] Fix test --- .../0_stateless/03198_group_array_intersect.reference | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03198_group_array_intersect.reference b/tests/queries/0_stateless/03198_group_array_intersect.reference index f6b2def2f55..6a3ffdba388 100644 --- a/tests/queries/0_stateless/03198_group_array_intersect.reference +++ b/tests/queries/0_stateless/03198_group_array_intersect.reference @@ -1,4 +1,4 @@ -['6\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','2\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','3\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] -['2\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] -['3\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] -['5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] +['0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','3\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] +['1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','2\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','3\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','6\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] +['1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','2\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] +['1\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','4\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','5\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'] From a61304508f07694354c1f698ad8c8c6ba5b65edc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jul 2024 21:47:09 +0200 Subject: [PATCH 051/145] Remove noise from unit tests --- src/Common/Exception.h | 9 +++++++++ src/Common/tests/gtest_shell_command.cpp | 13 ------------- .../MergeTree/MergeTreeBackgroundExecutor.cpp | 4 ++++ src/Storages/MergeTree/tests/gtest_executor.cpp | 4 ++-- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index a4774a89f6a..4e54c411bf1 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -244,6 +244,15 @@ private: const char * className() const noexcept override { return "DB::ErrnoException"; } }; +/// An exception to use in unit tests to test interfaces. +/// It is distinguished from others, so it does not have to be logged. +class TestException : public Exception +{ +public: + using Exception::Exception; +}; + + using Exceptions = std::vector; /** Try to write an exception to the log (and forget about it). diff --git a/src/Common/tests/gtest_shell_command.cpp b/src/Common/tests/gtest_shell_command.cpp index d6d0a544e9b..0ea96da9da2 100644 --- a/src/Common/tests/gtest_shell_command.cpp +++ b/src/Common/tests/gtest_shell_command.cpp @@ -54,16 +54,3 @@ TEST(ShellCommand, ExecuteWithInput) EXPECT_EQ(res, "Hello, world!\n"); } - -TEST(ShellCommand, AutoWait) -{ - // hunting: - for (int i = 0; i < 1000; ++i) - { - auto command = ShellCommand::execute("echo " + std::to_string(i)); - //command->wait(); // now automatic - } - - // std::cerr << "inspect me: ps auxwwf\n"; - // std::this_thread::sleep_for(std::chrono::seconds(100)); -} diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 8cb0badc19b..4e62c503d65 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -155,6 +155,10 @@ void printExceptionWithRespectToAbort(LoggerPtr log, const String & query_id) { std::rethrow_exception(ex); } + catch (const TestException &) + { + /// Exception from a unit test, ignore it. + } catch (const Exception & e) { NOEXCEPT_SCOPE({ diff --git a/src/Storages/MergeTree/tests/gtest_executor.cpp b/src/Storages/MergeTree/tests/gtest_executor.cpp index 6f34eb4dfbd..c7057ce87c6 100644 --- a/src/Storages/MergeTree/tests/gtest_executor.cpp +++ b/src/Storages/MergeTree/tests/gtest_executor.cpp @@ -34,7 +34,7 @@ public: auto choice = distribution(generator); if (choice == 0) - throw std::runtime_error("Unlucky..."); + throw TestException(); return false; } @@ -48,7 +48,7 @@ public: { auto choice = distribution(generator); if (choice == 0) - throw std::runtime_error("Unlucky..."); + throw TestException(); } Priority getPriority() const override { return {}; } From d4e7188750755a33148eb12536cd3687efb8354d Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 15 Jul 2024 16:32:49 +0200 Subject: [PATCH 052/145] CI: Report job start and finish to CI DB --- tests/ci/ci.py | 236 ++++++++++++++++++++++++++------------------- tests/ci/report.py | 14 ++- 2 files changed, 146 insertions(+), 104 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index cf285f4b97d..110a7b2a49c 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -48,7 +48,19 @@ from git_helper import GIT_PREFIX, Git from git_helper import Runner as GitRunner from github_helper import GitHub from pr_info import PRInfo -from report import ERROR, FAILURE, PENDING, SUCCESS, BuildResult, JobReport, TestResult +from report import ( + ERROR, + FAILURE, + PENDING, + SUCCESS, + BuildResult, + JobReport, + TestResult, + OK, + JOB_STARTED_TEST_NAME, + JOB_FINISHED_TEST_NAME, + FAIL, +) from s3_helper import S3Helper from stopwatch import Stopwatch from tee_popen import TeePopen @@ -263,7 +275,8 @@ def check_missing_images_on_dockerhub( return result -def _pre_action(s3, indata, pr_info): +def _pre_action(s3, job_name, batch, indata, pr_info): + no_cache = CiSettings.create_from_run_config(indata).no_ci_cache print("Clear dmesg") Utils.clear_dmesg() CommitStatusData.cleanup() @@ -282,6 +295,90 @@ def _pre_action(s3, indata, pr_info): ci_cache.dump_run_config(indata) + to_be_skipped = False + skip_status = SUCCESS + # check if job was run already + if CI.is_build_job(job_name): + # this is a build job - check if a build report is present + build_result = ( + BuildResult.load_any(job_name, pr_info.number, pr_info.head_ref) + if not no_cache + else None + ) + if build_result: + if build_result.status == SUCCESS: + to_be_skipped = True + else: + print( + "Build report found but status is unsuccessful - will try to rerun" + ) + print("::group::Build Report") + print(build_result.as_json()) + print("::endgroup::") + else: + # this is a test job - check if GH commit status or cache record is present + commit = get_commit(GitHub(get_best_robot_token(), per_page=100), pr_info.sha) + # rerun helper check + # FIXME: Find a way to identify if job restarted manually (by developer) or by automatic workflow restart (died spot-instance) + # disable rerun check for the former + if job_name not in ( + CI.JobNames.BUILD_CHECK, + ): # we might want to rerun build report job + rerun_helper = RerunHelper(commit, _get_ext_check_name(job_name)) + if rerun_helper.is_already_finished_by_status(): + print("WARNING: Rerunning job with GH status ") + status = rerun_helper.get_finished_status() + assert status + print("::group::Commit Status") + print(status) + print("::endgroup::") + to_be_skipped = True + skip_status = status.state + + # ci cache check + if not to_be_skipped and not no_cache: + ci_cache = CiCache(s3, indata["jobs_data"]["digests"]).update() + job_config = CI.get_job_config(job_name) + if ci_cache.is_successful( + job_name, + batch, + job_config.num_batches, + job_config.required_on_release_branch, + ): + print("CICache record has be found - job will be skipped") + job_status = ci_cache.get_successful( + job_name, batch, job_config.num_batches + ) + assert job_status, "BUG" + _create_gh_status( + commit, + job_name, + batch, + job_config.num_batches, + job_status, + ) + to_be_skipped = True + # skip_status = SUCCESS already there + GHActions.print_in_group("Commit Status Data", job_status) + + # create pre report + jr = JobReport.create_pre_report(status=skip_status, job_skipped=to_be_skipped) + jr.dump() + + if not to_be_skipped: + print("push start record to ci db") + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + [TestResult(JOB_STARTED_TEST_NAME, OK)], + SUCCESS, + 0.0, + JobReport.get_start_time_from_current(), + "", + _get_ext_check_name(job_name), + ) + ClickHouseHelper().insert_events_into( + db="default", table="checks", events=prepared_events + ) print(f"Pre action done. Report files [{reports_files}] have been downloaded") @@ -1045,108 +1142,23 @@ def main() -> int: ### PRE action: start elif args.pre: assert indata, "Run config must be provided via --infile" - _pre_action(s3, indata, pr_info) - JobReport.create_pre_report().dump() + _pre_action(s3, args.job_name, args.batch, indata, pr_info) ### RUN action: start elif args.run: assert indata - ci_settings = CiSettings.create_from_run_config(indata) + job_report = JobReport.load() check_name = args.job_name check_name_with_group = _get_ext_check_name(check_name) print( f"Check if rerun for name: [{check_name}], extended name [{check_name_with_group}]" ) - previous_status = None - if CI.is_build_job(check_name): - # this is a build job - check if a build report is present - build_result = ( - BuildResult.load_any(check_name, pr_info.number, pr_info.head_ref) - if not ci_settings.no_ci_cache - else None - ) - if build_result: - if build_result.status == SUCCESS: - previous_status = build_result.status - JobReport( - status=SUCCESS, - description="", - test_results=[], - start_time="", - duration=0.0, - additional_files=[], - job_skipped=True, - ).dump() - else: - # FIXME: Consider reusing failures for build jobs. - # Just remove this if/else - that makes build job starting and failing immediately - print( - "Build report found but status is unsuccessful - will try to rerun" - ) - print("::group::Build Report") - print(build_result.as_json()) - print("::endgroup::") - else: - # this is a test job - check if GH commit status or cache record is present - commit = get_commit( - GitHub(get_best_robot_token(), per_page=100), pr_info.sha - ) - # rerun helper check - # FIXME: Find a way to identify if job restarted manually (by developer) or by automatic workflow restart (died spot-instance) - # disable rerun check for the former - if check_name not in ( - CI.JobNames.BUILD_CHECK, - ): # we might want to rerun build report job - rerun_helper = RerunHelper(commit, check_name_with_group) - if rerun_helper.is_already_finished_by_status(): - print("WARNING: Rerunning job with GH status ") - status = rerun_helper.get_finished_status() - assert status - print("::group::Commit Status") - print(status) - print("::endgroup::") - previous_status = status.state - print("Create dummy job report with job_skipped flag") - JobReport( - status=status.state, - description="", - test_results=[], - start_time="", - duration=0.0, - additional_files=[], - job_skipped=True, - ).dump() - - # ci cache check - if not previous_status and not ci_settings.no_ci_cache: - ci_cache = CiCache(s3, indata["jobs_data"]["digests"]).update() - job_config = CI.get_job_config(check_name) - if ci_cache.is_successful( - check_name, - args.batch, - job_config.num_batches, - job_config.required_on_release_branch, - ): - job_status = ci_cache.get_successful( - check_name, args.batch, job_config.num_batches - ) - assert job_status, "BUG" - _create_gh_status( - commit, - check_name, - args.batch, - job_config.num_batches, - job_status, - ) - previous_status = job_status.status - GHActions.print_in_group("Commit Status Data", job_status) - - if previous_status and not args.force: + if job_report.job_skipped and not args.force: print( - f"Commit status or Build Report is already present - job will be skipped with status: [{previous_status}]" + f"Commit status or Build Report is already present - job will be skipped with status: [{job_report.status}]" ) - if previous_status == SUCCESS: + if job_report.status == SUCCESS: exit_code = 0 else: exit_code = 1 @@ -1166,7 +1178,8 @@ def main() -> int: assert ( job_report ), "BUG. There must be job report either real report, or pre-report if job was killed" - if not job_report.job_skipped and not job_report.pre_report: + error_description = "" + if not job_report.pre_report: # it's a real job report ch_helper = ClickHouseHelper() check_url = "" @@ -1244,7 +1257,6 @@ def main() -> int: pr_info, dump_to_file=True, ) - print(f"Job report url: [{check_url}]") prepared_events = prepare_tests_results_for_clickhouse( pr_info, @@ -1269,9 +1281,7 @@ def main() -> int: ) elif job_report.job_skipped: print(f"Skipped after rerun check {[args.job_name]} - do nothing") - elif job_report.job_skipped: - print(f"Job was skipped {[args.job_name]} - do nothing") - elif job_report.pre_report: + else: print(f"ERROR: Job was killed - generate evidence") job_report.update_duration() ret_code = os.getenv("JOB_EXIT_CODE", "") @@ -1282,10 +1292,13 @@ def main() -> int: pass if Utils.is_killed_with_oom(): print("WARNING: OOM while job execution") - error = f"Out Of Memory, exit_code {job_report.exit_code}, after {int(job_report.duration)}s" + error_description = f"Out Of Memory, exit_code {job_report.exit_code}" else: - error = f"Unknown, exit_code {job_report.exit_code}, after {int(job_report.duration)}s" - CIBuddy().post_error(error, job_name=_get_ext_check_name(args.job_name)) + error_description = f"Unknown, exit_code {job_report.exit_code}" + CIBuddy().post_error( + error_description + f" after {int(job_report.duration)}s", + job_name=_get_ext_check_name(args.job_name), + ) if CI.is_test_job(args.job_name): gh = GitHub(get_best_robot_token(), per_page=100) commit = get_commit(gh, pr_info.sha) @@ -1293,11 +1306,32 @@ def main() -> int: commit, ERROR, "", - "Error: " + error, + "Error: " + error_description, _get_ext_check_name(args.job_name), pr_info, dump_to_file=True, ) + + if not job_report.job_skipped: + print("push finish record to ci db") + prepared_events = prepare_tests_results_for_clickhouse( + pr_info, + [ + TestResult( + JOB_FINISHED_TEST_NAME, + FAIL if error_description else OK, + raw_logs=error_description or None, + ) + ], + SUCCESS if not error_description else ERROR, + 0.0, + JobReport.get_start_time_from_current(), + "", + _get_ext_check_name(args.job_name), + ) + ClickHouseHelper().insert_events_into( + db="default", table="checks", events=prepared_events + ) ### POST action: end ### MARK SUCCESS action: start diff --git a/tests/ci/report.py b/tests/ci/report.py index 4be7b438f4f..77043dfc8be 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -247,6 +247,9 @@ BASE_HEADERS = ["Test name", "Test status"] # should not be in TEMP directory or any directory that may be cleaned during the job execution JOB_REPORT_FILE = Path(GITHUB_WORKSPACE) / "job_report.json" +JOB_STARTED_TEST_NAME = "STARTED" +JOB_FINISHED_TEST_NAME = "COMPLETED" + @dataclass class TestResult: @@ -304,14 +307,19 @@ class JobReport: exit_code: int = -1 @staticmethod - def create_pre_report() -> "JobReport": + def get_start_time_from_current(): + return datetime.datetime.utcnow().strftime("%Y-%m-%d %H:%M:%S") + + @classmethod + def create_pre_report(cls, status: str, job_skipped: bool) -> "JobReport": return JobReport( - status=ERROR, + status=status, description="", test_results=[], - start_time=datetime.datetime.utcnow().strftime("%Y-%m-%d %H:%M:%S"), + start_time=cls.get_start_time_from_current(), duration=0.0, additional_files=[], + job_skipped=job_skipped, pre_report=True, ) From 3aca36c54585ea3fcb2f8a303b96c178fbdf7ecc Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 15 Jul 2024 19:23:29 +0200 Subject: [PATCH 053/145] Update AggregateFunctionGroupArrayIntersect.cpp From 2bc65fe2080260d16e27df965610197a38052705 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 09:44:05 +0000 Subject: [PATCH 054/145] Make ColumnSparse::updateWeakHash32 consistent with internal column. --- src/Columns/ColumnSparse.cpp | 38 +++++++++++++++++++++++++++++++++--- src/Columns/ColumnSparse.h | 3 +++ 2 files changed, 38 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 809586d8810..ea4d23c1678 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -684,13 +685,26 @@ void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " "column size is {}, hash size is {}", _size, hash.getData().size()); - auto offset_it = begin(); + size_t values_size = values->size(); + WeakHash32 values_hash(values_size); + auto & hash_data = hash.getData(); + auto & values_hash_data = values_hash.getData(); + const auto & offsets_data = getOffsetsData(); + + if (getNumberOfDefaultRows() > 0) + values_hash_data[0] = hash_data[getFirstDefaultValueIndex()]; + + for (size_t i = 0; i < values_size; ++i) + values_hash_data[i + 1] = hash_data[offsets_data[i]]; + + values->updateWeakHash32(values_hash); + + auto offset_it = begin(); for (size_t i = 0; i < _size; ++i, ++offset_it) { size_t value_index = offset_it.getValueIndex(); - auto data_ref = values->getDataAt(value_index); - hash_data[i] = ::updateWeakHash32(reinterpret_cast(data_ref.data), data_ref.size, hash_data[i]); + hash_data[i] = values_hash_data[value_index]; } } @@ -807,6 +821,24 @@ size_t ColumnSparse::getValueIndex(size_t n) const return it - offsets_data.begin() + 1; } +size_t ColumnSparse::getFirstDefaultValueIndex() const +{ + if (getNumberOfDefaultRows() == 0) + return size(); + + const auto & offsets_data = getOffsetsData(); + size_t off_size = offsets_data.size(); + + if (off_size == 0 || offsets_data[0] > 0) + return 0; + + size_t idx = 0; + while (idx + 1 < off_size && offsets_data[idx] + 1 == offsets_data[idx + 1]) + ++idx; + + return offsets_data[idx] + 1; +} + ColumnSparse::Iterator ColumnSparse::getIterator(size_t n) const { const auto & offsets_data = getOffsetsData(); diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 3e34d1de94a..4860f5171f7 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -173,6 +173,9 @@ public: /// O(log(offsets.size())) complexity, size_t getValueIndex(size_t n) const; + /// Returns an index of the first default value, or size() if there is no defaults. + size_t getFirstDefaultValueIndex() const; + const IColumn & getValuesColumn() const { return *values; } IColumn & getValuesColumn() { return *values; } From 24b9e1885216a62294e68a977437b4d4c62a12f9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 09:50:27 +0000 Subject: [PATCH 055/145] Add a test. --- ...l_window_finctions_and_column_sparce_bug.reference | 8 ++++++++ ...arallel_window_finctions_and_column_sparce_bug.sql | 11 +++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference create mode 100644 tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference new file mode 100644 index 00000000000..f11ec57a425 --- /dev/null +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference @@ -0,0 +1,8 @@ +false 1 1 +true 1 1 +--- +false 1 1 +false 1 2 +false 1 3 +true 1 1 +true 1 2 diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql new file mode 100644 index 00000000000..a4c0200813c --- /dev/null +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql @@ -0,0 +1,11 @@ +create table t(c Int32, d Bool) Engine=MergeTree order by c; +system stop merges t; + +insert into t values (1, 0); +insert into t values (1, 0); +insert into t values (1, 1); +insert into t values (1, 0)(1, 1); + +SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t qualify c8=1 order by d settings max_threads=2; +SELECT '---'; +SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t order by d, c8 settings max_threads=2; From 902e548a2daf087bdb4363694a91bf665a7f7f9b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 09:52:00 +0000 Subject: [PATCH 056/145] Fix typo. --- ...205_parallel_window_finctions_and_column_sparse_bug.reference} | 0 ... => 03205_parallel_window_finctions_and_column_sparse_bug.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03205_parallel_window_finctions_and_column_sparce_bug.reference => 03205_parallel_window_finctions_and_column_sparse_bug.reference} (100%) rename tests/queries/0_stateless/{03205_parallel_window_finctions_and_column_sparce_bug.sql => 03205_parallel_window_finctions_and_column_sparse_bug.sql} (100%) diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference similarity index 100% rename from tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.reference rename to tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql similarity index 100% rename from tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparce_bug.sql rename to tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql From 35725b666f6afdf115aa977fadd15d2076cba020 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 16 Jul 2024 13:23:46 +0000 Subject: [PATCH 057/145] 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 04f3c29b60658938a93aa6de6f15f8c50121e53e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 13:47:38 +0000 Subject: [PATCH 058/145] Fix crash. --- src/Columns/ColumnSparse.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index ea4d23c1678..0922eb5ea2d 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -695,7 +695,7 @@ void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const if (getNumberOfDefaultRows() > 0) values_hash_data[0] = hash_data[getFirstDefaultValueIndex()]; - for (size_t i = 0; i < values_size; ++i) + for (size_t i = 0; i + 1 < values_size; ++i) values_hash_data[i + 1] = hash_data[offsets_data[i]]; values->updateWeakHash32(values_hash); From 8acc5d90ca295c2ff6b574da15dec70268a19015 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 13:47:58 +0000 Subject: [PATCH 059/145] Add more test cases. --- ..._finctions_and_column_sparse_bug.reference | 75 ++++++++++++++++++ ...window_finctions_and_column_sparse_bug.sql | 76 +++++++++++++++++++ 2 files changed, 151 insertions(+) diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference index f11ec57a425..356329a392d 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference @@ -6,3 +6,78 @@ false 1 2 false 1 3 true 1 1 true 1 2 +--- +-755809149 0 +--- +1 -2081147898 +1 -1981899149 +2 -2051538534 +2 -1650266905 +3 -1975508531 +3 -1646738223 +4 -1700730666 +4 -1618912877 +5 -1465484835 +5 -1317193174 +6 -1458338029 +6 -1219769753 +7 -1450619195 +7 -1154269118 +8 -1365934326 +8 -1150980622 +9 -1203382363 +9 -1098155311 +10 -1197430632 +10 -841067875 +11 -1176267855 +11 -816935497 +12 -1020892864 +12 -599948807 +13 -991301833 +13 -526570556 +14 -685902265 +14 -504713125 +15 -653505826 +15 -411038390 +16 -451392958 +16 -331834394 +17 -262516786 +17 -176934810 +18 -222873194 +18 -2 +19 -153185515 +19 6 +20 -74234560 +20 255 +21 -41 +21 406615258 +22 -6 +22 541758331 +23 -5 +23 720985423 +24 -3 +24 745669725 +25 15 +25 897064234 +26 65535 +26 1116921321 +27 77089559 +27 1207796283 +28 100663045 +28 1603772265 +29 561061873 +29 1664059402 +30 643897141 +30 1688303275 +31 914629990 +31 1913361922 +32 1159852204 +32 1929066636 +33 1258218855 +33 1968095908 +34 1459407556 +34 2054878592 +35 1936334332 +35 2125075305 +36 1962246186 +37 2030467062 diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql index a4c0200813c..6e326d0a67f 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql @@ -9,3 +9,79 @@ insert into t values (1, 0)(1, 1); SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t qualify c8=1 order by d settings max_threads=2; SELECT '---'; SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t order by d, c8 settings max_threads=2; +SELECT '---'; + +drop table t; + +create table t ( + c Int32 primary key , + s Bool , + w Float64 + ); + +system stop merges t; + +insert into t values(439499072,true,0),(1393290072,true,0); +insert into t values(-1317193174,false,0),(1929066636,false,0); +insert into t values(-2,false,0),(1962246186,true,0),(2054878592,false,0); +insert into t values(-1893563136,true,41.55); +insert into t values(-1338380855,true,-0.7),(-991301833,true,0),(-755809149,false,43.18),(-41,true,0),(3,false,0),(255,false,0),(255,false,0),(189195893,false,0),(195550885,false,9223372036854776000); + +SELECT * FROM ( +SELECT c, min(w) OVER (PARTITION BY s ORDER BY c ASC, s ASC, w ASC) +FROM t limit toUInt64(-1)) +WHERE c = -755809149; + +SELECT '---'; + +create table t_vkx4cc ( + c_ylzjpt Int32, + c_hqfr9 Bool , + ) engine = MergeTree order by c_ylzjpt; + +system stop merges t_vkx4cc; + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-2081147898, coalesce((NOT NOT(cast( (53 < 539704722) as Nullable(Bool)))), true)), (-1219769753, coalesce((true) and (false), false)), (-1981899149, coalesce(false, false)), (-1646738223, coalesce((NOT NOT(cast( (23.5 <= -26) as Nullable(Bool)))), false)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (255, coalesce(false, false)), (-1317193174, coalesce(false, false)), (-41, coalesce(true, false)), (1929066636, coalesce(false, true)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-1700730666, coalesce((NOT NOT(cast( (-2022515471055597472 AND -29) as Nullable(Bool)))), false)), (1664059402, coalesce((NOT NOT(cast( (-19643 >= -122) as Nullable(Bool)))), false)), (1688303275, coalesce((NOT NOT(cast( (737275892 < 105) as Nullable(Bool)))), true)), (406615258, coalesce((NOT NOT(cast( (-657730213 = 82.86) as Nullable(Bool)))), false)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-2, coalesce(false, false)), (1962246186, coalesce(true, false)), (-991301833, coalesce(true, true)), (2054878592, coalesce(false, false)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (643897141, coalesce((NOT NOT(cast( (-60 AND cast(null as Nullable(Int64))) as Nullable(Bool)))), true)), (-2051538534, coalesce(((-1616816511 between 332225780 and -1883087387)) or ((-573375170 between -1427445977 and 615586748)), false)), (77089559, coalesce((NOT NOT(cast( ((true) and (true) != 925456787) as Nullable(Bool)))), false)), (1116921321, coalesce((0 is NULL), true)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-816935497, coalesce(false, false)), (1207796283, coalesce((-129 between -5 and -5), false)), (-1365934326, coalesce(true, false)), (-1618912877, coalesce((NOT NOT(cast( (false >= 31833) as Nullable(Bool)))), false)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-331834394, coalesce((NOT NOT(cast( (-63 <= -1822810052) as Nullable(Bool)))), true)), (-1020892864, coalesce((NOT NOT(cast( (40.31 <= 8146037365746019777) as Nullable(Bool)))), true)), (-1150980622, coalesce(((94019304 between -730556489 and 32)) and ((-956354236 is not NULL)), true)), (-1203382363, coalesce(true, true)); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-653505826, coalesce((true) or (true), false)), (-1975508531, coalesce(((-796885845 between 65536 and cast(null as Nullable(Int32)))) or ((NOT NOT(cast( (-7467729336434250795 < 100.20) as Nullable(Bool))))), false)), (-1465484835, coalesce(((NOT NOT(cast( (19209 <= 75.96) as Nullable(Bool))))) or (true), false)), (1968095908, coalesce((NOT NOT(cast( (-1309960412156062327 > 13102) as Nullable(Bool)))), true)); + +alter table t_vkx4cc add column c_zosphq2t1 Float64; + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-153185515, coalesce((NOT NOT(cast( (1291639145 >= 30.22) as Nullable(Bool)))), false), -1.8), (-411038390, coalesce(((-762326135 between 16 and 177530758)) or (false), true), 26.34), (914629990, coalesce((-1125832977 is not NULL), true), 59.2), (541758331, coalesce(false, true), -255.1); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (2125075305, coalesce(false, false), 55.36), (-1176267855, coalesce(true, true), 55.45), (1459407556, coalesce((true) and ((NOT NOT(cast( (95.96 != 65) as Nullable(Bool))))), true), 85.80), (-1098155311, coalesce(false, false), 2147483649.9); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (6, coalesce((NOT NOT(cast( (1546334968 < -4) as Nullable(Bool)))), true), 57.42), (-5, coalesce((NOT NOT(cast( (59 AND 13) as Nullable(Bool)))), false), 65536.3), (100663045, coalesce((-1190355242 is not NULL), true), 73.80), (-451392958, coalesce((NOT NOT(cast( (false != -443845933) as Nullable(Bool)))), false), -4294967294.0); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (561061873, coalesce(true, false), 12.17), (-526570556, coalesce(false, false), 64.73), (-1450619195, coalesce(true, true), 54.33), (-3, coalesce(true, true), 52.9); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-504713125, coalesce(false, true), 27.58), (897064234, coalesce((836516994 between cast(null as Nullable(Int32)) and -1832647080), true), 9223372036854775809.2), (65535, coalesce(true, true), 4294967297.5), (-599948807, coalesce((false) or ((NOT NOT(cast( (6.52 = 65.49) as Nullable(Bool))))), false), 256.5); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-1650266905, coalesce((NOT NOT(cast( (-83 = -218055084) as Nullable(Bool)))), true), 1.9), (-841067875, coalesce(false, true), -126.5), (15, coalesce(((NOT NOT(cast( (cast(null as Nullable(Decimal)) = cast(null as Nullable(Int32))) as Nullable(Bool))))) or (true), true), 33.65), (1913361922, coalesce((NOT NOT(cast( (false AND 0) as Nullable(Bool)))), false), 6.4); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (1159852204, coalesce((-2057115045 is not NULL), false), 20.61), (-6, coalesce(true, true), 66.33), (-1154269118, coalesce(false, true), 8.89), (1258218855, coalesce(true, false), 19.80); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (1603772265, coalesce(false, true), 57.87), (-176934810, coalesce(false, true), 128.8), (-1458338029, coalesce((NOT NOT(cast( (20908 != (NOT NOT(cast( (cast(null as Nullable(Decimal)) <= (true) or ((NOT NOT(cast( (973511022 <= -112) as Nullable(Bool)))))) as Nullable(Bool))))) as Nullable(Bool)))), true), 76.54), (-262516786, coalesce((cast(null as Nullable(Int32)) is NULL), false), 21.49); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-1197430632, coalesce(true, false), 45.40), (-685902265, coalesce((NOT NOT(cast( (cast(null as Nullable(Decimal)) < cast(null as Nullable(Decimal))) as Nullable(Bool)))), true), 5.55), (1936334332, coalesce((-1565552735 is not NULL), false), 26.28), (2030467062, coalesce((NOT NOT(cast( (127.3 != cast(null as Nullable(Int32))) as Nullable(Bool)))), true), 89.50); + +insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (720985423, coalesce((NOT NOT(cast( (-451448940 = cast(null as Nullable(Decimal))) as Nullable(Bool)))), false), 52.65), (-222873194, coalesce(((-20 between -1419620477 and 1616455043)) or ((25624502 between 1312431316 and 1757361651)), false), 127.2), (745669725, coalesce((NOT NOT(cast( ((NOT NOT(cast( (cast(null as Nullable(UInt64)) <= 42) as Nullable(Bool)))) >= 3233811255032796928) as Nullable(Bool)))), false), 7.74), (-74234560, coalesce((NOT NOT(cast( (cast(null as Nullable(Decimal)) >= cast(null as Nullable(Decimal))) as Nullable(Bool)))), true), 19.25); + +SELECT DISTINCT + count(ref_0.c_zosphq2t1) over (partition by ref_0.c_hqfr9 order by ref_0.c_ylzjpt, ref_0.c_hqfr9, ref_0.c_zosphq2t1) as c0, + ref_0.c_ylzjpt as c1 +FROM + t_vkx4cc as ref_0 + order by c0, c1; From 4dea89df763fb9504cb681379d46830f4ec98db3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Jul 2024 13:50:58 +0000 Subject: [PATCH 060/145] Cleanup. --- src/Columns/ColumnSparse.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0922eb5ea2d..98a66e87387 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include From e29d78e4cfbdd66bf4b5ed754f607267a57a9134 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 16 Jul 2024 14:43:08 +0000 Subject: [PATCH 061/145] Add option for validating the Primary key type in Dictionaries --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Databases/DatabaseDictionary.cpp | 2 +- src/Dictionaries/DictionaryStructure.cpp | 12 +----- src/Dictionaries/DictionaryStructure.h | 10 +---- .../getDictionaryConfigurationFromAST.cpp | 9 ++++ src/Storages/StorageDictionary.cpp | 13 ++++-- src/Storages/StorageDictionary.h | 2 +- .../TableFunctionDictionary.cpp | 2 +- ...ionary_validate_primary_key_type.reference | 4 ++ ...8_dictionary_validate_primary_key_type.sql | 41 +++++++++++++++++++ 11 files changed, 71 insertions(+), 26 deletions(-) create mode 100644 tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.reference create mode 100644 tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d84e5b149f6..c0bc1489b31 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -148,6 +148,7 @@ class IColumn; M(UInt64, max_local_write_bandwidth, 0, "The maximum speed of local writes in bytes per second.", 0) \ M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams, and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \ M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \ + M(Bool, dictionary_validate_id_type, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64.", 0) \ \ M(Bool, distributed_insert_skip_read_only_replicas, false, "If true, INSERT into Distributed will skip read-only replicas.", 0) \ M(Bool, distributed_foreground_insert, false, "If setting is enabled, insert query into distributed waits until data are sent to all nodes in a cluster. \n\nEnables or disables synchronous data insertion into a `Distributed` table.\n\nBy default, when inserting data into a Distributed table, the ClickHouse server sends data to cluster nodes in the background. When `distributed_foreground_insert` = 1, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).", 0) ALIAS(insert_distributed_sync) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b0725340f46..0b3082529aa 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,7 @@ static std::initializer_list tryGetAttributeUnderlyingType(TypeIndex i } - -DictionarySpecialAttribute::DictionarySpecialAttribute(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) - : name{config.getString(config_prefix + ".name", "")}, expression{config.getString(config_prefix + ".expression", "")} -{ - if (name.empty() && !expression.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty", config_prefix); -} - - DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { std::string structure_prefix = config_prefix + ".structure"; @@ -79,7 +70,8 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration if (has_id) { - id.emplace(config, structure_prefix + ".id"); + static constexpr auto id_default_type = "UInt64"; + id.emplace(makeDictionaryTypedSpecialAttribute(config, structure_prefix + ".id", id_default_type)); } else if (has_key) { diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index 55060b1592f..0d44b696d74 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -89,14 +89,6 @@ constexpr void callOnDictionaryAttributeType(AttributeUnderlyingType type, F && }); } -struct DictionarySpecialAttribute final -{ - const std::string name; - const std::string expression; - - DictionarySpecialAttribute(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix); -}; - struct DictionaryTypedSpecialAttribute final { const std::string name; @@ -108,7 +100,7 @@ struct DictionaryTypedSpecialAttribute final /// Name of identifier plus list of attributes struct DictionaryStructure final { - std::optional id; + std::optional id; std::optional> key; std::vector attributes; std::unordered_map attribute_name_to_index; diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 9ee2027afc7..4ec2e1f5260 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -382,6 +382,15 @@ void buildPrimaryKeyConfiguration( name_element->appendChild(name); buildAttributeExpressionIfNeeded(doc, id_element, dict_attr); + + if (!dict_attr->type) + return; + + AutoPtr type_element(doc->createElement("type")); + id_element->appendChild(type_element); + + AutoPtr type(doc->createTextNode(queryToString(dict_attr->type))); + type_element->appendChild(type); } else { diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 447fd87cdc9..69af73122ba 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -25,13 +25,14 @@ namespace ErrorCodes extern const int CANNOT_DETACH_DICTIONARY_AS_TABLE; extern const int DICTIONARY_ALREADY_EXISTS; extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; } namespace { void checkNamesAndTypesCompatibleWithDictionary(const String & dictionary_name, const ColumnsDescription & columns, const DictionaryStructure & dictionary_structure) { - auto dictionary_names_and_types = StorageDictionary::getNamesAndTypes(dictionary_structure); + auto dictionary_names_and_types = StorageDictionary::getNamesAndTypes(dictionary_structure, false); std::set names_and_types_set(dictionary_names_and_types.begin(), dictionary_names_and_types.end()); for (const auto & column : columns.getOrdinary()) @@ -47,13 +48,17 @@ namespace } -NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure & dictionary_structure) +NamesAndTypesList StorageDictionary::getNamesAndTypes(const DictionaryStructure & dictionary_structure, bool validate_id_type) { NamesAndTypesList dictionary_names_and_types; if (dictionary_structure.id) - dictionary_names_and_types.emplace_back(dictionary_structure.id->name, std::make_shared()); + { + if (validate_id_type && dictionary_structure.id->type->getTypeId() != TypeIndex::UInt64) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incorrect type of ID column: must be UInt64, but it is {}", dictionary_structure.id->type->getFamilyName()); + dictionary_names_and_types.emplace_back(dictionary_structure.id->name, std::make_shared()); + } /// In old-style (XML) configuration we don't have this attributes in the /// main attribute list, so we have to add them to columns list explicitly. /// In the new configuration (DDL) we have them both in range_* nodes and @@ -105,7 +110,7 @@ StorageDictionary::StorageDictionary( Location location_, ContextPtr context_) : StorageDictionary( - table_id_, dictionary_name_, ColumnsDescription{getNamesAndTypes(dictionary_structure_)}, comment, location_, context_) + table_id_, dictionary_name_, ColumnsDescription{getNamesAndTypes(dictionary_structure_, context_->getSettingsRef().dictionary_validate_id_type)}, comment, location_, context_) { } diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 17e4efda2cd..44a274cf97c 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -80,7 +80,7 @@ public: std::shared_ptr getDictionary() const; - static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure); + static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure, bool validate_id_type); bool isDictionary() const override { return true; } void shutdown(bool is_drop) override; diff --git a/src/TableFunctions/TableFunctionDictionary.cpp b/src/TableFunctions/TableFunctionDictionary.cpp index 867fbf5b11e..9e0491975ae 100644 --- a/src/TableFunctions/TableFunctionDictionary.cpp +++ b/src/TableFunctions/TableFunctionDictionary.cpp @@ -74,7 +74,7 @@ ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr c /// otherwise, we get table structure by dictionary structure. auto dictionary_structure = external_loader.getDictionaryStructure(dictionary_name, context); - return ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure)); + return ColumnsDescription(StorageDictionary::getNamesAndTypes(dictionary_structure, false)); } StoragePtr TableFunctionDictionary::executeImpl( diff --git a/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.reference b/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.reference new file mode 100644 index 00000000000..c7f2781aed9 --- /dev/null +++ b/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.reference @@ -0,0 +1,4 @@ +n1 UInt64 +n2 UInt32 +n1 UInt64 +n2 UInt32 diff --git a/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.sql b/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.sql new file mode 100644 index 00000000000..eaf51ef7ead --- /dev/null +++ b/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.sql @@ -0,0 +1,41 @@ +CREATE DICTIONARY `test_dictionary0` ( + `n1` String, + `n2` UInt32 +) +PRIMARY KEY n1 +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DB 'test_db' TABLE 'table_01' USER 'default')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +SET dictionary_validate_id_type=1; + +CREATE DICTIONARY `test_dictionary1` ( + `n1` String, + `n2` UInt32 +) +PRIMARY KEY n1 +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DB 'test_db' TABLE 'table_01' USER 'default')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); -- { serverError 36 } + +CREATE DICTIONARY `test_dictionary2` ( + `n1` UInt32, + `n2` UInt32 +) +PRIMARY KEY n1 +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DB 'test_db' TABLE 'table_01' USER 'default')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); -- { serverError 36 } + +CREATE DICTIONARY `test_dictionary3` ( + `n1` UInt64, + `n2` UInt32 +) +PRIMARY KEY n1 +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DB 'test_db' TABLE 'table_01' USER 'default')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +DESCRIBE `test_dictionary0`; +DESCRIBE `test_dictionary3`; + From 03381bdf22c93b7762f8ef1b63a6148b31599323 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Tue, 16 Jul 2024 16:44:04 +0000 Subject: [PATCH 062/145] Build fix --- src/Storages/StorageDictionary.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 69af73122ba..09ca53d7582 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include From 40b4a3de6dfddcb3ef206730334821183df06963 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 19:19:51 +0200 Subject: [PATCH 063/145] Update run.sh --- docker/test/fasttest/run.sh | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 58c436b469b..970bf12a81a 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -11,10 +11,6 @@ stage=${stage:-} # Compiler version, normally set by Dockerfile export LLVM_VERSION=${LLVM_VERSION:-18} -# For some reason it is needed for tests with 'expect' but I don't know, why -groupadd --system --gid 1000 clickhouse -useradd --system --gid 1000 --uid 1000 clickhouse - # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about # it being undefined. Also read it as array so that we can pass an empty list From 68526487ed51119a638af4554557fe813aa45bf7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 16 Jul 2024 19:35:24 +0200 Subject: [PATCH 064/145] fix --- src/Common/SignalHandlers.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/SignalHandlers.cpp b/src/Common/SignalHandlers.cpp index 97b890469b9..a4b7784df5c 100644 --- a/src/Common/SignalHandlers.cpp +++ b/src/Common/SignalHandlers.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #pragma clang diagnostic ignored "-Wreserved-identifier" From 187a9a623c2f23978d0b1161d6950a9cf7d9d8ad Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 06:56:32 +0200 Subject: [PATCH 065/145] Remove `is_deterministic` from system.functions --- .../System/StorageSystemFunctions.cpp | 67 ++++++------------- .../02117_show_create_table_system.reference | 1 - ...ckhouse_local_interactive_exception.expect | 2 +- .../02815_no_throw_in_simple_queries.sh | 7 +- 4 files changed, 21 insertions(+), 56 deletions(-) diff --git a/src/Storages/System/StorageSystemFunctions.cpp b/src/Storages/System/StorageSystemFunctions.cpp index 6e4ac8b2747..0a233dc052b 100644 --- a/src/Storages/System/StorageSystemFunctions.cpp +++ b/src/Storages/System/StorageSystemFunctions.cpp @@ -40,7 +40,6 @@ namespace MutableColumns & res_columns, const String & name, UInt64 is_aggregate, - std::optional is_deterministic, const String & create_query, FunctionOrigin function_origin, const Factory & factory) @@ -48,58 +47,53 @@ namespace res_columns[0]->insert(name); res_columns[1]->insert(is_aggregate); - if (!is_deterministic.has_value()) - res_columns[2]->insertDefault(); - else - res_columns[2]->insert(*is_deterministic); - if constexpr (std::is_same_v || std::is_same_v) { - res_columns[3]->insert(false); - res_columns[4]->insertDefault(); + res_columns[2]->insert(false); + res_columns[3]->insertDefault(); } else { - res_columns[3]->insert(factory.isCaseInsensitive(name)); + res_columns[2]->insert(factory.isCaseInsensitive(name)); if (factory.isAlias(name)) - res_columns[4]->insert(factory.aliasTo(name)); + res_columns[3]->insert(factory.aliasTo(name)); else - res_columns[4]->insertDefault(); + res_columns[3]->insertDefault(); } - res_columns[5]->insert(create_query); - res_columns[6]->insert(static_cast(function_origin)); + res_columns[4]->insert(create_query); + res_columns[5]->insert(static_cast(function_origin)); if constexpr (std::is_same_v) { if (factory.isAlias(name)) { + res_columns[6]->insertDefault(); res_columns[7]->insertDefault(); res_columns[8]->insertDefault(); res_columns[9]->insertDefault(); res_columns[10]->insertDefault(); res_columns[11]->insertDefault(); - res_columns[12]->insertDefault(); } else { auto documentation = factory.getDocumentation(name); - res_columns[7]->insert(documentation.description); - res_columns[8]->insert(documentation.syntax); - res_columns[9]->insert(documentation.argumentsAsString()); - res_columns[10]->insert(documentation.returned_value); - res_columns[11]->insert(documentation.examplesAsString()); - res_columns[12]->insert(documentation.categoriesAsString()); + res_columns[6]->insert(documentation.description); + res_columns[7]->insert(documentation.syntax); + res_columns[8]->insert(documentation.argumentsAsString()); + res_columns[9]->insert(documentation.returned_value); + res_columns[10]->insert(documentation.examplesAsString()); + res_columns[11]->insert(documentation.categoriesAsString()); } } else { + res_columns[6]->insertDefault(); res_columns[7]->insertDefault(); res_columns[8]->insertDefault(); res_columns[9]->insertDefault(); res_columns[10]->insertDefault(); res_columns[11]->insertDefault(); - res_columns[12]->insertDefault(); } } } @@ -120,7 +114,6 @@ ColumnsDescription StorageSystemFunctions::getColumnsDescription() { {"name", std::make_shared(), "The name of the function."}, {"is_aggregate", std::make_shared(), "Whether the function is an aggregate function."}, - {"is_deterministic", std::make_shared(std::make_shared()), "Whether the function is deterministic."}, {"case_insensitive", std::make_shared(), "Whether the function name can be used case-insensitively."}, {"alias_to", std::make_shared(), "The original function name, if the function name is an alias."}, {"create_query", std::make_shared(), "Obsolete."}, @@ -140,36 +133,14 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c const auto & function_names = functions_factory.getAllRegisteredNames(); for (const auto & function_name : function_names) { - std::optional is_deterministic; - try - { - DO_NOT_UPDATE_ERROR_STATISTICS(); - is_deterministic = functions_factory.tryGet(function_name, context)->isDeterministic(); - } - catch (const Exception & e) - { - /// Some functions throw because they need special configuration or setup before use. - if (e.code() == ErrorCodes::DICTIONARIES_WAS_NOT_LOADED - || e.code() == ErrorCodes::FUNCTION_NOT_ALLOWED - || e.code() == ErrorCodes::NOT_IMPLEMENTED - || e.code() == ErrorCodes::SUPPORT_IS_DISABLED - || e.code() == ErrorCodes::ACCESS_DENIED - || e.code() == ErrorCodes::DEPRECATED_FUNCTION) - { - /// Ignore exception, show is_deterministic = NULL. - } - else - throw; - } - - fillRow(res_columns, function_name, 0, is_deterministic, "", FunctionOrigin::SYSTEM, functions_factory); + fillRow(res_columns, function_name, 0, "", FunctionOrigin::SYSTEM, functions_factory); } const auto & aggregate_functions_factory = AggregateFunctionFactory::instance(); const auto & aggregate_function_names = aggregate_functions_factory.getAllRegisteredNames(); for (const auto & function_name : aggregate_function_names) { - fillRow(res_columns, function_name, 1, {1}, "", FunctionOrigin::SYSTEM, aggregate_functions_factory); + fillRow(res_columns, function_name, 1, "", FunctionOrigin::SYSTEM, aggregate_functions_factory); } const auto & user_defined_sql_functions_factory = UserDefinedSQLFunctionFactory::instance(); @@ -177,14 +148,14 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c for (const auto & function_name : user_defined_sql_functions_names) { auto create_query = queryToString(user_defined_sql_functions_factory.get(function_name)); - fillRow(res_columns, function_name, 0, {0}, create_query, FunctionOrigin::SQL_USER_DEFINED, user_defined_sql_functions_factory); + fillRow(res_columns, function_name, 0, create_query, FunctionOrigin::SQL_USER_DEFINED, user_defined_sql_functions_factory); } const auto & user_defined_executable_functions_factory = UserDefinedExecutableFunctionFactory::instance(); const auto & user_defined_executable_functions_names = user_defined_executable_functions_factory.getRegisteredNames(context); /// NOLINT(readability-static-accessed-through-instance) for (const auto & function_name : user_defined_executable_functions_names) { - fillRow(res_columns, function_name, 0, {0}, "", FunctionOrigin::EXECUTABLE_USER_DEFINED, user_defined_executable_functions_factory); + fillRow(res_columns, function_name, 0, "", FunctionOrigin::EXECUTABLE_USER_DEFINED, user_defined_executable_functions_factory); } } diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 8f62eda9233..cfae4fee6c2 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -286,7 +286,6 @@ CREATE TABLE system.functions ( `name` String, `is_aggregate` UInt8, - `is_deterministic` Nullable(UInt8), `case_insensitive` UInt8, `alias_to` String, `create_query` String, diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect index add977c4fce..76902bdc69d 100755 --- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect @@ -9,7 +9,7 @@ if {[info exists env(CLICKHOUSE_TMP)]} { } exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 -log_user 0 +log_user 1 set timeout 20 match_max 100000 diff --git a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh index 50f5e565088..68c55f9b66a 100755 --- a/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh +++ b/tests/queries/0_stateless/02815_no_throw_in_simple_queries.sh @@ -45,9 +45,4 @@ expect eof EOF } -run "$CLICKHOUSE_LOCAL --disable_suggestion" -# Suggestions are off because the suggestion feature initializes itself by reading all available function -# names from "system.functions". Getting the value for field "is_obsolete" occasionally throws (e.g. for -# certain dictionary functions when dictionaries are not set up yet). Exceptions are properly handled, but -# they exist for a short time. This, in combination with CLICKHOUSE_TERMINATE_ON_ANY_EXCEPTION, terminates -# clickhouse-local and clickhouse-client when run in interactive mode *with* suggestions. +run "$CLICKHOUSE_LOCAL" From 855f03d1d3c5b34d3de1b0919b9f7975d64b3cb8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 06:57:16 +0200 Subject: [PATCH 066/145] Remove `is_deterministic` from system.functions --- docs/en/operations/system-tables/functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/operations/system-tables/functions.md b/docs/en/operations/system-tables/functions.md index d52bf24f289..da0ef04bd27 100644 --- a/docs/en/operations/system-tables/functions.md +++ b/docs/en/operations/system-tables/functions.md @@ -9,7 +9,6 @@ Columns: - `name` ([String](../../sql-reference/data-types/string.md)) – The name of the function. - `is_aggregate` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Whether the function is an aggregate function. -- `is_deterministic` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md))) - Whether the function is deterministic. - `case_insensitive`, ([UInt8](../../sql-reference/data-types/int-uint.md)) - Whether the function name can be used case-insensitively. - `alias_to`, ([String](../../sql-reference/data-types/string.md)) - The original function name, if the function name is an alias. - `create_query`, ([String](../../sql-reference/data-types/enum.md)) - Unused. From 4a69bd78819c1bb97988fce96d17be99bbffe00e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 07:37:45 +0200 Subject: [PATCH 067/145] Fix terrible test @arthurpassos --- ...tiple_batches_array_inconsistent_offsets.sh | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh index 83196458a84..c96531ffea9 100755 --- a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh +++ b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-ubsan, no-fasttest +# Tags: long, no-ubsan, no-fasttest, no-parallel, no-asan, no-msan, no-tsan +# This test requires around 10 GB of memory and it is just too much. CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -121,9 +122,12 @@ echo "Parquet" #} DATA_FILE=$CUR_DIR/data_parquet/string_int_list_inconsistent_offset_multiple_batches.parquet -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (ints Array(Int64), strings Nullable(String)) ENGINE = Memory" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load FORMAT Parquet" -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load" | md5sum -${CLICKHOUSE_CLIENT} --query="SELECT count() FROM parquet_load" -${CLICKHOUSE_CLIENT} --query="drop table parquet_load" \ No newline at end of file + +${CLICKHOUSE_LOCAL} --multiquery " +DROP TABLE IF EXISTS parquet_load; +CREATE TABLE parquet_load (ints Array(Int64), strings Nullable(String)) ENGINE = Memory; +INSERT INTO parquet_load FROM INFILE '$DATA_FILE'; +SELECT sum(cityHash64(*)) FROM parquet_load; +SELECT count() FROM parquet_load; +DROP TABLE parquet_load; +" From 73ecbefd4123bae9ca9902111c28ace0cc05b994 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 07:52:33 +0200 Subject: [PATCH 068/145] 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 88f9a87ce6960c97990b801d8c6f99e614f329a8 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 17 Jul 2024 08:43:38 +0200 Subject: [PATCH 069/145] unused error codes --- src/Storages/System/StorageSystemFunctions.cpp | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/Storages/System/StorageSystemFunctions.cpp b/src/Storages/System/StorageSystemFunctions.cpp index 0a233dc052b..f10ce9e3987 100644 --- a/src/Storages/System/StorageSystemFunctions.cpp +++ b/src/Storages/System/StorageSystemFunctions.cpp @@ -16,16 +16,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int DICTIONARIES_WAS_NOT_LOADED; - extern const int FUNCTION_NOT_ALLOWED; - extern const int NOT_IMPLEMENTED; - extern const int SUPPORT_IS_DISABLED; - extern const int ACCESS_DENIED; - extern const int DEPRECATED_FUNCTION; -}; - enum class FunctionOrigin : int8_t { SYSTEM = 0, From b4755b36de41e493e121f5846a71ede11d5155d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Tue, 16 Jul 2024 15:22:41 +0000 Subject: [PATCH 070/145] Fix dropping named collection in local storage --- .../NamedCollectionsMetadataStorage.cpp | 54 +++++++++---------- 1 file changed, 27 insertions(+), 27 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index 3c6561fa51e..79b674965de 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -116,32 +116,32 @@ public: return elements; } - bool exists(const std::string & path) const override + bool exists(const std::string & file_name) const override { - return fs::exists(getPath(path)); + return fs::exists(getPath(file_name)); } - std::string read(const std::string & path) const override + std::string read(const std::string & file_name) const override { - ReadBufferFromFile in(getPath(path)); + ReadBufferFromFile in(getPath(file_name)); std::string data; readStringUntilEOF(data, in); return data; } - void write(const std::string & path, const std::string & data, bool replace) override + void write(const std::string & file_name, const std::string & data, bool replace) override { - if (!replace && fs::exists(path)) + if (!replace && fs::exists(file_name)) { throw Exception( ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, "Metadata file {} for named collection already exists", - path); + file_name); } fs::create_directories(root_path); - auto tmp_path = getPath(path + ".tmp"); + auto tmp_path = getPath(file_name + ".tmp"); WriteBufferFromFile out(tmp_path, data.size(), O_WRONLY | O_CREAT | O_EXCL); writeString(data, out); @@ -150,22 +150,22 @@ public: out.sync(); out.close(); - fs::rename(tmp_path, getPath(path)); + fs::rename(tmp_path, getPath(file_name)); } - void remove(const std::string & path) override + void remove(const std::string & file_name) override { - if (!removeIfExists(getPath(path))) + if (!removeIfExists(file_name)) { throw Exception( ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, - "Cannot remove `{}`, because it doesn't exist", path); + "Cannot remove `{}`, because it doesn't exist", file_name); } } - bool removeIfExists(const std::string & path) override + bool removeIfExists(const std::string & file_name) override { - return fs::remove(getPath(path)); + return fs::remove(getPath(file_name)); } private: @@ -264,49 +264,49 @@ public: return children; } - bool exists(const std::string & path) const override + bool exists(const std::string & file_name) const override { - return getClient()->exists(getPath(path)); + return getClient()->exists(getPath(file_name)); } - std::string read(const std::string & path) const override + std::string read(const std::string & file_name) const override { - return getClient()->get(getPath(path)); + return getClient()->get(getPath(file_name)); } - void write(const std::string & path, const std::string & data, bool replace) override + void write(const std::string & file_name, const std::string & data, bool replace) override { if (replace) { - getClient()->createOrUpdate(getPath(path), data, zkutil::CreateMode::Persistent); + getClient()->createOrUpdate(getPath(file_name), data, zkutil::CreateMode::Persistent); } else { - auto code = getClient()->tryCreate(getPath(path), data, zkutil::CreateMode::Persistent); + auto code = getClient()->tryCreate(getPath(file_name), data, zkutil::CreateMode::Persistent); if (code == Coordination::Error::ZNODEEXISTS) { throw Exception( ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, "Metadata file {} for named collection already exists", - path); + file_name); } } } - void remove(const std::string & path) override + void remove(const std::string & file_name) override { - getClient()->remove(getPath(path)); + getClient()->remove(getPath(file_name)); } - bool removeIfExists(const std::string & path) override + bool removeIfExists(const std::string & file_name) override { - auto code = getClient()->tryRemove(getPath(path)); + auto code = getClient()->tryRemove(getPath(file_name)); if (code == Coordination::Error::ZOK) return true; if (code == Coordination::Error::ZNONODE) return false; - throw Coordination::Exception::fromPath(code, getPath(path)); + throw Coordination::Exception::fromPath(code, getPath(file_name)); } private: From b8d6c68f5fdc691f18000878203971689d2f812c Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 17 Jul 2024 10:30:03 +0200 Subject: [PATCH 071/145] Update 02874_parquet_multiple_batches_array_inconsistent_offsets.reference --- ...arquet_multiple_batches_array_inconsistent_offsets.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference index ba63f2f7e9c..a9eae234dba 100644 --- a/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference +++ b/tests/queries/0_stateless/02874_parquet_multiple_batches_array_inconsistent_offsets.reference @@ -1,3 +1,3 @@ Parquet -e76a749f346078a6a43e0cbd25f0d18a - +3249508141921544766 400 From f1350b442c79854de25a06efca66fc888225f93b Mon Sep 17 00:00:00 2001 From: cangyin Date: Wed, 17 Jul 2024 17:30:27 +0800 Subject: [PATCH 072/145] 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 67852402aa2eb0b57e1f89ff78ab4a831cbf33c6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 17 Jul 2024 10:30:56 +0000 Subject: [PATCH 073/145] Better (perhaps, if the stars align) --- .../InterpreterShowIndexesQuery.cpp | 37 +++++++++++++++---- 1 file changed, 29 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index c6b5d5ac727..31f0404e123 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -33,12 +33,33 @@ String InterpreterShowIndexesQuery::getRewrittenQuery() String rewritten_query = fmt::format(R"( SELECT * FROM ( - (SELECT + (WITH + t1 AS ( + SELECT + name, + arrayJoin(splitByString(', ', primary_key)) AS pk_col + FROM + system.tables + WHERE + database = '{0}' + AND name = '{1}' + ), + t2 AS ( + SELECT + name, + pk_col, + row_number() OVER (ORDER BY 1) AS row_num + FROM + t1 + ) + SELECT name AS table, 1 AS non_unique, 'PRIMARY' AS key_name, - row_number() over (order by database) AS seq_in_index, - arrayJoin(splitByString(', ', primary_key)) AS column_name, + -- row_number() over (order by database) AS seq_in_index, + row_num AS seq_in_index, + -- arrayJoin(splitByString(', ', primary_key)) AS column_name, + pk_col, 'A' AS collation, 0 AS cardinality, NULL AS sub_part, @@ -49,10 +70,9 @@ FROM ( '' AS index_comment, 'YES' AS visible, '' AS expression - FROM system.tables - WHERE - database = '{0}' - AND name = '{1}') + FROM + t2 + ) UNION ALL ( SELECT table AS table, @@ -70,7 +90,8 @@ FROM ( '' AS index_comment, 'YES' AS visible, expr AS expression - FROM system.data_skipping_indices + FROM + system.data_skipping_indices WHERE database = '{0}' AND table = '{1}')) From 8a81946033eac461c155f388fa1595f8e42db794 Mon Sep 17 00:00:00 2001 From: MikhailBurdukov Date: Wed, 17 Jul 2024 10:52:01 +0000 Subject: [PATCH 074/145] Rename opt --- src/Core/Settings.h | 3 +-- src/Core/SettingsChangesHistory.cpp | 2 +- src/Storages/StorageDictionary.cpp | 2 +- .../0_stateless/03198_dictionary_validate_primary_key_type.sql | 2 +- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c116c888f28..9d2e3606e67 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -151,8 +151,7 @@ class IColumn; M(UInt64, max_local_write_bandwidth, 0, "The maximum speed of local writes in bytes per second.", 0) \ M(Bool, stream_like_engine_allow_direct_select, false, "Allow direct SELECT query for Kafka, RabbitMQ, FileLog, Redis Streams, and NATS engines. In case there are attached materialized views, SELECT query is not allowed even if this setting is enabled.", 0) \ M(String, stream_like_engine_insert_queue, "", "When stream like engine reads from multiple queues, user will need to select one queue to insert into when writing. Used by Redis Streams and NATS.", 0) \ - M(Bool, dictionary_validate_id_type, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64.", 0) \ - \ + M(Bool, dictionary_validate_primary_key_type, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64.", 0) \ M(Bool, distributed_insert_skip_read_only_replicas, false, "If true, INSERT into Distributed will skip read-only replicas.", 0) \ M(Bool, distributed_foreground_insert, false, "If setting is enabled, insert query into distributed waits until data are sent to all nodes in a cluster. \n\nEnables or disables synchronous data insertion into a `Distributed` table.\n\nBy default, when inserting data into a Distributed table, the ClickHouse server sends data to cluster nodes in the background. When `distributed_foreground_insert` = 1, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true).", 0) ALIAS(insert_distributed_sync) \ M(UInt64, distributed_background_insert_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) ALIAS(insert_distributed_timeout) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 4e9e402f1d4..b2669a0e1c5 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -67,7 +67,7 @@ static std::initializer_listgetSettingsRef().dictionary_validate_id_type)}, comment, location_, context_) + table_id_, dictionary_name_, ColumnsDescription{getNamesAndTypes(dictionary_structure_, context_->getSettingsRef().dictionary_validate_primary_key_type)}, comment, location_, context_) { } diff --git a/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.sql b/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.sql index eaf51ef7ead..e7b8730abbd 100644 --- a/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.sql +++ b/tests/queries/0_stateless/03198_dictionary_validate_primary_key_type.sql @@ -7,7 +7,7 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 DB 'test_db' TABLE 'table_01' USER LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SET dictionary_validate_id_type=1; +SET dictionary_validate_primary_key_type=1; CREATE DICTIONARY `test_dictionary1` ( `n1` String, From 7356e809aba191cdcff1785032be7c0e44f9eea3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 17 Jul 2024 11:32:02 +0000 Subject: [PATCH 075/145] Ensure that `getPath` is not called with an absolute path --- .../NamedCollectionsMetadataStorage.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index 79b674965de..189eb58196d 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -169,9 +169,13 @@ public: } private: - std::string getPath(const std::string & path) const + std::string getPath(const std::string & file_name) const { - return fs::path(root_path) / path; + const auto file_name_as_path = fs::path(file_name); + if (file_name_as_path.is_absolute()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Filename {} cannot be an absolute path!", file_name); + + return fs::path(root_path) / file_name_as_path; } /// Delete .tmp files. They could be left undeleted in case of @@ -320,9 +324,13 @@ private: return zookeeper_client; } - std::string getPath(const std::string & path) const + std::string getPath(const std::string & file_name) const { - return fs::path(root_path) / path; + const auto file_name_as_path = fs::path(file_name); + if (file_name_as_path.is_absolute()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Filename {} cannot be an absolute path!", file_name); + + return fs::path(root_path) / file_name_as_path; } }; From e4816e5c2283dd6254e9d36991b60ad58c2c2dc0 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 17 Jul 2024 13:42:32 +0200 Subject: [PATCH 076/145] 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 8211d8eabfed2fee115ec9d60b0692918547cd32 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Wed, 17 Jul 2024 12:04:37 +0000 Subject: [PATCH 077/145] fix data race --- src/IO/S3/Client.cpp | 11 +++++++++++ src/IO/S3/Client.h | 11 ++++------- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 3b958dea046..b86e181c369 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -825,6 +825,17 @@ void Client::updateURIForBucket(const std::string & bucket, S3::URI new_uri) con cache->uri_for_bucket_cache.emplace(bucket, std::move(new_uri)); } +ClientCache::ClientCache(const ClientCache & other) +{ + { + std::lock_guard lock(region_cache_mutex); + region_for_bucket_cache = other.region_for_bucket_cache; + } + { + std::lock_guard lock(uri_cache_mutex); + uri_for_bucket_cache = other.uri_for_bucket_cache; + } +} void ClientCache::clearCache() { diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 2fd0a9cd2d1..6d97ec29ae6 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -54,10 +54,7 @@ struct ClientCache { ClientCache() = default; - ClientCache(const ClientCache & other) - : region_for_bucket_cache(other.region_for_bucket_cache) - , uri_for_bucket_cache(other.uri_for_bucket_cache) - {} + ClientCache(const ClientCache & other); ClientCache(ClientCache && other) = delete; @@ -67,10 +64,10 @@ struct ClientCache void clearCache(); std::mutex region_cache_mutex; - std::unordered_map region_for_bucket_cache; + std::unordered_map region_for_bucket_cache TSA_GUARDED_BY(region_cache_mutex); std::mutex uri_cache_mutex; - std::unordered_map uri_for_bucket_cache; + std::unordered_map uri_for_bucket_cache TSA_GUARDED_BY(uri_cache_mutex); }; class ClientCacheRegistry @@ -89,7 +86,7 @@ private: ClientCacheRegistry() = default; std::mutex clients_mutex; - std::unordered_map> client_caches; + std::unordered_map> client_caches TSA_GUARDED_BY(clients_mutex); }; bool isS3ExpressEndpoint(const std::string & endpoint); From 0de54b9d296927b7a8bc84bf1f8bfe5a5a01f080 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Wed, 17 Jul 2024 13:01:04 +0000 Subject: [PATCH 078/145] fix comma --- src/IO/S3/Client.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index b86e181c369..6e659c82def 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -828,11 +828,11 @@ void Client::updateURIForBucket(const std::string & bucket, S3::URI new_uri) con ClientCache::ClientCache(const ClientCache & other) { { - std::lock_guard lock(region_cache_mutex); + std::lock_guard lock(other.region_cache_mutex); region_for_bucket_cache = other.region_for_bucket_cache; } { - std::lock_guard lock(uri_cache_mutex); + std::lock_guard lock(other.uri_cache_mutex); uri_for_bucket_cache = other.uri_for_bucket_cache; } } From baade8baf45c7919f106160b8b8633d2c59e3ae8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Jul 2024 13:22:50 +0000 Subject: [PATCH 079/145] Replace updateWeakHash to getWeakHash --- src/Columns/ColumnAggregateFunction.cpp | 9 +-- src/Columns/ColumnAggregateFunction.h | 2 +- src/Columns/ColumnArray.cpp | 11 ++- src/Columns/ColumnArray.h | 2 +- src/Columns/ColumnCompressed.h | 3 +- src/Columns/ColumnConst.cpp | 14 +--- src/Columns/ColumnConst.h | 2 +- src/Columns/ColumnDecimal.cpp | 9 +-- src/Columns/ColumnDecimal.h | 2 +- src/Columns/ColumnDynamic.h | 5 +- src/Columns/ColumnFixedString.cpp | 10 +-- src/Columns/ColumnFixedString.h | 2 +- src/Columns/ColumnFunction.h | 5 +- src/Columns/ColumnLowCardinality.cpp | 26 +++--- src/Columns/ColumnLowCardinality.h | 4 +- src/Columns/ColumnMap.cpp | 4 +- src/Columns/ColumnMap.h | 2 +- src/Columns/ColumnNullable.cpp | 16 ++-- src/Columns/ColumnNullable.h | 2 +- src/Columns/ColumnObject.h | 3 +- src/Columns/ColumnSparse.cpp | 21 ++--- src/Columns/ColumnSparse.h | 2 +- src/Columns/ColumnString.cpp | 9 +-- src/Columns/ColumnString.h | 2 +- src/Columns/ColumnTuple.cpp | 11 ++- src/Columns/ColumnTuple.h | 2 +- src/Columns/ColumnVariant.cpp | 27 ++----- src/Columns/ColumnVariant.h | 2 +- src/Columns/ColumnVector.cpp | 9 +-- src/Columns/ColumnVector.h | 2 +- src/Columns/IColumn.h | 4 +- src/Columns/IColumnDummy.h | 4 +- src/Columns/IColumnUnique.h | 5 +- src/Columns/tests/gtest_weak_hash_32.cpp | 81 +++++++------------ src/Common/WeakHash.cpp | 22 +++++ src/Common/WeakHash.h | 5 +- src/Interpreters/ConcurrentHashJoin.cpp | 2 +- src/Interpreters/JoinUtils.cpp | 2 +- .../ScatterByPartitionTransform.cpp | 2 +- 39 files changed, 150 insertions(+), 197 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index cfd07c27765..33bd1266c90 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -366,13 +366,10 @@ void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) cons hash.update(wbuf.str().c_str(), wbuf.str().size()); } -void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnAggregateFunction::getWeakHash32() const { auto s = data.size(); - if (hash.getData().size() != data.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), hash.getData().size()); - + WeakHash32 hash(s); auto & hash_data = hash.getData(); std::vector v; @@ -383,6 +380,8 @@ void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const wbuf.finalize(); hash_data[i] = ::updateWeakHash32(v.data(), v.size(), hash_data[i]); } + + return hash; } void ColumnAggregateFunction::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 1be7a862438..330a707b75c 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -177,7 +177,7 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 5d7350f3a79..9203fb8042f 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -271,15 +271,12 @@ void ColumnArray::updateHashWithValue(size_t n, SipHash & hash) const getData().updateHashWithValue(offset + i, hash); } -void ColumnArray::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnArray::getWeakHash32() const { auto s = offsets->size(); - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", s, hash.getData().size()); + WeakHash32 hash(s); - WeakHash32 internal_hash(data->size()); - data->updateWeakHash32(internal_hash); + WeakHash32 internal_hash = data->getWeakHash32(); Offset prev_offset = 0; const auto & offsets_data = getOffsets(); @@ -300,6 +297,8 @@ void ColumnArray::updateWeakHash32(WeakHash32 & hash) const prev_offset = offsets_data[i]; } + + return hash; } void ColumnArray::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 6cd3e2f6c3b..5e01b9144d7 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -82,7 +82,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; #if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index 5e455709fec..19470113394 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -98,7 +99,7 @@ public: const char * deserializeAndInsertFromArena(const char *) override { throwMustBeDecompressed(); } const char * skipSerializedInArena(const char *) const override { throwMustBeDecompressed(); } void updateHashWithValue(size_t, SipHash &) const override { throwMustBeDecompressed(); } - void updateWeakHash32(WeakHash32 &) const override { throwMustBeDecompressed(); } + WeakHash32 getWeakHash32() const override { throwMustBeDecompressed(); } void updateHashFast(SipHash &) const override { throwMustBeDecompressed(); } ColumnPtr filter(const Filter &, ssize_t) const override { throwMustBeDecompressed(); } void expand(const Filter &, bool) override { throwMustBeDecompressed(); } diff --git a/src/Columns/ColumnConst.cpp b/src/Columns/ColumnConst.cpp index f2cea83db0e..84427e7be2b 100644 --- a/src/Columns/ColumnConst.cpp +++ b/src/Columns/ColumnConst.cpp @@ -137,18 +137,10 @@ void ColumnConst::updatePermutation(PermutationSortDirection /*direction*/, Perm { } -void ColumnConst::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnConst::getWeakHash32() const { - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); - - WeakHash32 element_hash(1); - data->updateWeakHash32(element_hash); - size_t data_hash = element_hash.getData()[0]; - - for (auto & value : hash.getData()) - value = static_cast(intHashCRC32(data_hash, value)); + WeakHash32 element_hash = data->getWeakHash32(); + return WeakHash32(s, element_hash.getData()[0]); } void ColumnConst::compareColumn( diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index b55a1f42037..65ce53687b9 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -204,7 +204,7 @@ public: data->updateHashWithValue(0, hash); } - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override { diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index cf413f790a7..ed9c699a841 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -76,13 +76,10 @@ void ColumnDecimal::updateHashWithValue(size_t n, SipHash & hash) const } template -void ColumnDecimal::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnDecimal::getWeakHash32() const { auto s = data.size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); const T * begin = data.data(); const T * end = begin + s; @@ -94,6 +91,8 @@ void ColumnDecimal::updateWeakHash32(WeakHash32 & hash) const ++begin; ++hash_data; } + + return hash; } template diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 32efeb643a6..eb8a305a822 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -102,7 +102,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; #if !defined(ABORT_ON_LOGICAL_ERROR) int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 9abddc7a26d..6f09abb945a 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -174,9 +175,9 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override + WeakHash32 getWeakHash32() const override { - variant_column->updateWeakHash32(hash); + return variant_column->getWeakHash32(); } void updateHashFast(SipHash & hash) const override diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 1c2de203a94..4d17eb0bebd 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -137,14 +137,10 @@ void ColumnFixedString::updateHashWithValue(size_t index, SipHash & hash) const hash.update(reinterpret_cast(&chars[n * index]), n); } -void ColumnFixedString::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnFixedString::getWeakHash32() const { auto s = size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, " - "hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); const UInt8 * pos = chars.data(); UInt32 * hash_data = hash.getData().data(); @@ -156,6 +152,8 @@ void ColumnFixedString::updateWeakHash32(WeakHash32 & hash) const pos += n; ++hash_data; } + + return hash; } void ColumnFixedString::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 6e88136fc50..630c6c1c0a6 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -133,7 +133,7 @@ public: void updateHashWithValue(size_t index, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index ba924c49a82..dcd67aecad7 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -130,9 +131,9 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "updateHashWithValue is not implemented for {}", getName()); } - void updateWeakHash32(WeakHash32 &) const override + WeakHash32 getWeakHash32() const override { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "updateWeakHash32 is not implemented for {}", getName()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getWeakHash32 is not implemented for {}", getName()); } void updateHashFast(SipHash &) const override diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index eb694a10b0f..49ce948bf78 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -7,8 +7,7 @@ #include #include #include -#include "Storages/IndicesDescription.h" -#include "base/types.h" +#include #include #include @@ -320,19 +319,10 @@ const char * ColumnLowCardinality::skipSerializedInArena(const char * pos) const return getDictionary().skipSerializedInArena(pos); } -void ColumnLowCardinality::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnLowCardinality::getWeakHash32() const { - auto s = size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); - - const auto & dict = getDictionary().getNestedColumn(); - WeakHash32 dict_hash(dict->size()); - dict->updateWeakHash32(dict_hash); - - idx.updateWeakHash(hash, dict_hash); + WeakHash32 dict_hash = getDictionary().getNestedColumn()->getWeakHash32(); + return idx.getWeakHash(dict_hash); } void ColumnLowCardinality::updateHashFast(SipHash & hash) const @@ -832,10 +822,11 @@ bool ColumnLowCardinality::Index::containsDefault() const return contains; } -void ColumnLowCardinality::Index::updateWeakHash(WeakHash32 & hash, WeakHash32 & dict_hash) const +WeakHash32 ColumnLowCardinality::Index::getWeakHash(const WeakHash32 & dict_hash) const { + WeakHash32 hash(positions->size()); auto & hash_data = hash.getData(); - auto & dict_hash_data = dict_hash.getData(); + const auto & dict_hash_data = dict_hash.getData(); auto update_weak_hash = [&](auto x) { @@ -844,10 +835,11 @@ void ColumnLowCardinality::Index::updateWeakHash(WeakHash32 & hash, WeakHash32 & auto size = data.size(); for (size_t i = 0; i < size; ++i) - hash_data[i] = static_cast(intHashCRC32(dict_hash_data[data[i]], hash_data[i])); + hash_data[i] = dict_hash_data[data[i]]; }; callForType(std::move(update_weak_hash), size_of_type); + return hash; } void ColumnLowCardinality::Index::collectSerializedValueSizes( diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index e99be07cd8d..fb0c1237fcf 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -111,7 +111,7 @@ public: getDictionary().updateHashWithValue(getIndexes().getUInt(n), hash); } - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash &) const override; @@ -325,7 +325,7 @@ public: bool containsDefault() const; - void updateWeakHash(WeakHash32 & hash, WeakHash32 & dict_hash) const; + WeakHash32 getWeakHash(const WeakHash32 & dict_hash) const; void collectSerializedValueSizes(PaddedPODArray & sizes, const PaddedPODArray & dict_sizes) const; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 2dffddb2dc9..08d7734ac6b 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -143,9 +143,9 @@ void ColumnMap::updateHashWithValue(size_t n, SipHash & hash) const nested->updateHashWithValue(n, hash); } -void ColumnMap::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnMap::getWeakHash32() const { - nested->updateWeakHash32(hash); + return nested->getWeakHash32(); } void ColumnMap::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index a54071a2974..29275e1b5f7 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -64,7 +64,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; #if !defined(ABORT_ON_LOGICAL_ERROR) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index f060e74b315..64e99a3bbe8 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -56,25 +56,21 @@ void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const getNestedColumn().updateHashWithValue(n, hash); } -void ColumnNullable::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnNullable::getWeakHash32() const { auto s = size(); - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); - - WeakHash32 old_hash = hash; - nested_column->updateWeakHash32(hash); + WeakHash32 hash = nested_column->getWeakHash32(); const auto & null_map_data = getNullMapData(); auto & hash_data = hash.getData(); - auto & old_hash_data = old_hash.getData(); - /// Use old data for nulls. + /// Use defualt for nulls. for (size_t row = 0; row < s; ++row) if (null_map_data[row]) - hash_data[row] = old_hash_data[row]; + hash_data[row] = WeakHash32::kDefaultInitialValue; + + return hash; } void ColumnNullable::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index a6d0483e527..15bbd8c3b57 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -133,7 +133,7 @@ public: void protect() override; ColumnPtr replicate(const Offsets & replicate_offsets) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; void getExtremes(Field & min, Field & max) const override; // Special function for nullable minmax index diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index fadf2e18779..21607e003f2 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -252,7 +253,7 @@ public: const char * deserializeAndInsertFromArena(const char *) override { throwMustBeConcrete(); } const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); } void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); } - void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); } + WeakHash32 getWeakHash32() const override { throwMustBeConcrete(); } void updateHashFast(SipHash &) const override { throwMustBeConcrete(); } void expand(const Filter &, bool) override { throwMustBeConcrete(); } bool hasEqualValues() const override { throwMustBeConcrete(); } diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 98a66e87387..0d103a263dd 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -678,26 +678,13 @@ void ColumnSparse::updateHashWithValue(size_t n, SipHash & hash) const values->updateHashWithValue(getValueIndex(n), hash); } -void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnSparse::getWeakHash32() const { - if (hash.getData().size() != _size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", _size, hash.getData().size()); - - size_t values_size = values->size(); - WeakHash32 values_hash(values_size); + WeakHash32 values_hash = values->getWeakHash32(); + WeakHash32 hash(size()); auto & hash_data = hash.getData(); auto & values_hash_data = values_hash.getData(); - const auto & offsets_data = getOffsetsData(); - - if (getNumberOfDefaultRows() > 0) - values_hash_data[0] = hash_data[getFirstDefaultValueIndex()]; - - for (size_t i = 0; i + 1 < values_size; ++i) - values_hash_data[i + 1] = hash_data[offsets_data[i]]; - - values->updateWeakHash32(values_hash); auto offset_it = begin(); for (size_t i = 0; i < _size; ++i, ++offset_it) @@ -705,6 +692,8 @@ void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const size_t value_index = offset_it.getValueIndex(); hash_data[i] = values_hash_data[value_index]; } + + return hash; } void ColumnSparse::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 4860f5171f7..a5d4d788b17 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -139,7 +139,7 @@ public: void protect() override; ColumnPtr replicate(const Offsets & replicate_offsets) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; void getExtremes(Field & min, Field & max) const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 1eda9714d62..4accfbe8f41 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -108,13 +108,10 @@ MutableColumnPtr ColumnString::cloneResized(size_t to_size) const return res; } -void ColumnString::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnString::getWeakHash32() const { auto s = offsets.size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); const UInt8 * pos = chars.data(); UInt32 * hash_data = hash.getData().data(); @@ -130,6 +127,8 @@ void ColumnString::updateWeakHash32(WeakHash32 & hash) const prev_offset = offset; ++hash_data; } + + return hash; } diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 602ffac65e8..faaaa8848ca 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -212,7 +212,7 @@ public: hash.update(reinterpret_cast(&chars[offset]), string_size); } - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override { diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 9b822d7f570..cb0b05d2154 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -308,16 +308,15 @@ void ColumnTuple::updateHashWithValue(size_t n, SipHash & hash) const column->updateHashWithValue(n, hash); } -void ColumnTuple::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnTuple::getWeakHash32() const { auto s = size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); for (const auto & column : columns) - column->updateWeakHash32(hash); + hash.update(column->getWeakHash32()); + + return hash; } void ColumnTuple::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 38e479791d4..2fafd93f776 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -81,7 +81,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; #if !defined(ABORT_ON_LOGICAL_ERROR) void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index ee5de4c2dde..8fd6e1bbac1 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -789,36 +789,26 @@ void ColumnVariant::updateHashWithValue(size_t n, SipHash & hash) const variants[localDiscriminatorByGlobal(global_discr)]->updateHashWithValue(offsetAt(n), hash); } -void ColumnVariant::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnVariant::getWeakHash32() const { auto s = size(); - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); - /// If we have only NULLs, keep hash unchanged. if (hasOnlyNulls()) - return; + return WeakHash32(s); /// Optimization for case when there is only 1 non-empty variant and no NULLs. /// In this case we can just calculate weak hash for this variant. if (auto non_empty_local_discr = getLocalDiscriminatorOfOneNoneEmptyVariantNoNulls()) - { - variants[*non_empty_local_discr]->updateWeakHash32(hash); - return; - } + return variants[*non_empty_local_discr]->getWeakHash32(); /// Calculate weak hash for all variants. std::vector nested_hashes; for (const auto & variant : variants) - { - WeakHash32 nested_hash(variant->size()); - variant->updateWeakHash32(nested_hash); - nested_hashes.emplace_back(std::move(nested_hash)); - } + nested_hashes.emplace_back(variant->getWeakHash32()); /// For each row hash is a hash of corresponding row from corresponding variant. + WeakHash32 hash(s); auto & hash_data = hash.getData(); const auto & local_discriminators_data = getLocalDiscriminators(); const auto & offsets_data = getOffsets(); @@ -827,11 +817,10 @@ void ColumnVariant::updateWeakHash32(WeakHash32 & hash) const Discriminator discr = local_discriminators_data[i]; /// Update hash only for non-NULL values if (discr != NULL_DISCRIMINATOR) - { - auto nested_hash = nested_hashes[local_discriminators_data[i]].getData()[offsets_data[i]]; - hash_data[i] = static_cast(hashCRC32(nested_hash, hash_data[i])); - } + hash_data[i] = nested_hashes[discr].getData()[offsets_data[i]]; } + + return hash; } void ColumnVariant::updateHashFast(SipHash & hash) const diff --git a/src/Columns/ColumnVariant.h b/src/Columns/ColumnVariant.h index d91b8e93a7d..94f3066e676 100644 --- a/src/Columns/ColumnVariant.h +++ b/src/Columns/ColumnVariant.h @@ -213,7 +213,7 @@ public: const char * deserializeVariantAndInsertFromArena(Discriminator global_discr, const char * pos); const char * skipSerializedInArena(const char * pos) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; void expand(const Filter & mask, bool inverted) override; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 19d1b800961..185a1e0f615 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -73,13 +73,10 @@ void ColumnVector::updateHashWithValue(size_t n, SipHash & hash) const } template -void ColumnVector::updateWeakHash32(WeakHash32 & hash) const +WeakHash32 ColumnVector::getWeakHash32() const { auto s = data.size(); - - if (hash.getData().size() != s) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " - "column size is {}, hash size is {}", std::to_string(s), std::to_string(hash.getData().size())); + WeakHash32 hash(s); const T * begin = data.data(); const T * end = begin + s; @@ -91,6 +88,8 @@ void ColumnVector::updateWeakHash32(WeakHash32 & hash) const ++begin; ++hash_data; } + + return hash; } template diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 3a0acf5e312..c01778ecf32 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -114,7 +114,7 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; - void updateWeakHash32(WeakHash32 & hash) const override; + WeakHash32 getWeakHash32() const override; void updateHashFast(SipHash & hash) const override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 4b6f34e5aa2..3798d3b7466 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -300,10 +300,10 @@ public: /// passed bytes to hash must identify sequence of values unambiguously. virtual void updateHashWithValue(size_t n, SipHash & hash) const = 0; - /// Update hash function value. Hash is calculated for each element. + /// Get hash function value. Hash is calculated for each element. /// It's a fast weak hash function. Mainly need to scatter data between threads. /// WeakHash32 must have the same size as column. - virtual void updateWeakHash32(WeakHash32 & hash) const = 0; + virtual WeakHash32 getWeakHash32() const = 0; /// Update state of hash with all column. virtual void updateHashFast(SipHash & hash) const = 0; diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index c19fb704d9b..b18f4fdb302 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -63,8 +64,9 @@ public: { } - void updateWeakHash32(WeakHash32 & /*hash*/) const override + WeakHash32 getWeakHash32() const override { + return WeakHash32(s); } void updateHashFast(SipHash & /*hash*/) const override diff --git a/src/Columns/IColumnUnique.h b/src/Columns/IColumnUnique.h index 3398452b7ee..1b86204f5b1 100644 --- a/src/Columns/IColumnUnique.h +++ b/src/Columns/IColumnUnique.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -166,9 +167,9 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method scatter is not supported for ColumnUnique."); } - void updateWeakHash32(WeakHash32 &) const override + WeakHash32 getWeakHash32() const override { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method updateWeakHash32 is not supported for ColumnUnique."); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getWeakHash32 is not supported for ColumnUnique."); } void updateHashFast(SipHash &) const override diff --git a/src/Columns/tests/gtest_weak_hash_32.cpp b/src/Columns/tests/gtest_weak_hash_32.cpp index 2c95998761b..3143d0ff83c 100644 --- a/src/Columns/tests/gtest_weak_hash_32.cpp +++ b/src/Columns/tests/gtest_weak_hash_32.cpp @@ -60,8 +60,7 @@ TEST(WeakHash32, ColumnVectorU8) data.push_back(i); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -77,8 +76,7 @@ TEST(WeakHash32, ColumnVectorI8) data.push_back(i); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -94,8 +92,7 @@ TEST(WeakHash32, ColumnVectorU16) data.push_back(i); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -111,8 +108,7 @@ TEST(WeakHash32, ColumnVectorI16) data.push_back(i); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -128,8 +124,7 @@ TEST(WeakHash32, ColumnVectorU32) data.push_back(i << 16u); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -145,8 +140,7 @@ TEST(WeakHash32, ColumnVectorI32) data.push_back(i << 16); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -162,8 +156,7 @@ TEST(WeakHash32, ColumnVectorU64) data.push_back(i << 32u); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -179,8 +172,7 @@ TEST(WeakHash32, ColumnVectorI64) data.push_back(i << 32); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -204,8 +196,7 @@ TEST(WeakHash32, ColumnVectorU128) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), eq_data); } @@ -221,8 +212,7 @@ TEST(WeakHash32, ColumnVectorI128) data.push_back(i << 32); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -238,8 +228,7 @@ TEST(WeakHash32, ColumnDecimal32) data.push_back(i << 16); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -255,8 +244,7 @@ TEST(WeakHash32, ColumnDecimal64) data.push_back(i << 32); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -272,8 +260,7 @@ TEST(WeakHash32, ColumnDecimal128) data.push_back(i << 32); } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), col->getData()); } @@ -294,8 +281,7 @@ TEST(WeakHash32, ColumnString1) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -331,8 +317,7 @@ TEST(WeakHash32, ColumnString2) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -369,8 +354,7 @@ TEST(WeakHash32, ColumnString3) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -397,8 +381,7 @@ TEST(WeakHash32, ColumnFixedString) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -444,8 +427,7 @@ TEST(WeakHash32, ColumnArray) auto col_arr = ColumnArray::create(std::move(val), std::move(off)); - WeakHash32 hash(col_arr->size()); - col_arr->updateWeakHash32(hash); + WeakHash32 hash = col_arr->getWeakHash32(); checkColumn(hash.getData(), eq_data); } @@ -479,8 +461,7 @@ TEST(WeakHash32, ColumnArray2) auto col_arr = ColumnArray::create(std::move(val), std::move(off)); - WeakHash32 hash(col_arr->size()); - col_arr->updateWeakHash32(hash); + WeakHash32 hash = col_arr->getWeakHash32(); checkColumn(hash.getData(), eq_data); } @@ -536,8 +517,7 @@ TEST(WeakHash32, ColumnArrayArray) auto col_arr = ColumnArray::create(std::move(val), std::move(off)); auto col_arr_arr = ColumnArray::create(std::move(col_arr), std::move(off2)); - WeakHash32 hash(col_arr_arr->size()); - col_arr_arr->updateWeakHash32(hash); + WeakHash32 hash = col_arr_arr->getWeakHash32(); checkColumn(hash.getData(), eq_data); } @@ -555,8 +535,7 @@ TEST(WeakHash32, ColumnConst) auto col_const = ColumnConst::create(std::move(inner_col), 256); - WeakHash32 hash(col_const->size()); - col_const->updateWeakHash32(hash); + WeakHash32 hash = col_const->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -576,8 +555,7 @@ TEST(WeakHash32, ColumnLowcardinality) } } - WeakHash32 hash(col->size()); - col->updateWeakHash32(hash); + WeakHash32 hash = col->getWeakHash32(); checkColumn(hash.getData(), data); } @@ -602,8 +580,7 @@ TEST(WeakHash32, ColumnNullable) auto col_null = ColumnNullable::create(std::move(col), std::move(mask)); - WeakHash32 hash(col_null->size()); - col_null->updateWeakHash32(hash); + WeakHash32 hash = col_null->getWeakHash32(); checkColumn(hash.getData(), eq); } @@ -633,8 +610,7 @@ TEST(WeakHash32, ColumnTupleUInt64UInt64) columns.emplace_back(std::move(col2)); auto col_tuple = ColumnTuple::create(std::move(columns)); - WeakHash32 hash(col_tuple->size()); - col_tuple->updateWeakHash32(hash); + WeakHash32 hash = col_tuple->getWeakHash32(); checkColumn(hash.getData(), eq); } @@ -671,8 +647,7 @@ TEST(WeakHash32, ColumnTupleUInt64String) columns.emplace_back(std::move(col2)); auto col_tuple = ColumnTuple::create(std::move(columns)); - WeakHash32 hash(col_tuple->size()); - col_tuple->updateWeakHash32(hash); + WeakHash32 hash = col_tuple->getWeakHash32(); checkColumn(hash.getData(), eq); } @@ -709,8 +684,7 @@ TEST(WeakHash32, ColumnTupleUInt64FixedString) columns.emplace_back(std::move(col2)); auto col_tuple = ColumnTuple::create(std::move(columns)); - WeakHash32 hash(col_tuple->size()); - col_tuple->updateWeakHash32(hash); + WeakHash32 hash = col_tuple->getWeakHash32(); checkColumn(hash.getData(), eq); } @@ -756,8 +730,7 @@ TEST(WeakHash32, ColumnTupleUInt64Array) columns.emplace_back(ColumnArray::create(std::move(val), std::move(off))); auto col_tuple = ColumnTuple::create(std::move(columns)); - WeakHash32 hash(col_tuple->size()); - col_tuple->updateWeakHash32(hash); + WeakHash32 hash = col_tuple->getWeakHash32(); checkColumn(hash.getData(), eq_data); } diff --git a/src/Common/WeakHash.cpp b/src/Common/WeakHash.cpp index 54d973b6296..cb12df84db1 100644 --- a/src/Common/WeakHash.cpp +++ b/src/Common/WeakHash.cpp @@ -1,2 +1,24 @@ #include +#include +#include +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void WeakHash32::update(const WeakHash32 & other) +{ + size_t size = data.size(); + if (size != other.data.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match:" + "left size is {}, right size is {}", size, other.data.size()); + + for (size_t i = 0; i < size; ++i) + data[i] = static_cast(intHashCRC32(other.data[i], data[i])); +} + +} diff --git a/src/Common/WeakHash.h b/src/Common/WeakHash.h index b59624e64f2..d4a8d63868c 100644 --- a/src/Common/WeakHash.h +++ b/src/Common/WeakHash.h @@ -11,9 +11,8 @@ namespace DB /// The main purpose why this class needed is to support data initialization. Initially, every bit is 1. class WeakHash32 { - static constexpr UInt32 kDefaultInitialValue = ~UInt32(0); - public: + static constexpr UInt32 kDefaultInitialValue = ~UInt32(0); using Container = PaddedPODArray; @@ -22,6 +21,8 @@ public: void reset(size_t size, UInt32 initial_value = kDefaultInitialValue) { data.assign(size, initial_value); } + void update(const WeakHash32 & other); + const Container & getData() const { return data; } Container & getData() { return data; } diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 4493a9f4dbd..ac940c62a1a 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -310,7 +310,7 @@ IColumn::Selector ConcurrentHashJoin::selectDispatchBlock(const Strings & key_co { const auto & key_col = from_block.getByName(key_name).column->convertToFullColumnIfConst(); const auto & key_col_no_lc = recursiveRemoveLowCardinality(recursiveRemoveSparse(key_col)); - key_col_no_lc->updateWeakHash32(hash); + hash.update(key_col_no_lc->getWeakHash32()); } return hashToSelector(hash, num_shards); } diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index 1788c9aca48..180a45d4295 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -554,7 +554,7 @@ static Blocks scatterBlockByHashImpl(const Strings & key_columns_names, const Bl for (const auto & key_name : key_columns_names) { ColumnPtr key_col = materializeColumn(block, key_name); - key_col->updateWeakHash32(hash); + hash.update(key_col->getWeakHash32()); } auto selector = hashToSelector(hash, sharder); diff --git a/src/Processors/Transforms/ScatterByPartitionTransform.cpp b/src/Processors/Transforms/ScatterByPartitionTransform.cpp index 6e3cdc0fda1..16d265c9bcb 100644 --- a/src/Processors/Transforms/ScatterByPartitionTransform.cpp +++ b/src/Processors/Transforms/ScatterByPartitionTransform.cpp @@ -109,7 +109,7 @@ void ScatterByPartitionTransform::generateOutputChunks() hash.reset(num_rows); for (const auto & column_number : key_columns) - columns[column_number]->updateWeakHash32(hash); + hash.update(columns[column_number]->getWeakHash32()); const auto & hash_data = hash.getData(); IColumn::Selector selector(num_rows); From fd443a1721439aa455d431763e38eec2e51822a9 Mon Sep 17 00:00:00 2001 From: Konstantin Morozov Date: Wed, 17 Jul 2024 13:43:36 +0000 Subject: [PATCH 080/145] add mutable --- src/IO/S3/Client.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index 6d97ec29ae6..94f1fd05c05 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -63,10 +63,10 @@ struct ClientCache void clearCache(); - std::mutex region_cache_mutex; + mutable std::mutex region_cache_mutex; std::unordered_map region_for_bucket_cache TSA_GUARDED_BY(region_cache_mutex); - std::mutex uri_cache_mutex; + mutable std::mutex uri_cache_mutex; std::unordered_map uri_for_bucket_cache TSA_GUARDED_BY(uri_cache_mutex); }; From e25fbb6d478cc36af44d6ea60d35c28309bcb603 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 17 Jul 2024 15:28:08 +0000 Subject: [PATCH 081/145] Fix style --- .../NamedCollections/NamedCollectionsMetadataStorage.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp index 189eb58196d..36191b89e86 100644 --- a/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp +++ b/src/Common/NamedCollections/NamedCollectionsMetadataStorage.cpp @@ -173,7 +173,7 @@ private: { const auto file_name_as_path = fs::path(file_name); if (file_name_as_path.is_absolute()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Filename {} cannot be an absolute path!", file_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Filename {} cannot be an absolute path", file_name); return fs::path(root_path) / file_name_as_path; } @@ -328,7 +328,7 @@ private: { const auto file_name_as_path = fs::path(file_name); if (file_name_as_path.is_absolute()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Filename {} cannot be an absolute path!", file_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Filename {} cannot be an absolute path", file_name); return fs::path(root_path) / file_name_as_path; } From 05874d0b85d78c1067c1db5332c7cc74b94d88cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Jul 2024 15:37:55 +0000 Subject: [PATCH 082/145] Fixing style. --- src/Columns/ColumnDecimal.cpp | 1 - src/Columns/ColumnNullable.cpp | 2 +- src/Columns/ColumnSparse.cpp | 18 ------------------ src/Columns/ColumnSparse.h | 3 --- 4 files changed, 1 insertion(+), 23 deletions(-) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index ed9c699a841..e27807950ae 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int NOT_IMPLEMENTED; - extern const int LOGICAL_ERROR; } template diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 64e99a3bbe8..867c9149242 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -65,7 +65,7 @@ WeakHash32 ColumnNullable::getWeakHash32() const const auto & null_map_data = getNullMapData(); auto & hash_data = hash.getData(); - /// Use defualt for nulls. + /// Use default for nulls. for (size_t row = 0; row < s; ++row) if (null_map_data[row]) hash_data[row] = WeakHash32::kDefaultInitialValue; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0d103a263dd..8f98a4433d3 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -809,24 +809,6 @@ size_t ColumnSparse::getValueIndex(size_t n) const return it - offsets_data.begin() + 1; } -size_t ColumnSparse::getFirstDefaultValueIndex() const -{ - if (getNumberOfDefaultRows() == 0) - return size(); - - const auto & offsets_data = getOffsetsData(); - size_t off_size = offsets_data.size(); - - if (off_size == 0 || offsets_data[0] > 0) - return 0; - - size_t idx = 0; - while (idx + 1 < off_size && offsets_data[idx] + 1 == offsets_data[idx + 1]) - ++idx; - - return offsets_data[idx] + 1; -} - ColumnSparse::Iterator ColumnSparse::getIterator(size_t n) const { const auto & offsets_data = getOffsetsData(); diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index a5d4d788b17..392a6910956 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -173,9 +173,6 @@ public: /// O(log(offsets.size())) complexity, size_t getValueIndex(size_t n) const; - /// Returns an index of the first default value, or size() if there is no defaults. - size_t getFirstDefaultValueIndex() const; - const IColumn & getValuesColumn() const { return *values; } IColumn & getValuesColumn() { return *values; } From 8e013a1c5938e7cecba5804ac8e9f653a95e4ddc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Jul 2024 16:23:36 +0000 Subject: [PATCH 083/145] 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 084/145] 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 085/145] 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 c7fa082eac37bd109085f2c65f85f752781864a2 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 17 Jul 2024 19:32:54 +0200 Subject: [PATCH 086/145] add log for splitBlockIntoParts --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 73244b714bf..ee3ac4207cc 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -315,6 +315,12 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( for (size_t i = 0; i < async_insert_info_with_partition.size(); ++i) { + if (async_insert_info_with_partition[i] == nullptr) + { + LOG_ERROR(getLogger("MergeTreeDataWriter"), "The {}th element in async_insert_info_with_partition is nullptr. There are totally {} partitions in the insert. Selector content is {}", + i, partitions_count, fmt::join(selector.begin(), selector.end(), ",")); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected error for async deduplicated insert, please check error logs"); + } result[i].offsets = std::move(async_insert_info_with_partition[i]->offsets); result[i].tokens = std::move(async_insert_info_with_partition[i]->tokens); } From eb129232ff27c8103aa5e71d4ce8a9cdbc905dde Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 17 Jul 2024 18:47:33 +0800 Subject: [PATCH 087/145] Avoid generating named tuple for special keywords --- src/Parsers/isUnquotedIdentifier.cpp | 13 +++++++++++++ src/Parsers/isUnquotedIdentifier.h | 8 ++++++++ .../02890_named_tuple_functions.reference | 1 + .../0_stateless/02890_named_tuple_functions.sql | 5 ++++- 4 files changed, 26 insertions(+), 1 deletion(-) diff --git a/src/Parsers/isUnquotedIdentifier.cpp b/src/Parsers/isUnquotedIdentifier.cpp index 6f2442635ec..26cb3992a50 100644 --- a/src/Parsers/isUnquotedIdentifier.cpp +++ b/src/Parsers/isUnquotedIdentifier.cpp @@ -1,5 +1,6 @@ #include +#include #include namespace DB @@ -7,6 +8,18 @@ namespace DB bool isUnquotedIdentifier(const String & name) { + auto is_keyword = [&name](Keyword keyword) + { + auto s = toStringView(keyword); + if (name.size() != s.size()) + return false; + return strncasecmp(s.data(), name.data(), s.size()) == 0; + }; + + /// Special keywords are parsed as literals instead of identifiers. + if (is_keyword(Keyword::NULL_KEYWORD) || is_keyword(Keyword::TRUE_KEYWORD) || is_keyword(Keyword::FALSE_KEYWORD)) + return false; + Lexer lexer(name.data(), name.data() + name.size()); auto maybe_ident = lexer.nextToken(); diff --git a/src/Parsers/isUnquotedIdentifier.h b/src/Parsers/isUnquotedIdentifier.h index 839e5860ad3..9c9f9239eb3 100644 --- a/src/Parsers/isUnquotedIdentifier.h +++ b/src/Parsers/isUnquotedIdentifier.h @@ -5,6 +5,14 @@ namespace DB { +/// Checks if the input string @name is a valid unquoted identifier. +/// +/// Example Usage: +/// abc -> true (valid unquoted identifier) +/// 123 -> false (identifiers cannot start with digits) +/// `123` -> false (quoted identifiers are not considered) +/// `abc` -> false (quoted identifiers are not considered) +/// null -> false (reserved literal keyword) bool isUnquotedIdentifier(const String & name); } diff --git a/tests/queries/0_stateless/02890_named_tuple_functions.reference b/tests/queries/0_stateless/02890_named_tuple_functions.reference index f7a0c440b5a..6b36ff2c54c 100644 --- a/tests/queries/0_stateless/02890_named_tuple_functions.reference +++ b/tests/queries/0_stateless/02890_named_tuple_functions.reference @@ -7,3 +7,4 @@ Tuple(\n k UInt8,\n j Int32) Tuple(Int32, Int32, Int32, Int32) ['1','2','3','4'] (1,2,3) +Tuple(Nullable(Nothing)) Tuple(Bool) Tuple(Bool) diff --git a/tests/queries/0_stateless/02890_named_tuple_functions.sql b/tests/queries/0_stateless/02890_named_tuple_functions.sql index 8e0c9c2b10e..0033102bd53 100644 --- a/tests/queries/0_stateless/02890_named_tuple_functions.sql +++ b/tests/queries/0_stateless/02890_named_tuple_functions.sql @@ -28,4 +28,7 @@ create table tbl (x Tuple(a Int32, b Int32, c Int32)) engine MergeTree order by insert into tbl values (tuple(1, 2, 3)); -- without tuple it's interpreted differently inside values block. select * from tbl; -drop table tbl +drop table tbl; + +-- Avoid generating named tuple for special keywords +select toTypeName(tuple(null)), toTypeName(tuple(true)), toTypeName(tuple(false)); 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 088/145] 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 29d3f9598d58aa6010509fbf4e2344cefd4cef24 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 20:53:05 +0200 Subject: [PATCH 089/145] Fix test --- .../02164_clickhouse_local_interactive_exception.expect | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect index 76902bdc69d..add977c4fce 100755 --- a/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect +++ b/tests/queries/0_stateless/02164_clickhouse_local_interactive_exception.expect @@ -9,7 +9,7 @@ if {[info exists env(CLICKHOUSE_TMP)]} { } exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 -log_user 1 +log_user 0 set timeout 20 match_max 100000 From 79402aa71b62d2a3f9cbd462c40e8d710714e37e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 17 Jul 2024 20:39:37 +0000 Subject: [PATCH 090/145] Make CaseSensitiveness an enum class --- .../AggregateFunctionAnalysisOfVariance.cpp | 4 +- .../AggregateFunctionAny.cpp | 6 +- .../AggregateFunctionAnyRespectNulls.cpp | 6 +- .../AggregateFunctionAvg.cpp | 2 +- .../AggregateFunctionBitwise.cpp | 6 +- .../AggregateFunctionCorr.cpp | 2 +- .../AggregateFunctionCount.cpp | 2 +- .../AggregateFunctionCovar.cpp | 4 +- .../AggregateFunctionFactory.cpp | 4 +- .../AggregateFunctionFactory.h | 2 +- .../AggregateFunctionGroupArray.cpp | 4 +- .../AggregateFunctionGroupConcat.cpp | 2 +- ...AggregateFunctionKolmogorovSmirnovTest.cpp | 2 +- .../AggregateFunctionSecondMoment.cpp | 10 +-- .../AggregateFunctionSum.cpp | 2 +- .../AggregateFunctionTopK.cpp | 6 +- .../AggregateFunctionsMinMax.cpp | 4 +- src/Common/IFactoryWithAliases.h | 12 ++-- src/DataTypes/DataTypeDate.cpp | 2 +- src/DataTypes/DataTypeDate32.cpp | 2 +- src/DataTypes/DataTypeDomainBool.cpp | 4 +- src/DataTypes/DataTypeEnum.cpp | 2 +- src/DataTypes/DataTypeFactory.cpp | 10 +-- src/DataTypes/DataTypeFactory.h | 8 +-- src/DataTypes/DataTypeFixedString.cpp | 2 +- src/DataTypes/DataTypeIPv4andIPv6.cpp | 4 +- src/DataTypes/DataTypeObject.cpp | 2 +- src/DataTypes/DataTypeString.cpp | 62 ++++++++--------- src/DataTypes/DataTypesDecimal.cpp | 16 ++--- src/DataTypes/DataTypesNumber.cpp | 66 +++++++++---------- src/DataTypes/registerDataTypeDateTime.cpp | 8 +-- src/Functions/CRC.cpp | 6 +- src/Functions/CastOverloadResolver.cpp | 4 +- src/Functions/FunctionChar.cpp | 2 +- src/Functions/FunctionFQDN.cpp | 2 +- src/Functions/FunctionFactory.cpp | 6 +- src/Functions/FunctionFactory.h | 8 +-- .../FunctionGenerateRandomStructure.cpp | 3 +- .../FunctionsBinaryRepresentation.cpp | 8 +-- src/Functions/FunctionsCodingIP.cpp | 8 +-- src/Functions/FunctionsCodingULID.cpp | 3 +- src/Functions/FunctionsCodingUUID.cpp | 7 +- src/Functions/FunctionsConversion.cpp | 2 +- src/Functions/FunctionsHashingMisc.cpp | 3 +- src/Functions/FunctionsLogical.cpp | 2 +- src/Functions/FunctionsOpDate.cpp | 4 +- src/Functions/FunctionsRound.cpp | 14 ++-- .../FunctionsStringHashFixedString.cpp | 3 +- src/Functions/JSONArrayLength.cpp | 2 +- src/Functions/UTCTimestamp.cpp | 4 +- src/Functions/UTCTimestampTransform.cpp | 4 +- src/Functions/abs.cpp | 2 +- src/Functions/acos.cpp | 2 +- src/Functions/array/arrayFlatten.cpp | 2 +- src/Functions/array/arrayShuffle.cpp | 4 +- src/Functions/array/length.cpp | 4 +- src/Functions/ascii.cpp | 2 +- src/Functions/asin.cpp | 2 +- src/Functions/atan.cpp | 2 +- src/Functions/atan2.cpp | 2 +- src/Functions/base64Decode.cpp | 2 +- src/Functions/base64Encode.cpp | 2 +- src/Functions/byteSwap.cpp | 2 +- src/Functions/coalesce.cpp | 2 +- src/Functions/concat.cpp | 2 +- src/Functions/concatWithSeparator.cpp | 2 +- src/Functions/connectionId.cpp | 4 +- src/Functions/cos.cpp | 2 +- src/Functions/countMatches.cpp | 4 +- src/Functions/countSubstrings.cpp | 2 +- src/Functions/currentDatabase.cpp | 6 +- src/Functions/currentSchemas.cpp | 4 +- src/Functions/currentUser.cpp | 4 +- src/Functions/dateDiff.cpp | 6 +- src/Functions/dateName.cpp | 2 +- src/Functions/date_trunc.cpp | 2 +- src/Functions/degrees.cpp | 2 +- src/Functions/exp.cpp | 2 +- src/Functions/extractAllGroupsVertical.cpp | 2 +- src/Functions/factorial.cpp | 2 +- src/Functions/formatDateTime.cpp | 4 +- src/Functions/formatReadableDecimalSize.cpp | 3 +- src/Functions/formatReadableSize.cpp | 2 +- src/Functions/fromDaysSinceYearZero.cpp | 2 +- src/Functions/generateULID.cpp | 3 +- src/Functions/greatest.cpp | 2 +- src/Functions/hasSubsequence.cpp | 2 +- .../hasSubsequenceCaseInsensitive.cpp | 2 +- .../hasSubsequenceCaseInsensitiveUTF8.cpp | 2 +- src/Functions/hasSubsequenceUTF8.cpp | 2 +- src/Functions/hasToken.cpp | 4 +- src/Functions/hasTokenCaseInsensitive.cpp | 4 +- src/Functions/hypot.cpp | 2 +- src/Functions/if.cpp | 2 +- src/Functions/ifNull.cpp | 2 +- src/Functions/initcap.cpp | 2 +- src/Functions/initialQueryID.cpp | 2 +- src/Functions/isNull.cpp | 2 +- .../keyvaluepair/extractKeyValuePairs.cpp | 2 +- src/Functions/least.cpp | 2 +- src/Functions/left.cpp | 4 +- src/Functions/lengthUTF8.cpp | 4 +- src/Functions/locate.cpp | 2 +- src/Functions/log.cpp | 4 +- src/Functions/log10.cpp | 2 +- src/Functions/log2.cpp | 2 +- src/Functions/lower.cpp | 4 +- src/Functions/makeDate.cpp | 2 +- src/Functions/match.cpp | 2 +- src/Functions/mathConstants.cpp | 2 +- src/Functions/max2.cpp | 2 +- src/Functions/min2.cpp | 2 +- src/Functions/modulo.cpp | 8 +-- src/Functions/monthName.cpp | 2 +- src/Functions/now.cpp | 4 +- src/Functions/now64.cpp | 2 +- src/Functions/nullIf.cpp | 2 +- src/Functions/padString.cpp | 4 +- src/Functions/parseDateTime.cpp | 4 +- src/Functions/position.cpp | 2 +- src/Functions/positionCaseInsensitive.cpp | 2 +- src/Functions/pow.cpp | 4 +- src/Functions/queryID.cpp | 2 +- src/Functions/radians.cpp | 2 +- src/Functions/rand.cpp | 2 +- src/Functions/regexpExtract.cpp | 2 +- src/Functions/repeat.cpp | 2 +- src/Functions/replaceAll.cpp | 2 +- src/Functions/replaceRegexpAll.cpp | 2 +- src/Functions/reverse.cpp | 2 +- src/Functions/right.cpp | 4 +- src/Functions/serverConstants.cpp | 7 +- src/Functions/sign.cpp | 2 +- src/Functions/sin.cpp | 2 +- src/Functions/soundex.cpp | 2 +- src/Functions/space.cpp | 2 +- src/Functions/sqrt.cpp | 2 +- src/Functions/structureToFormatSchema.cpp | 6 +- src/Functions/substring.cpp | 10 +-- src/Functions/substringIndex.cpp | 2 +- src/Functions/synonyms.cpp | 2 +- src/Functions/tan.cpp | 2 +- src/Functions/tanh.cpp | 2 +- src/Functions/timestamp.cpp | 2 +- src/Functions/toCustomWeek.cpp | 4 +- src/Functions/toDayOfMonth.cpp | 4 +- src/Functions/toDayOfWeek.cpp | 2 +- src/Functions/toDayOfYear.cpp | 2 +- src/Functions/toDaysSinceYearZero.cpp | 2 +- src/Functions/toDecimalString.cpp | 2 +- src/Functions/toHour.cpp | 2 +- src/Functions/toLastDayOfMonth.cpp | 2 +- src/Functions/toMillisecond.cpp | 2 +- src/Functions/toMinute.cpp | 2 +- src/Functions/toMonth.cpp | 2 +- src/Functions/toQuarter.cpp | 2 +- src/Functions/toSecond.cpp | 2 +- src/Functions/toYear.cpp | 2 +- src/Functions/today.cpp | 4 +- src/Functions/upper.cpp | 4 +- src/Functions/vectorFunctions.cpp | 34 +++++----- src/Functions/widthBucket.cpp | 2 +- src/Processors/Transforms/WindowTransform.cpp | 12 ++-- src/TableFunctions/TableFunctionFactory.cpp | 6 +- src/TableFunctions/TableFunctionFactory.h | 4 +- src/TableFunctions/TableFunctionFormat.cpp | 2 +- src/TableFunctions/TableFunctionValues.cpp | 2 +- 167 files changed, 355 insertions(+), 365 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.cpp b/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.cpp index 934a8dffd90..5d833796510 100644 --- a/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.cpp +++ b/src/AggregateFunctions/AggregateFunctionAnalysisOfVariance.cpp @@ -118,10 +118,10 @@ AggregateFunctionPtr createAggregateFunctionAnalysisOfVariance(const std::string void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .is_order_dependent = false }; - factory.registerFunction("analysisOfVariance", {createAggregateFunctionAnalysisOfVariance, properties}, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("analysisOfVariance", {createAggregateFunctionAnalysisOfVariance, properties}, AggregateFunctionFactory::Case::Insensitive); /// This is widely used term - factory.registerAlias("anova", "analysisOfVariance", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("anova", "analysisOfVariance", AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionAny.cpp b/src/AggregateFunctions/AggregateFunctionAny.cpp index f727ab04aa9..2bcee0fdd5f 100644 --- a/src/AggregateFunctions/AggregateFunctionAny.cpp +++ b/src/AggregateFunctions/AggregateFunctionAny.cpp @@ -361,9 +361,9 @@ void registerAggregateFunctionsAny(AggregateFunctionFactory & factory) AggregateFunctionProperties default_properties = {.returns_default_when_only_null = false, .is_order_dependent = true}; factory.registerFunction("any", {createAggregateFunctionAny, default_properties}); - factory.registerAlias("any_value", "any", AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("first_value", "any", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("any_value", "any", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("first_value", "any", AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("anyLast", {createAggregateFunctionAnyLast, default_properties}); - factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("last_value", "anyLast", AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp index 7275409c151..0b6642bffac 100644 --- a/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp +++ b/src/AggregateFunctions/AggregateFunctionAnyRespectNulls.cpp @@ -221,11 +221,11 @@ void registerAggregateFunctionsAnyRespectNulls(AggregateFunctionFactory & factor = {.returns_default_when_only_null = false, .is_order_dependent = true, .is_window_function = true}; factory.registerFunction("any_respect_nulls", {createAggregateFunctionAnyRespectNulls, default_properties_for_respect_nulls}); - factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("any_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("first_value_respect_nulls", "any_respect_nulls", AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("anyLast_respect_nulls", {createAggregateFunctionAnyLastRespectNulls, default_properties_for_respect_nulls}); - factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("last_value_respect_nulls", "anyLast_respect_nulls", AggregateFunctionFactory::Case::Insensitive); /// Must happen after registering any and anyLast factory.registerNullsActionTransformation("any", "any_respect_nulls"); diff --git a/src/AggregateFunctions/AggregateFunctionAvg.cpp b/src/AggregateFunctions/AggregateFunctionAvg.cpp index ac6d2cf7fb4..57b14921c99 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvg.cpp @@ -46,6 +46,6 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const void registerAggregateFunctionAvg(AggregateFunctionFactory & factory) { - factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionBitwise.cpp b/src/AggregateFunctions/AggregateFunctionBitwise.cpp index 619251552e4..ecced5f3e32 100644 --- a/src/AggregateFunctions/AggregateFunctionBitwise.cpp +++ b/src/AggregateFunctions/AggregateFunctionBitwise.cpp @@ -234,9 +234,9 @@ void registerAggregateFunctionsBitwise(AggregateFunctionFactory & factory) factory.registerFunction("groupBitXor", createAggregateFunctionBitwise); /// Aliases for compatibility with MySQL. - factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("BIT_OR", "groupBitOr", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("BIT_AND", "groupBitAnd", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("BIT_XOR", "groupBitXor", AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionCorr.cpp b/src/AggregateFunctions/AggregateFunctionCorr.cpp index 2e8ff3af933..02d3a4aa912 100644 --- a/src/AggregateFunctions/AggregateFunctionCorr.cpp +++ b/src/AggregateFunctions/AggregateFunctionCorr.cpp @@ -9,7 +9,7 @@ template using AggregateFunctionCorr = AggregateFunct void registerAggregateFunctionsStatisticsCorr(AggregateFunctionFactory & factory) { - factory.registerFunction("corr", createAggregateFunctionStatisticsBinary, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("corr", createAggregateFunctionStatisticsBinary, AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionCount.cpp b/src/AggregateFunctions/AggregateFunctionCount.cpp index 25f991ab693..ad3aee90c37 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.cpp +++ b/src/AggregateFunctions/AggregateFunctionCount.cpp @@ -37,7 +37,7 @@ AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, cons void registerAggregateFunctionCount(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("count", {createAggregateFunctionCount, properties}, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("count", {createAggregateFunctionCount, properties}, AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionCovar.cpp b/src/AggregateFunctions/AggregateFunctionCovar.cpp index 9645685483f..e4877a0aed3 100644 --- a/src/AggregateFunctions/AggregateFunctionCovar.cpp +++ b/src/AggregateFunctions/AggregateFunctionCovar.cpp @@ -13,8 +13,8 @@ void registerAggregateFunctionsStatisticsCovar(AggregateFunctionFactory & factor factory.registerFunction("covarPop", createAggregateFunctionStatisticsBinary); /// Synonyms for compatibility. - factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("COVAR_SAMP", "covarSamp", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("COVAR_POP", "covarPop", AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index b5c6440a69c..082fa11ca8a 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -29,7 +29,7 @@ const String & getAggregateFunctionCanonicalNameIfAny(const String & name) return AggregateFunctionFactory::instance().getCanonicalNameIfAny(name); } -void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, CaseSensitiveness case_sensitiveness) +void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, Case case_sensitiveness) { if (creator_with_properties.creator == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: " @@ -39,7 +39,7 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory: the aggregate function name '{}' is not unique", name); - if (case_sensitiveness == CaseInsensitive) + if (case_sensitiveness == Case::Insensitive) { auto key = Poco::toLower(name); if (!case_insensitive_aggregate_functions.emplace(key, creator_with_properties).second) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index b1dc422fcb0..a5fa3424543 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -60,7 +60,7 @@ public: void registerFunction( const String & name, Value creator, - CaseSensitiveness case_sensitiveness = CaseSensitive); + Case case_sensitiveness = Case::Sensitive); /// Register how to transform from one aggregate function to other based on NullsAction /// Registers them both ways: diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 16907e0f24f..7034e6373b1 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -840,8 +840,8 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupArray", { createAggregateFunctionGroupArray, properties }); - factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::CaseInsensitive); - factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("array_agg", "groupArray", AggregateFunctionFactory::Case::Insensitive); + factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties }); factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray, properties }); } diff --git a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp index 1c059dc52aa..5494ef74705 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp @@ -277,7 +277,7 @@ void registerAggregateFunctionGroupConcat(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupConcat", { createAggregateFunctionGroupConcat, properties }); - factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp index 736cca11f1e..04eebe9f485 100644 --- a/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp +++ b/src/AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.cpp @@ -350,7 +350,7 @@ AggregateFunctionPtr createAggregateFunctionKolmogorovSmirnovTest( void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory) { - factory.registerFunction("kolmogorovSmirnovTest", createAggregateFunctionKolmogorovSmirnovTest, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("kolmogorovSmirnovTest", createAggregateFunctionKolmogorovSmirnovTest, AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionSecondMoment.cpp b/src/AggregateFunctions/AggregateFunctionSecondMoment.cpp index 80fbe2511d9..4aa6a0a4429 100644 --- a/src/AggregateFunctions/AggregateFunctionSecondMoment.cpp +++ b/src/AggregateFunctions/AggregateFunctionSecondMoment.cpp @@ -15,11 +15,11 @@ void registerAggregateFunctionsStatisticsSecondMoment(AggregateFunctionFactory & factory.registerFunction("stddevPop", createAggregateFunctionStatisticsUnary); /// Synonyms for compatibility. - factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("STD", "stddevPop", AggregateFunctionFactory::CaseInsensitive); + factory.registerAlias("VAR_SAMP", "varSamp", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("VAR_POP", "varPop", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("STDDEV_SAMP", "stddevSamp", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("STDDEV_POP", "stddevPop", AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("STD", "stddevPop", AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionSum.cpp b/src/AggregateFunctions/AggregateFunctionSum.cpp index e393cb6dd38..910e49f388d 100644 --- a/src/AggregateFunctions/AggregateFunctionSum.cpp +++ b/src/AggregateFunctions/AggregateFunctionSum.cpp @@ -72,7 +72,7 @@ AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const void registerAggregateFunctionSum(AggregateFunctionFactory & factory) { - factory.registerFunction("sum", createAggregateFunctionSum, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("sum", createAggregateFunctionSum, AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("sumWithOverflow", createAggregateFunctionSum); factory.registerFunction("sumKahan", createAggregateFunctionSum); } diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index 26f756abe18..f949f6b7e4a 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -535,9 +535,9 @@ void registerAggregateFunctionTopK(AggregateFunctionFactory & factory) factory.registerFunction("topK", { createAggregateFunctionTopK, properties }); factory.registerFunction("topKWeighted", { createAggregateFunctionTopK, properties }); - factory.registerFunction("approx_top_k", { createAggregateFunctionTopK, properties }, AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("approx_top_sum", { createAggregateFunctionTopK, properties }, AggregateFunctionFactory::CaseInsensitive); - factory.registerAlias("approx_top_count", "approx_top_k", AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("approx_top_k", { createAggregateFunctionTopK, properties }, AggregateFunctionFactory::Case::Insensitive); + factory.registerFunction("approx_top_sum", { createAggregateFunctionTopK, properties }, AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("approx_top_count", "approx_top_k", AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/AggregateFunctions/AggregateFunctionsMinMax.cpp b/src/AggregateFunctions/AggregateFunctionsMinMax.cpp index 03e21c15a75..5fa9a4ff5d1 100644 --- a/src/AggregateFunctions/AggregateFunctionsMinMax.cpp +++ b/src/AggregateFunctions/AggregateFunctionsMinMax.cpp @@ -195,8 +195,8 @@ AggregateFunctionPtr createAggregateFunctionMinMax( void registerAggregateFunctionsMinMax(AggregateFunctionFactory & factory) { - factory.registerFunction("min", createAggregateFunctionMinMax, AggregateFunctionFactory::CaseInsensitive); - factory.registerFunction("max", createAggregateFunctionMinMax, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("min", createAggregateFunctionMinMax, AggregateFunctionFactory::Case::Insensitive); + factory.registerFunction("max", createAggregateFunctionMinMax, AggregateFunctionFactory::Case::Insensitive); } } diff --git a/src/Common/IFactoryWithAliases.h b/src/Common/IFactoryWithAliases.h index 74d4b6e3bcb..431e5c7b733 100644 --- a/src/Common/IFactoryWithAliases.h +++ b/src/Common/IFactoryWithAliases.h @@ -39,16 +39,16 @@ protected: public: /// For compatibility with SQL, it's possible to specify that certain function name is case insensitive. - enum CaseSensitiveness + enum Case { - CaseSensitive, - CaseInsensitive + Sensitive, + Insensitive }; /** Register additional name for value * real_name have to be already registered. */ - void registerAlias(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive) + void registerAlias(const String & alias_name, const String & real_name, Case case_sensitiveness = Sensitive) { const auto & creator_map = getMap(); const auto & case_insensitive_creator_map = getCaseInsensitiveMap(); @@ -66,12 +66,12 @@ public: } /// We need sure the real_name exactly exists when call the function directly. - void registerAliasUnchecked(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive) + void registerAliasUnchecked(const String & alias_name, const String & real_name, Case case_sensitiveness = Sensitive) { String alias_name_lowercase = Poco::toLower(alias_name); const String factory_name = getFactoryName(); - if (case_sensitiveness == CaseInsensitive) + if (case_sensitiveness == Insensitive) { if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", factory_name, alias_name); diff --git a/src/DataTypes/DataTypeDate.cpp b/src/DataTypes/DataTypeDate.cpp index ee4b0065e59..0a7aa7deac6 100644 --- a/src/DataTypes/DataTypeDate.cpp +++ b/src/DataTypes/DataTypeDate.cpp @@ -17,7 +17,7 @@ SerializationPtr DataTypeDate::doGetDefaultSerialization() const void registerDataTypeDate(DataTypeFactory & factory) { - factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive); + factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeDate32.cpp b/src/DataTypes/DataTypeDate32.cpp index 343e498d303..b2b8e7c0c1c 100644 --- a/src/DataTypes/DataTypeDate32.cpp +++ b/src/DataTypes/DataTypeDate32.cpp @@ -24,7 +24,7 @@ Field DataTypeDate32::getDefault() const void registerDataTypeDate32(DataTypeFactory & factory) { factory.registerSimpleDataType( - "Date32", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::CaseInsensitive); + "Date32", [] { return DataTypePtr(std::make_shared()); }, DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeDomainBool.cpp b/src/DataTypes/DataTypeDomainBool.cpp index 3d19b6262d8..30dbba2d8c0 100644 --- a/src/DataTypes/DataTypeDomainBool.cpp +++ b/src/DataTypes/DataTypeDomainBool.cpp @@ -15,8 +15,8 @@ void registerDataTypeDomainBool(DataTypeFactory & factory) std::make_unique("Bool"), std::make_unique(type->getDefaultSerialization()))); }); - factory.registerAlias("bool", "Bool", DataTypeFactory::CaseInsensitive); - factory.registerAlias("boolean", "Bool", DataTypeFactory::CaseInsensitive); + factory.registerAlias("bool", "Bool", DataTypeFactory::Case::Insensitive); + factory.registerAlias("boolean", "Bool", DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index a1d5e4b39b7..08e0c0d2045 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -318,7 +318,7 @@ void registerDataTypeEnum(DataTypeFactory & factory) factory.registerDataType("Enum", create); /// MySQL - factory.registerAlias("ENUM", "Enum", DataTypeFactory::CaseInsensitive); + factory.registerAlias("ENUM", "Enum", DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 07dc4395bfe..af37cde2846 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -175,7 +175,7 @@ DataTypePtr DataTypeFactory::getCustom(DataTypeCustomDescPtr customization) cons } -void DataTypeFactory::registerDataType(const String & family_name, Value creator, CaseSensitiveness case_sensitiveness) +void DataTypeFactory::registerDataType(const String & family_name, Value creator, Case case_sensitiveness) { if (creator == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type family {} has been provided a null constructor", family_name); @@ -189,12 +189,12 @@ void DataTypeFactory::registerDataType(const String & family_name, Value creator throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type family name '{}' is not unique", family_name); - if (case_sensitiveness == CaseInsensitive + if (case_sensitiveness == Case::Insensitive && !case_insensitive_data_types.emplace(family_name_lowercase, creator).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the case insensitive data type family name '{}' is not unique", family_name); } -void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness) +void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, Case case_sensitiveness) { if (creator == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "DataTypeFactory: the data type {} has been provided a null constructor", @@ -208,7 +208,7 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator }, case_sensitiveness); } -void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness) +void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, Case case_sensitiveness) { registerDataType(family_name, [creator](const ASTPtr & ast) { @@ -219,7 +219,7 @@ void DataTypeFactory::registerDataTypeCustom(const String & family_name, Creator }, case_sensitiveness); } -void DataTypeFactory::registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness) +void DataTypeFactory::registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, Case case_sensitiveness) { registerDataTypeCustom(name, [name, creator](const ASTPtr & ast) { diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index 86e0203358d..edba9886d1c 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -41,16 +41,16 @@ public: DataTypePtr tryGet(const ASTPtr & ast) const; /// Register a type family by its name. - void registerDataType(const String & family_name, Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + void registerDataType(const String & family_name, Value creator, Case case_sensitiveness = Case::Sensitive); /// Register a simple data type, that have no parameters. - void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + void registerSimpleDataType(const String & name, SimpleCreator creator, Case case_sensitiveness = Case::Sensitive); /// Register a customized type family - void registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + void registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, Case case_sensitiveness = Case::Sensitive); /// Register a simple customized data type - void registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + void registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, Case case_sensitiveness = Case::Sensitive); private: template diff --git a/src/DataTypes/DataTypeFixedString.cpp b/src/DataTypes/DataTypeFixedString.cpp index 85af59e852d..080ff8826a5 100644 --- a/src/DataTypes/DataTypeFixedString.cpp +++ b/src/DataTypes/DataTypeFixedString.cpp @@ -64,7 +64,7 @@ void registerDataTypeFixedString(DataTypeFactory & factory) factory.registerDataType("FixedString", create); /// Compatibility alias. - factory.registerAlias("BINARY", "FixedString", DataTypeFactory::CaseInsensitive); + factory.registerAlias("BINARY", "FixedString", DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeIPv4andIPv6.cpp b/src/DataTypes/DataTypeIPv4andIPv6.cpp index 4c0b45f472a..de11cc50107 100644 --- a/src/DataTypes/DataTypeIPv4andIPv6.cpp +++ b/src/DataTypes/DataTypeIPv4andIPv6.cpp @@ -9,9 +9,9 @@ namespace DB void registerDataTypeIPv4andIPv6(DataTypeFactory & factory) { factory.registerSimpleDataType("IPv4", [] { return DataTypePtr(std::make_shared()); }); - factory.registerAlias("INET4", "IPv4", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INET4", "IPv4", DataTypeFactory::Case::Insensitive); factory.registerSimpleDataType("IPv6", [] { return DataTypePtr(std::make_shared()); }); - factory.registerAlias("INET6", "IPv6", DataTypeFactory::CaseInsensitive); + factory.registerAlias("INET6", "IPv6", DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 720436d0e0d..5636a46373f 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -76,7 +76,7 @@ void registerDataTypeObject(DataTypeFactory & factory) factory.registerDataType("Object", create); factory.registerSimpleDataType("JSON", [] { return std::make_shared("JSON", false); }, - DataTypeFactory::CaseInsensitive); + DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypeString.cpp b/src/DataTypes/DataTypeString.cpp index 95e49420009..ca65fb42cc8 100644 --- a/src/DataTypes/DataTypeString.cpp +++ b/src/DataTypes/DataTypeString.cpp @@ -62,38 +62,38 @@ void registerDataTypeString(DataTypeFactory & factory) /// These synonims are added for compatibility. - factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NCHAR", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("CHARACTER", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NVARCHAR", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("VARCHAR2", "String", DataTypeFactory::CaseInsensitive); /// Oracle - factory.registerAlias("TEXT", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("TINYTEXT", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("LONGTEXT", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BLOB", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("CLOB", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("TINYBLOB", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("LONGBLOB", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BYTEA", "String", DataTypeFactory::CaseInsensitive); /// PostgreSQL + factory.registerAlias("CHAR", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NCHAR", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("CHARACTER", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("VARCHAR", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NVARCHAR", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("VARCHAR2", "String", DataTypeFactory::Case::Insensitive); /// Oracle + factory.registerAlias("TEXT", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("TINYTEXT", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("LONGTEXT", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("BLOB", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("CLOB", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("TINYBLOB", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("LONGBLOB", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("BYTEA", "String", DataTypeFactory::Case::Insensitive); /// PostgreSQL - factory.registerAlias("CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("CHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("CHAR VARYING", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NATIONAL CHAR", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NATIONAL CHARACTER", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NATIONAL CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NATIONAL CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NATIONAL CHAR VARYING", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NCHAR VARYING", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NCHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("VARBINARY", "String", DataTypeFactory::CaseInsensitive); - factory.registerAlias("GEOMETRY", "String", DataTypeFactory::CaseInsensitive); //mysql + factory.registerAlias("CHARACTER LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("CHARACTER VARYING", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("CHAR LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("CHAR VARYING", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NATIONAL CHAR", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NATIONAL CHARACTER", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NATIONAL CHARACTER LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NATIONAL CHARACTER VARYING", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NATIONAL CHAR VARYING", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NCHAR VARYING", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NCHAR LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("VARBINARY", "String", DataTypeFactory::Case::Insensitive); + factory.registerAlias("GEOMETRY", "String", DataTypeFactory::Case::Insensitive); //mysql } } diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 77a7a3e7237..a427fd0717a 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -364,15 +364,15 @@ template class DataTypeDecimal; void registerDataTypeDecimal(DataTypeFactory & factory) { - factory.registerDataType("Decimal32", createExact, DataTypeFactory::CaseInsensitive); - factory.registerDataType("Decimal64", createExact, DataTypeFactory::CaseInsensitive); - factory.registerDataType("Decimal128", createExact, DataTypeFactory::CaseInsensitive); - factory.registerDataType("Decimal256", createExact, DataTypeFactory::CaseInsensitive); + factory.registerDataType("Decimal32", createExact, DataTypeFactory::Case::Insensitive); + factory.registerDataType("Decimal64", createExact, DataTypeFactory::Case::Insensitive); + factory.registerDataType("Decimal128", createExact, DataTypeFactory::Case::Insensitive); + factory.registerDataType("Decimal256", createExact, DataTypeFactory::Case::Insensitive); - factory.registerDataType("Decimal", create, DataTypeFactory::CaseInsensitive); - factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive); - factory.registerAlias("NUMERIC", "Decimal", DataTypeFactory::CaseInsensitive); - factory.registerAlias("FIXED", "Decimal", DataTypeFactory::CaseInsensitive); + factory.registerDataType("Decimal", create, DataTypeFactory::Case::Insensitive); + factory.registerAlias("DEC", "Decimal", DataTypeFactory::Case::Insensitive); + factory.registerAlias("NUMERIC", "Decimal", DataTypeFactory::Case::Insensitive); + factory.registerAlias("FIXED", "Decimal", DataTypeFactory::Case::Insensitive); } } diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp index 99446d24eed..72020b0a5aa 100644 --- a/src/DataTypes/DataTypesNumber.cpp +++ b/src/DataTypes/DataTypesNumber.cpp @@ -65,41 +65,41 @@ void registerDataTypeNumbers(DataTypeFactory & factory) /// These synonyms are added for compatibility. - factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INT1", "Int8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BYTE", "Int8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive); - factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INT", "Int32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INTEGER", "Int32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("MEDIUMINT", "Int32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("MEDIUMINT SIGNED", "Int32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BIGINT", "Int64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("SIGNED", "Int64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("TIME", "Int64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("TINYINT", "Int8", DataTypeFactory::Case::Insensitive); + factory.registerAlias("INT1", "Int8", DataTypeFactory::Case::Insensitive); + factory.registerAlias("BYTE", "Int8", DataTypeFactory::Case::Insensitive); + factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::Case::Insensitive); + factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::Case::Insensitive); + factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::Case::Insensitive); + factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::Case::Insensitive); + factory.registerAlias("INT", "Int32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("INTEGER", "Int32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("MEDIUMINT", "Int32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("MEDIUMINT SIGNED", "Int32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("BIGINT", "Int64", DataTypeFactory::Case::Insensitive); + factory.registerAlias("SIGNED", "Int64", DataTypeFactory::Case::Insensitive); + factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::Case::Insensitive); + factory.registerAlias("TIME", "Int64", DataTypeFactory::Case::Insensitive); - factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::CaseInsensitive); - factory.registerAlias("YEAR", "UInt16", DataTypeFactory::CaseInsensitive); - factory.registerAlias("MEDIUMINT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BIT", "UInt64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("SET", "UInt64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::Case::Insensitive); + factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::Case::Insensitive); + factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::Case::Insensitive); + factory.registerAlias("YEAR", "UInt16", DataTypeFactory::Case::Insensitive); + factory.registerAlias("MEDIUMINT UNSIGNED", "UInt32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("UNSIGNED", "UInt64", DataTypeFactory::Case::Insensitive); + factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::Case::Insensitive); + factory.registerAlias("BIT", "UInt64", DataTypeFactory::Case::Insensitive); + factory.registerAlias("SET", "UInt64", DataTypeFactory::Case::Insensitive); - factory.registerAlias("FLOAT", "Float32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("REAL", "Float32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("SINGLE", "Float32", DataTypeFactory::CaseInsensitive); - factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive); - factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive); + factory.registerAlias("FLOAT", "Float32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("REAL", "Float32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("SINGLE", "Float32", DataTypeFactory::Case::Insensitive); + factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::Case::Insensitive); + factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::Case::Insensitive); } /// Explicit template instantiations. diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 802356cc108..84a52d4affb 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -108,11 +108,11 @@ static DataTypePtr create64(const ASTPtr & arguments) void registerDataTypeDateTime(DataTypeFactory & factory) { - factory.registerDataType("DateTime", create, DataTypeFactory::CaseInsensitive); - factory.registerDataType("DateTime32", create32, DataTypeFactory::CaseInsensitive); - factory.registerDataType("DateTime64", create64, DataTypeFactory::CaseInsensitive); + factory.registerDataType("DateTime", create, DataTypeFactory::Case::Insensitive); + factory.registerDataType("DateTime32", create32, DataTypeFactory::Case::Insensitive); + factory.registerDataType("DateTime64", create64, DataTypeFactory::Case::Insensitive); - factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::CaseInsensitive); + factory.registerAlias("TIMESTAMP", "DateTime", DataTypeFactory::Case::Insensitive); } } diff --git a/src/Functions/CRC.cpp b/src/Functions/CRC.cpp index 49d6dd6fa52..0ba976669a3 100644 --- a/src/Functions/CRC.cpp +++ b/src/Functions/CRC.cpp @@ -150,9 +150,9 @@ using FunctionCRC64ECMA = FunctionCRC; REGISTER_FUNCTION(CRC) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/CastOverloadResolver.cpp b/src/Functions/CastOverloadResolver.cpp index 10a08c6e35f..49f63073aaf 100644 --- a/src/Functions/CastOverloadResolver.cpp +++ b/src/Functions/CastOverloadResolver.cpp @@ -137,10 +137,10 @@ FunctionOverloadResolverPtr createInternalCastOverloadResolver(CastType type, st REGISTER_FUNCTION(CastOverloadResolvers) { - factory.registerFunction("_CAST", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::nonAccurate, true, {}); }, {}, FunctionFactory::CaseInsensitive); + factory.registerFunction("_CAST", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::nonAccurate, true, {}); }, {}, FunctionFactory::Case::Insensitive); /// Note: "internal" (not affected by null preserving setting) versions of accurate cast functions are unneeded. - factory.registerFunction("CAST", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::nonAccurate, false, {}); }, {}, FunctionFactory::CaseInsensitive); + factory.registerFunction("CAST", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::nonAccurate, false, {}); }, {}, FunctionFactory::Case::Insensitive); factory.registerFunction("accurateCast", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::accurate, false, {}); }, {}); factory.registerFunction("accurateCastOrNull", [](ContextPtr context){ return CastOverloadResolverImpl::create(context, CastType::accurateOrNull, false, {}); }, {}); } diff --git a/src/Functions/FunctionChar.cpp b/src/Functions/FunctionChar.cpp index 055eb08f0c7..0ebe1442f08 100644 --- a/src/Functions/FunctionChar.cpp +++ b/src/Functions/FunctionChar.cpp @@ -116,7 +116,7 @@ private: REGISTER_FUNCTION(Char) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/FunctionFQDN.cpp b/src/Functions/FunctionFQDN.cpp index 108a96216fd..8948c948265 100644 --- a/src/Functions/FunctionFQDN.cpp +++ b/src/Functions/FunctionFQDN.cpp @@ -46,7 +46,7 @@ public: REGISTER_FUNCTION(FQDN) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); factory.registerAlias("fullHostName", "FQDN"); } diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index de6d5a9e1c1..501cf6e725c 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -31,7 +31,7 @@ void FunctionFactory::registerFunction( const std::string & name, FunctionCreator creator, FunctionDocumentation doc, - CaseSensitiveness case_sensitiveness) + Case case_sensitiveness) { if (!functions.emplace(name, FunctionFactoryData{creator, doc}).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "FunctionFactory: the function name '{}' is not unique", name); @@ -41,7 +41,7 @@ void FunctionFactory::registerFunction( throw Exception(ErrorCodes::LOGICAL_ERROR, "FunctionFactory: the function name '{}' is already registered as alias", name); - if (case_sensitiveness == CaseInsensitive) + if (case_sensitiveness == Case::Insensitive) { if (!case_insensitive_functions.emplace(function_name_lowercase, FunctionFactoryData{creator, doc}).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "FunctionFactory: the case insensitive function name '{}' is not unique", @@ -54,7 +54,7 @@ void FunctionFactory::registerFunction( const std::string & name, FunctionSimpleCreator creator, FunctionDocumentation doc, - CaseSensitiveness case_sensitiveness) + Case case_sensitiveness) { registerFunction(name, [my_creator = std::move(creator)](ContextPtr context) { diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index bb43d4719b8..d05e84439be 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -30,7 +30,7 @@ public: static FunctionFactory & instance(); template - void registerFunction(FunctionDocumentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) + void registerFunction(FunctionDocumentation doc = {}, Case case_sensitiveness = Case::Sensitive) { registerFunction(Function::name, std::move(doc), case_sensitiveness); } @@ -56,13 +56,13 @@ public: const std::string & name, FunctionCreator creator, FunctionDocumentation doc = {}, - CaseSensitiveness case_sensitiveness = CaseSensitive); + Case case_sensitiveness = Case::Sensitive); void registerFunction( const std::string & name, FunctionSimpleCreator creator, FunctionDocumentation doc = {}, - CaseSensitiveness case_sensitiveness = CaseSensitive); + Case case_sensitiveness = Case::Sensitive); FunctionDocumentation getDocumentation(const std::string & name) const; @@ -79,7 +79,7 @@ private: String getFactoryName() const override { return "FunctionFactory"; } template - void registerFunction(const std::string & name, FunctionDocumentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) + void registerFunction(const std::string & name, FunctionDocumentation doc = {}, Case case_sensitiveness = Case::Sensitive) { registerFunction(name, &Function::create, std::move(doc), case_sensitiveness); } diff --git a/src/Functions/FunctionGenerateRandomStructure.cpp b/src/Functions/FunctionGenerateRandomStructure.cpp index 0c6f4287ecb..2bead8737fd 100644 --- a/src/Functions/FunctionGenerateRandomStructure.cpp +++ b/src/Functions/FunctionGenerateRandomStructure.cpp @@ -445,8 +445,7 @@ The function returns a value of type String. {"with specified seed", "SELECT generateRandomStructure(1, 42)", "c1 UInt128"}, }, .categories{"Random"} - }, - FunctionFactory::CaseSensitive); + }); } } diff --git a/src/Functions/FunctionsBinaryRepresentation.cpp b/src/Functions/FunctionsBinaryRepresentation.cpp index ab10d402df4..f77d2f1f350 100644 --- a/src/Functions/FunctionsBinaryRepresentation.cpp +++ b/src/Functions/FunctionsBinaryRepresentation.cpp @@ -728,10 +728,10 @@ public: REGISTER_FUNCTION(BinaryRepr) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/FunctionsCodingIP.cpp b/src/Functions/FunctionsCodingIP.cpp index a134e39fbcd..0a97d029f84 100644 --- a/src/Functions/FunctionsCodingIP.cpp +++ b/src/Functions/FunctionsCodingIP.cpp @@ -1169,10 +1169,10 @@ REGISTER_FUNCTION(Coding) factory.registerFunction>(); /// MySQL compatibility aliases: - factory.registerAlias("INET_ATON", FunctionIPv4StringToNum::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("INET6_NTOA", FunctionIPv6NumToString::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("INET6_ATON", FunctionIPv6StringToNum::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("INET_NTOA", NameFunctionIPv4NumToString::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("INET_ATON", FunctionIPv4StringToNum::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("INET6_NTOA", FunctionIPv6NumToString::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("INET6_ATON", FunctionIPv6StringToNum::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("INET_NTOA", NameFunctionIPv4NumToString::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/FunctionsCodingULID.cpp b/src/Functions/FunctionsCodingULID.cpp index ff040945a15..b67224a5625 100644 --- a/src/Functions/FunctionsCodingULID.cpp +++ b/src/Functions/FunctionsCodingULID.cpp @@ -180,8 +180,7 @@ An optional second argument can be passed to specify a timezone for the timestam {"ulid", "SELECT ULIDStringToDateTime(generateULID())", ""}, {"timezone", "SELECT ULIDStringToDateTime(generateULID(), 'Asia/Istanbul')", ""}}, .categories{"ULID"} - }, - FunctionFactory::CaseSensitive); + }); } } diff --git a/src/Functions/FunctionsCodingUUID.cpp b/src/Functions/FunctionsCodingUUID.cpp index 6a44f4263a8..83fdcbc4af9 100644 --- a/src/Functions/FunctionsCodingUUID.cpp +++ b/src/Functions/FunctionsCodingUUID.cpp @@ -496,8 +496,8 @@ This function accepts a UUID and returns a FixedString(16) as its binary represe │ 612f3c40-5d3b-217e-707b-6a546a3d7b29 │ a/<@];!~p{jTj={) │ @( FunctionDocumentation{ @@ -509,8 +509,7 @@ An optional second argument can be passed to specify a timezone for the timestam .examples{ {"uuid","select UUIDv7ToDateTime(generateUUIDv7())", ""}, {"uuid","select generateUUIDv7() as uuid, UUIDv7ToDateTime(uuid), UUIDv7ToDateTime(uuid, 'America/New_York')", ""}}, - .categories{"UUID"}}, - FunctionFactory::CaseSensitive); + .categories{"UUID"}}); } } diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 1342e3f2c5d..675283d011e 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -5224,7 +5224,7 @@ REGISTER_FUNCTION(Conversion) /// MySQL compatibility alias. Cannot be registered as alias, /// because we don't want it to be normalized to toDate in queries, /// otherwise CREATE DICTIONARY query breaks. - factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::CaseInsensitive); + factory.registerFunction("DATE", &FunctionToDate::create, {}, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsHashingMisc.cpp b/src/Functions/FunctionsHashingMisc.cpp index 38f16af0e6d..5cc29215fe3 100644 --- a/src/Functions/FunctionsHashingMisc.cpp +++ b/src/Functions/FunctionsHashingMisc.cpp @@ -41,8 +41,7 @@ REGISTER_FUNCTION(Hashing) .description="Calculates value of XXH3 64-bit hash function. Refer to https://github.com/Cyan4973/xxHash for detailed documentation.", .examples{{"hash", "SELECT xxh3('ClickHouse')", ""}}, .categories{"Hash"} - }, - FunctionFactory::CaseSensitive); + }); factory.registerFunction(); diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index 2f5ce6deebf..e1814150da6 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -29,7 +29,7 @@ REGISTER_FUNCTION(Logical) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction({}, FunctionFactory::CaseInsensitive); /// Operator NOT(x) can be parsed as a function. + factory.registerFunction({}, FunctionFactory::Case::Insensitive); /// Operator NOT(x) can be parsed as a function. } namespace ErrorCodes diff --git a/src/Functions/FunctionsOpDate.cpp b/src/Functions/FunctionsOpDate.cpp index 7355848f73f..c4b154736e0 100644 --- a/src/Functions/FunctionsOpDate.cpp +++ b/src/Functions/FunctionsOpDate.cpp @@ -99,8 +99,8 @@ using FunctionSubDate = FunctionOpDate; REGISTER_FUNCTION(AddInterval) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/FunctionsRound.cpp b/src/Functions/FunctionsRound.cpp index 059476acb40..d87a9e7ca43 100644 --- a/src/Functions/FunctionsRound.cpp +++ b/src/Functions/FunctionsRound.cpp @@ -7,16 +7,16 @@ namespace DB REGISTER_FUNCTION(Round) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerFunction({}, FunctionFactory::CaseSensitive); - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerFunction({}, FunctionFactory::Case::Sensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); factory.registerFunction(); /// Compatibility aliases. - factory.registerAlias("ceiling", "ceil", FunctionFactory::CaseInsensitive); - factory.registerAlias("truncate", "trunc", FunctionFactory::CaseInsensitive); + factory.registerAlias("ceiling", "ceil", FunctionFactory::Case::Insensitive); + factory.registerAlias("truncate", "trunc", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/FunctionsStringHashFixedString.cpp b/src/Functions/FunctionsStringHashFixedString.cpp index e3b1b82c92f..01e989a7f2c 100644 --- a/src/Functions/FunctionsStringHashFixedString.cpp +++ b/src/Functions/FunctionsStringHashFixedString.cpp @@ -428,8 +428,7 @@ REGISTER_FUNCTION(HashFixedStrings) It returns a BLAKE3 hash as a byte array with type FixedString(32). )", .examples{{"hash", "SELECT hex(BLAKE3('ABC'))", ""}}, - .categories{"Hash"}}, - FunctionFactory::CaseSensitive); + .categories{"Hash"}}); # endif } #endif diff --git a/src/Functions/JSONArrayLength.cpp b/src/Functions/JSONArrayLength.cpp index 73dd55f1266..24e93440454 100644 --- a/src/Functions/JSONArrayLength.cpp +++ b/src/Functions/JSONArrayLength.cpp @@ -104,7 +104,7 @@ REGISTER_FUNCTION(JSONArrayLength) .description="Returns the number of elements in the outermost JSON array. The function returns NULL if input JSON string is invalid."}); /// For Spark compatibility. - factory.registerAlias("JSON_ARRAY_LENGTH", "JSONArrayLength", FunctionFactory::CaseInsensitive); + factory.registerAlias("JSON_ARRAY_LENGTH", "JSONArrayLength", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp index acc34b0a974..bc8e1b28431 100644 --- a/src/Functions/UTCTimestamp.cpp +++ b/src/Functions/UTCTimestamp.cpp @@ -117,8 +117,8 @@ Example: )", .examples{ {"typical", "SELECT UTCTimestamp();", ""}}, - .categories{"Dates and Times"}}, FunctionFactory::CaseInsensitive); - factory.registerAlias("UTC_timestamp", UTCTimestampOverloadResolver::name, FunctionFactory::CaseInsensitive); + .categories{"Dates and Times"}}, FunctionFactory::Case::Insensitive); + factory.registerAlias("UTC_timestamp", UTCTimestampOverloadResolver::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/UTCTimestampTransform.cpp b/src/Functions/UTCTimestampTransform.cpp index 6d301270d8e..36ec520068f 100644 --- a/src/Functions/UTCTimestampTransform.cpp +++ b/src/Functions/UTCTimestampTransform.cpp @@ -144,8 +144,8 @@ REGISTER_FUNCTION(UTCTimestampTransform) { factory.registerFunction(); factory.registerFunction(); - factory.registerAlias("to_utc_timestamp", NameToUTCTimestamp::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("from_utc_timestamp", NameFromUTCTimestamp::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("to_utc_timestamp", NameToUTCTimestamp::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("from_utc_timestamp", NameFromUTCTimestamp::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/abs.cpp b/src/Functions/abs.cpp index 9ac2363f765..742d3b85619 100644 --- a/src/Functions/abs.cpp +++ b/src/Functions/abs.cpp @@ -51,7 +51,7 @@ template <> struct FunctionUnaryArithmeticMonotonicity REGISTER_FUNCTION(Abs) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/acos.cpp b/src/Functions/acos.cpp index bc300ee77fb..39895fed64a 100644 --- a/src/Functions/acos.cpp +++ b/src/Functions/acos.cpp @@ -14,7 +14,7 @@ using FunctionAcos = FunctionMathUnary>; REGISTER_FUNCTION(Acos) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/array/arrayFlatten.cpp b/src/Functions/array/arrayFlatten.cpp index d4eb8eebeee..553ad82bd53 100644 --- a/src/Functions/array/arrayFlatten.cpp +++ b/src/Functions/array/arrayFlatten.cpp @@ -123,7 +123,7 @@ private: REGISTER_FUNCTION(ArrayFlatten) { factory.registerFunction(); - factory.registerAlias("flatten", "arrayFlatten", FunctionFactory::CaseInsensitive); + factory.registerAlias("flatten", "arrayFlatten", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/array/arrayShuffle.cpp b/src/Functions/array/arrayShuffle.cpp index 10cb51d27d2..fa17aa46464 100644 --- a/src/Functions/array/arrayShuffle.cpp +++ b/src/Functions/array/arrayShuffle.cpp @@ -196,7 +196,7 @@ It is possible to override the seed to produce stable results: {"explicit_seed", "SELECT arrayShuffle([1, 2, 3, 4], 41)", ""}, {"materialize", "SELECT arrayShuffle(materialize([1, 2, 3]), 42), arrayShuffle([1, 2, 3], 42) FROM numbers(10)", ""}}, .categories{"Array"}}, - FunctionFactory::CaseInsensitive); + FunctionFactory::Case::Insensitive); factory.registerFunction>( FunctionDocumentation{ @@ -224,7 +224,7 @@ It is possible to override the seed to produce stable results: {"materialize", "SELECT arrayPartialShuffle(materialize([1, 2, 3, 4]), 2, 42), arrayPartialShuffle([1, 2, 3], 2, 42) FROM numbers(10)", ""}}, .categories{"Array"}}, - FunctionFactory::CaseInsensitive); + FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/array/length.cpp b/src/Functions/array/length.cpp index 91a5e5fdec2..d81c071b55e 100644 --- a/src/Functions/array/length.cpp +++ b/src/Functions/array/length.cpp @@ -100,8 +100,8 @@ It is ok to have ASCII NUL bytes in strings, and they will be counted as well. }, .categories{"String", "Array"} }, - FunctionFactory::CaseInsensitive); - factory.registerAlias("OCTET_LENGTH", "length", FunctionFactory::CaseInsensitive); + FunctionFactory::Case::Insensitive); + factory.registerAlias("OCTET_LENGTH", "length", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/ascii.cpp b/src/Functions/ascii.cpp index b43c3221391..7c8158b53d4 100644 --- a/src/Functions/ascii.cpp +++ b/src/Functions/ascii.cpp @@ -90,7 +90,7 @@ If s is empty, the result is 0. If the first character is not an ASCII character )", .examples{{"ascii", "SELECT ascii('234')", ""}}, .categories{"String"} - }, FunctionFactory::CaseInsensitive); + }, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/asin.cpp b/src/Functions/asin.cpp index 3049b025d5e..85faf8c275d 100644 --- a/src/Functions/asin.cpp +++ b/src/Functions/asin.cpp @@ -41,7 +41,7 @@ For more details, see [https://en.wikipedia.org/wiki/Inverse_trigonometric_funct {"nan", "SELECT asin(1.1), asin(-2), asin(inf), asin(nan)", ""}}, .categories{"Mathematical", "Trigonometric"} }, - FunctionFactory::CaseInsensitive); + FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/atan.cpp b/src/Functions/atan.cpp index 32a0f06db8a..3f74c510487 100644 --- a/src/Functions/atan.cpp +++ b/src/Functions/atan.cpp @@ -14,7 +14,7 @@ using FunctionAtan = FunctionMathUnary>; REGISTER_FUNCTION(Atan) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/atan2.cpp b/src/Functions/atan2.cpp index 7be177f6dfb..42294e11458 100644 --- a/src/Functions/atan2.cpp +++ b/src/Functions/atan2.cpp @@ -15,7 +15,7 @@ namespace REGISTER_FUNCTION(Atan2) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/base64Decode.cpp b/src/Functions/base64Decode.cpp index 50278c4b0b2..4d06ac99d6f 100644 --- a/src/Functions/base64Decode.cpp +++ b/src/Functions/base64Decode.cpp @@ -17,7 +17,7 @@ REGISTER_FUNCTION(Base64Decode) factory.registerFunction>>({description, syntax, arguments, returned_value, examples, categories}); /// MySQL compatibility alias. - factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::CaseInsensitive); + factory.registerAlias("FROM_BASE64", "base64Decode", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/base64Encode.cpp b/src/Functions/base64Encode.cpp index d6e63c38a4c..64142995552 100644 --- a/src/Functions/base64Encode.cpp +++ b/src/Functions/base64Encode.cpp @@ -17,7 +17,7 @@ REGISTER_FUNCTION(Base64Encode) factory.registerFunction>>({description, syntax, arguments, returned_value, examples, categories}); /// MySQL compatibility alias. - factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::CaseInsensitive); + factory.registerAlias("TO_BASE64", "base64Encode", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/byteSwap.cpp b/src/Functions/byteSwap.cpp index 2a343a07720..6c824b851b0 100644 --- a/src/Functions/byteSwap.cpp +++ b/src/Functions/byteSwap.cpp @@ -100,7 +100,7 @@ One use-case of this function is reversing IPv4s: {"64-bit", "SELECT byteSwap(123294967295)", "18439412204227788800"}, }, .categories{"Mathematical", "Arithmetic"}}, - FunctionFactory::CaseInsensitive); + FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/coalesce.cpp b/src/Functions/coalesce.cpp index 722f32af523..19da6a85b38 100644 --- a/src/Functions/coalesce.cpp +++ b/src/Functions/coalesce.cpp @@ -180,7 +180,7 @@ private: REGISTER_FUNCTION(Coalesce) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index b011c33e02a..5c5e089e740 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -240,7 +240,7 @@ private: REGISTER_FUNCTION(Concat) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); factory.registerFunction(); } diff --git a/src/Functions/concatWithSeparator.cpp b/src/Functions/concatWithSeparator.cpp index ed02f331192..1d38ef87558 100644 --- a/src/Functions/concatWithSeparator.cpp +++ b/src/Functions/concatWithSeparator.cpp @@ -193,7 +193,7 @@ The function is named “injective” if it always returns different result for .categories{"String"}}); /// Compatibility with Spark and MySQL: - factory.registerAlias("concat_ws", "concatWithSeparator", FunctionFactory::CaseInsensitive); + factory.registerAlias("concat_ws", "concatWithSeparator", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/connectionId.cpp b/src/Functions/connectionId.cpp index 9c53482482b..c1036b2ddbe 100644 --- a/src/Functions/connectionId.cpp +++ b/src/Functions/connectionId.cpp @@ -33,8 +33,8 @@ public: REGISTER_FUNCTION(ConnectionId) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerAlias("connection_id", "connectionID", FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerAlias("connection_id", "connectionID", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/cos.cpp b/src/Functions/cos.cpp index 3496373a9d5..40fdede0e1c 100644 --- a/src/Functions/cos.cpp +++ b/src/Functions/cos.cpp @@ -13,7 +13,7 @@ using FunctionCos = FunctionMathUnary>; REGISTER_FUNCTION(Cos) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/countMatches.cpp b/src/Functions/countMatches.cpp index a8620080012..4db48b1305f 100644 --- a/src/Functions/countMatches.cpp +++ b/src/Functions/countMatches.cpp @@ -22,8 +22,8 @@ namespace DB REGISTER_FUNCTION(CountMatches) { - factory.registerFunction>({}, FunctionFactory::CaseSensitive); - factory.registerFunction>({}, FunctionFactory::CaseSensitive); + factory.registerFunction>(); + factory.registerFunction>(); } } diff --git a/src/Functions/countSubstrings.cpp b/src/Functions/countSubstrings.cpp index 843b81437f5..137edb179b2 100644 --- a/src/Functions/countSubstrings.cpp +++ b/src/Functions/countSubstrings.cpp @@ -19,6 +19,6 @@ using FunctionCountSubstrings = FunctionsStringSearch({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/currentDatabase.cpp b/src/Functions/currentDatabase.cpp index 954899c3c2b..16cb43ebb04 100644 --- a/src/Functions/currentDatabase.cpp +++ b/src/Functions/currentDatabase.cpp @@ -54,9 +54,9 @@ public: REGISTER_FUNCTION(CurrentDatabase) { factory.registerFunction(); - factory.registerAlias("DATABASE", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("SCHEMA", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("current_database", FunctionCurrentDatabase::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("DATABASE", FunctionCurrentDatabase::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("SCHEMA", FunctionCurrentDatabase::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("current_database", FunctionCurrentDatabase::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/currentSchemas.cpp b/src/Functions/currentSchemas.cpp index 322e719eb17..0a128d0e908 100644 --- a/src/Functions/currentSchemas.cpp +++ b/src/Functions/currentSchemas.cpp @@ -80,8 +80,8 @@ Requires a boolean parameter, but it is ignored actually. It is required just fo {"common", "SELECT current_schemas(true);", "['default']"} } }, - FunctionFactory::CaseInsensitive); - factory.registerAlias("current_schemas", FunctionCurrentSchemas::name, FunctionFactory::CaseInsensitive); + FunctionFactory::Case::Insensitive); + factory.registerAlias("current_schemas", FunctionCurrentSchemas::name, FunctionFactory::Case::Insensitive); } diff --git a/src/Functions/currentUser.cpp b/src/Functions/currentUser.cpp index 1679c56a929..9f48f15ffb3 100644 --- a/src/Functions/currentUser.cpp +++ b/src/Functions/currentUser.cpp @@ -54,8 +54,8 @@ public: REGISTER_FUNCTION(CurrentUser) { factory.registerFunction(); - factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("current_user", FunctionCurrentUser::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("current_user", FunctionCurrentUser::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 8e8865db7ed..f49e8dee6b7 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -490,7 +490,7 @@ private: REGISTER_FUNCTION(DateDiff) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); factory.registerAlias("date_diff", FunctionDateDiff::name); factory.registerAlias("DATE_DIFF", FunctionDateDiff::name); factory.registerAlias("timestampDiff", FunctionDateDiff::name); @@ -509,12 +509,12 @@ Example: )", .examples{ {"typical", "SELECT timeDiff(UTCTimestamp(), now());", ""}}, - .categories{"Dates and Times"}}, FunctionFactory::CaseInsensitive); + .categories{"Dates and Times"}}, FunctionFactory::Case::Insensitive); } REGISTER_FUNCTION(Age) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/dateName.cpp b/src/Functions/dateName.cpp index c06dfe15dc4..8165ea1b8d3 100644 --- a/src/Functions/dateName.cpp +++ b/src/Functions/dateName.cpp @@ -354,7 +354,7 @@ private: REGISTER_FUNCTION(DateName) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index b8c60dd164e..dd3ea0b877b 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -178,7 +178,7 @@ REGISTER_FUNCTION(DateTrunc) factory.registerFunction(); /// Compatibility alias. - factory.registerAlias("DATE_TRUNC", "dateTrunc", FunctionFactory::CaseInsensitive); + factory.registerAlias("DATE_TRUNC", "dateTrunc", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/degrees.cpp b/src/Functions/degrees.cpp index 3aa20a77a0d..8646eb54d9a 100644 --- a/src/Functions/degrees.cpp +++ b/src/Functions/degrees.cpp @@ -23,7 +23,7 @@ namespace REGISTER_FUNCTION(Degrees) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/exp.cpp b/src/Functions/exp.cpp index d352cda7460..e67cbd6d819 100644 --- a/src/Functions/exp.cpp +++ b/src/Functions/exp.cpp @@ -36,7 +36,7 @@ using FunctionExp = FunctionMathUnary>; REGISTER_FUNCTION(Exp) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/extractAllGroupsVertical.cpp b/src/Functions/extractAllGroupsVertical.cpp index 87a0b4cf7bc..6a968d89354 100644 --- a/src/Functions/extractAllGroupsVertical.cpp +++ b/src/Functions/extractAllGroupsVertical.cpp @@ -18,7 +18,7 @@ namespace DB REGISTER_FUNCTION(ExtractAllGroupsVertical) { factory.registerFunction>(); - factory.registerAlias("extractAllGroups", VerticalImpl::Name, FunctionFactory::CaseSensitive); + factory.registerAlias("extractAllGroups", VerticalImpl::Name); } } diff --git a/src/Functions/factorial.cpp b/src/Functions/factorial.cpp index 7ff9126c004..9b319caad63 100644 --- a/src/Functions/factorial.cpp +++ b/src/Functions/factorial.cpp @@ -106,7 +106,7 @@ The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any ne )", .examples{{"factorial", "SELECT factorial(10)", ""}}, .categories{"Mathematical"}}, - FunctionFactory::CaseInsensitive); + FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index c3a5fe036c3..f89afd67e78 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -1834,10 +1834,10 @@ using FunctionFromUnixTimestampInJodaSyntax = FunctionFormatDateTimeImpl(); - factory.registerAlias("DATE_FORMAT", FunctionFormatDateTime::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("DATE_FORMAT", FunctionFormatDateTime::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); - factory.registerAlias("FROM_UNIXTIME", FunctionFromUnixTimestamp::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("FROM_UNIXTIME", FunctionFromUnixTimestamp::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/formatReadableDecimalSize.cpp b/src/Functions/formatReadableDecimalSize.cpp index 1aa5abc526e..9298360aebc 100644 --- a/src/Functions/formatReadableDecimalSize.cpp +++ b/src/Functions/formatReadableDecimalSize.cpp @@ -29,8 +29,7 @@ Accepts the size (number of bytes). Returns a rounded size with a suffix (KB, MB .examples{ {"formatReadableDecimalSize", "SELECT formatReadableDecimalSize(1000)", ""}}, .categories{"OtherFunctions"} - }, - FunctionFactory::CaseSensitive); + }); } } diff --git a/src/Functions/formatReadableSize.cpp b/src/Functions/formatReadableSize.cpp index 5c11603e9d7..ee66a0396df 100644 --- a/src/Functions/formatReadableSize.cpp +++ b/src/Functions/formatReadableSize.cpp @@ -22,7 +22,7 @@ namespace REGISTER_FUNCTION(FormatReadableSize) { factory.registerFunction>(); - factory.registerAlias("FORMAT_BYTES", Impl::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("FORMAT_BYTES", Impl::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/fromDaysSinceYearZero.cpp b/src/Functions/fromDaysSinceYearZero.cpp index 0543e6bf229..e1ba9ea533e 100644 --- a/src/Functions/fromDaysSinceYearZero.cpp +++ b/src/Functions/fromDaysSinceYearZero.cpp @@ -125,7 +125,7 @@ The calculation is the same as in MySQL's FROM_DAYS() function. .examples{{"typical", "SELECT fromDaysSinceYearZero32(713569)", "2023-09-08"}}, .categories{"Dates and Times"}}); - factory.registerAlias("FROM_DAYS", FunctionFromDaysSinceYearZero::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("FROM_DAYS", FunctionFromDaysSinceYearZero::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/generateULID.cpp b/src/Functions/generateULID.cpp index f2f2d8ae3b9..933618ccec3 100644 --- a/src/Functions/generateULID.cpp +++ b/src/Functions/generateULID.cpp @@ -85,8 +85,7 @@ The function returns a value of type FixedString(26). {"ulid", "SELECT generateULID()", ""}, {"multiple", "SELECT generateULID(1), generateULID(2)", ""}}, .categories{"ULID"} - }, - FunctionFactory::CaseSensitive); + }); } } diff --git a/src/Functions/greatest.cpp b/src/Functions/greatest.cpp index 87a48c887b4..88539bda4a5 100644 --- a/src/Functions/greatest.cpp +++ b/src/Functions/greatest.cpp @@ -65,7 +65,7 @@ using FunctionGreatest = FunctionBinaryArithmetic; REGISTER_FUNCTION(Greatest) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/hasSubsequence.cpp b/src/Functions/hasSubsequence.cpp index 4bcce53b4db..1426e8cb7a9 100644 --- a/src/Functions/hasSubsequence.cpp +++ b/src/Functions/hasSubsequence.cpp @@ -24,7 +24,7 @@ using FunctionHasSubsequence = HasSubsequenceImpl({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/hasSubsequenceCaseInsensitive.cpp b/src/Functions/hasSubsequenceCaseInsensitive.cpp index c93bbead58c..8e5751066a9 100644 --- a/src/Functions/hasSubsequenceCaseInsensitive.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitive.cpp @@ -23,7 +23,7 @@ using FunctionHasSubsequenceCaseInsensitive = HasSubsequenceImpl({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp index 18438bc8b16..039af061bf5 100644 --- a/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp +++ b/src/Functions/hasSubsequenceCaseInsensitiveUTF8.cpp @@ -25,7 +25,7 @@ using FunctionHasSubsequenceCaseInsensitiveUTF8 = HasSubsequenceImpl({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/hasSubsequenceUTF8.cpp b/src/Functions/hasSubsequenceUTF8.cpp index 7a22211eb8c..636fbfab85f 100644 --- a/src/Functions/hasSubsequenceUTF8.cpp +++ b/src/Functions/hasSubsequenceUTF8.cpp @@ -24,7 +24,7 @@ using FunctionHasSubsequenceUTF8 = HasSubsequenceImpl({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/hasToken.cpp b/src/Functions/hasToken.cpp index fa41abf2641..299a8a16b35 100644 --- a/src/Functions/hasToken.cpp +++ b/src/Functions/hasToken.cpp @@ -25,10 +25,10 @@ using FunctionHasTokenOrNull REGISTER_FUNCTION(HasToken) { factory.registerFunction(FunctionDocumentation - {.description="Performs lookup of needle in haystack using tokenbf_v1 index."}, FunctionFactory::CaseSensitive); + {.description="Performs lookup of needle in haystack using tokenbf_v1 index."}); factory.registerFunction(FunctionDocumentation - {.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."}, FunctionFactory::CaseSensitive); + {.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."}); } } diff --git a/src/Functions/hasTokenCaseInsensitive.cpp b/src/Functions/hasTokenCaseInsensitive.cpp index 32675b9384d..6ff134194e3 100644 --- a/src/Functions/hasTokenCaseInsensitive.cpp +++ b/src/Functions/hasTokenCaseInsensitive.cpp @@ -26,11 +26,11 @@ REGISTER_FUNCTION(HasTokenCaseInsensitive) { factory.registerFunction( FunctionDocumentation{.description="Performs case insensitive lookup of needle in haystack using tokenbf_v1 index."}, - DB::FunctionFactory::CaseInsensitive); + DB::FunctionFactory::Case::Insensitive); factory.registerFunction( FunctionDocumentation{.description="Performs case insensitive lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."}, - DB::FunctionFactory::CaseInsensitive); + DB::FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/hypot.cpp b/src/Functions/hypot.cpp index 465471cb09b..8845d1fa8ae 100644 --- a/src/Functions/hypot.cpp +++ b/src/Functions/hypot.cpp @@ -15,7 +15,7 @@ namespace REGISTER_FUNCTION(Hypot) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 30eaa26fa20..07dbee27a9d 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -1309,7 +1309,7 @@ public: REGISTER_FUNCTION(If) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } FunctionOverloadResolverPtr createInternalFunctionIfOverloadResolver(bool allow_experimental_variant_type, bool use_variant_as_common_type) diff --git a/src/Functions/ifNull.cpp b/src/Functions/ifNull.cpp index 1093f3f817f..358a52c8394 100644 --- a/src/Functions/ifNull.cpp +++ b/src/Functions/ifNull.cpp @@ -91,7 +91,7 @@ private: REGISTER_FUNCTION(IfNull) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/initcap.cpp b/src/Functions/initcap.cpp index 6b2958227bc..4661ce117c0 100644 --- a/src/Functions/initcap.cpp +++ b/src/Functions/initcap.cpp @@ -60,7 +60,7 @@ using FunctionInitcap = FunctionStringToString; REGISTER_FUNCTION(Initcap) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/initialQueryID.cpp b/src/Functions/initialQueryID.cpp index 9c9390d4e50..f32f92a2f46 100644 --- a/src/Functions/initialQueryID.cpp +++ b/src/Functions/initialQueryID.cpp @@ -41,6 +41,6 @@ public: REGISTER_FUNCTION(InitialQueryID) { factory.registerFunction(); - factory.registerAlias("initial_query_id", FunctionInitialQueryID::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("initial_query_id", FunctionInitialQueryID::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/isNull.cpp b/src/Functions/isNull.cpp index 95d659b103b..9347d81c45f 100644 --- a/src/Functions/isNull.cpp +++ b/src/Functions/isNull.cpp @@ -101,7 +101,7 @@ public: REGISTER_FUNCTION(IsNull) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/keyvaluepair/extractKeyValuePairs.cpp b/src/Functions/keyvaluepair/extractKeyValuePairs.cpp index 02e8412bbf3..1c5164e132d 100644 --- a/src/Functions/keyvaluepair/extractKeyValuePairs.cpp +++ b/src/Functions/keyvaluepair/extractKeyValuePairs.cpp @@ -241,7 +241,7 @@ REGISTER_FUNCTION(ExtractKeyValuePairs) └──────────────────┘ ```)"} ); - factory.registerAlias("str_to_map", NameExtractKeyValuePairs::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("str_to_map", NameExtractKeyValuePairs::name, FunctionFactory::Case::Insensitive); factory.registerAlias("mapFromString", NameExtractKeyValuePairs::name); } diff --git a/src/Functions/least.cpp b/src/Functions/least.cpp index babb8378d80..091a868e8e2 100644 --- a/src/Functions/least.cpp +++ b/src/Functions/least.cpp @@ -65,7 +65,7 @@ using FunctionLeast = FunctionBinaryArithmetic; REGISTER_FUNCTION(Least) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/left.cpp b/src/Functions/left.cpp index 006706c8f21..c9f62a0f8f1 100644 --- a/src/Functions/left.cpp +++ b/src/Functions/left.cpp @@ -6,8 +6,8 @@ namespace DB REGISTER_FUNCTION(Left) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerFunction>({}, FunctionFactory::CaseSensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); + factory.registerFunction>({}, FunctionFactory::Case::Sensitive); } } diff --git a/src/Functions/lengthUTF8.cpp b/src/Functions/lengthUTF8.cpp index 5a4af4934df..59a0d532602 100644 --- a/src/Functions/lengthUTF8.cpp +++ b/src/Functions/lengthUTF8.cpp @@ -83,8 +83,8 @@ REGISTER_FUNCTION(LengthUTF8) factory.registerFunction(); /// Compatibility aliases. - factory.registerAlias("CHAR_LENGTH", "lengthUTF8", FunctionFactory::CaseInsensitive); - factory.registerAlias("CHARACTER_LENGTH", "lengthUTF8", FunctionFactory::CaseInsensitive); + factory.registerAlias("CHAR_LENGTH", "lengthUTF8", FunctionFactory::Case::Insensitive); + factory.registerAlias("CHARACTER_LENGTH", "lengthUTF8", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/locate.cpp b/src/Functions/locate.cpp index d9a727ab3ef..076aa1bdc6d 100644 --- a/src/Functions/locate.cpp +++ b/src/Functions/locate.cpp @@ -29,6 +29,6 @@ REGISTER_FUNCTION(Locate) FunctionDocumentation::Categories doc_categories = {"String search"}; - factory.registerFunction({doc_description, doc_syntax, doc_arguments, doc_returned_value, doc_examples, doc_categories}, FunctionFactory::CaseInsensitive); + factory.registerFunction({doc_description, doc_syntax, doc_arguments, doc_returned_value, doc_examples, doc_categories}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/log.cpp b/src/Functions/log.cpp index 9096b8c6f22..8bebdb8d7bd 100644 --- a/src/Functions/log.cpp +++ b/src/Functions/log.cpp @@ -34,8 +34,8 @@ using FunctionLog = FunctionMathUnary>; REGISTER_FUNCTION(Log) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerAlias("ln", "log", FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerAlias("ln", "log", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/log10.cpp b/src/Functions/log10.cpp index 5dfe4ac9357..6241df3e092 100644 --- a/src/Functions/log10.cpp +++ b/src/Functions/log10.cpp @@ -13,7 +13,7 @@ using FunctionLog10 = FunctionMathUnary({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/log2.cpp b/src/Functions/log2.cpp index 9457ac64bc6..52b3ab52ea7 100644 --- a/src/Functions/log2.cpp +++ b/src/Functions/log2.cpp @@ -13,7 +13,7 @@ using FunctionLog2 = FunctionMathUnary>; REGISTER_FUNCTION(Log2) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/lower.cpp b/src/Functions/lower.cpp index 38ae5a8a7f0..5210a20b026 100644 --- a/src/Functions/lower.cpp +++ b/src/Functions/lower.cpp @@ -19,8 +19,8 @@ using FunctionLower = FunctionStringToString, NameLower REGISTER_FUNCTION(Lower) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerAlias("lcase", NameLower::name, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerAlias("lcase", NameLower::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/makeDate.cpp b/src/Functions/makeDate.cpp index 41a09793994..21d466d7708 100644 --- a/src/Functions/makeDate.cpp +++ b/src/Functions/makeDate.cpp @@ -724,7 +724,7 @@ public: REGISTER_FUNCTION(MakeDate) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); factory.registerFunction>(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/match.cpp b/src/Functions/match.cpp index c719cc6dd82..6cd65597032 100644 --- a/src/Functions/match.cpp +++ b/src/Functions/match.cpp @@ -20,7 +20,7 @@ using FunctionMatch = FunctionsStringSearch(); - factory.registerAlias("REGEXP_MATCHES", NameMatch::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("REGEXP_MATCHES", NameMatch::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/mathConstants.cpp b/src/Functions/mathConstants.cpp index 2b199a30616..37ababbc0e5 100644 --- a/src/Functions/mathConstants.cpp +++ b/src/Functions/mathConstants.cpp @@ -44,7 +44,7 @@ REGISTER_FUNCTION(E) REGISTER_FUNCTION(Pi) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/max2.cpp b/src/Functions/max2.cpp index 928e6f22918..88b5c7c08c0 100644 --- a/src/Functions/max2.cpp +++ b/src/Functions/max2.cpp @@ -21,6 +21,6 @@ namespace REGISTER_FUNCTION(Max2) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/min2.cpp b/src/Functions/min2.cpp index f031530edf5..8ab56dbe90d 100644 --- a/src/Functions/min2.cpp +++ b/src/Functions/min2.cpp @@ -22,6 +22,6 @@ namespace REGISTER_FUNCTION(Min2) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/modulo.cpp b/src/Functions/modulo.cpp index ebc1c4f5275..76a07aeda2e 100644 --- a/src/Functions/modulo.cpp +++ b/src/Functions/modulo.cpp @@ -155,7 +155,7 @@ using FunctionModulo = BinaryArithmeticOverloadResolver(); - factory.registerAlias("mod", "modulo", FunctionFactory::CaseInsensitive); + factory.registerAlias("mod", "modulo", FunctionFactory::Case::Insensitive); } struct NameModuloLegacy { static constexpr auto name = "moduloLegacy"; }; @@ -183,11 +183,11 @@ In other words, the function returning the modulus (modulo) in the terms of Modu )", .examples{{"positiveModulo", "SELECT positiveModulo(-1, 10);", ""}}, .categories{"Arithmetic"}}, - FunctionFactory::CaseInsensitive); + FunctionFactory::Case::Insensitive); - factory.registerAlias("positive_modulo", "positiveModulo", FunctionFactory::CaseInsensitive); + factory.registerAlias("positive_modulo", "positiveModulo", FunctionFactory::Case::Insensitive); /// Compatibility with Spark: - factory.registerAlias("pmod", "positiveModulo", FunctionFactory::CaseInsensitive); + factory.registerAlias("pmod", "positiveModulo", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/monthName.cpp b/src/Functions/monthName.cpp index f49f77bd6e7..ae444460170 100644 --- a/src/Functions/monthName.cpp +++ b/src/Functions/monthName.cpp @@ -74,7 +74,7 @@ private: REGISTER_FUNCTION(MonthName) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index 352c05f8f32..7b2150e3534 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -138,8 +138,8 @@ private: REGISTER_FUNCTION(Now) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerAlias("current_timestamp", NowOverloadResolver::name, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerAlias("current_timestamp", NowOverloadResolver::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index 23ab8cad003..9786a0c9f39 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -170,7 +170,7 @@ private: REGISTER_FUNCTION(Now64) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/nullIf.cpp b/src/Functions/nullIf.cpp index 392cc20cfcf..550287885a1 100644 --- a/src/Functions/nullIf.cpp +++ b/src/Functions/nullIf.cpp @@ -69,7 +69,7 @@ public: REGISTER_FUNCTION(NullIf) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/padString.cpp b/src/Functions/padString.cpp index 8670c837e21..23554c3fbbc 100644 --- a/src/Functions/padString.cpp +++ b/src/Functions/padString.cpp @@ -335,8 +335,8 @@ REGISTER_FUNCTION(PadString) factory.registerFunction>(); /// rightPad factory.registerFunction>(); /// rightPadUTF8 - factory.registerAlias("lpad", "leftPad", FunctionFactory::CaseInsensitive); - factory.registerAlias("rpad", "rightPad", FunctionFactory::CaseInsensitive); + factory.registerAlias("lpad", "leftPad", FunctionFactory::Case::Insensitive); + factory.registerAlias("rpad", "rightPad", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/parseDateTime.cpp b/src/Functions/parseDateTime.cpp index d2353c19a61..bdca0151bba 100644 --- a/src/Functions/parseDateTime.cpp +++ b/src/Functions/parseDateTime.cpp @@ -2098,10 +2098,10 @@ namespace REGISTER_FUNCTION(ParseDateTime) { factory.registerFunction(); - factory.registerAlias("TO_UNIXTIME", FunctionParseDateTime::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("TO_UNIXTIME", FunctionParseDateTime::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); - factory.registerAlias("str_to_date", FunctionParseDateTimeOrNull::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("str_to_date", FunctionParseDateTimeOrNull::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/position.cpp b/src/Functions/position.cpp index 29a5db2eb24..aad47cc5b3f 100644 --- a/src/Functions/position.cpp +++ b/src/Functions/position.cpp @@ -19,6 +19,6 @@ using FunctionPosition = FunctionsStringSearch({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/positionCaseInsensitive.cpp b/src/Functions/positionCaseInsensitive.cpp index f71ce0078cc..7c59ffa83cd 100644 --- a/src/Functions/positionCaseInsensitive.cpp +++ b/src/Functions/positionCaseInsensitive.cpp @@ -20,6 +20,6 @@ using FunctionPositionCaseInsensitive = FunctionsStringSearch(); - factory.registerAlias("instr", NamePositionCaseInsensitive::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("instr", NamePositionCaseInsensitive::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/pow.cpp b/src/Functions/pow.cpp index 9b383da97e7..f2976b4812e 100644 --- a/src/Functions/pow.cpp +++ b/src/Functions/pow.cpp @@ -13,8 +13,8 @@ using FunctionPow = FunctionMathBinaryFloat64({}, FunctionFactory::CaseInsensitive); - factory.registerAlias("power", "pow", FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerAlias("power", "pow", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/queryID.cpp b/src/Functions/queryID.cpp index 5d0ac719797..7299714e464 100644 --- a/src/Functions/queryID.cpp +++ b/src/Functions/queryID.cpp @@ -41,6 +41,6 @@ public: REGISTER_FUNCTION(QueryID) { factory.registerFunction(); - factory.registerAlias("query_id", FunctionQueryID::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("query_id", FunctionQueryID::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/radians.cpp b/src/Functions/radians.cpp index 2c2c2743532..9185340be15 100644 --- a/src/Functions/radians.cpp +++ b/src/Functions/radians.cpp @@ -23,7 +23,7 @@ namespace REGISTER_FUNCTION(Radians) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/rand.cpp b/src/Functions/rand.cpp index ea30922d731..35b325e59fd 100644 --- a/src/Functions/rand.cpp +++ b/src/Functions/rand.cpp @@ -13,7 +13,7 @@ using FunctionRand = FunctionRandom; REGISTER_FUNCTION(Rand) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); factory.registerAlias("rand32", NameRand::name); } diff --git a/src/Functions/regexpExtract.cpp b/src/Functions/regexpExtract.cpp index 3cc5393296c..6bedac54e39 100644 --- a/src/Functions/regexpExtract.cpp +++ b/src/Functions/regexpExtract.cpp @@ -253,7 +253,7 @@ REGISTER_FUNCTION(RegexpExtract) FunctionDocumentation{.description="Extracts the first string in haystack that matches the regexp pattern and corresponds to the regex group index."}); /// For Spark compatibility. - factory.registerAlias("REGEXP_EXTRACT", "regexpExtract", FunctionFactory::CaseInsensitive); + factory.registerAlias("REGEXP_EXTRACT", "regexpExtract", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index aa90bf2490d..c001959b465 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -278,7 +278,7 @@ public: REGISTER_FUNCTION(Repeat) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/replaceAll.cpp b/src/Functions/replaceAll.cpp index 6c06f5984b3..9ce525390bf 100644 --- a/src/Functions/replaceAll.cpp +++ b/src/Functions/replaceAll.cpp @@ -20,7 +20,7 @@ using FunctionReplaceAll = FunctionStringReplace(); - factory.registerAlias("replace", NameReplaceAll::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("replace", NameReplaceAll::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/replaceRegexpAll.cpp b/src/Functions/replaceRegexpAll.cpp index f5f56fb0f35..77f21b6efee 100644 --- a/src/Functions/replaceRegexpAll.cpp +++ b/src/Functions/replaceRegexpAll.cpp @@ -20,7 +20,7 @@ using FunctionReplaceRegexpAll = FunctionStringReplace(); - factory.registerAlias("REGEXP_REPLACE", NameReplaceRegexpAll::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("REGEXP_REPLACE", NameReplaceRegexpAll::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/reverse.cpp b/src/Functions/reverse.cpp index 39608b77997..d23e48b8d42 100644 --- a/src/Functions/reverse.cpp +++ b/src/Functions/reverse.cpp @@ -113,7 +113,7 @@ private: REGISTER_FUNCTION(Reverse) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/right.cpp b/src/Functions/right.cpp index a8ab4bf9685..ef3303ab968 100644 --- a/src/Functions/right.cpp +++ b/src/Functions/right.cpp @@ -6,8 +6,8 @@ namespace DB REGISTER_FUNCTION(Right) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerFunction>({}, FunctionFactory::CaseSensitive); + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); + factory.registerFunction>({}, FunctionFactory::Case::Sensitive); } } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 761c8964f12..fe999d66701 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -206,12 +206,12 @@ REGISTER_FUNCTION(Uptime) REGISTER_FUNCTION(Version) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } REGISTER_FUNCTION(Revision) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } REGISTER_FUNCTION(ZooKeeperSessionUptime) @@ -237,8 +237,7 @@ Returns the value of `display_name` from config or server FQDN if not set. )", .examples{{"displayName", "SELECT displayName();", ""}}, .categories{"Constant", "Miscellaneous"} - }, - FunctionFactory::CaseSensitive); + }); } diff --git a/src/Functions/sign.cpp b/src/Functions/sign.cpp index 3dd2ac8e3aa..914e1ad9e1f 100644 --- a/src/Functions/sign.cpp +++ b/src/Functions/sign.cpp @@ -44,7 +44,7 @@ struct FunctionUnaryArithmeticMonotonicity REGISTER_FUNCTION(Sign) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/sin.cpp b/src/Functions/sin.cpp index 914f431adb4..945486b26a8 100644 --- a/src/Functions/sin.cpp +++ b/src/Functions/sin.cpp @@ -21,7 +21,7 @@ REGISTER_FUNCTION(Sin) .returned_value = "The sine of x.", .examples = {{.name = "simple", .query = "SELECT sin(1.23)", .result = "0.9424888019316975"}}, .categories{"Mathematical", "Trigonometric"}}, - FunctionFactory::CaseInsensitive); + FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/soundex.cpp b/src/Functions/soundex.cpp index 77ddb14a6ec..fcf1523d1a3 100644 --- a/src/Functions/soundex.cpp +++ b/src/Functions/soundex.cpp @@ -112,7 +112,7 @@ struct NameSoundex REGISTER_FUNCTION(Soundex) { factory.registerFunction>( - FunctionDocumentation{.description="Returns Soundex code of a string."}, FunctionFactory::CaseInsensitive); + FunctionDocumentation{.description="Returns Soundex code of a string."}, FunctionFactory::Case::Insensitive); } diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index ce12f2f541c..cd6ca73c088 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -173,7 +173,7 @@ public: REGISTER_FUNCTION(Space) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/sqrt.cpp b/src/Functions/sqrt.cpp index 3c50f994391..a6e2dee71d9 100644 --- a/src/Functions/sqrt.cpp +++ b/src/Functions/sqrt.cpp @@ -13,7 +13,7 @@ using FunctionSqrt = FunctionMathUnary>; REGISTER_FUNCTION(Sqrt) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/structureToFormatSchema.cpp b/src/Functions/structureToFormatSchema.cpp index 406da372c04..4fc2bf707a4 100644 --- a/src/Functions/structureToFormatSchema.cpp +++ b/src/Functions/structureToFormatSchema.cpp @@ -116,8 +116,7 @@ Function that converts ClickHouse table structure to CapnProto format schema "}"}, }, .categories{"Other"} - }, - FunctionFactory::CaseSensitive); + }); } @@ -138,8 +137,7 @@ Function that converts ClickHouse table structure to Protobuf format schema "}"}, }, .categories{"Other"} - }, - FunctionFactory::CaseSensitive); + }); } } diff --git a/src/Functions/substring.cpp b/src/Functions/substring.cpp index f1dea7db018..51980eb6b9c 100644 --- a/src/Functions/substring.cpp +++ b/src/Functions/substring.cpp @@ -201,12 +201,12 @@ public: REGISTER_FUNCTION(Substring) { - factory.registerFunction>({}, FunctionFactory::CaseInsensitive); - factory.registerAlias("substr", "substring", FunctionFactory::CaseInsensitive); // MySQL alias - factory.registerAlias("mid", "substring", FunctionFactory::CaseInsensitive); /// MySQL alias - factory.registerAlias("byteSlice", "substring", FunctionFactory::CaseInsensitive); /// resembles PostgreSQL's get_byte function, similar to ClickHouse's bitSlice + factory.registerFunction>({}, FunctionFactory::Case::Insensitive); + factory.registerAlias("substr", "substring", FunctionFactory::Case::Insensitive); // MySQL alias + factory.registerAlias("mid", "substring", FunctionFactory::Case::Insensitive); /// MySQL alias + factory.registerAlias("byteSlice", "substring", FunctionFactory::Case::Insensitive); /// resembles PostgreSQL's get_byte function, similar to ClickHouse's bitSlice - factory.registerFunction>({}, FunctionFactory::CaseSensitive); + factory.registerFunction>(); } } diff --git a/src/Functions/substringIndex.cpp b/src/Functions/substringIndex.cpp index 15a321bd5b0..eccd849059b 100644 --- a/src/Functions/substringIndex.cpp +++ b/src/Functions/substringIndex.cpp @@ -314,7 +314,7 @@ REGISTER_FUNCTION(SubstringIndex) factory.registerFunction>(); /// substringIndex factory.registerFunction>(); /// substringIndexUTF8 - factory.registerAlias("SUBSTRING_INDEX", "substringIndex", FunctionFactory::CaseInsensitive); + factory.registerAlias("SUBSTRING_INDEX", "substringIndex", FunctionFactory::Case::Insensitive); } diff --git a/src/Functions/synonyms.cpp b/src/Functions/synonyms.cpp index 28dd83627d9..18c1557115f 100644 --- a/src/Functions/synonyms.cpp +++ b/src/Functions/synonyms.cpp @@ -121,7 +121,7 @@ public: REGISTER_FUNCTION(Synonyms) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/tan.cpp b/src/Functions/tan.cpp index e39f8598419..51cf0bbcceb 100644 --- a/src/Functions/tan.cpp +++ b/src/Functions/tan.cpp @@ -13,7 +13,7 @@ using FunctionTan = FunctionMathUnary>; REGISTER_FUNCTION(Tan) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/tanh.cpp b/src/Functions/tanh.cpp index bdefa5263d7..62755737f70 100644 --- a/src/Functions/tanh.cpp +++ b/src/Functions/tanh.cpp @@ -39,7 +39,7 @@ using FunctionTanh = FunctionMathUnary>; REGISTER_FUNCTION(Tanh) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/timestamp.cpp b/src/Functions/timestamp.cpp index 6f2bd2030d5..c2e10a2d220 100644 --- a/src/Functions/timestamp.cpp +++ b/src/Functions/timestamp.cpp @@ -187,7 +187,7 @@ If the second argument 'expr_time' is provided, it adds the specified time to th {"timestamp", "SELECT timestamp('2013-12-31 12:00:00')", "2013-12-31 12:00:00.000000"}, {"timestamp", "SELECT timestamp('2013-12-31 12:00:00', '12:00:00.11')", "2014-01-01 00:00:00.110000"}, }, - .categories{"DateTime"}}, FunctionFactory::CaseInsensitive); + .categories{"DateTime"}}, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toCustomWeek.cpp b/src/Functions/toCustomWeek.cpp index 98e7aaf1d6b..61c0767654e 100644 --- a/src/Functions/toCustomWeek.cpp +++ b/src/Functions/toCustomWeek.cpp @@ -21,8 +21,8 @@ REGISTER_FUNCTION(ToCustomWeek) factory.registerFunction(); /// Compatibility aliases for mysql. - factory.registerAlias("week", "toWeek", FunctionFactory::CaseInsensitive); - factory.registerAlias("yearweek", "toYearWeek", FunctionFactory::CaseInsensitive); + factory.registerAlias("week", "toWeek", FunctionFactory::Case::Insensitive); + factory.registerAlias("yearweek", "toYearWeek", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toDayOfMonth.cpp b/src/Functions/toDayOfMonth.cpp index c20b0b75797..93013c3528b 100644 --- a/src/Functions/toDayOfMonth.cpp +++ b/src/Functions/toDayOfMonth.cpp @@ -14,8 +14,8 @@ REGISTER_FUNCTION(ToDayOfMonth) factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("DAY", "toDayOfMonth", FunctionFactory::CaseInsensitive); - factory.registerAlias("DAYOFMONTH", "toDayOfMonth", FunctionFactory::CaseInsensitive); + factory.registerAlias("DAY", "toDayOfMonth", FunctionFactory::Case::Insensitive); + factory.registerAlias("DAYOFMONTH", "toDayOfMonth", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toDayOfWeek.cpp b/src/Functions/toDayOfWeek.cpp index dc508d70814..d1f55bbddab 100644 --- a/src/Functions/toDayOfWeek.cpp +++ b/src/Functions/toDayOfWeek.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToDayOfWeek) factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("DAYOFWEEK", "toDayOfWeek", FunctionFactory::CaseInsensitive); + factory.registerAlias("DAYOFWEEK", "toDayOfWeek", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toDayOfYear.cpp b/src/Functions/toDayOfYear.cpp index 0cbafd6275a..9a27c41b0ed 100644 --- a/src/Functions/toDayOfYear.cpp +++ b/src/Functions/toDayOfYear.cpp @@ -14,7 +14,7 @@ REGISTER_FUNCTION(ToDayOfYear) factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("DAYOFYEAR", "toDayOfYear", FunctionFactory::CaseInsensitive); + factory.registerAlias("DAYOFYEAR", "toDayOfYear", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toDaysSinceYearZero.cpp b/src/Functions/toDaysSinceYearZero.cpp index f6239b2900b..b5c053a11b3 100644 --- a/src/Functions/toDaysSinceYearZero.cpp +++ b/src/Functions/toDaysSinceYearZero.cpp @@ -20,7 +20,7 @@ The calculation is the same as in MySQL's TO_DAYS() function. .categories{"Dates and Times"}}); /// MySQL compatibility alias. - factory.registerAlias("TO_DAYS", FunctionToDaysSinceYearZero::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("TO_DAYS", FunctionToDaysSinceYearZero::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toDecimalString.cpp b/src/Functions/toDecimalString.cpp index 4ee664ad237..523948a5396 100644 --- a/src/Functions/toDecimalString.cpp +++ b/src/Functions/toDecimalString.cpp @@ -273,7 +273,7 @@ second argument is the desired number of digits in fractional part. Returns Stri )", .examples{{"toDecimalString", "SELECT toDecimalString(2.1456,2)", ""}}, .categories{"String"} - }, FunctionFactory::CaseInsensitive); + }, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toHour.cpp b/src/Functions/toHour.cpp index fc9ec657adf..bc122538661 100644 --- a/src/Functions/toHour.cpp +++ b/src/Functions/toHour.cpp @@ -14,7 +14,7 @@ REGISTER_FUNCTION(ToHour) factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("HOUR", "toHour", FunctionFactory::CaseInsensitive); + factory.registerAlias("HOUR", "toHour", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toLastDayOfMonth.cpp b/src/Functions/toLastDayOfMonth.cpp index 9365880bfb8..004ae2718e7 100644 --- a/src/Functions/toLastDayOfMonth.cpp +++ b/src/Functions/toLastDayOfMonth.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToLastDayOfMonth) factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("LAST_DAY", "toLastDayOfMonth", FunctionFactory::CaseInsensitive); + factory.registerAlias("LAST_DAY", "toLastDayOfMonth", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toMillisecond.cpp b/src/Functions/toMillisecond.cpp index aaef517c996..efa08c322a2 100644 --- a/src/Functions/toMillisecond.cpp +++ b/src/Functions/toMillisecond.cpp @@ -27,7 +27,7 @@ Returns the millisecond component (0-999) of a date with time. ); /// MySQL compatibility alias. - factory.registerAlias("MILLISECOND", "toMillisecond", FunctionFactory::CaseInsensitive); + factory.registerAlias("MILLISECOND", "toMillisecond", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toMinute.cpp b/src/Functions/toMinute.cpp index 162ecb282df..291da33d2e8 100644 --- a/src/Functions/toMinute.cpp +++ b/src/Functions/toMinute.cpp @@ -14,7 +14,7 @@ REGISTER_FUNCTION(ToMinute) factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("MINUTE", "toMinute", FunctionFactory::CaseInsensitive); + factory.registerAlias("MINUTE", "toMinute", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toMonth.cpp b/src/Functions/toMonth.cpp index 422f21e7df8..3ef73bf1be3 100644 --- a/src/Functions/toMonth.cpp +++ b/src/Functions/toMonth.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToMonth) { factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("MONTH", "toMonth", FunctionFactory::CaseInsensitive); + factory.registerAlias("MONTH", "toMonth", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toQuarter.cpp b/src/Functions/toQuarter.cpp index 3c301095ff2..2e6d4fa93de 100644 --- a/src/Functions/toQuarter.cpp +++ b/src/Functions/toQuarter.cpp @@ -13,7 +13,7 @@ REGISTER_FUNCTION(ToQuarter) { factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("QUARTER", "toQuarter", FunctionFactory::CaseInsensitive); + factory.registerAlias("QUARTER", "toQuarter", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toSecond.cpp b/src/Functions/toSecond.cpp index 372097fd488..1ad3b46fbd7 100644 --- a/src/Functions/toSecond.cpp +++ b/src/Functions/toSecond.cpp @@ -14,7 +14,7 @@ REGISTER_FUNCTION(ToSecond) factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("SECOND", "toSecond", FunctionFactory::CaseInsensitive); + factory.registerAlias("SECOND", "toSecond", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/toYear.cpp b/src/Functions/toYear.cpp index 75479adb82c..0d2c8136337 100644 --- a/src/Functions/toYear.cpp +++ b/src/Functions/toYear.cpp @@ -14,7 +14,7 @@ REGISTER_FUNCTION(ToYear) factory.registerFunction(); /// MySQL compatibility alias. - factory.registerAlias("YEAR", "toYear", FunctionFactory::CaseInsensitive); + factory.registerAlias("YEAR", "toYear", FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/today.cpp b/src/Functions/today.cpp index 356660fa7b5..88eddc9b60e 100644 --- a/src/Functions/today.cpp +++ b/src/Functions/today.cpp @@ -84,8 +84,8 @@ public: REGISTER_FUNCTION(Today) { factory.registerFunction(); - factory.registerAlias("current_date", TodayOverloadResolver::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("curdate", TodayOverloadResolver::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("current_date", TodayOverloadResolver::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("curdate", TodayOverloadResolver::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/upper.cpp b/src/Functions/upper.cpp index 3e1c7b1d800..5af0f059e3f 100644 --- a/src/Functions/upper.cpp +++ b/src/Functions/upper.cpp @@ -18,8 +18,8 @@ using FunctionUpper = FunctionStringToString, NameUpper REGISTER_FUNCTION(Upper) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); - factory.registerAlias("ucase", FunctionUpper::name, FunctionFactory::CaseInsensitive); + factory.registerFunction({}, FunctionFactory::Case::Insensitive); + factory.registerAlias("ucase", FunctionUpper::name, FunctionFactory::Case::Insensitive); } } diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 7a2598f5f4b..5e23493c86d 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -1576,9 +1576,9 @@ using TupleOrArrayFunctionCosineDistance = TupleOrArrayFunction(); - factory.registerAlias("vectorSum", FunctionTuplePlus::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("vectorSum", FunctionTuplePlus::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); - factory.registerAlias("vectorDifference", FunctionTupleMinus::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("vectorDifference", FunctionTupleMinus::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); @@ -1652,7 +1652,7 @@ If the types of the first interval (or the interval in the tuple) and the second factory.registerFunction(); factory.registerFunction(); - factory.registerAlias("scalarProduct", TupleOrArrayFunctionDotProduct::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("scalarProduct", TupleOrArrayFunctionDotProduct::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); @@ -1660,11 +1660,11 @@ If the types of the first interval (or the interval in the tuple) and the second factory.registerFunction(); factory.registerFunction(); - factory.registerAlias("normL1", TupleOrArrayFunctionL1Norm::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("normL2", TupleOrArrayFunctionL2Norm::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("normL2Squared", TupleOrArrayFunctionL2SquaredNorm::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("normLinf", TupleOrArrayFunctionLinfNorm::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("normLp", FunctionLpNorm::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("normL1", TupleOrArrayFunctionL1Norm::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("normL2", TupleOrArrayFunctionL2Norm::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("normL2Squared", TupleOrArrayFunctionL2SquaredNorm::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("normLinf", TupleOrArrayFunctionLinfNorm::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("normLp", FunctionLpNorm::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); @@ -1672,21 +1672,21 @@ If the types of the first interval (or the interval in the tuple) and the second factory.registerFunction(); factory.registerFunction(); - factory.registerAlias("distanceL1", FunctionL1Distance::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("distanceL2", FunctionL2Distance::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("distanceL2Squared", FunctionL2SquaredDistance::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("distanceLinf", FunctionLinfDistance::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("distanceLp", FunctionLpDistance::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("distanceL1", FunctionL1Distance::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("distanceL2", FunctionL2Distance::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("distanceL2Squared", FunctionL2SquaredDistance::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("distanceLinf", FunctionLinfDistance::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("distanceLp", FunctionLpDistance::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerAlias("normalizeL1", FunctionL1Normalize::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("normalizeL2", FunctionL2Normalize::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("normalizeLinf", FunctionLinfNormalize::name, FunctionFactory::CaseInsensitive); - factory.registerAlias("normalizeLp", FunctionLpNormalize::name, FunctionFactory::CaseInsensitive); + factory.registerAlias("normalizeL1", FunctionL1Normalize::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("normalizeL2", FunctionL2Normalize::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("normalizeLinf", FunctionLinfNormalize::name, FunctionFactory::Case::Insensitive); + factory.registerAlias("normalizeLp", FunctionLpNormalize::name, FunctionFactory::Case::Insensitive); factory.registerFunction(); } diff --git a/src/Functions/widthBucket.cpp b/src/Functions/widthBucket.cpp index e804808b699..d007cc968f0 100644 --- a/src/Functions/widthBucket.cpp +++ b/src/Functions/widthBucket.cpp @@ -287,7 +287,7 @@ Result: .categories{"Mathematical"}, }); - factory.registerAlias("width_bucket", "widthBucket", FunctionFactory::CaseInsensitive); + factory.registerAlias("width_bucket", "widthBucket", FunctionFactory::Case::Insensitive); } } diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index a694fa43e46..86421adf4fb 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2716,42 +2716,42 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::CaseInsensitive); + }, properties}, AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("dense_rank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::CaseInsensitive); + }, properties}, AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("percent_rank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::CaseInsensitive); + }, properties}, AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::CaseInsensitive); + }, properties}, AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("ntile", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::CaseInsensitive); + }, properties}, AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("nth_value", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared( name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::CaseInsensitive); + }, properties}, AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("lagInFrame", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 6ecdeb47779..e505535ae76 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -19,17 +19,17 @@ namespace ErrorCodes } void TableFunctionFactory::registerFunction( - const std::string & name, Value value, CaseSensitiveness case_sensitiveness) + const std::string & name, Value value, Case case_sensitiveness) { if (!table_functions.emplace(name, value).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "TableFunctionFactory: the table function name '{}' is not unique", name); - if (case_sensitiveness == CaseInsensitive + if (case_sensitiveness == Case::Insensitive && !case_insensitive_table_functions.emplace(Poco::toLower(name), value).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "TableFunctionFactory: " "the case insensitive table function name '{}' is not unique", name); - KnownTableFunctionNames::instance().add(name, (case_sensitiveness == CaseInsensitive)); + KnownTableFunctionNames::instance().add(name, (case_sensitiveness == Case::Insensitive)); } TableFunctionPtr TableFunctionFactory::get( diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index 2cc648ba181..adc74c2e735 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -48,10 +48,10 @@ public: void registerFunction( const std::string & name, Value value, - CaseSensitiveness case_sensitiveness = CaseSensitive); + Case case_sensitiveness = Case::Sensitive); template - void registerFunction(TableFunctionProperties properties = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) + void registerFunction(TableFunctionProperties properties = {}, Case case_sensitiveness = Case::Sensitive) { auto creator = []() -> TableFunctionPtr { return std::make_shared(); }; registerFunction(Function::name, diff --git a/src/TableFunctions/TableFunctionFormat.cpp b/src/TableFunctions/TableFunctionFormat.cpp index 5a0f47653d5..7e4fdea1ff3 100644 --- a/src/TableFunctions/TableFunctionFormat.cpp +++ b/src/TableFunctions/TableFunctionFormat.cpp @@ -219,7 +219,7 @@ Result: void registerTableFunctionFormat(TableFunctionFactory & factory) { - factory.registerFunction({format_table_function_documentation, false}, TableFunctionFactory::CaseInsensitive); + factory.registerFunction({format_table_function_documentation, false}, TableFunctionFactory::Case::Insensitive); } } diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index 4b56fa57091..95c531f8a3b 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -174,7 +174,7 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, Context void registerTableFunctionValues(TableFunctionFactory & factory) { - factory.registerFunction({.documentation = {}, .allow_readonly = true}, TableFunctionFactory::CaseInsensitive); + factory.registerFunction({.documentation = {}, .allow_readonly = true}, TableFunctionFactory::Case::Insensitive); } } From 3985a4012d70762480d50083eb0d30a3b70e0981 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jul 2024 00:06:25 +0200 Subject: [PATCH 091/145] Fix tidy --- src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 4e62c503d65..4a4deb07eee 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -155,7 +155,7 @@ void printExceptionWithRespectToAbort(LoggerPtr log, const String & query_id) { std::rethrow_exception(ex); } - catch (const TestException &) + catch (const TestException &) // NOLINT { /// Exception from a unit test, ignore it. } From 787a9137df53d6c46f0b6f47ba21b23f49f8e749 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jul 2024 03:08:34 +0200 Subject: [PATCH 092/145] 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 4217d029c0f6b69b0b67c414f2b6df750392561b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jul 2024 03:33:36 +0200 Subject: [PATCH 093/145] Fix something --- programs/server/Server.cpp | 3 +- programs/server/Server.h | 3 +- src/Databases/IDatabase.h | 1 - src/Storages/MergeTree/AlterConversions.h | 1 - .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 +- src/Storages/StorageMergeTree.cpp | 50 +++++++------------ src/Storages/StorageMergeTree.h | 2 +- 7 files changed, 23 insertions(+), 41 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 053ddaf8d8b..0b695c3dde6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2731,8 +2731,7 @@ void Server::createInterserverServers( void Server::stopServers( std::vector & servers, - const ServerType & server_type -) const + const ServerType & server_type) const { LoggerRawPtr log = &logger(); diff --git a/programs/server/Server.h b/programs/server/Server.h index 3f03dd137ef..feaf61f1ffd 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -129,8 +129,7 @@ private: void stopServers( std::vector & servers, - const ServerType & server_type - ) const; + const ServerType & server_type) const; }; } 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 diff --git a/src/Storages/MergeTree/AlterConversions.h b/src/Storages/MergeTree/AlterConversions.h index 0f857d351dd..7bcd4af8b36 100644 --- a/src/Storages/MergeTree/AlterConversions.h +++ b/src/Storages/MergeTree/AlterConversions.h @@ -8,7 +8,6 @@ namespace DB { - /// Alter conversions which should be applied on-fly for part. /// Built from of the most recent mutation commands for part. /// Now only ALTER RENAME COLUMN is applied. diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3c223b8d748..140a226f2d1 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -118,11 +118,11 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() const occupied >= data_settings->max_number_of_mutations_for_replica) return 0; - /// DataPart can be store only at one disk. Get maximum reservable free space at all disks. + /// A DataPart can be stored only at a single disk. Get the maximum reservable free space at all disks. UInt64 disk_space = data.getStoragePolicy()->getMaxUnreservedFreeSpace(); auto max_tasks_count = data.getContext()->getMergeMutateExecutor()->getMaxTasksCount(); - /// Allow mutations only if there are enough threads, leave free threads for merges else + /// Allow mutations only if there are enough threads, otherwise, leave free threads for merges. if (occupied <= 1 || max_tasks_count - occupied >= data_settings->number_of_free_entries_in_pool_to_execute_mutation) return static_cast(disk_space / DISK_USAGE_COEFFICIENT_TO_RESERVE); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 40b3a12297b..444b3fbae4c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -11,22 +11,17 @@ #include #include #include -#include #include #include #include #include #include -#include #include -#include #include -#include #include #include #include #include -#include #include #include #include @@ -35,19 +30,14 @@ #include #include #include -#include #include #include #include -#include #include -#include #include #include #include -#include #include -#include namespace DB @@ -1197,7 +1187,6 @@ bool StorageMergeTree::merge( task->setCurrentTransaction(MergeTreeTransactionHolder{}, MergeTreeTransactionPtr{txn}); executeHere(task); - return true; } @@ -1493,7 +1482,6 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign scheduled = true; } - return scheduled; } @@ -1600,9 +1588,7 @@ bool StorageMergeTree::optimize( if (!partition && final) { if (cleanup && this->merging_params.mode != MergingParams::Mode::Replacing) - { throw Exception(ErrorCodes::CANNOT_ASSIGN_OPTIMIZE, "Cannot OPTIMIZE with CLEANUP table: only ReplacingMergeTree can be CLEANUP"); - } if (cleanup && !getSettings()->allow_experimental_replacing_merge_with_cleanup) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed"); @@ -1616,15 +1602,15 @@ bool StorageMergeTree::optimize( for (const String & partition_id : partition_ids) { if (!merge( - true, - partition_id, - true, - deduplicate, - deduplicate_by_columns, - cleanup, - txn, - disable_reason, - local_context->getSettingsRef().optimize_skip_merged_partitions)) + true, + partition_id, + true, + deduplicate, + deduplicate_by_columns, + cleanup, + txn, + disable_reason, + local_context->getSettingsRef().optimize_skip_merged_partitions)) { constexpr auto message = "Cannot OPTIMIZE table: {}"; if (disable_reason.text.empty()) @@ -1644,15 +1630,15 @@ bool StorageMergeTree::optimize( partition_id = getPartitionIDFromQuery(partition, local_context); if (!merge( - true, - partition_id, - final, - deduplicate, - deduplicate_by_columns, - cleanup, - txn, - disable_reason, - local_context->getSettingsRef().optimize_skip_merged_partitions)) + true, + partition_id, + final, + deduplicate, + deduplicate_by_columns, + cleanup, + txn, + disable_reason, + local_context->getSettingsRef().optimize_skip_merged_partitions)) { constexpr auto message = "Cannot OPTIMIZE table: {}"; if (disable_reason.text.empty()) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 4d819508934..064b51739bd 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -211,7 +211,6 @@ private: bool optimize_skip_merged_partitions = false, SelectPartsDecision * select_decision_out = nullptr); - MergeMutateSelectedEntryPtr selectPartsToMutate( const StorageMetadataPtr & metadata_snapshot, PreformattedMessage & disable_reason, TableLockHolder & table_lock_holder, std::unique_lock & currently_processing_in_background_mutex_lock); @@ -310,6 +309,7 @@ private: }; protected: + /// Collect mutations that have to be applied on the fly: currently they are only RENAME COLUMN. MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const override; }; From f16a05cf2d31c1306e89e2fa3e990233a5d34288 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 10:51:53 +0000 Subject: [PATCH 094/145] 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 87fafaa9f5f8406228440631acf597c782a3ecdd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Jul 2024 11:00:50 +0000 Subject: [PATCH 095/145] Remove flaky case from the test. --- ..._finctions_and_column_sparse_bug.reference | 73 ------------------- ...window_finctions_and_column_sparse_bug.sql | 54 -------------- 2 files changed, 127 deletions(-) diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference index 356329a392d..13e229432ae 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.reference @@ -8,76 +8,3 @@ true 1 1 true 1 2 --- -755809149 0 ---- -1 -2081147898 -1 -1981899149 -2 -2051538534 -2 -1650266905 -3 -1975508531 -3 -1646738223 -4 -1700730666 -4 -1618912877 -5 -1465484835 -5 -1317193174 -6 -1458338029 -6 -1219769753 -7 -1450619195 -7 -1154269118 -8 -1365934326 -8 -1150980622 -9 -1203382363 -9 -1098155311 -10 -1197430632 -10 -841067875 -11 -1176267855 -11 -816935497 -12 -1020892864 -12 -599948807 -13 -991301833 -13 -526570556 -14 -685902265 -14 -504713125 -15 -653505826 -15 -411038390 -16 -451392958 -16 -331834394 -17 -262516786 -17 -176934810 -18 -222873194 -18 -2 -19 -153185515 -19 6 -20 -74234560 -20 255 -21 -41 -21 406615258 -22 -6 -22 541758331 -23 -5 -23 720985423 -24 -3 -24 745669725 -25 15 -25 897064234 -26 65535 -26 1116921321 -27 77089559 -27 1207796283 -28 100663045 -28 1603772265 -29 561061873 -29 1664059402 -30 643897141 -30 1688303275 -31 914629990 -31 1913361922 -32 1159852204 -32 1929066636 -33 1258218855 -33 1968095908 -34 1459407556 -34 2054878592 -35 1936334332 -35 2125075305 -36 1962246186 -37 2030467062 diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql index 6e326d0a67f..f2391e0d165 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql @@ -31,57 +31,3 @@ SELECT * FROM ( SELECT c, min(w) OVER (PARTITION BY s ORDER BY c ASC, s ASC, w ASC) FROM t limit toUInt64(-1)) WHERE c = -755809149; - -SELECT '---'; - -create table t_vkx4cc ( - c_ylzjpt Int32, - c_hqfr9 Bool , - ) engine = MergeTree order by c_ylzjpt; - -system stop merges t_vkx4cc; - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-2081147898, coalesce((NOT NOT(cast( (53 < 539704722) as Nullable(Bool)))), true)), (-1219769753, coalesce((true) and (false), false)), (-1981899149, coalesce(false, false)), (-1646738223, coalesce((NOT NOT(cast( (23.5 <= -26) as Nullable(Bool)))), false)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (255, coalesce(false, false)), (-1317193174, coalesce(false, false)), (-41, coalesce(true, false)), (1929066636, coalesce(false, true)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-1700730666, coalesce((NOT NOT(cast( (-2022515471055597472 AND -29) as Nullable(Bool)))), false)), (1664059402, coalesce((NOT NOT(cast( (-19643 >= -122) as Nullable(Bool)))), false)), (1688303275, coalesce((NOT NOT(cast( (737275892 < 105) as Nullable(Bool)))), true)), (406615258, coalesce((NOT NOT(cast( (-657730213 = 82.86) as Nullable(Bool)))), false)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-2, coalesce(false, false)), (1962246186, coalesce(true, false)), (-991301833, coalesce(true, true)), (2054878592, coalesce(false, false)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (643897141, coalesce((NOT NOT(cast( (-60 AND cast(null as Nullable(Int64))) as Nullable(Bool)))), true)), (-2051538534, coalesce(((-1616816511 between 332225780 and -1883087387)) or ((-573375170 between -1427445977 and 615586748)), false)), (77089559, coalesce((NOT NOT(cast( ((true) and (true) != 925456787) as Nullable(Bool)))), false)), (1116921321, coalesce((0 is NULL), true)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-816935497, coalesce(false, false)), (1207796283, coalesce((-129 between -5 and -5), false)), (-1365934326, coalesce(true, false)), (-1618912877, coalesce((NOT NOT(cast( (false >= 31833) as Nullable(Bool)))), false)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-331834394, coalesce((NOT NOT(cast( (-63 <= -1822810052) as Nullable(Bool)))), true)), (-1020892864, coalesce((NOT NOT(cast( (40.31 <= 8146037365746019777) as Nullable(Bool)))), true)), (-1150980622, coalesce(((94019304 between -730556489 and 32)) and ((-956354236 is not NULL)), true)), (-1203382363, coalesce(true, true)); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9) values (-653505826, coalesce((true) or (true), false)), (-1975508531, coalesce(((-796885845 between 65536 and cast(null as Nullable(Int32)))) or ((NOT NOT(cast( (-7467729336434250795 < 100.20) as Nullable(Bool))))), false)), (-1465484835, coalesce(((NOT NOT(cast( (19209 <= 75.96) as Nullable(Bool))))) or (true), false)), (1968095908, coalesce((NOT NOT(cast( (-1309960412156062327 > 13102) as Nullable(Bool)))), true)); - -alter table t_vkx4cc add column c_zosphq2t1 Float64; - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-153185515, coalesce((NOT NOT(cast( (1291639145 >= 30.22) as Nullable(Bool)))), false), -1.8), (-411038390, coalesce(((-762326135 between 16 and 177530758)) or (false), true), 26.34), (914629990, coalesce((-1125832977 is not NULL), true), 59.2), (541758331, coalesce(false, true), -255.1); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (2125075305, coalesce(false, false), 55.36), (-1176267855, coalesce(true, true), 55.45), (1459407556, coalesce((true) and ((NOT NOT(cast( (95.96 != 65) as Nullable(Bool))))), true), 85.80), (-1098155311, coalesce(false, false), 2147483649.9); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (6, coalesce((NOT NOT(cast( (1546334968 < -4) as Nullable(Bool)))), true), 57.42), (-5, coalesce((NOT NOT(cast( (59 AND 13) as Nullable(Bool)))), false), 65536.3), (100663045, coalesce((-1190355242 is not NULL), true), 73.80), (-451392958, coalesce((NOT NOT(cast( (false != -443845933) as Nullable(Bool)))), false), -4294967294.0); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (561061873, coalesce(true, false), 12.17), (-526570556, coalesce(false, false), 64.73), (-1450619195, coalesce(true, true), 54.33), (-3, coalesce(true, true), 52.9); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-504713125, coalesce(false, true), 27.58), (897064234, coalesce((836516994 between cast(null as Nullable(Int32)) and -1832647080), true), 9223372036854775809.2), (65535, coalesce(true, true), 4294967297.5), (-599948807, coalesce((false) or ((NOT NOT(cast( (6.52 = 65.49) as Nullable(Bool))))), false), 256.5); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-1650266905, coalesce((NOT NOT(cast( (-83 = -218055084) as Nullable(Bool)))), true), 1.9), (-841067875, coalesce(false, true), -126.5), (15, coalesce(((NOT NOT(cast( (cast(null as Nullable(Decimal)) = cast(null as Nullable(Int32))) as Nullable(Bool))))) or (true), true), 33.65), (1913361922, coalesce((NOT NOT(cast( (false AND 0) as Nullable(Bool)))), false), 6.4); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (1159852204, coalesce((-2057115045 is not NULL), false), 20.61), (-6, coalesce(true, true), 66.33), (-1154269118, coalesce(false, true), 8.89), (1258218855, coalesce(true, false), 19.80); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (1603772265, coalesce(false, true), 57.87), (-176934810, coalesce(false, true), 128.8), (-1458338029, coalesce((NOT NOT(cast( (20908 != (NOT NOT(cast( (cast(null as Nullable(Decimal)) <= (true) or ((NOT NOT(cast( (973511022 <= -112) as Nullable(Bool)))))) as Nullable(Bool))))) as Nullable(Bool)))), true), 76.54), (-262516786, coalesce((cast(null as Nullable(Int32)) is NULL), false), 21.49); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (-1197430632, coalesce(true, false), 45.40), (-685902265, coalesce((NOT NOT(cast( (cast(null as Nullable(Decimal)) < cast(null as Nullable(Decimal))) as Nullable(Bool)))), true), 5.55), (1936334332, coalesce((-1565552735 is not NULL), false), 26.28), (2030467062, coalesce((NOT NOT(cast( (127.3 != cast(null as Nullable(Int32))) as Nullable(Bool)))), true), 89.50); - -insert into t_vkx4cc (c_ylzjpt, c_hqfr9, c_zosphq2t1) values (720985423, coalesce((NOT NOT(cast( (-451448940 = cast(null as Nullable(Decimal))) as Nullable(Bool)))), false), 52.65), (-222873194, coalesce(((-20 between -1419620477 and 1616455043)) or ((25624502 between 1312431316 and 1757361651)), false), 127.2), (745669725, coalesce((NOT NOT(cast( ((NOT NOT(cast( (cast(null as Nullable(UInt64)) <= 42) as Nullable(Bool)))) >= 3233811255032796928) as Nullable(Bool)))), false), 7.74), (-74234560, coalesce((NOT NOT(cast( (cast(null as Nullable(Decimal)) >= cast(null as Nullable(Decimal))) as Nullable(Bool)))), true), 19.25); - -SELECT DISTINCT - count(ref_0.c_zosphq2t1) over (partition by ref_0.c_hqfr9 order by ref_0.c_ylzjpt, ref_0.c_hqfr9, ref_0.c_zosphq2t1) as c0, - ref_0.c_ylzjpt as c1 -FROM - t_vkx4cc as ref_0 - order by c0, c1; From 959ac9a768530cb9e2c7b013df37bfee000a1644 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 18 Jul 2024 13:16:35 +0200 Subject: [PATCH 096/145] ci: dump dmesg in case of OOM Without additional info it is unclear how to tune paralelism or maybe split some modules. Signed-off-by: Azat Khuzhin --- tests/ci/ci.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index cf285f4b97d..c5ad97088aa 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1281,7 +1281,8 @@ def main() -> int: except ValueError: pass if Utils.is_killed_with_oom(): - print("WARNING: OOM while job execution") + print("WARNING: OOM while job execution:") + print(subprocess.run("sudo dmesg -T", check=False)) error = f"Out Of Memory, exit_code {job_report.exit_code}, after {int(job_report.duration)}s" else: error = f"Unknown, exit_code {job_report.exit_code}, after {int(job_report.duration)}s" From e3be4a95bd6118ca789642c56edf88b6a880c02e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 18 Jul 2024 13:29:02 +0200 Subject: [PATCH 097/145] fix clang 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 5c1db5fc66170fd2b194962b2914f48b21e15453 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 18 Jul 2024 13:30:22 +0200 Subject: [PATCH 098/145] Fix column injection in merges after drop column --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 ++----- src/Storages/MergeTree/IMergeTreeDataPart.h | 4 +++- src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h | 2 +- .../MergeTree/LoadedMergeTreeDataPartInfoForReader.h | 5 ++++- src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp | 3 ++- src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 5 ++++- ...parallel_alter_add_drop_column_zookeeper_on_steroids.sh | 4 ++-- 7 files changed, 18 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 15863e74455..c2f87018872 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -652,15 +652,12 @@ size_t IMergeTreeDataPart::getFileSizeOrZero(const String & file_name) const return checksum->second.file_size; } -String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const +String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(const NamesAndTypesList & available_columns) const { - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns(with_subcolumns); - auto columns_list = columns_description.get(options); - std::optional minimum_size_column; UInt64 minimum_size = std::numeric_limits::max(); - for (const auto & column : columns_list) + for (const auto & column : available_columns) { if (!hasColumnFiles(column)) continue; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 9fd481b0d8e..85ef0472ce7 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -196,7 +196,9 @@ public: /// Returns the name of a column with minimum compressed size (as returned by getColumnSize()). /// If no checksums are present returns the name of the first physically existing column. - String getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const; + /// We pass a list of available columns since the ones available in the current storage snapshot might be smaller + /// than the one the table has (e.g a DROP COLUMN happened) and we don't want to get a column not in the snapshot + String getColumnNameWithMinimumCompressedSize(const NamesAndTypesList & available_columns) const; bool contains(const IMergeTreeDataPart & other) const { return info.contains(other.info); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h index ccc88079daa..7d4fb1df1c2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartInfoForReader.h @@ -47,7 +47,7 @@ public: virtual std::optional getColumnPosition(const String & column_name) const = 0; - virtual String getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const = 0; + virtual String getColumnNameWithMinimumCompressedSize(const NamesAndTypesList & available_columns) const = 0; virtual const MergeTreeDataPartChecksums & getChecksums() const = 0; diff --git a/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h b/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h index f5111ccaacc..aff1cf0edb0 100644 --- a/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h +++ b/src/Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h @@ -36,7 +36,10 @@ public: AlterConversionsPtr getAlterConversions() const override { return alter_conversions; } - String getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const override { return data_part->getColumnNameWithMinimumCompressedSize(with_subcolumns); } + String getColumnNameWithMinimumCompressedSize(const NamesAndTypesList & available_columns) const override + { + return data_part->getColumnNameWithMinimumCompressedSize(available_columns); + } const MergeTreeDataPartChecksums & getChecksums() const override { return data_part->checksums; } diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 570387a7046..aaa4ecd8eee 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -127,7 +127,8 @@ NameSet injectRequiredColumns( */ if (!have_at_least_one_physical_column) { - const auto minimum_size_column_name = data_part_info_for_reader.getColumnNameWithMinimumCompressedSize(with_subcolumns); + auto available_columns = storage_snapshot->metadata->getColumns().get(options); + const auto minimum_size_column_name = data_part_info_for_reader.getColumnNameWithMinimumCompressedSize(available_columns); columns.push_back(minimum_size_column_name); /// correctly report added column injected_columns.insert(columns.back()); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index ee1a9b7f8ed..5bab523a9f1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -266,10 +266,13 @@ void MergeTreeDataPartWide::doCheckConsistency(bool require_part_metadata) const bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const { + auto serialization = tryGetSerialization(column.name); + if (!serialization) + return false; auto marks_file_extension = index_granularity_info.mark_type.getFileExtension(); bool res = true; - getSerialization(column.name)->enumerateStreams([&](const auto & substream_path) + serialization->enumerateStreams([&](const auto & substream_path) { auto stream_name = getStreamNameForColumn(column, substream_path, checksums); if (!stream_name || !checksums.files.contains(*stream_name + marks_file_extension)) diff --git a/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh b/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh index c27dfffcfc2..b3d0b08948b 100755 --- a/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh +++ b/tests/queries/0_stateless/03144_parallel_alter_add_drop_column_zookeeper_on_steroids.sh @@ -30,9 +30,9 @@ function alter_thread() while true; do REPLICA=$(($RANDOM % 3 + 1)) ADD=$(($RANDOM % 5 + 1)) - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_steroids_$REPLICA ADD COLUMN value$ADD UInt32 DEFAULT 42 SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_steroids_$REPLICA ADD COLUMN value$ADD UInt32 DEFAULT 42 SETTINGS replication_alter_partitions_sync=0"; # additionally we don't wait anything for more heavy concurrency DROP=$(($RANDOM % 5 + 1)) - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_steroids_$REPLICA DROP COLUMN value$DROP SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_steroids_$REPLICA DROP COLUMN value$DROP SETTINGS replication_alter_partitions_sync=0"; # additionally we don't wait anything for more heavy concurrency sleep 0.$RANDOM done } From 617840c7270bf90068e04d348db138fd8c1f3456 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 18 Jul 2024 15:19:41 +0200 Subject: [PATCH 099/145] add test --- ...duplication_remote_insert_select.reference | 0 ...008_deduplication_remote_insert_select.sql | 45 +++++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 tests/queries/0_stateless/03008_deduplication_remote_insert_select.reference create mode 100644 tests/queries/0_stateless/03008_deduplication_remote_insert_select.sql diff --git a/tests/queries/0_stateless/03008_deduplication_remote_insert_select.reference b/tests/queries/0_stateless/03008_deduplication_remote_insert_select.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03008_deduplication_remote_insert_select.sql b/tests/queries/0_stateless/03008_deduplication_remote_insert_select.sql new file mode 100644 index 00000000000..1c229cdbc26 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_remote_insert_select.sql @@ -0,0 +1,45 @@ +DROP TABLE IF EXISTS tt; +CREATE TABLE src (a UInt64, b UInt64) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_remote_insert_select/src', '{replica}') + ORDER BY tuple(); + +INSERT INTO src SELECT number % 10 as a, number as b FROM numbers(50); + +SET allow_experimental_parallel_reading_from_replicas=1; +SET max_parallel_replicas=3; +SET parallel_replicas_for_non_replicated_merge_tree=1; +SET cluster_for_parallel_replicas='parallel_replicas'; + +SELECT count() FROM remote('127.0.0.{1..6}', currentDatabase(), src); + +CREATE TABLE dst_null(a UInt64, b UInt64) ENGINE = Null; + +set allow_deprecated_syntax_for_merge_tree=1; +CREATE MATERIALIZED VIEW mv_dst +ENGINE = AggregatingMergeTree() +ORDER BY a +AS SELECT + a, + sumState(b) AS sum_b, + uniqState(b) AS uniq_b +FROM dst_null +GROUP BY a; + +INSERT INTO dst_null +SELECT + a, + b +FROM src; + +SELECT + a, + sumMerge(sum_b) AS sum_b, + uniqMerge(uniq_b) AS uniq_b +FROM mv_dst +GROUP BY a +ORDER BY a; + +DROP TABLE src; +DROP TABLE mv_dst; +DROP TABLE dst_null; + From e128d88ff11f78be47e0b6ae52164f1313e2ed6c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 18 Jul 2024 16:42:47 +0200 Subject: [PATCH 100/145] fix CollectionOfDerivedItems::append to appendIfUniq --- src/Common/CollectionOfDerived.h | 13 +++--- src/Interpreters/Squashing.cpp | 2 +- .../DeduplicationTokenTransforms.cpp | 2 +- ...duplication_remote_insert_select.reference | 36 +++++++++++++++ ...008_deduplication_remote_insert_select.sql | 44 +++++++++++-------- 5 files changed, 71 insertions(+), 26 deletions(-) diff --git a/src/Common/CollectionOfDerived.h b/src/Common/CollectionOfDerived.h index 97c0c3fbc06..9f80ff727b4 100644 --- a/src/Common/CollectionOfDerived.h +++ b/src/Common/CollectionOfDerived.h @@ -84,12 +84,18 @@ public: return result; } - void append(Self && other) + // append items for other inscnace only if there is no such item in current instance + void appendIfUniq(Self && other) { auto middle_idx = records.size(); std::move(other.records.begin(), other.records.end(), std::back_inserter(records)); + // merge is stable std::inplace_merge(records.begin(), records.begin() + middle_idx, records.end()); - chassert(isUniqTypes()); + // remove duplicates + records.erase(std::unique(records.begin(), records.end()), records.end()); + + assert(std::is_sorted(records.begin(), records.end())); + assert(isUniqTypes()); } template @@ -142,7 +148,6 @@ private: bool isUniqTypes() const { auto uniq_it = std::adjacent_find(records.begin(), records.end()); - return uniq_it == records.end(); } @@ -161,8 +166,6 @@ private: records.emplace(it, type_idx, item); - - chassert(isUniqTypes()); } Records::const_iterator getImpl(std::type_index type_idx) const diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 3a7f28ed837..488177c3b4f 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -134,7 +134,7 @@ Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoColl Chunk result; result.setColumns(std::move(mutable_columns), rows); result.setChunkInfos(infos); - result.getChunkInfos().append(std::move(input_chunks.back().getChunkInfos())); + result.getChunkInfos().appendIfUniq(std::move(input_chunks.back().getChunkInfos())); chassert(result); return result; diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp index e6f7e44e026..841090f029e 100644 --- a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes void RestoreChunkInfosTransform::transform(Chunk & chunk) { - chunk.getChunkInfos().append(chunk_infos.clone()); + chunk.getChunkInfos().appendIfUniq(chunk_infos.clone()); } namespace DeduplicationToken diff --git a/tests/queries/0_stateless/03008_deduplication_remote_insert_select.reference b/tests/queries/0_stateless/03008_deduplication_remote_insert_select.reference index e69de29bb2d..9dd45974e40 100644 --- a/tests/queries/0_stateless/03008_deduplication_remote_insert_select.reference +++ b/tests/queries/0_stateless/03008_deduplication_remote_insert_select.reference @@ -0,0 +1,36 @@ +-- { echoOn } +SELECT count() FROM src; +100 +SELECT a, sum(b), uniq(b), FROM src GROUP BY a ORDER BY a; +0 450 10 +1 460 10 +2 470 10 +3 480 10 +4 490 10 +5 500 10 +6 510 10 +7 520 10 +8 530 10 +9 540 10 +SELECT count() FROM remote('127.0.0.{1..2}', currentDatabase(), src); +200 +-- { echoOn } +INSERT INTO dst_null + SELECT a, b FROM src; +SELECT + a, + sumMerge(sum_b) AS sum_b, + uniqMerge(uniq_b) AS uniq_b +FROM mv_dst +GROUP BY a +ORDER BY a; +0 450 10 +1 460 10 +2 470 10 +3 480 10 +4 490 10 +5 500 10 +6 510 10 +7 520 10 +8 530 10 +9 540 10 diff --git a/tests/queries/0_stateless/03008_deduplication_remote_insert_select.sql b/tests/queries/0_stateless/03008_deduplication_remote_insert_select.sql index 1c229cdbc26..c8e092822da 100644 --- a/tests/queries/0_stateless/03008_deduplication_remote_insert_select.sql +++ b/tests/queries/0_stateless/03008_deduplication_remote_insert_select.sql @@ -1,43 +1,49 @@ -DROP TABLE IF EXISTS tt; +DROP TABLE IF EXISTS src; + CREATE TABLE src (a UInt64, b UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_remote_insert_select/src', '{replica}') ORDER BY tuple(); -INSERT INTO src SELECT number % 10 as a, number as b FROM numbers(50); +INSERT INTO src SELECT number % 10 as a, number as b FROM numbers(100); SET allow_experimental_parallel_reading_from_replicas=1; SET max_parallel_replicas=3; SET parallel_replicas_for_non_replicated_merge_tree=1; SET cluster_for_parallel_replicas='parallel_replicas'; -SELECT count() FROM remote('127.0.0.{1..6}', currentDatabase(), src); +-- { echoOn } +SELECT count() FROM src; +SELECT a, sum(b), uniq(b), FROM src GROUP BY a ORDER BY a; +SELECT count() FROM remote('127.0.0.{1..2}', currentDatabase(), src); +-- { echoOff } -CREATE TABLE dst_null(a UInt64, b UInt64) ENGINE = Null; +DROP TABLE IF EXISTS dst_null; +CREATE TABLE dst_null(a UInt64, b UInt64) + ENGINE = Null; -set allow_deprecated_syntax_for_merge_tree=1; +DROP TABLE IF EXISTS mv_dst; CREATE MATERIALIZED VIEW mv_dst -ENGINE = AggregatingMergeTree() -ORDER BY a -AS SELECT - a, - sumState(b) AS sum_b, - uniqState(b) AS uniq_b -FROM dst_null -GROUP BY a; + ENGINE = AggregatingMergeTree() + ORDER BY a + AS SELECT + a, + sumState(b) AS sum_b, + uniqState(b) AS uniq_b + FROM dst_null + GROUP BY a; +-- { echoOn } INSERT INTO dst_null -SELECT - a, - b -FROM src; + SELECT a, b FROM src; SELECT a, - sumMerge(sum_b) AS sum_b, - uniqMerge(uniq_b) AS uniq_b + sumMerge(sum_b) AS sum_b, + uniqMerge(uniq_b) AS uniq_b FROM mv_dst GROUP BY a ORDER BY a; +-- { echoOff } DROP TABLE src; DROP TABLE mv_dst; From 81fb9b18f623bc85b1044242ab908d50731ae624 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 18 Jul 2024 17:41:36 +0200 Subject: [PATCH 101/145] 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 102/145] 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 103/145] 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 9eba5975d8d5738998418bb0be613aede5f77c6b Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 18 Jul 2024 18:43:34 +0200 Subject: [PATCH 104/145] CI: Fix issue with a skipped Build report --- tests/ci/ci_cache.py | 5 ----- tests/ci/ci_settings.py | 4 ++++ tests/ci/test_ci_options.py | 16 +++++++++++++--- 3 files changed, 17 insertions(+), 8 deletions(-) diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index 9486a286a8d..cfefb954fcd 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -737,17 +737,12 @@ class CiCache: if job_name not in required_builds: remove_from_to_do.append(job_name) - if not required_builds: - remove_from_to_do.append(CI.JobNames.BUILD_CHECK) - for job in remove_from_to_do: print(f"Filter job [{job}] - not affected by the change") if job in self.jobs_to_do: del self.jobs_to_do[job] if job in self.jobs_to_wait: del self.jobs_to_wait[job] - if job in self.jobs_to_skip: - self.jobs_to_skip.remove(job) def await_pending_jobs(self, is_release: bool, dry_run: bool = False) -> None: """ diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index 7b2dd12c310..54323ef868f 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -160,6 +160,10 @@ class CiSettings: else: return False + if CI.is_build_job(job): + print(f"Build job [{job}] - always run") + return True + if self.exclude_keywords: for keyword in self.exclude_keywords: if keyword in normalize_string(job): diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index f4d14a17512..f71320abf2c 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -197,6 +197,10 @@ class TestCIOptions(unittest.TestCase): "package_debug", "package_msan", "package_ubsan", + "package_aarch64", + "package_release_coverage", + "package_tsan", + "binary_release", "Stateless tests (asan)", "Stateless tests (azure, asan)", "Stateless tests flaky check (asan)", @@ -276,6 +280,7 @@ class TestCIOptions(unittest.TestCase): filtered_jobs, [ "Style check", + "fuzzers", ], ) @@ -291,9 +296,7 @@ class TestCIOptions(unittest.TestCase): ) self.assertCountEqual( filtered_jobs, - [ - "Style check", - ], + ["Style check", "fuzzers"], ) def test_options_applied_4(self): @@ -329,5 +332,12 @@ class TestCIOptions(unittest.TestCase): "Stateless tests (release, old analyzer, s3, DatabaseReplicated)", "package_asan", "fuzzers", + "package_aarch64", + "package_release_coverage", + "package_debug", + "package_tsan", + "package_msan", + "package_ubsan", + "binary_release", ], ) From f50538c596050974c7afef5a87e10c97c2b1656b Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 18 Jul 2024 19:31:49 +0200 Subject: [PATCH 105/145] 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 106/145] 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 4249d869a952595a124528e1af6b6a91d21e3d3e Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 18 Jul 2024 19:48:13 +0200 Subject: [PATCH 107/145] fix tests --- tests/ci/test_ci_config.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 4336783e0d5..12e863c4d8d 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -295,6 +295,13 @@ class TestCIConfig(unittest.TestCase): continue expected_jobs_to_do.append(job) for job, config in CI.JOB_CONFIGS.items(): + if ( + CI.is_build_job(job) + and not config.run_by_label + and job not in expected_jobs_to_do + ): + # expected to run all builds jobs + expected_jobs_to_do.append(job) if not any( keyword in normalize_string(job) for keyword in settings.include_keywords From 36468c528debe37bb6023a37d7ed9b93e7b56836 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 18 Jul 2024 19:55:09 +0200 Subject: [PATCH 108/145] 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 109/145] 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 110/145] 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 111/145] 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 112/145] 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 113/145] 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 114/145] 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 115/145] 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 116/145] 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 117/145] 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 118/145] 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 119/145] 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 69ad57a2a52d50510b4ec5beea08c0c599846859 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jul 2024 03:58:07 +0200 Subject: [PATCH 120/145] Update 03205_parallel_window_finctions_and_column_sparse_bug.sql --- .../03205_parallel_window_finctions_and_column_sparse_bug.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql index f2391e0d165..4cc54bb5ac2 100644 --- a/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql +++ b/tests/queries/0_stateless/03205_parallel_window_finctions_and_column_sparse_bug.sql @@ -6,7 +6,7 @@ insert into t values (1, 0); insert into t values (1, 1); insert into t values (1, 0)(1, 1); -SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t qualify c8=1 order by d settings max_threads=2; +SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t qualify c8=1 order by d settings max_threads=2, allow_experimental_analyzer = 1; SELECT '---'; SELECT d, c, row_number() over (partition by d order by c) as c8 FROM t order by d, c8 settings max_threads=2; SELECT '---'; From f6380dac0cbf08f913069606beeea433fb7f8857 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jul 2024 04:03:28 +0200 Subject: [PATCH 121/145] 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 122/145] 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 123/145] 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 124/145] 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 125/145] 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 3fb01ed2c9154d79cd9d23e3ae2e8708e86d0a34 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Fri, 19 Jul 2024 12:10:28 +0000 Subject: [PATCH 126/145] Use nonexistent address to check connection error at table creation --- tests/integration/test_storage_rabbitmq/test.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 3240039ee81..c163f3f7aed 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2220,13 +2220,11 @@ def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): def test_rabbitmq_no_connection_at_startup_1(rabbitmq_cluster): - # no connection when table is initialized - rabbitmq_cluster.pause_container("rabbitmq1") - instance.query_and_get_error( + error = instance.query_and_get_error( """ CREATE TABLE test.cs (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = 'no_connection_at_startup:5672', rabbitmq_exchange_name = 'cs', rabbitmq_format = 'JSONEachRow', rabbitmq_flush_interval_ms=1000, @@ -2234,7 +2232,7 @@ def test_rabbitmq_no_connection_at_startup_1(rabbitmq_cluster): rabbitmq_row_delimiter = '\\n'; """ ) - rabbitmq_cluster.unpause_container("rabbitmq1") + assert "CANNOT_CONNECT_RABBITMQ" in error def test_rabbitmq_no_connection_at_startup_2(rabbitmq_cluster): From 09cf5b1188d8b7a7485647ad62c936a7b28d41c7 Mon Sep 17 00:00:00 2001 From: lgbo-ustc Date: Wed, 17 Jul 2024 09:31:31 +0800 Subject: [PATCH 127/145] 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 128/145] 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 129/145] 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 130/145] 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 131/145] 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 132/145] 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 133/145] 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 134/145] 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 135/145] 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 136/145] 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 137/145] 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 138/145] 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 139/145] 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 140/145] 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 141/145] 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 142/145] 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" --- From ee3c0e7e1f37a4e9388866145e9cbb8f0220e42c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jul 2024 21:50:53 +0200 Subject: [PATCH 143/145] Better diagnostics in `test_disk_configuration` --- .../test_disk_configuration/test.py | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_disk_configuration/test.py b/tests/integration/test_disk_configuration/test.py index c003ff85755..afc5303298c 100644 --- a/tests/integration/test_disk_configuration/test.py +++ b/tests/integration/test_disk_configuration/test.py @@ -208,13 +208,21 @@ def test_merge_tree_custom_disk_setting(start_cluster): secret_access_key='minio123'); """ ) - count = len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) + + list1 = list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)) + count1 = len(list1) + node1.query(f"INSERT INTO {TABLE_NAME}_3 SELECT number FROM numbers(100)") assert int(node1.query(f"SELECT count() FROM {TABLE_NAME}_3")) == 100 - assert ( - len(list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True))) - == count - ) + + list2 = list(minio.list_objects(cluster.minio_bucket, "data/", recursive=True)) + count2 = len(list2) + + if count1 != count2: + print("list1: ", list1) + print("list2: ", list2) + + assert count1 == count2 assert ( len(list(minio.list_objects(cluster.minio_bucket, "data2/", recursive=True))) > 0 From f9b97aac84d6ab5f377261e5b57c2d6e8a432cef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jul 2024 23:19:33 +0200 Subject: [PATCH 144/145] Fix bad test `02950_part_log_bytes_uncompressed` --- .../0_stateless/02950_part_log_bytes_uncompressed.sql | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql b/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql index 248475ab84b..cfed02eaeeb 100644 --- a/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql +++ b/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql @@ -1,3 +1,6 @@ +-- Tags: no-random-merge-tree-settings, no-random-settings +-- Because we compare part sizes, and they could be affected by index granularity and index compression settings. + CREATE TABLE part_log_bytes_uncompressed ( key UInt8, value UInt8 @@ -17,7 +20,8 @@ ALTER TABLE part_log_bytes_uncompressed DROP PART 'all_4_4_0' SETTINGS mutations SYSTEM FLUSH LOGS; -SELECT event_type, table, part_name, bytes_uncompressed > 0, size_in_bytes < bytes_uncompressed FROM system.part_log +SELECT event_type, table, part_name, bytes_uncompressed > 0, size_in_bytes < bytes_uncompressed ? '1' : toString((size_in_bytes, bytes_uncompressed)) +FROM system.part_log WHERE event_date >= yesterday() AND database = currentDatabase() AND table = 'part_log_bytes_uncompressed' AND (event_type != 'RemovePart' OR part_name = 'all_4_4_0') -- ignore removal of other parts ORDER BY part_name, event_type; From 67f51153661f37370f4e365ed3f19c6a4b84c2f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 04:16:31 +0200 Subject: [PATCH 145/145] Update 02950_part_log_bytes_uncompressed.sql --- tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql b/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql index cfed02eaeeb..24425062116 100644 --- a/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql +++ b/tests/queries/0_stateless/02950_part_log_bytes_uncompressed.sql @@ -20,7 +20,7 @@ ALTER TABLE part_log_bytes_uncompressed DROP PART 'all_4_4_0' SETTINGS mutations SYSTEM FLUSH LOGS; -SELECT event_type, table, part_name, bytes_uncompressed > 0, size_in_bytes < bytes_uncompressed ? '1' : toString((size_in_bytes, bytes_uncompressed)) +SELECT event_type, table, part_name, bytes_uncompressed > 0, (bytes_uncompressed > 0 ? (size_in_bytes < bytes_uncompressed ? '1' : toString((size_in_bytes, bytes_uncompressed))) : '0') FROM system.part_log WHERE event_date >= yesterday() AND database = currentDatabase() AND table = 'part_log_bytes_uncompressed' AND (event_type != 'RemovePart' OR part_name = 'all_4_4_0') -- ignore removal of other parts