From 7338b560a82be77ccd4c2a880899798906c9b0ca Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Sep 2023 14:37:24 +0200 Subject: [PATCH 001/120] S3Queue fixes --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 535 ++++++++---------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 125 ++-- src/Storages/S3Queue/S3QueueSettings.cpp | 4 +- src/Storages/S3Queue/S3QueueSource.cpp | 246 ++------ src/Storages/S3Queue/S3QueueSource.h | 108 +--- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 20 +- src/Storages/S3Queue/S3QueueTableMetadata.h | 9 +- src/Storages/S3Queue/StorageS3Queue.cpp | 470 ++++++--------- src/Storages/S3Queue/StorageS3Queue.h | 107 ++-- .../integration/test_storage_s3_queue/test.py | 48 +- 10 files changed, 619 insertions(+), 1053 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 4624566a517..d0e4d0f88cc 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,18 +1,18 @@ -#include "IO/VarInt.h" #include "config.h" #if USE_AWS_S3 -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include - +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { @@ -30,151 +30,6 @@ namespace } } -void S3QueueFilesMetadata::S3QueueCollection::read(ReadBuffer & in) -{ - files = {}; - if (in.eof()) - return; - - size_t files_num; - in >> files_num >> "\n"; - while (files_num--) - { - TrackedCollectionItem item; - in >> item.file_path >> "\n"; - in >> item.timestamp >> "\n"; - in >> item.retries_count >> "\n"; - in >> item.last_exception >> "\n"; - files.push_back(item); - } -} - -void S3QueueFilesMetadata::S3QueueCollection::write(WriteBuffer & out) const -{ - out << files.size() << "\n"; - for (const auto & processed_file : files) - { - out << processed_file.file_path << "\n"; - out << processed_file.timestamp << "\n"; - out << processed_file.retries_count << "\n"; - out << processed_file.last_exception << "\n"; - } -} - -String S3QueueFilesMetadata::S3QueueCollection::toString() const -{ - WriteBufferFromOwnString out; - write(out); - return out.str(); -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueCollection::getFileNames() -{ - S3FilesCollection keys = {}; - for (const auto & pair : files) - keys.insert(pair.file_path); - return keys; -} - - -S3QueueFilesMetadata::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) - : max_size(max_size_), max_age(max_age_) -{ -} - -void S3QueueFilesMetadata::S3QueueProcessedCollection::parse(const String & collection_str) -{ - ReadBufferFromString buf(collection_str); - read(buf); - if (max_age > 0) // Remove old items - { - std::erase_if( - files, - [timestamp = getCurrentTime(), this](const TrackedCollectionItem & processed_file) - { return (timestamp - processed_file.timestamp) > max_age; }); - } -} - - -void S3QueueFilesMetadata::S3QueueProcessedCollection::add(const String & file_name) -{ - TrackedCollectionItem processed_file; - processed_file.file_path = file_name; - processed_file.timestamp = getCurrentTime(); - files.push_back(processed_file); - - /// TODO: it is strange that in parse() we take into account only max_age, but here only max_size. - while (files.size() > max_size) - { - files.pop_front(); - } -} - - -S3QueueFilesMetadata::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & max_retries_count_) - : max_retries_count(max_retries_count_) -{ -} - -void S3QueueFilesMetadata::S3QueueFailedCollection::parse(const String & collection_str) -{ - ReadBufferFromString buf(collection_str); - read(buf); -} - - -bool S3QueueFilesMetadata::S3QueueFailedCollection::add(const String & file_name, const String & exception_message) -{ - auto failed_it = std::find_if( - files.begin(), files.end(), - [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); - - if (failed_it == files.end()) - { - files.emplace_back(file_name, 0, max_retries_count, exception_message); - } - else if (failed_it->retries_count == 0 || --failed_it->retries_count == 0) - { - return false; - } - return true; -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueFailedCollection::getFileNames() -{ - S3FilesCollection failed_keys; - for (const auto & pair : files) - { - if (pair.retries_count == 0) - failed_keys.insert(pair.file_path); - } - return failed_keys; -} - -void S3QueueFilesMetadata::S3QueueProcessingCollection::parse(const String & collection_str) -{ - ReadBufferFromString rb(collection_str); - Strings result; - readQuoted(result, rb); - files = S3FilesCollection(result.begin(), result.end()); -} - -void S3QueueFilesMetadata::S3QueueProcessingCollection::add(const Strings & file_names) -{ - files.insert(file_names.begin(), file_names.end()); -} - -void S3QueueFilesMetadata::S3QueueProcessingCollection::remove(const String & file_name) -{ - files.erase(file_name); -} - -String S3QueueFilesMetadata::S3QueueProcessingCollection::toString() const -{ - return DB::toString(Strings(files.begin(), files.end())); -} - - S3QueueFilesMetadata::S3QueueFilesMetadata( const StorageS3Queue * storage_, const S3QueueSettings & settings_) @@ -183,171 +38,273 @@ S3QueueFilesMetadata::S3QueueFilesMetadata( , max_set_size(settings_.s3queue_tracked_files_limit.value) , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) , max_loading_retries(settings_.s3queue_loading_retries.value) - , zookeeper_processing_path(fs::path(storage->getZooKeeperPath()) / "processing") - , zookeeper_processed_path(fs::path(storage->getZooKeeperPath()) / "processed") - , zookeeper_failed_path(fs::path(storage->getZooKeeperPath()) / "failed") - , zookeeper_lock_path(fs::path(storage->getZooKeeperPath()) / "lock") + , zookeeper_processing_path(storage->getZooKeeperPath() / "processing") + , zookeeper_processed_path(storage->getZooKeeperPath() / "processed") + , zookeeper_failed_path(storage->getZooKeeperPath() / "failed") , log(&Poco::Logger::get("S3QueueFilesMetadata")) { } -void S3QueueFilesMetadata::setFileProcessed(const String & file_path) +std::string S3QueueFilesMetadata::NodeMetadata::toString() const { - auto zookeeper = storage->getZooKeeper(); - auto lock = acquireLock(zookeeper); + Poco::JSON::Object json; + json.set("file_path", file_path); + json.set("last_processed_timestamp", getCurrentTime()); + json.set("last_exception", last_exception); + json.set("retries", retries); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); +} + +S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::NodeMetadata::fromString(const std::string & metadata_str) +{ + Poco::JSON::Parser parser; + auto json = parser.parse(metadata_str).extract(); + + NodeMetadata metadata; + metadata.file_path = json->getValue("file_path"); + metadata.last_processed_timestamp = json->getValue("last_processed_timestamp"); + metadata.last_exception = json->getValue("last_exception"); + metadata.retries = json->getValue("retries"); + return metadata; +} + +std::string S3QueueFilesMetadata::getNodeName(const std::string & path) +{ + SipHash path_hash; + path_hash.update(path); + return toString(path_hash.get64()); +} + +S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( + const std::string & path, + const std::string & exception, + size_t retries) +{ + NodeMetadata metadata; + metadata.file_path = path; + metadata.last_processed_timestamp = getCurrentTime(); + metadata.last_exception = exception; + metadata.retries = retries; + return metadata; +} + +bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) +{ switch (mode) { - case S3QueueMode::UNORDERED: - { - S3QueueProcessedCollection processed_files(max_set_size, max_set_age_sec); - processed_files.parse(zookeeper->get(zookeeper_processed_path)); - processed_files.add(file_path); - zookeeper->set(zookeeper_processed_path, processed_files.toString()); - break; - } case S3QueueMode::ORDERED: { - // Check that we set in ZooKeeper node only maximum processed file path. - // This check can be useful, when multiple table engines consume in ordered mode. - String max_file = getMaxProcessedFile(); - if (max_file.compare(file_path) <= 0) - zookeeper->set(zookeeper_processed_path, file_path); - break; + return trySetFileAsProcessingForOrderedMode(path); } - } - removeProcessingFile(file_path); -} - - -bool S3QueueFilesMetadata::setFileFailed(const String & file_path, const String & exception_message) -{ - auto zookeeper = storage->getZooKeeper(); - auto lock = acquireLock(zookeeper); - - S3QueueFailedCollection failed_collection(max_loading_retries); - failed_collection.parse(zookeeper->get(zookeeper_failed_path)); - const bool can_be_retried = failed_collection.add(file_path, exception_message); - zookeeper->set(zookeeper_failed_path, failed_collection.toString()); - removeProcessingFile(file_path); - return can_be_retried; -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getFailedFiles() -{ - auto zookeeper = storage->getZooKeeper(); - String failed_files = zookeeper->get(zookeeper_failed_path); - - S3QueueFailedCollection failed_collection(max_loading_retries); - failed_collection.parse(failed_files); - return failed_collection.getFileNames(); -} - -String S3QueueFilesMetadata::getMaxProcessedFile() -{ - auto zookeeper = storage->getZooKeeper(); - return zookeeper->get(zookeeper_processed_path); -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessingFiles() -{ - auto zookeeper = storage->getZooKeeper(); - String processing_files; - if (!zookeeper->tryGet(zookeeper_processing_path, processing_files)) - return {}; - - S3QueueProcessingCollection processing_collection; - if (!processing_files.empty()) - processing_collection.parse(processing_files); - return processing_collection.getFileNames(); -} - -void S3QueueFilesMetadata::setFilesProcessing(const Strings & file_paths) -{ - auto zookeeper = storage->getZooKeeper(); - String processing_files; - zookeeper->tryGet(zookeeper_processing_path, processing_files); - - S3QueueProcessingCollection processing_collection; - if (!processing_files.empty()) - processing_collection.parse(processing_files); - processing_collection.add(file_paths); - - if (zookeeper->exists(zookeeper_processing_path)) - zookeeper->set(zookeeper_processing_path, processing_collection.toString()); - else - zookeeper->create(zookeeper_processing_path, processing_collection.toString(), zkutil::CreateMode::Ephemeral); -} - -void S3QueueFilesMetadata::removeProcessingFile(const String & file_path) -{ - auto zookeeper = storage->getZooKeeper(); - String processing_files; - zookeeper->tryGet(zookeeper_processing_path, processing_files); - - S3QueueProcessingCollection processing_collection; - processing_collection.parse(processing_files); - processing_collection.remove(file_path); - zookeeper->set(zookeeper_processing_path, processing_collection.toString()); -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getUnorderedProcessedFiles() -{ - auto zookeeper = storage->getZooKeeper(); - S3QueueProcessedCollection processed_collection(max_set_size, max_set_age_sec); - processed_collection.parse(zookeeper->get(zookeeper_processed_path)); - return processed_collection.getFileNames(); -} - -S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessedFailedAndProcessingFiles() -{ - S3FilesCollection processed_and_failed_files = getFailedFiles(); - switch (mode) - { case S3QueueMode::UNORDERED: { - processed_and_failed_files.merge(getUnorderedProcessedFiles()); - break; - } - case S3QueueMode::ORDERED: - { - processed_and_failed_files.insert(getMaxProcessedFile()); - break; + return trySetFileAsProcessingForUnorderedMode(path); } } - processed_and_failed_files.merge(getProcessingFiles()); - return processed_and_failed_files; } -std::shared_ptr S3QueueFilesMetadata::acquireLock(zkutil::ZooKeeperPtr zookeeper) +bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { - UInt32 retry_count = 200; - UInt32 sleep_ms = 100; - UInt32 retries = 0; + const auto node_name = getNodeName(path); + const auto node_metadata = createNodeMetadata(path).toString(); + const auto zk_client = storage->getZooKeeper(); + + /// The following requests to the following: + /// If !exists(processed_node) && !exists(failed_node) && !exists(processing_node) => create(processing_node) + Coordination::Requests requests; + /// Check that processed node does not appear. + requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, "", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processed_path / node_name, -1)); + /// Check that failed node does not appear. + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, "", zkutil::CreateMode::Persistent)); + requests.push_back(zkutil::makeRemoveRequest(zookeeper_failed_path / node_name, -1)); + /// Check that processing node does not exist and create if not. + requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + return code == Coordination::Error::ZOK; +} + +bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) +{ + const auto node_name = getNodeName(path); + const auto node_metadata = createNodeMetadata(path).toString(); + const auto zk_client = storage->getZooKeeper(); while (true) { - Coordination::Error code = zookeeper->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + Coordination::Requests requests; + zkutil::addCheckNotExistsRequest(requests, zk_client, zookeeper_failed_path / node_name); + zkutil::addCheckNotExistsRequest(requests, zk_client, zookeeper_processing_path / node_name); + requests.push_back(zkutil::makeGetRequest(zookeeper_processed_path)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + + if (code != Coordination::Error::ZOK) { - retries++; - if (retries > retry_count) + if (responses[0]->error != Coordination::Error::ZOK + || responses[1]->error != Coordination::Error::ZOK) { - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); + /// Path is already in Failed or Processing. + return false; } - sleepForMilliseconds(sleep_ms); + /// GetRequest for zookeeper_processed_path should never fail, + /// because this is persistent node created at the creation of S3Queue storage. + throw zkutil::KeeperException::fromPath(code, requests.back()->getPath()); } - else if (code != Coordination::Error::ZOK) + + Coordination::Stat processed_node_stat; + NodeMetadata processed_node_metadata; + if (const auto * get_response = dynamic_cast(responses.back().get())) { - throw Coordination::Exception::fromPath(code, zookeeper_lock_path); + processed_node_stat = get_response->stat; + if (!get_response->data.empty()) + processed_node_metadata = NodeMetadata::fromString(get_response->data); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected response type with error: {}", responses.back()->error); + + auto max_processed_file_path = processed_node_metadata.file_path; + if (!max_processed_file_path.empty() && path <= max_processed_file_path) + return false; + + requests.clear(); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); + requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); + + code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return true; + + if (responses[0]->error != Coordination::Error::ZOK + || responses[1]->error != Coordination::Error::ZOK) + { + /// Path is already in Failed or Processing. + return false; + } + /// Max processed path changed. Retry. + } +} + +void S3QueueFilesMetadata::setFileProcessed(const String & path) +{ + switch (mode) + { + case S3QueueMode::ORDERED: + { + return setFileProcessedForOrderedMode(path); + } + case S3QueueMode::UNORDERED: + { + return setFileProcessedForUnorderedMode(path); + } + } +} + +void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) +{ + /// List results in s3 are always returned in UTF-8 binary order. + /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) + + const auto node_name = getNodeName(path); + const auto node_metadata = createNodeMetadata(path).toString(); + const auto zk_client = storage->getZooKeeper(); + + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, node_metadata, zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + /// TODO this could be because of the expired session. + if (responses[0]->error != Coordination::Error::ZOK) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as processed but it is not processing"); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as processed but it is already processed"); +} + +void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) +{ + const auto node_name = getNodeName(path); + const auto node_metadata = createNodeMetadata(path).toString(); + const auto zk_client = storage->getZooKeeper(); + + while (true) + { + std::string res; + Coordination::Stat stat; + bool exists = zk_client->tryGet(zookeeper_processed_path, res, &stat); + Coordination::Requests requests; + if (exists) + { + if (!res.empty()) + { + auto metadata = NodeMetadata::fromString(res); + if (metadata.file_path >= path) + return; + } + requests.push_back(zkutil::makeSetRequest(zookeeper_processed_path, node_metadata, stat.version)); } else { - return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zookeeper); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path, node_metadata, zkutil::CreateMode::Persistent)); } + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; } } +void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exception_message) +{ + const auto node_name = getNodeName(path); + auto node_metadata = createNodeMetadata(path, exception_message); + const auto zk_client = storage->getZooKeeper(); + + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, node_metadata.toString(), zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + if (responses[0]->error != Coordination::Error::ZOK) + { + /// TODO this could be because of the expired session. + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as filed but it is not processing"); + } + + Coordination::Stat stat; + auto failed_node_metadata = NodeMetadata::fromString(zk_client->get(zookeeper_failed_path / node_name, &stat)); + node_metadata.retries = failed_node_metadata.retries + 1; + + /// Failed node already exists, update it. + requests.clear(); + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeSetRequest(zookeeper_failed_path / node_name, node_metadata.toString(), stat.version)); + + responses.clear(); + code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); +} + } #endif diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 577c71b2227..302feab6028 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -1,102 +1,29 @@ #pragma once +#include "config.h" #if USE_AWS_S3 +#include +#include +#include -# include -# include -# include -# include +namespace fs = std::filesystem; +namespace Poco { class Logger; } namespace DB { -class StorageS3Queue; struct S3QueueSettings; +class StorageS3Queue; class S3QueueFilesMetadata { public: - struct TrackedCollectionItem - { - TrackedCollectionItem() = default; - TrackedCollectionItem(const String & file_path_, UInt64 timestamp_, UInt64 retries_count_, const String & last_exception_) - : file_path(file_path_), timestamp(timestamp_), retries_count(retries_count_), last_exception(last_exception_) {} - String file_path; - UInt64 timestamp = 0; - UInt64 retries_count = 0; - String last_exception; - }; - - using S3FilesCollection = std::unordered_set; - using TrackedFiles = std::deque; - S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_); - void setFilesProcessing(const Strings & file_paths); - void setFileProcessed(const String & file_path); - bool setFileFailed(const String & file_path, const String & exception_message); + bool trySetFileAsProcessing(const std::string & path); - S3FilesCollection getProcessedFailedAndProcessingFiles(); - String getMaxProcessedFile(); - std::shared_ptr acquireLock(zkutil::ZooKeeperPtr zookeeper); + void setFileProcessed(const std::string & path); - struct S3QueueCollection - { - public: - virtual ~S3QueueCollection() = default; - virtual String toString() const; - S3FilesCollection getFileNames(); - - virtual void parse(const String & collection_str) = 0; - - protected: - TrackedFiles files; - - void read(ReadBuffer & in); - void write(WriteBuffer & out) const; - }; - - struct S3QueueProcessedCollection : public S3QueueCollection - { - public: - S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); - - void parse(const String & collection_str) override; - void add(const String & file_name); - - private: - const UInt64 max_size; - const UInt64 max_age; - }; - - struct S3QueueFailedCollection : S3QueueCollection - { - public: - S3QueueFailedCollection(const UInt64 & max_retries_count_); - - void parse(const String & collection_str) override; - bool add(const String & file_name, const String & exception_message); - - S3FilesCollection getFileNames(); - - private: - UInt64 max_retries_count; - }; - - struct S3QueueProcessingCollection - { - public: - S3QueueProcessingCollection() = default; - - void parse(const String & collection_str); - void add(const Strings & file_names); - void remove(const String & file_name); - - String toString() const; - const S3FilesCollection & getFileNames() const { return files; } - - private: - S3FilesCollection files; - }; + void setFileFailed(const std::string & path, const std::string & exception_message); private: const StorageS3Queue * storage; @@ -105,23 +32,35 @@ private: const UInt64 max_set_age_sec; const UInt64 max_loading_retries; - const String zookeeper_processing_path; - const String zookeeper_processed_path; - const String zookeeper_failed_path; - const String zookeeper_lock_path; + const fs::path zookeeper_processing_path; + const fs::path zookeeper_processed_path; + const fs::path zookeeper_failed_path; mutable std::mutex mutex; Poco::Logger * log; - S3FilesCollection getFailedFiles(); - S3FilesCollection getProcessingFiles(); - S3FilesCollection getUnorderedProcessedFiles(); + bool trySetFileAsProcessingForOrderedMode(const std::string & path); + bool trySetFileAsProcessingForUnorderedMode(const std::string & path); - void removeProcessingFile(const String & file_path); + void setFileProcessedForOrderedMode(const std::string & path); + void setFileProcessedForUnorderedMode(const std::string & path); + + std::string getNodeName(const std::string & path); + + struct NodeMetadata + { + std::string file_path; + UInt64 last_processed_timestamp = 0; + std::string last_exception; + UInt64 retries = 0; + + std::string toString() const; + static NodeMetadata fromString(const std::string & metadata_str); + }; + + NodeMetadata createNodeMetadata(const std::string & path, const std::string & exception = "", size_t retries = 0); }; - } - #endif diff --git a/src/Storages/S3Queue/S3QueueSettings.cpp b/src/Storages/S3Queue/S3QueueSettings.cpp index b74cf8d39bb..cb312adc5d9 100644 --- a/src/Storages/S3Queue/S3QueueSettings.cpp +++ b/src/Storages/S3Queue/S3QueueSettings.cpp @@ -1,8 +1,8 @@ +#include +#include #include #include #include -#include -#include namespace DB diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 54a863aeb2c..6704345ea59 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -1,47 +1,12 @@ -#include -#include -#include -#include "IO/ParallelReadBuffer.h" -#include "Parsers/ASTCreateQuery.h" #include "config.h" #if USE_AWS_S3 - -# include - -# include - -# include -# include - -# include - -# include -# include - -# include -# include -# include -# include -# include -# include - -# include - -# include -# include -# include - -# include - -# include - -# include -# include -# include - -# include -# include +#include +#include +#include +#include +#include +#include namespace CurrentMetrics @@ -64,138 +29,43 @@ namespace ErrorCodes extern const int S3_ERROR; } - -StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - ASTPtr query, - const NamesAndTypesList & virtual_columns, - ContextPtr context, - UInt64 & max_poll_size_, - const S3Settings::RequestSettings & request_settings_) - : max_poll_size(max_poll_size_) - , glob_iterator(std::make_unique( - client_, globbed_uri_, query, virtual_columns, context, nullptr, request_settings_)) +StorageS3QueueSource::FileIterator::FileIterator( + std::shared_ptr metadata_, std::unique_ptr glob_iterator_) + : metadata(metadata_) , glob_iterator(std::move(glob_iterator_)) +{ +} + +StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::FileIterator::next() { - /// todo(kssenii): remove this loop, it should not be here while (true) { KeyWithInfo val = glob_iterator->next(); if (val.key.empty()) - break; - keys_buf.push_back(val); + return {}; + if (metadata->trySetFileAsProcessing(val.key)) + return val; } } -Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( - const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) -{ - for (const KeyWithInfo & val : keys_buf) - { - auto full_path = val.key; - if (exclude_keys.find(full_path) != exclude_keys.end()) - { - LOG_TEST(log, "File {} will be skipped, because it was found in exclude files list " - "(either already processed or failed to be processed)", val.key); - continue; - } - - if ((engine_mode == S3QueueMode::ORDERED) && (full_path.compare(max_file) <= 0)) - continue; - - if ((processing_keys.size() < max_poll_size) || (engine_mode == S3QueueMode::ORDERED)) - { - processing_keys.push_back(val); - } - else - { - break; - } - } - - if (engine_mode == S3QueueMode::ORDERED) - { - std::sort( - processing_keys.begin(), - processing_keys.end(), - [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key.compare(rhs.key) < 0; }); - - if (processing_keys.size() > max_poll_size) - { - processing_keys.erase(processing_keys.begin() + max_poll_size, processing_keys.end()); - } - } - - Strings keys; - for (const auto & key_info : processing_keys) - keys.push_back(key_info.key); - - processing_keys.push_back(KeyWithInfo()); - processing_iterator = processing_keys.begin(); - return keys; -} - - -StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next() -{ - std::lock_guard lock(mutex); - if (processing_iterator != processing_keys.end()) - { - return *processing_iterator++; - } - - return KeyWithInfo(); -} - StorageS3QueueSource::StorageS3QueueSource( - const ReadFromFormatInfo & info, - const String & format_, String name_, - ContextPtr context_, - std::optional format_settings_, - UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, - String compression_hint_, - const std::shared_ptr & client_, - const String & bucket_, - const String & version_id_, - const String & url_host_and_port, - std::shared_ptr file_iterator_, + const Block & header_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, const S3QueueAction & action_, - const size_t download_thread_num_) - : ISource(info.source_header) + RemoveFileFunc remove_file_func_, + const NamesAndTypesList & requested_virtual_columns_, + ContextPtr context_) + : ISource(header_) , WithContext(context_) , name(std::move(name_)) - , bucket(bucket_) - , version_id(version_id_) - , format(format_) - , columns_desc(info.columns_description) - , request_settings(request_settings_) - , client(client_) - , files_metadata(files_metadata_) - , requested_virtual_columns(info.requested_virtual_columns) - , requested_columns(info.requested_columns) - , file_iterator(file_iterator_) , action(action_) + , files_metadata(files_metadata_) + , internal_source(std::move(internal_source_)) + , requested_virtual_columns(requested_virtual_columns_) + , remove_file_func(remove_file_func_) + , log(&Poco::Logger::get("StorageS3QueueSource")) { - internal_source = std::make_shared( - info, - format_, - name_, - context_, - format_settings_, - max_block_size_, - request_settings_, - compression_hint_, - client_, - bucket_, - version_id_, - url_host_and_port, - file_iterator, - download_thread_num_, - false, - /* query_info */ std::nullopt); reader = std::move(internal_source->reader); if (reader) reader_future = std::move(internal_source->reader_future); @@ -213,7 +83,6 @@ String StorageS3QueueSource::getName() const Chunk StorageS3QueueSource::generate() { - auto file_progress = getContext()->getFileProgressCallback(); while (true) { if (isCancelled() || !reader) @@ -223,46 +92,27 @@ Chunk StorageS3QueueSource::generate() break; } - Chunk chunk; - bool success_in_pulling = false; try { + Chunk chunk; if (reader->pull(chunk)) { - UInt64 num_rows = chunk.getNumRows(); - auto file_path = reader.getPath(); - - for (const auto & virtual_column : requested_virtual_columns) - { - if (virtual_column.name == "_path") - { - chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_path)->convertToFullColumnIfConst()); - } - else if (virtual_column.name == "_file") - { - size_t last_slash_pos = file_path.find_last_of('/'); - auto column = virtual_column.type->createColumnConst(num_rows, file_path.substr(last_slash_pos + 1)); - chunk.addColumn(column->convertToFullColumnIfConst()); - } - } - success_in_pulling = true; + LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getPath()); + VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath()); + return chunk; } } catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); files_metadata->setFileFailed(reader.getFile(), e.message()); - success_in_pulling = false; - } - if (success_in_pulling) - { - applyActionAfterProcessing(reader.getFile()); - files_metadata->setFileProcessed(reader.getFile()); - return chunk; + throw; } + files_metadata->setFileProcessed(reader.getFile()); + applyActionAfterProcessing(reader.getFile()); - assert(reader_future.valid()); + chassert(reader_future.valid()); reader = reader_future.get(); if (!reader) @@ -277,37 +127,21 @@ Chunk StorageS3QueueSource::generate() return {}; } - -void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) +void StorageS3QueueSource::applyActionAfterProcessing(const String & path) { switch (action) { case S3QueueAction::DELETE: - deleteProcessedObject(file_path); + { + assert(remove_file_func); + remove_file_func(path); break; + } case S3QueueAction::KEEP: break; } } -void StorageS3QueueSource::deleteProcessedObject(const String & file_path) -{ - LOG_INFO(log, "Delete processed file {} from bucket {}", file_path, bucket); - - S3::DeleteObjectRequest request; - request.WithKey(file_path).WithBucket(bucket); - auto outcome = client->DeleteObject(request); - if (!outcome.IsSuccess()) - { - const auto & err = outcome.GetError(); - LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); - } - else - { - LOG_TRACE(log, "Object with path {} was removed from S3", file_path); - } -} - } #endif diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index f89384fb096..1ec762d6477 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -2,29 +2,13 @@ #include "config.h" #if USE_AWS_S3 +#include +#include +#include +#include -# include - -# include - -# include -# include -# include -# include -# include - -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +namespace Poco { class Logger; } namespace DB { @@ -34,56 +18,37 @@ class StorageS3QueueSource : public ISource, WithContext { public: using IIterator = StorageS3Source::IIterator; - using DisclosedGlobIterator = StorageS3Source::DisclosedGlobIterator; - using KeysWithInfo = StorageS3Source::KeysWithInfo; + using GlobIterator = StorageS3Source::DisclosedGlobIterator; using KeyWithInfo = StorageS3Source::KeyWithInfo; - class QueueGlobIterator : public IIterator + using ZooKeeperGetter = std::function; + using RemoveFileFunc = std::function; + + class FileIterator : public IIterator { public: - QueueGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - ASTPtr query, - const NamesAndTypesList & virtual_columns, - ContextPtr context, - UInt64 & max_poll_size_, - const S3Settings::RequestSettings & request_settings_ = {}); + FileIterator( + std::shared_ptr metadata_, + std::unique_ptr glob_iterator_); KeyWithInfo next() override; - Strings - filterProcessingFiles(const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); - private: - UInt64 max_poll_size; - KeysWithInfo keys_buf; - KeysWithInfo processing_keys; - mutable std::mutex mutex; - std::unique_ptr glob_iterator; - std::vector::iterator processing_iterator; - - Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSourceIterator"); + const std::shared_ptr metadata; + const std::unique_ptr glob_iterator; + std::mutex mutex; }; static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); StorageS3QueueSource( - const ReadFromFormatInfo & info, - const String & format, String name_, - ContextPtr context_, - std::optional format_settings_, - UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, - String compression_hint_, - const std::shared_ptr & client_, - const String & bucket, - const String & version_id, - const String & url_host_and_port, - std::shared_ptr file_iterator_, + const Block & header_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, const S3QueueAction & action_, - size_t download_thread_num); + RemoveFileFunc remove_file_func_, + const NamesAndTypesList & requested_virtual_columns_, + ContextPtr context_); ~StorageS3QueueSource() override; @@ -91,34 +56,21 @@ public: Chunk generate() override; - private: - String name; - String bucket; - String version_id; - String format; - ColumnsDescription columns_desc; - S3Settings::RequestSettings request_settings; - std::shared_ptr client; + const String name; + const S3QueueAction action; + const std::shared_ptr files_metadata; + const std::shared_ptr internal_source; + const NamesAndTypesList requested_virtual_columns; + + RemoveFileFunc remove_file_func; + Poco::Logger * log; - std::shared_ptr files_metadata; using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; - - NamesAndTypesList requested_virtual_columns; - NamesAndTypesList requested_columns; - std::shared_ptr file_iterator; - const S3QueueAction action; - - Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); - std::future reader_future; - mutable std::mutex mutex; - - std::shared_ptr internal_source; - void deleteProcessedObject(const String & file_path); - void applyActionAfterProcessing(const String & file_path); + void applyActionAfterProcessing(const String & path); }; } diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 23eebb6ded9..f9c89f4d87d 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -2,12 +2,12 @@ #if USE_AWS_S3 -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include namespace DB @@ -18,13 +18,17 @@ namespace ErrorCodes extern const int METADATA_MISMATCH; } -S3QueueTableMetadata::S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings) +S3QueueTableMetadata::S3QueueTableMetadata( + const StorageS3::Configuration & configuration, + const S3QueueSettings & engine_settings, + const StorageInMemoryMetadata & storage_metadata) { format_name = configuration.format; after_processing = engine_settings.after_processing.toString(); mode = engine_settings.mode.toString(); s3queue_tracked_files_limit = engine_settings.s3queue_tracked_files_limit; s3queue_tracked_file_ttl_sec = engine_settings.s3queue_tracked_file_ttl_sec; + columns = storage_metadata.getColumns().toString(); } @@ -36,6 +40,7 @@ String S3QueueTableMetadata::toString() const json.set("s3queue_tracked_files_limit", s3queue_tracked_files_limit); json.set("s3queue_tracked_file_ttl_sec", s3queue_tracked_file_ttl_sec); json.set("format_name", format_name); + json.set("columns", columns); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); @@ -52,6 +57,7 @@ void S3QueueTableMetadata::read(const String & metadata_str) s3queue_tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); s3queue_tracked_file_ttl_sec = json->getValue("s3queue_tracked_file_ttl_sec"); format_name = json->getValue("format_name"); + columns = json->getValue("columns"); } S3QueueTableMetadata S3QueueTableMetadata::parse(const String & metadata_str) diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 4b6fbc54825..f15665692c4 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -2,9 +2,9 @@ #if USE_AWS_S3 -# include -# include -# include +#include +#include +#include namespace DB { @@ -18,13 +18,14 @@ class ReadBuffer; struct S3QueueTableMetadata { String format_name; + String columns; String after_processing; String mode; UInt64 s3queue_tracked_files_limit; UInt64 s3queue_tracked_file_ttl_sec; S3QueueTableMetadata() = default; - S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); + S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings, const StorageInMemoryMetadata & storage_metadata); void read(const String & metadata_str); static S3QueueTableMetadata parse(const String & metadata_str); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 08cbff96cd0..be71af24601 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -2,69 +2,38 @@ #if USE_AWS_S3 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include -# include -# include -# include -# include -# include -# include -# include -# include -# include "IO/ParallelReadBuffer.h" - -# include - -# include - -# include - -# include -# include - -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include - - -# include - -# include -# include -# include - -# include - -# include - -# include - -# include -# include -# include -# include namespace fs = std::filesystem; namespace ProfileEvents { -extern const Event S3DeleteObjects; -extern const Event S3ListObjects; + extern const Event S3DeleteObjects; + extern const Event S3ListObjects; } namespace DB { -static const String PARTITION_ID_WILDCARD = "{_partition_id}"; static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes @@ -78,6 +47,33 @@ namespace ErrorCodes extern const int INCOMPATIBLE_COLUMNS; } +namespace +{ + bool containsGlobs(const S3::URI & url) + { + return url.key.find_first_of("*?{") != std::string::npos; + } + + std::string chooseZooKeeperPath(const StorageID & table_id, const Settings & settings, const S3QueueSettings & s3queue_settings) + { + std::string zk_path_prefix = settings.s3queue_default_zookeeper_path.value; + if (zk_path_prefix.empty()) + zk_path_prefix = "/"; + + std::string result_zk_path; + if (s3queue_settings.keeper_path.changed) + { + /// We do not add table uuid here on purpose. + result_zk_path = fs::path(zk_path_prefix) / s3queue_settings.keeper_path.value; + } + else + { + auto database_uuid = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getUUID(); + result_zk_path = fs::path(zk_path_prefix) / toString(database_uuid) / toString(table_id.uuid); + } + return zkutil::extractZooKeeperPath(result_zk_path, true); + } +} StorageS3Queue::StorageS3Queue( std::unique_ptr s3queue_settings_, @@ -87,79 +83,64 @@ StorageS3Queue::StorageS3Queue( const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_, - std::optional format_settings_, - ASTPtr partition_by_) + std::optional format_settings_) : IStorage(table_id_) , WithContext(context_) , s3queue_settings(std::move(s3queue_settings_)) + , zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *s3queue_settings)) , after_processing(s3queue_settings->after_processing) + , files_metadata(std::make_shared(this, *s3queue_settings)) , configuration{configuration_} - , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) , format_settings(format_settings_) - , partition_by(partition_by_) + , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { if (configuration.url.key.ends_with('/')) + { configuration.url.key += '*'; - - if (!withGlobs()) + } + else if (!containsGlobs(configuration.url)) + { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); - - std::string zk_path_prefix = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; - if (zk_path_prefix.empty()) - zk_path_prefix = "/"; - - std::string result_zk_path; - if (s3queue_settings->keeper_path.changed) - { - /// We do not add table uuid here on purpose. - result_zk_path = fs::path(zk_path_prefix) / s3queue_settings->keeper_path.value; - } - else - { - auto database_uuid = DatabaseCatalog::instance().getDatabase(table_id_.database_name)->getUUID(); - result_zk_path = fs::path(zk_path_prefix) / toString(database_uuid) / toString(table_id_.uuid); } - zk_path = zkutil::extractZooKeeperPath(result_zk_path, true/* check_starts_with_slash */, log); - LOG_INFO(log, "Using zookeeper path: {}", zk_path); - - FormatFactory::instance().checkFormatName(configuration.format); - context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri); - StorageInMemoryMetadata storage_metadata; configuration.update(context_); + FormatFactory::instance().checkFormatName(configuration.format); + context_->getRemoteHostFilter().checkURL(configuration.url.uri); + StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { auto columns = StorageS3::getTableStructureFromDataImpl(configuration, format_settings, context_); storage_metadata.setColumns(columns); } else + { storage_metadata.setColumns(columns_); - + } storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); + + createOrCheckMetadata(storage_metadata); setInMemoryMetadata(storage_metadata); - auto metadata_snapshot = getInMemoryMetadataPtr(); - const bool is_first_replica = createTableIfNotExists(metadata_snapshot); - - if (!is_first_replica) - { - checkTableStructure(zk_path, metadata_snapshot); - } - - files_metadata = std::make_shared(this, *s3queue_settings); virtual_columns = VirtualColumnUtils::getPathAndFileVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); + task = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); - auto poll_thread = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); - task = std::make_shared(std::move(poll_thread)); + LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); } - -bool StorageS3Queue::supportsSubcolumns() const +void StorageS3Queue::startup() { - return true; + if (task) + task->activateAndSchedule(); +} + +void StorageS3Queue::shutdown() +{ + shutdown_called = true; + if (task) + task->deactivate(); } bool StorageS3Queue::supportsSubsetOfColumns() const @@ -177,80 +158,62 @@ Pipe StorageS3Queue::read( size_t /* num_streams */) { if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) - throw Exception( - ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + { + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. " + "To enable use setting `stream_like_engine_allow_direct_select`"); + } if (mv_attached) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageS3Queue with attached materialized views"); + { + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, + "Cannot read from {} with attached materialized views", getName()); + } - auto query_configuration = updateConfigurationAndGetCopy(local_context); - - std::shared_ptr iterator_wrapper = createFileIterator(local_context, query_info.query); + Pipes pipes; + pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); + return Pipe::unitePipes(std::move(pipes)); +} +std::shared_ptr StorageS3Queue::createSource( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + ASTPtr query, + size_t max_block_size, + ContextPtr local_context) +{ + auto configuration_snapshot = updateConfigurationAndGetCopy(local_context); + auto file_iterator = createFileIterator(local_context, query); auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); - const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; - - return Pipe(std::make_shared( - read_from_format_info, - configuration.format, - getName(), - local_context, - format_settings, + auto internal_source = std::make_unique( + read_from_format_info, configuration.format, getName(), local_context, format_settings, max_block_size, - query_configuration.request_settings, - configuration.compression_method, - query_configuration.client, - query_configuration.url.bucket, - query_configuration.url.version_id, - query_configuration.url.uri.getHost() + std::to_string(query_configuration.url.uri.getPort()), - iterator_wrapper, - files_metadata, - after_processing, - max_download_threads)); -} + configuration_snapshot.request_settings, + configuration_snapshot.compression_method, + configuration_snapshot.client, + configuration_snapshot.url.bucket, + configuration_snapshot.url.version_id, + configuration_snapshot.url.uri.getHost() + std::to_string(configuration_snapshot.url.uri.getPort()), + file_iterator, local_context->getSettingsRef().max_download_threads, false, /* query_info */ std::nullopt); -SinkToStoragePtr StorageS3Queue::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, bool) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); -} - -void StorageS3Queue::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); -} - -NamesAndTypesList StorageS3Queue::getVirtuals() const -{ - return virtual_columns; -} - -bool StorageS3Queue::supportsPartitionBy() const -{ - return true; -} - -void StorageS3Queue::startup() -{ - if (task) - task->holder->activateAndSchedule(); -} - -void StorageS3Queue::shutdown() -{ - shutdown_called = true; - if (task) + auto file_deleter = [this, bucket = configuration_snapshot.url.bucket, client = configuration_snapshot.client](const std::string & path) { - task->stream_cancelled = true; - task->holder->deactivate(); - } -} - -size_t StorageS3Queue::getTableDependentCount() const -{ - auto table_id = getStorageID(); - // Check if at least one direct dependency is attached - return DatabaseCatalog::instance().getDependentViews(table_id).size(); + S3::DeleteObjectRequest request; + request.WithKey(path).WithBucket(bucket); + auto outcome = client->DeleteObject(request); + if (!outcome.IsSuccess()) + { + const auto & err = outcome.GetError(); + LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); + } + else + { + LOG_TRACE(log, "Object with path {} was removed from S3", path); + } + }; + return std::make_shared( + getName(), read_from_format_info.source_header, std::move(internal_source), + files_metadata, after_processing, file_deleter, read_from_format_info.requested_virtual_columns, local_context); } bool StorageS3Queue::hasDependencies(const StorageID & table_id) @@ -280,40 +243,33 @@ bool StorageS3Queue::hasDependencies(const StorageID & table_id) void StorageS3Queue::threadFunc() { - bool reschedule = true; + SCOPE_EXIT({ mv_attached.store(false); }); try { auto table_id = getStorageID(); - - auto dependencies_count = getTableDependentCount(); + size_t dependencies_count = DatabaseCatalog::instance().getDependentViews(table_id).size(); if (dependencies_count) { auto start_time = std::chrono::steady_clock::now(); - + /// Reset reschedule interval. + reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; + /// Disallow parallel selects while streaming to mv. mv_attached.store(true); - // Keep streaming as long as there are attached views and streaming is not cancelled - while (!task->stream_cancelled) - { - if (!hasDependencies(table_id)) - { - /// For this case, we can not wait for watch thread to wake up - reschedule = true; - break; - } + /// Keep streaming as long as there are attached views and streaming is not cancelled + while (!shutdown_called && hasDependencies(table_id)) + { LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + streamToViews(); - auto ts = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(ts - start_time); + auto now = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(now - start_time); if (duration.count() > MAX_THREAD_WORK_DURATION_MS) { LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); - reschedule = true; break; } - - reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; } } } @@ -322,19 +278,16 @@ void StorageS3Queue::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } - mv_attached.store(false); - - if (reschedule && !shutdown_called) + if (!shutdown_called) { - LOG_TRACE(log, "Reschedule S3 Queue thread func."); - /// Reschedule with backoff. if (reschedule_processing_interval_ms < s3queue_settings->s3queue_polling_max_timeout_ms) reschedule_processing_interval_ms += s3queue_settings->s3queue_polling_backoff_ms; - task->holder->scheduleAfter(reschedule_processing_interval_ms); + + LOG_TRACE(log, "Reschedule S3 Queue processing thread in {} ms", reschedule_processing_interval_ms); + task->scheduleAfter(reschedule_processing_interval_ms); } } - void StorageS3Queue::streamToViews() { auto table_id = getStorageID(); @@ -348,8 +301,6 @@ void StorageS3Queue::streamToViews() auto insert = std::make_shared(); insert->table_id = table_id; - size_t block_size = 100; - auto s3queue_context = Context::createCopy(getContext()); s3queue_context->makeQueryContext(); auto query_configuration = updateConfigurationAndGetCopy(s3queue_context); @@ -358,40 +309,14 @@ void StorageS3Queue::streamToViews() // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); auto block_io = interpreter.execute(); - auto column_names = block_io.pipeline.getHeader().getNames(); - - // Create a stream for each consumer and join them in a union stream - - std::shared_ptr iterator_wrapper = createFileIterator(s3queue_context, nullptr); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); - const size_t max_download_threads = s3queue_context->getSettingsRef().max_download_threads; - - auto pipe = Pipe(std::make_shared( - read_from_format_info, - configuration.format, - getName(), - s3queue_context, - format_settings, - block_size, - query_configuration.request_settings, - configuration.compression_method, - query_configuration.client, - query_configuration.url.bucket, - query_configuration.url.version_id, - query_configuration.url.uri.getHost() + std::to_string(query_configuration.url.uri.getPort()), - iterator_wrapper, - files_metadata, - after_processing, - max_download_threads)); + auto pipe = Pipe(createSource(block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context)); std::atomic_size_t rows = 0; - { - block_io.pipeline.complete(std::move(pipe)); - block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); - CompletedPipelineExecutor executor(block_io.pipeline); - executor.execute(); - } + block_io.pipeline.complete(std::move(pipe)); + block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); } StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) @@ -411,49 +336,40 @@ zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const return zk_client; } - -bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot) +void StorageS3Queue::createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata) { auto zookeeper = getZooKeeper(); zookeeper->createAncestors(zk_path); - for (size_t i = 0; i < zk_create_table_retries; ++i) + for (size_t i = 0; i < 1000; ++i) { - Coordination::Requests ops; - bool is_first_replica = true; - if (zookeeper->exists(zk_path + "/metadata")) + Coordination::Requests requests; + if (zookeeper->exists(zk_path / "metadata")) { - if (!zookeeper->exists(zk_path + "/processing")) - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral)); - LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zk_path); - is_first_replica = false; + checkTableStructure(zk_path, storage_metadata); } else { - String metadata_str = S3QueueTableMetadata(configuration, *s3queue_settings).toString(); - ops.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processed", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/failed", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral)); - ops.emplace_back(zkutil::makeCreateRequest( - zk_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); + std::string metadata = S3QueueTableMetadata(configuration, *s3queue_settings, storage_metadata).toString(); + requests.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zk_path / "processed", "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zk_path / "failed", "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zk_path / "processing", "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(zk_path / "metadata", metadata, zkutil::CreateMode::Persistent)); } Coordination::Responses responses; - auto code = zookeeper->tryMulti(ops, responses); + auto code = zookeeper->tryMulti(requests, responses); if (code == Coordination::Error::ZNODEEXISTS) { - LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zk_path); + LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zk_path.string()); continue; } else if (code != Coordination::Error::ZOK) { - zkutil::KeeperMultiException::check(code, ops, responses); + zkutil::KeeperMultiException::check(code, requests, responses); } - - return is_first_replica; + return; } throw Exception( @@ -463,24 +379,20 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ } -/** Verify that list of columns and table settings match those specified in ZK (/metadata). - * If not, throw an exception. - */ -void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot) +void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const StorageInMemoryMetadata & storage_metadata) { + // Verify that list of columns and table settings match those specified in ZK (/metadata). + // If not, throw an exception. + auto zookeeper = getZooKeeper(); - - S3QueueTableMetadata old_metadata(configuration, *s3queue_settings); - - Coordination::Stat metadata_stat; - String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata", &metadata_stat); + String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata"); auto metadata_from_zk = S3QueueTableMetadata::parse(metadata_str); + + S3QueueTableMetadata old_metadata(configuration, *s3queue_settings, storage_metadata); old_metadata.checkEquals(metadata_from_zk); - Coordination::Stat columns_stat; - auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns", &columns_stat)); - - const ColumnsDescription & old_columns = metadata_snapshot->getColumns(); + auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns")); + const ColumnsDescription & old_columns = storage_metadata.getColumns(); if (columns_from_zk != old_columns) { throw Exception( @@ -492,38 +404,12 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const } } - -std::shared_ptr -StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query) +std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query) { - auto it = std::make_shared( - *configuration.client, - configuration.url, - query, - virtual_columns, - local_context, - s3queue_settings->s3queue_polling_size.value, - configuration.request_settings); - - auto zookeeper = getZooKeeper(); - auto lock = files_metadata->acquireLock(zookeeper); - S3QueueFilesMetadata::S3FilesCollection files_to_skip = files_metadata->getProcessedFailedAndProcessingFiles(); - - Strings files_to_process; - if (s3queue_settings->mode == S3QueueMode::UNORDERED) - { - files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip); - } - else - { - String max_processed_file = files_metadata->getMaxProcessedFile(); - files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip, max_processed_file); - } - - LOG_TEST(log, "Found files to process: {}", fmt::join(files_to_process, ", ")); - - files_metadata->setFilesProcessing(files_to_process); - return it; + auto glob_iterator = std::make_unique( + *configuration.client, configuration.url, query, virtual_columns, local_context, + /* read_keys */nullptr, configuration.request_settings); + return std::make_shared(files_metadata, std::move(glob_iterator)); } void StorageS3Queue::drop() @@ -540,11 +426,15 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) [](const StorageFactory::Arguments & args) { if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_experimental_s3queue) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3Queue is experimental. You can enable it with the `allow_experimental_s3queue` setting."); + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3Queue is experimental. " + "You can enable it with the `allow_experimental_s3queue` setting."); + } auto & engine_args = args.engine_args; if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); // Use format settings from global server context + settings from @@ -582,10 +472,6 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) format_settings = getFormatSettings(args.getContext()); } - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); - return std::make_shared( std::move(s3queue_settings), std::move(configuration), @@ -594,12 +480,10 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) args.constraints, args.comment, args.getContext(), - format_settings, - partition_by); + format_settings); }, { .supports_settings = true, - .supports_sort_order = true, // for partition by .supports_schema_inference = true, .source_access_type = AccessType::S3, }); diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 712fe9e530b..9151473a9bc 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -4,29 +4,14 @@ #if USE_AWS_S3 -# include - -# include -# include - -# include -# include -# include -# include -# include -# include - -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include namespace Aws::S3 { @@ -35,7 +20,7 @@ class Client; namespace DB { - +class S3QueueFilesMetadata; class StorageS3Queue : public IStorage, WithContext { @@ -50,8 +35,7 @@ public: const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_, - std::optional format_settings_, - ASTPtr partition_by_ = nullptr); + std::optional format_settings_); String getName() const override { return "S3Queue"; } @@ -64,79 +48,58 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr context, - bool async_insert) override; - - void truncate( - const ASTPtr & /*query*/, - const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr /*local_context*/, - TableExclusiveLockHolder &) override; - - NamesAndTypesList getVirtuals() const override; - - bool supportsPartitionBy() const override; + NamesAndTypesList getVirtuals() const override { return virtual_columns; } const auto & getFormatName() const { return configuration.format; } - const String & getZooKeeperPath() const { return zk_path; } + const fs::path & getZooKeeperPath() const { return zk_path; } zkutil::ZooKeeperPtr getZooKeeper() const; private: + using FileIterator = StorageS3QueueSource::FileIterator; + const std::unique_ptr s3queue_settings; + const fs::path zk_path; const S3QueueAction after_processing; std::shared_ptr files_metadata; Configuration configuration; + + const std::optional format_settings; NamesAndTypesList virtual_columns; - UInt64 reschedule_processing_interval_ms; - std::optional format_settings; - ASTPtr partition_by; - - String zk_path; mutable zkutil::ZooKeeperPtr zk_client; mutable std::mutex zk_mutex; + BackgroundSchedulePool::TaskHolder task; + std::atomic stream_cancelled{false}; + UInt64 reschedule_processing_interval_ms; + std::atomic mv_attached = false; - std::atomic shutdown_called{false}; + std::atomic shutdown_called = false; Poco::Logger * log; - bool supportsSubcolumns() const override; - bool withGlobs() const { return configuration.url.key.find_first_of("*?{") != std::string::npos; } - - void threadFunc(); - size_t getTableDependentCount() const; - bool hasDependencies(const StorageID & table_id); - void startup() override; void shutdown() override; void drop() override; - - struct TaskContext - { - BackgroundSchedulePool::TaskHolder holder; - std::atomic stream_cancelled{false}; - explicit TaskContext(BackgroundSchedulePool::TaskHolder && task_) : holder(std::move(task_)) { } - }; - std::shared_ptr task; - bool supportsSubsetOfColumns() const override; + bool supportsSubcolumns() const override { return true; } - const UInt32 zk_create_table_retries = 1000; - bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); - void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); - - using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; - - std::shared_ptr - createFileIterator(ContextPtr local_context, ASTPtr query); + std::shared_ptr createFileIterator(ContextPtr local_context, ASTPtr query); + std::shared_ptr createSource( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + ASTPtr query, + size_t max_block_size, + ContextPtr local_context); + bool hasDependencies(const StorageID & table_id); void streamToViews(); + void threadFunc(); + + void createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata); + void checkTableStructure(const String & zookeeper_prefix, const StorageInMemoryMetadata & storage_metadata); Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index c11bbd43dc6..65c31acb5d8 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -111,6 +111,7 @@ def generate_random_files( to_generate = [ (f"{prefix}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) ] + print(f"Generating files: {to_generate}") to_generate.sort(key=lambda x: x[0]) for filename, i in to_generate: @@ -179,29 +180,58 @@ def run_query(instance, query, stdin=None, settings=None): @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_delete_after_processing(started_cluster, mode): - prefix = "delete" bucket = started_cluster.minio_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] - total_values = generate_random_files(5, prefix, started_cluster, bucket) - instance.query( + table_name = "test.delete_after_processing" + dst_table_name = "test.delete_after_processing_dst" + mv_name = "test.delete_after_processing_mv" + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + files_num = 5 + row_num = 10 + + prefix = "delete" + total_values = generate_random_files( + files_num, prefix, started_cluster, bucket, row_num=row_num + ) + node.query( f""" - DROP TABLE IF EXISTS test.s3_queue; - CREATE TABLE test.s3_queue ({table_format}) + DROP TABLE IF EXISTS {table_name}; + DROP TABLE IF EXISTS {dst_table_name}; + DROP TABLE IF EXISTS {mv_name}; + CREATE TABLE {table_name} ({table_format}) ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_delete_{mode}', s3queue_loading_retries = 3, after_processing='delete'; + + CREATE TABLE {dst_table_name} ({table_format}, _path String) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE MATERIALIZED VIEW {mv_name} TO {dst_table_name} AS SELECT *, _path FROM {table_name}; """ ) - get_query = f"SELECT * FROM test.s3_queue ORDER BY column1, column2, column3" + expected_count = files_num * row_num + for _ in range(100): + count = int(node.query(f"SELECT count() FROM {dst_table_name}")) + print(f"{count}/{expected_count}") + if count == expected_count: + break + time.sleep(1) + + assert int(node.query(f"SELECT count() FROM {dst_table_name}")) == expected_count + assert int(node.query(f"SELECT uniq(_path) FROM {dst_table_name}")) == files_num assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name} ORDER BY column1, column2, column3" + ).splitlines() ] == sorted(total_values, key=lambda x: (x[0], x[1], x[2])) + minio = started_cluster.minio_client objects = list(minio.list_objects(started_cluster.minio_bucket, recursive=True)) assert len(objects) == 0 From 220a67eca7c47b211590de6187152428c856f19d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Sep 2023 15:49:34 +0200 Subject: [PATCH 002/120] Better --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 148 +++-- src/Storages/S3Queue/StorageS3Queue.cpp | 3 +- .../configs/zookeeper.xml | 16 + .../integration/test_storage_s3_queue/test.py | 593 +++++++++--------- 4 files changed, 391 insertions(+), 369 deletions(-) create mode 100644 tests/integration/test_storage_s3_queue/configs/zookeeper.xml diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index d0e4d0f88cc..99c0924968c 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -19,7 +19,7 @@ namespace DB namespace ErrorCodes { - extern const int TIMEOUT_EXCEEDED; + extern const int LOGICAL_ERROR; } namespace @@ -113,16 +113,9 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::str const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = storage->getZooKeeper(); - /// The following requests to the following: - /// If !exists(processed_node) && !exists(failed_node) && !exists(processing_node) => create(processing_node) Coordination::Requests requests; - /// Check that processed node does not appear. - requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, "", zkutil::CreateMode::Persistent)); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processed_path / node_name, -1)); - /// Check that failed node does not appear. - requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, "", zkutil::CreateMode::Persistent)); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_failed_path / node_name, -1)); - /// Check that processing node does not exist and create if not. + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processed_path / node_name); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); Coordination::Responses responses; @@ -139,42 +132,30 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin while (true) { Coordination::Requests requests; - zkutil::addCheckNotExistsRequest(requests, zk_client, zookeeper_failed_path / node_name); - zkutil::addCheckNotExistsRequest(requests, zk_client, zookeeper_processing_path / node_name); - requests.push_back(zkutil::makeGetRequest(zookeeper_processed_path)); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processing_path / node_name); Coordination::Responses responses; auto code = zk_client->tryMulti(requests, responses); - if (code != Coordination::Error::ZOK) { - if (responses[0]->error != Coordination::Error::ZOK - || responses[1]->error != Coordination::Error::ZOK) - { - /// Path is already in Failed or Processing. - return false; - } - /// GetRequest for zookeeper_processed_path should never fail, - /// because this is persistent node created at the creation of S3Queue storage. - throw zkutil::KeeperException::fromPath(code, requests.back()->getPath()); + LOG_TEST(log, "Skipping file `{}`: {}", + path, responses[0]->error != Coordination::Error::ZOK ? "failed" : "processing"); + return false; } Coordination::Stat processed_node_stat; + auto data = zk_client->get(zookeeper_processed_path, &processed_node_stat); NodeMetadata processed_node_metadata; - if (const auto * get_response = dynamic_cast(responses.back().get())) - { - processed_node_stat = get_response->stat; - if (!get_response->data.empty()) - processed_node_metadata = NodeMetadata::fromString(get_response->data); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected response type with error: {}", responses.back()->error); + if (!data.empty()) + processed_node_metadata = NodeMetadata::fromString(data); auto max_processed_file_path = processed_node_metadata.file_path; if (!max_processed_file_path.empty() && path <= max_processed_file_path) return false; requests.clear(); + responses.clear(); zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); @@ -186,10 +167,14 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin if (responses[0]->error != Coordination::Error::ZOK || responses[1]->error != Coordination::Error::ZOK) { - /// Path is already in Failed or Processing. + LOG_TEST(log, "Skipping file `{}`: {}", + path, responses[0]->error != Coordination::Error::ZOK ? "failed" : "processing"); return false; } - /// Max processed path changed. Retry. + else + { + LOG_TEST(log, "Version of max processed file changed. Retring the check for file `{}`", path); + } } } @@ -228,9 +213,9 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) /// TODO this could be because of the expired session. if (responses[0]->error != Coordination::Error::ZOK) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as processed but it is not processing"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is not processing"); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as processed but it is already processed"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is already processed"); } void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) @@ -273,36 +258,83 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc auto node_metadata = createNodeMetadata(path, exception_message); const auto zk_client = storage->getZooKeeper(); - Coordination::Requests requests; - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); - requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, node_metadata.toString(), zkutil::CreateMode::Persistent)); - - Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) - return; - - if (responses[0]->error != Coordination::Error::ZOK) + if (max_loading_retries == 0) { - /// TODO this could be because of the expired session. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attemp to set file as filed but it is not processing"); + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, + node_metadata.toString(), + zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + { + LOG_TEST(log, "File `{}` failed to process and will not be retried. " + "Error: {}", path, exception_message); + return; + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); } + const auto node_name_with_retriable_suffix = node_name + ".retriable"; + Coordination::Stat stat; - auto failed_node_metadata = NodeMetadata::fromString(zk_client->get(zookeeper_failed_path / node_name, &stat)); - node_metadata.retries = failed_node_metadata.retries + 1; + std::string res; + if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) + { + auto failed_node_metadata = NodeMetadata::fromString(res); + node_metadata.retries = failed_node_metadata.retries + 1; + } - /// Failed node already exists, update it. - requests.clear(); - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); - requests.push_back(zkutil::makeSetRequest(zookeeper_failed_path / node_name, node_metadata.toString(), stat.version)); + LOG_TEST(log, "File `{}` failed to process, try {}/{} (Error: {})", + path, node_metadata.retries, max_loading_retries, exception_message); - responses.clear(); - code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) - return; + if (node_metadata.retries >= max_loading_retries) + { + /// File is no longer retriable. + /// Make a failed/node_name node and remove failed/node_name.retriable node. + /// TODO always add version for processing node. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + requests.push_back(zkutil::makeRemoveRequest(zookeeper_failed_path / node_name_with_retriable_suffix, + stat.version)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, + node_metadata.toString(), + zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); + } + else + { + Coordination::Requests requests; + requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); + if (node_metadata.retries == 0) + { + requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name_with_retriable_suffix, + node_metadata.toString(), + zkutil::CreateMode::Persistent)); + } + else + { + requests.push_back(zkutil::makeSetRequest(zookeeper_failed_path / node_name_with_retriable_suffix, + node_metadata.toString(), + stat.version)); + } + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); + if (code == Coordination::Error::ZOK) + return; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); + } } } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index be71af24601..9aa83a1aa97 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -41,7 +41,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int S3_ERROR; - extern const int NOT_IMPLEMENTED; extern const int QUERY_NOT_ALLOWED; extern const int REPLICA_ALREADY_EXISTS; extern const int INCOMPATIBLE_COLUMNS; @@ -391,7 +390,7 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const S3QueueTableMetadata old_metadata(configuration, *s3queue_settings, storage_metadata); old_metadata.checkEquals(metadata_from_zk); - auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns")); + auto columns_from_zk = ColumnsDescription::parse(metadata_from_zk.columns); const ColumnsDescription & old_columns = storage_metadata.getColumns(); if (columns_from_zk != old_columns) { diff --git a/tests/integration/test_storage_s3_queue/configs/zookeeper.xml b/tests/integration/test_storage_s3_queue/configs/zookeeper.xml new file mode 100644 index 00000000000..27334dca590 --- /dev/null +++ b/tests/integration/test_storage_s3_queue/configs/zookeeper.xml @@ -0,0 +1,16 @@ + + + + zoo1 + 2181 + + + zoo2 + 2181 + + + zoo3 + 2181 + + + diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 65c31acb5d8..9793f2b7191 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -17,6 +17,11 @@ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/home/sergey/vkr/ClickHouse/programs/ser """ +MINIO_INTERNAL_PORT = 9001 +AVAILABLE_MODES = ["unordered", "ordered"] +AUTH = "'minio','minio123'," +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. @@ -92,53 +97,6 @@ def s3_queue_setup_teardown(started_cluster): yield # run test -MINIO_INTERNAL_PORT = 9001 -AVAILABLE_MODES = ["unordered", "ordered"] -AUTH = "'minio','minio123'," - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - - -def put_s3_file_content(started_cluster, bucket, filename, data): - buf = io.BytesIO(data) - started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) - - -def generate_random_files( - count, prefix, cluster, bucket, column_num=3, row_num=10, start_ind=0 -): - total_values = [] - to_generate = [ - (f"{prefix}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) - ] - print(f"Generating files: {to_generate}") - to_generate.sort(key=lambda x: x[0]) - - for filename, i in to_generate: - rand_values = [ - [random.randint(0, 50) for _ in range(column_num)] for _ in range(row_num) - ] - total_values += rand_values - values_csv = ( - "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" - ).encode() - put_s3_file_content(cluster, bucket, filename, values_csv) - return total_values - - -# Returns content of given S3 file as string. -def get_s3_file_content(started_cluster, bucket, filename, decode=True): - # type: (ClickHouseCluster, str, str, bool) -> str - - data = started_cluster.minio_client.get_object(bucket, filename) - data_str = b"" - for chunk in data.stream(): - data_str += chunk - if decode: - return data_str.decode() - return data_str - - @pytest.fixture(scope="module") def started_cluster(): try: @@ -148,7 +106,11 @@ def started_cluster(): user_configs=["configs/users.xml"], with_minio=True, with_zookeeper=True, - main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"], + main_configs=[ + "configs/defaultS3.xml", + "configs/named_collections.xml", + "configs/zookeeper.xml", + ], ) cluster.add_instance( "instance2", @@ -178,43 +140,119 @@ def run_query(instance, query, stdin=None, settings=None): return result -@pytest.mark.parametrize("mode", AVAILABLE_MODES) -def test_delete_after_processing(started_cluster, mode): - bucket = started_cluster.minio_bucket - node = started_cluster.instances["instance"] +def generate_random_files( + started_cluster, files_path, count, column_num=3, row_num=10, start_ind=0 +): + files = [ + (f"{files_path}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) + ] + files.sort(key=lambda x: x[0]) - table_name = "test.delete_after_processing" - dst_table_name = "test.delete_after_processing_dst" - mv_name = "test.delete_after_processing_mv" - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - files_num = 5 - row_num = 10 + print(f"Generating files: {files}") - prefix = "delete" - total_values = generate_random_files( - files_num, prefix, started_cluster, bucket, row_num=row_num + total_values = [] + for filename, i in files: + rand_values = [ + [random.randint(0, 1000) for _ in range(column_num)] for _ in range(row_num) + ] + total_values += rand_values + values_csv = ( + "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" + ).encode() + put_s3_file_content(started_cluster, filename, values_csv) + return total_values + + +def put_s3_file_content(started_cluster, filename, data): + buf = io.BytesIO(data) + started_cluster.minio_client.put_object( + started_cluster.minio_bucket, filename, buf, len(data) ) + + +def get_s3_file_content(started_cluster, bucket, filename, decode=True): + # type: (ClickHouseCluster, str, str, bool) -> str + # Returns content of given S3 file as string. + + data = started_cluster.minio_client.get_object(bucket, filename) + data_str = b"" + for chunk in data.stream(): + data_str += chunk + if decode: + return data_str.decode() + return data_str + + +def create_table( + started_cluster, + node, + table_name, + mode, + files_path, + format="column1 UInt32, column2 UInt32, column3 UInt32", + additional_settings={}, +): + settings = { + "s3queue_loading_retries": 0, + "after_processing": "keep", + "keeper_path": f"/clickhouse/test_{table_name}", + "mode": f"{mode}", + } + settings.update(additional_settings) + + url = f"http://{started_cluster.minio_host}:{started_cluster.minio_port}/{started_cluster.minio_bucket}/{files_path}/" + node.query(f"DROP TABLE IF EXISTS {table_name}") + create_query = f""" + CREATE TABLE {table_name} ({format}) + ENGINE = S3Queue('{url}', {AUTH}'CSV') + SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))} + """ + node.query(create_query) + + +def create_mv( + node, + src_table_name, + dst_table_name, + format="column1 UInt32, column2 UInt32, column3 UInt32", +): + mv_name = f"{dst_table_name}_mv" node.query( f""" - DROP TABLE IF EXISTS {table_name}; DROP TABLE IF EXISTS {dst_table_name}; DROP TABLE IF EXISTS {mv_name}; - CREATE TABLE {table_name} ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_delete_{mode}', - s3queue_loading_retries = 3, - after_processing='delete'; - CREATE TABLE {dst_table_name} ({table_format}, _path String) + CREATE TABLE {dst_table_name} ({format}, _path String) ENGINE = MergeTree() ORDER BY column1; - CREATE MATERIALIZED VIEW {mv_name} TO {dst_table_name} AS SELECT *, _path FROM {table_name}; + CREATE MATERIALIZED VIEW {mv_name} TO {dst_table_name} AS SELECT *, _path FROM {src_table_name}; """ ) + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_delete_after_processing(started_cluster, mode): + node = started_cluster.instances["instance"] + table_name = f"test.delete_after_processing_{mode}" + dst_table_name = f"{table_name}_dst" + files_path = f"{table_name}_data" + files_num = 5 + row_num = 10 + + total_values = generate_random_files( + started_cluster, files_path, files_num, row_num=row_num + ) + create_table( + started_cluster, + node, + table_name, + mode, + files_path, + additional_settings={"after_processing": "delete"}, + ) + create_mv(node, table_name, dst_table_name) + expected_count = files_num * row_num for _ in range(100): count = int(node.query(f"SELECT count() FROM {dst_table_name}")) @@ -239,9 +277,13 @@ def test_delete_after_processing(started_cluster, mode): @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_failed_retry(started_cluster, mode): - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] + table_name = f"test.failed_retry_{mode}" + dst_table_name = f"{table_name}_dst" + files_path = f"{table_name}_data" + file_path = f"{files_path}/trash_test.csv" + keeper_path = f"/clickhouse/test_{table_name}" + retries_num = 3 values = [ ["failed", 1, 1], @@ -249,54 +291,55 @@ def test_failed_retry(started_cluster, mode): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() - filename = f"test.csv" - put_s3_file_content(started_cluster, bucket, filename, values_csv) + put_s3_file_content(started_cluster, file_path, values_csv) - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - CREATE TABLE test.s3_queue ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/select_failed_retry_{mode}', - s3queue_loading_retries = 3; - """ + create_table( + started_cluster, + node, + table_name, + mode, + files_path, + additional_settings={ + "s3queue_loading_retries": retries_num, + "keeper_path": keeper_path, + }, ) + create_mv(node, table_name, dst_table_name) - # first try - get_query = f"SELECT * FROM test.s3_queue" - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] - # second try - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] - # upload correct file - values = [ - [1, 1, 1], - ] - values_csv = ( - "\n".join((",".join(map(str, row)) for row in values)) + "\n" - ).encode() - put_s3_file_content(started_cluster, bucket, filename, values_csv) + failed_node_path = "" + for _ in range(20): + zk = started_cluster.get_kazoo_client("zoo1") + failed_nodes = zk.get_children(f"{keeper_path}/failed/") + if len(failed_nodes) > 0: + assert len(failed_nodes) == 1 + failed_node_path = f"{keeper_path}/failed/{failed_nodes[0]}" + time.sleep(1) - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == values + assert failed_node_path != "" - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] + retries = 0 + for _ in range(20): + data, stat = zk.get(failed_node_path) + json_data = json.loads(data) + print(f"Failed node metadata: {json_data}") + assert json_data["file_path"] == file_path + retries = int(json_data["retries"]) + if retries == retries_num: + break + time.sleep(1) + + assert retries == retries_num + assert 0 == int(node.query(f"SELECT count() FROM {dst_table_name}")) @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_direct_select_file(started_cluster, mode): - auth = "'minio','minio123'," - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] + table_name = f"test.direct_select_file_{mode}" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + file_path = f"{files_path}/test.csv" + values = [ [12549, 2463, 19893], [64021, 38652, 66703], @@ -305,63 +348,68 @@ def test_direct_select_file(started_cluster, mode): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() - filename = f"test.csv" - put_s3_file_content(started_cluster, bucket, filename, values_csv) - instance.query( - """ - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.s3_queue_2; - DROP TABLE IF EXISTS test.s3_queue_3; - """ - ) + put_s3_file_content(started_cluster, file_path, values_csv) - instance.query( - f""" - CREATE TABLE test.s3_queue ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/select_{mode}' - """ - ) + for i in range(3): + create_table( + started_cluster, + node, + f"{table_name}_{i + 1}", + mode, + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, + ) - get_query = f"SELECT * FROM test.s3_queue" assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_1").splitlines() ] == values - instance.query( - f""" - CREATE TABLE test.s3_queue_2 ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/select_{mode}' - """ + assert [ + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_2").splitlines() + ] == [] + + assert [ + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_3").splitlines() + ] == [] + + # New table with same zookeeper path + create_table( + started_cluster, + node, + f"{table_name}_4", + mode, + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) - get_query = f"SELECT * FROM test.s3_queue" assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] - # New table with same zookeeper path - get_query = f"SELECT * FROM test.s3_queue_2" - assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_4").splitlines() ] == [] + # New table with different zookeeper path - instance.query( - f""" - CREATE TABLE test.s3_queue_3 ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') - SETTINGS - mode = '{mode}', - keeper_path='/clickhouse/select_{mode}_2' - """ + keeper_path = f"/clickhouse/test_{table_name}_{mode}_2" + create_table( + started_cluster, + node, + f"{table_name}_4", + mode, + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) - get_query = f"SELECT * FROM test.s3_queue_3" + assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}_4").splitlines() ] == values values = [ @@ -370,189 +418,116 @@ def test_direct_select_file(started_cluster, mode): values_csv = ( "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() - filename = f"t.csv" - put_s3_file_content(started_cluster, bucket, filename, values_csv) + file_path = f"{files_path}/t.csv" + put_s3_file_content(started_cluster, file_path, values_csv) - get_query = f"SELECT * FROM test.s3_queue_3" if mode == "unordered": assert [ list(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() + for l in node.query(f"SELECT * FROM {table_name}_4").splitlines() ] == values elif mode == "ordered": assert [ list(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() + for l in node.query(f"SELECT * FROM {table_name}_4").splitlines() ] == [] @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_direct_select_multiple_files(started_cluster, mode): - prefix = f"multiple_files_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - instance.query("drop table if exists test.s3_queue") - instance.query( - f""" - CREATE TABLE test.s3_queue ({table_format}) - ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/select_multiple_{mode}' - """ - ) + node = started_cluster.instances["instance"] + table_name = f"direct_select_multiple_files_{mode}" + files_path = f"{table_name}_data" + create_table(started_cluster, node, table_name, mode, files_path) for i in range(5): rand_values = [[random.randint(0, 50) for _ in range(3)] for _ in range(10)] - values_csv = ( "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" ).encode() - filename = f"{prefix}/test_{i}.csv" - put_s3_file_content(started_cluster, bucket, filename, values_csv) - get_query = f"SELECT * FROM test.s3_queue" + file_path = f"{files_path}/test_{i}.csv" + put_s3_file_content(started_cluster, file_path, values_csv) + assert [ list(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() + for l in node.query(f"SELECT * FROM {table_name}").splitlines() ] == rand_values - total_values = generate_random_files( - 4, prefix, started_cluster, bucket, start_ind=5 - ) - get_query = f"SELECT * FROM test.s3_queue" + total_values = generate_random_files(started_cluster, files_path, 4, start_ind=5) assert { - tuple(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + tuple(map(int, l.split())) + for l in node.query(f"SELECT * FROM {table_name}").splitlines() } == set([tuple(i) for i in total_values]) @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_streaming_to_view_(started_cluster, mode): - prefix = f"streaming_files_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] + table_name = f"streaming_to_view_{mode}" + dst_table_name = f"{table_name}_dst" + files_path = f"{table_name}_data" - total_values = generate_random_files(10, prefix, started_cluster, bucket) - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue_persistent; - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + total_values = generate_random_files(started_cluster, files_path, 10) + create_table(started_cluster, node, table_name, mode, files_path) + create_mv(node, table_name, dst_table_name) - CREATE TABLE test.s3_queue_persistent ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/view_{mode}'; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS - SELECT - * - FROM test.s3_queue; - """ - ) expected_values = set([tuple(i) for i in total_values]) for i in range(10): - get_query = f"SELECT * FROM test.persistent_s3_queue_mv" - selected_values = { tuple(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}" + ).splitlines() } - if selected_values != expected_values: - time.sleep(1) - else: + if selected_values == expected_values: break - + time.sleep(1) assert selected_values == expected_values @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_streaming_to_many_views(started_cluster, mode): - prefix = f"streaming_files_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - retry_cnt = 10 + node = started_cluster.instances["instance"] + table_name = f"streaming_to_many_views_{mode}" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue_persistent; - DROP TABLE IF EXISTS test.s3_queue_persistent_2; - DROP TABLE IF EXISTS test.s3_queue_persistent_3; - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv_2; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv_3; + for i in range(3): + table = f"{table_name}_{i + 1}" + create_table( + started_cluster, + node, + table, + mode, + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, + ) + create_mv(node, table, dst_table_name) - - CREATE TABLE test.s3_queue_persistent ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue_persistent_2 ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue_persistent_3 ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/multiple_view_{mode}'; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS - SELECT - * - FROM test.s3_queue; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_2 TO test.s3_queue_persistent_2 AS - SELECT - * - FROM test.s3_queue; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_3 TO test.s3_queue_persistent_3 AS - SELECT - * - FROM test.s3_queue; - """ - ) - total_values = generate_random_files(5, prefix, started_cluster, bucket) + total_values = generate_random_files(started_cluster, files_path, 5) expected_values = set([tuple(i) for i in total_values]) - for i in range(retry_cnt): - retry = False - for get_query in [ - f"SELECT * FROM test.s3_queue_persistent", - f"SELECT * FROM test.s3_queue_persistent_2", - f"SELECT * FROM test.s3_queue_persistent_3", - ]: - selected_values = { - tuple(map(int, l.split())) - for l in run_query(instance, get_query).splitlines() - } - if i == retry_cnt - 1: - assert selected_values == expected_values - if selected_values != expected_values: - retry = True - break - if retry: - time.sleep(1) - else: + def select(): + return { + tuple(map(int, l.split())) + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}" + ).splitlines() + } + + for _ in range(20): + if select() == expected_values: break + time.sleep(1) + assert select() == expected_values def test_multiple_tables_meta_mismatch(started_cluster): - prefix = f"test_meta" + files_path = f"test_meta" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -562,7 +537,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): DROP TABLE IF EXISTS test.s3_queue; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'ordered', keeper_path = '/clickhouse/test_meta'; @@ -574,7 +549,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_meta'; @@ -591,7 +566,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" CREATE TABLE test.s3_queue_copy ({table_format_copy}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'ordered', keeper_path = '/clickhouse/test_meta'; @@ -611,7 +586,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'TSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'TSV') SETTINGS mode = 'ordered', keeper_path = '/clickhouse/test_meta'; @@ -626,7 +601,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'ordered', keeper_path = '/clickhouse/test_meta'; @@ -637,7 +612,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): def test_max_set_age(started_cluster): files_to_generate = 10 max_age = 1 - prefix = f"test_multiple" + files_path = f"test_multiple" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -647,7 +622,7 @@ def test_max_set_age(started_cluster): DROP TABLE IF EXISTS test.s3_queue; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_set_age', @@ -657,7 +632,7 @@ def test_max_set_age(started_cluster): ) total_values = generate_random_files( - files_to_generate, prefix, started_cluster, bucket, row_num=1 + files_to_generate, files_path, started_cluster, bucket, row_num=1 ) get_query = f"SELECT * FROM test.s3_queue" res1 = [ @@ -677,7 +652,7 @@ def test_max_set_age(started_cluster): def test_multiple_tables_streaming_sync(started_cluster, mode): files_to_generate = 300 poll_size = 30 - prefix = f"test_multiple_{mode}" + files_path = f"test_multiple_{mode}" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -697,21 +672,21 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy_2; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', s3queue_polling_size = {poll_size}; CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', s3queue_polling_size = {poll_size}; CREATE TABLE test.s3_queue_copy_2 ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', @@ -746,7 +721,7 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): """ ) total_values = generate_random_files( - files_to_generate, prefix, started_cluster, bucket, row_num=1 + files_to_generate, files_path, started_cluster, bucket, row_num=1 ) def get_count(table_name): @@ -792,7 +767,7 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): files_to_generate = 100 poll_size = 2 - prefix = f"test_multiple_{mode}" + files_path = f"test_multiple_{mode}" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] instance_2 = started_cluster.instances["instance2"] @@ -806,7 +781,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): DROP TABLE IF EXISTS test.persistent_s3_queue_mv; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_{mode}', @@ -829,7 +804,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): ) total_values = generate_random_files( - files_to_generate, prefix, started_cluster, bucket, row_num=1 + files_to_generate, files_path, started_cluster, bucket, row_num=1 ) def get_count(node, table_name): @@ -869,7 +844,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): def test_max_set_size(started_cluster): files_to_generate = 10 - prefix = f"test_multiple" + files_path = f"test_multiple" bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -879,7 +854,7 @@ def test_max_set_size(started_cluster): DROP TABLE IF EXISTS test.s3_queue; CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_set_size', @@ -888,7 +863,7 @@ def test_max_set_size(started_cluster): ) total_values = generate_random_files( - files_to_generate, prefix, started_cluster, bucket, start_ind=0, row_num=1 + files_to_generate, files_path, started_cluster, bucket, start_ind=0, row_num=1 ) get_query = f"SELECT * FROM test.s3_queue" res1 = [ From 342755d35e9ecef55df306007f6f5b95b8d2b8db Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 14 Sep 2023 18:41:31 +0200 Subject: [PATCH 003/120] Better --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 173 +++++++- src/Storages/S3Queue/S3QueueFilesMetadata.h | 13 +- src/Storages/S3Queue/S3QueueSource.cpp | 3 + src/Storages/S3Queue/StorageS3Queue.cpp | 13 +- src/Storages/S3Queue/StorageS3Queue.h | 2 - .../integration/test_storage_s3_queue/test.py | 371 ++++++++---------- 6 files changed, 357 insertions(+), 218 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 99c0924968c..d4c2c116a47 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,8 +1,13 @@ +#include "Common/Exception.h" +#include "Common/ZooKeeper/Types.h" +#include "Interpreters/Context_fwd.h" +#include "Storages/S3Queue/S3QueueSettings.h" #include "config.h" #if USE_AWS_S3 #include #include +#include #include #include #include @@ -28,11 +33,22 @@ namespace { return std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); } + + size_t generateRescheduleInterval() + { + /// Use more or less random interval for unordered mode cleanup task. + /// So that distributed processing cleanup tasks would not schedule cleanup at the same time. + /// TODO: make lower and upper boundary configurable by settings + pcg64 rng(randomSeed()); + //return 5000 + rng() % 30000; + return rng() % 100; + } } S3QueueFilesMetadata::S3QueueFilesMetadata( const StorageS3Queue * storage_, - const S3QueueSettings & settings_) + const S3QueueSettings & settings_, + ContextPtr context) : storage(storage_) , mode(settings_.mode) , max_set_size(settings_.s3queue_tracked_files_limit.value) @@ -43,6 +59,27 @@ S3QueueFilesMetadata::S3QueueFilesMetadata( , zookeeper_failed_path(storage->getZooKeeperPath() / "failed") , log(&Poco::Logger::get("S3QueueFilesMetadata")) { + if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) + { + task = context->getSchedulePool().createTask("S3QueueCleanupFunc", [this] { cleanupThreadFunc(); }); + task->activate(); + + auto schedule_ms = generateRescheduleInterval(); + LOG_TEST(log, "Scheduling a cleanup task in {} ms", schedule_ms); + task->scheduleAfter(schedule_ms); + } +} + +S3QueueFilesMetadata::~S3QueueFilesMetadata() +{ + deactivateCleanupTask(); +} + +void S3QueueFilesMetadata::deactivateCleanupTask() +{ + shutdown = true; + if (task) + task->deactivate(); } std::string S3QueueFilesMetadata::NodeMetadata::toString() const @@ -109,6 +146,10 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { + /// Create an ephemenral node in /processing + /// if corresponding node does not exist in failed/, processed/ and processing/. + /// Return false otherwise. + const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = storage->getZooKeeper(); @@ -125,6 +166,10 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::str bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { + /// Create an ephemenral node in /processing + /// if corresponding it does not exist in failed/, processing/ and satisfied max processed file check. + /// Return false otherwise. + const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = storage->getZooKeeper(); @@ -195,8 +240,7 @@ void S3QueueFilesMetadata::setFileProcessed(const String & path) void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) { - /// List results in s3 are always returned in UTF-8 binary order. - /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) + /// Create a persistent node in /processed and remove ephemeral node from /processing. const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); @@ -337,6 +381,129 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc } } +void S3QueueFilesMetadata::cleanupThreadFunc() +{ + /// A background task is responsible for maintaining + /// max_set_size and max_set_age settings for `unordered` processing mode. + + if (shutdown) + return; + + try + { + cleanupThreadFuncImpl(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + if (shutdown) + return; + + task->scheduleAfter(generateRescheduleInterval()); +} + +void S3QueueFilesMetadata::cleanupThreadFuncImpl() +{ + chassert(max_set_size || max_set_age_sec); + + const bool check_nodes_limit = max_set_size > 0; + const bool check_nodes_ttl = max_set_age_sec > 0; + + const auto zk_client = storage->getZooKeeper(); + auto nodes = zk_client->getChildren(zookeeper_processed_path); + if (nodes.empty()) + { + LOG_TEST(log, "A set of nodes is empty"); + return; + } + + const bool nodes_limit_exceeded = nodes.size() > max_set_size; + if (!nodes_limit_exceeded && check_nodes_limit && !check_nodes_ttl) + { + LOG_TEST(log, "No limit exceeded"); + return; + } + + struct Node + { + std::string name; + NodeMetadata metadata; + }; + auto node_cmp = [](const Node & a, const Node & b) + { + return a.metadata.last_processed_timestamp < b.metadata.last_processed_timestamp; + }; + + /// Ordered in ascending order of timestamps. + std::set sorted_nodes(node_cmp); + + for (const auto & node : nodes) + { + try + { + std::string metadata_str; + if (zk_client->tryGet(zookeeper_processed_path / node, metadata_str)) + { + bool inserted = sorted_nodes.emplace(node, NodeMetadata::fromString(metadata_str)).second; + chassert(inserted); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + /// TODO add a zookeeper lock for cleanup + + LOG_TRACE(log, "Checking node limits"); + + size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes.size() - max_set_size : 0; + for (const auto & node : sorted_nodes) + { + if (nodes_to_remove) + { + auto path = zookeeper_processed_path / node.name; + LOG_TEST(log, "Removing node at path `{}` because max files limit is reached", path.string()); + + auto code = zk_client->tryRemove(path); + if (code == Coordination::Error::ZOK) + --nodes_to_remove; + else + LOG_ERROR(log, "Failed to remove a node `{}`", path.string()); + } + else if (check_nodes_ttl) + { + UInt64 node_age = getCurrentTime() - node.metadata.last_processed_timestamp; + if (node_age >= max_set_age_sec) + { + auto path = zookeeper_processed_path / node.name; + LOG_TEST(log, "Removing node at path `{}` because file ttl is reached", path.string()); + + auto code = zk_client->tryRemove(path); + if (code != Coordination::Error::ZOK) + LOG_ERROR(log, "Failed to remove a node `{}`", path.string()); + } + else if (!nodes_to_remove) + { + /// Nodes limit satisfied. + /// Nodes ttl satisfied as well as if current node is under tll, then all remaining as well + /// (because we are iterating in timestamp ascending order). + break; + } + } + else + { + /// Nodes limit and ttl are satisfied. + break; + } + } + + LOG_TRACE(log, "Node limits check finished"); +} + } #endif diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 302feab6028..b8e172bcd88 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace fs = std::filesystem; namespace Poco { class Logger; } @@ -17,7 +18,9 @@ class StorageS3Queue; class S3QueueFilesMetadata { public: - S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_); + S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_, ContextPtr context); + + ~S3QueueFilesMetadata(); bool trySetFileAsProcessing(const std::string & path); @@ -25,6 +28,8 @@ public: void setFileFailed(const std::string & path, const std::string & exception_message); + void deactivateCleanupTask(); + private: const StorageS3Queue * storage; const S3QueueMode mode; @@ -39,6 +44,9 @@ private: mutable std::mutex mutex; Poco::Logger * log; + std::atomic_bool shutdown = false; + BackgroundSchedulePool::TaskHolder task; + bool trySetFileAsProcessingForOrderedMode(const std::string & path); bool trySetFileAsProcessingForUnorderedMode(const std::string & path); @@ -59,6 +67,9 @@ private: }; NodeMetadata createNodeMetadata(const std::string & path, const std::string & exception = "", size_t retries = 0); + + void cleanupThreadFunc(); + void cleanupThreadFuncImpl(); }; } diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 6704345ea59..3a834dae8d0 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -37,6 +37,9 @@ StorageS3QueueSource::FileIterator::FileIterator( StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::FileIterator::next() { + /// List results in s3 are always returned in UTF-8 binary order. + /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) + while (true) { KeyWithInfo val = glob_iterator->next(); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 53a79aa9cff..bbaf5ae5311 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -88,7 +88,7 @@ StorageS3Queue::StorageS3Queue( , s3queue_settings(std::move(s3queue_settings_)) , zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *s3queue_settings)) , after_processing(s3queue_settings->after_processing) - , files_metadata(std::make_shared(this, *s3queue_settings)) + , files_metadata(std::make_shared(this, *s3queue_settings, context_)) , configuration{configuration_} , format_settings(format_settings_) , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) @@ -138,8 +138,17 @@ void StorageS3Queue::startup() void StorageS3Queue::shutdown() { shutdown_called = true; + if (task) + { task->deactivate(); + } + + if (files_metadata) + { + files_metadata->deactivateCleanupTask(); + files_metadata.reset(); + } } bool StorageS3Queue::supportsSubsetOfColumns(const ContextPtr & context_) const @@ -182,7 +191,7 @@ std::shared_ptr StorageS3Queue::createSource( { auto configuration_snapshot = updateConfigurationAndGetCopy(local_context); auto file_iterator = createFileIterator(local_context, query); - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(), getVirtuals()); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); auto internal_source = std::make_unique( read_from_format_info, configuration.format, getName(), local_context, format_settings, diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 5cdac607645..07f52c434de 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -1,9 +1,7 @@ #pragma once - #include "config.h" #if USE_AWS_S3 - #include #include #include diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 9793f2b7191..dd59138d935 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -191,6 +191,7 @@ def create_table( files_path, format="column1 UInt32, column2 UInt32, column3 UInt32", additional_settings={}, + file_format="CSV", ): settings = { "s3queue_loading_retries": 0, @@ -204,7 +205,7 @@ def create_table( node.query(f"DROP TABLE IF EXISTS {table_name}") create_query = f""" CREATE TABLE {table_name} ({format}) - ENGINE = S3Queue('{url}', {AUTH}'CSV') + ENGINE = S3Queue('{url}', {AUTH}'{file_format}') SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))} """ node.query(create_query) @@ -527,50 +528,52 @@ def test_streaming_to_many_views(started_cluster, mode): def test_multiple_tables_meta_mismatch(started_cluster): - files_path = f"test_meta" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + node = started_cluster.instances["instance"] + table_name = f"multiple_tables_meta_mismatch" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'ordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + table_name, + "ordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) # check mode failed = False try: - instance.query( - f""" - CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'unordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + f"{table_name}_copy", + "unordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) except QueryRuntimeException as e: assert "Existing table metadata in ZooKeeper differs in engine mode" in str(e) failed = True + assert failed is True # check columns - table_format_copy = table_format + ", column4 UInt32" try: - instance.query( - f""" - CREATE TABLE test.s3_queue_copy ({table_format_copy}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'ordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + f"{table_name}_copy", + "ordered", + files_path, + format="column1 UInt32, column2 UInt32, column3 UInt32, column4 UInt32", + additional_settings={ + "keeper_path": keeper_path, + }, ) except QueryRuntimeException as e: assert ( @@ -583,172 +586,96 @@ def test_multiple_tables_meta_mismatch(started_cluster): # check format try: - instance.query( - f""" - CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'TSV') - SETTINGS - mode = 'ordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + f"{table_name}_copy", + "ordered", + files_path, + format="column1 UInt32, column2 UInt32, column3 UInt32, column4 UInt32", + additional_settings={ + "keeper_path": keeper_path, + }, + file_format="TSV", ) except QueryRuntimeException as e: assert "Existing table metadata in ZooKeeper differs in format name" in str(e) failed = True + assert failed is True # create working engine - instance.query( - f""" - CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'ordered', - keeper_path = '/clickhouse/test_meta'; - """ + create_table( + started_cluster, + node, + f"{table_name}_copy", + "ordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, ) -def test_max_set_age(started_cluster): - files_to_generate = 10 - max_age = 1 - files_path = f"test_multiple" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'unordered', - keeper_path = '/clickhouse/test_set_age', - s3queue_tracked_files_limit = 10, - s3queue_tracked_file_ttl_sec = {max_age}; - """ - ) - - total_values = generate_random_files( - files_to_generate, files_path, started_cluster, bucket, row_num=1 - ) - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] - assert res1 == total_values - time.sleep(max_age + 1) - - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] - assert res1 == total_values - - @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_multiple_tables_streaming_sync(started_cluster, mode): + node = started_cluster.instances["instance"] + table_name = f"multiple_tables_streaming_sync_{mode}" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" files_to_generate = 300 poll_size = 30 - files_path = f"test_multiple_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.s3_queue_copy; - DROP TABLE IF EXISTS test.s3_queue_copy_2; + for i in range(3): + table = f"{table_name}_{i + 1}" + dst_table = f"{dst_table_name}_{i + 1}" + create_table( + started_cluster, + node, + table, + mode, + files_path, + additional_settings={ + "s3queue_polling_size": poll_size, + "keeper_path": keeper_path, + }, + ) + create_mv(node, table, dst_table) - DROP TABLE IF EXISTS test.s3_queue_persistent; - DROP TABLE IF EXISTS test.s3_queue_persistent_copy; - DROP TABLE IF EXISTS test.s3_queue_persistent_copy_2; - - DROP TABLE IF EXISTS test.persistent_s3_queue_mv; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy_2; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', - s3queue_polling_size = {poll_size}; - - CREATE TABLE test.s3_queue_copy ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', - s3queue_polling_size = {poll_size}; - - CREATE TABLE test.s3_queue_copy_2 ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', - s3queue_polling_size = {poll_size}; - - CREATE TABLE test.s3_queue_persistent ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue_persistent_copy ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE TABLE test.s3_queue_persistent_copy_2 ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS - SELECT - * - FROM test.s3_queue; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy TO test.s3_queue_persistent_copy AS - SELECT - * - FROM test.s3_queue_copy; - - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy_2 TO test.s3_queue_persistent_copy_2 AS - SELECT - * - FROM test.s3_queue_copy_2; - """ - ) total_values = generate_random_files( - files_to_generate, files_path, started_cluster, bucket, row_num=1 + started_cluster, files_path, files_to_generate, row_num=1 ) def get_count(table_name): - return int(run_query(instance, f"SELECT count() FROM {table_name}")) + return int(run_query(node, f"SELECT count() FROM {table_name}")) for _ in range(100): if ( - get_count("test.s3_queue_persistent") - + get_count("test.s3_queue_persistent_copy") - + get_count("test.s3_queue_persistent_copy_2") + get_count(f"{dst_table_name}_1") + + get_count(f"{dst_table_name}_2") + + get_count(f"{dst_table_name}_3") ) == files_to_generate: break time.sleep(1) - get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + list(map(int, l.split())) + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}_1" + ).splitlines() ] - get_query_copy = f"SELECT * FROM test.s3_queue_persistent_copy" res2 = [ list(map(int, l.split())) - for l in run_query(instance, get_query_copy).splitlines() + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}_2" + ).splitlines() ] - get_query_copy_2 = f"SELECT * FROM test.s3_queue_persistent_copy_2" res3 = [ list(map(int, l.split())) - for l in run_query(instance, get_query_copy_2).splitlines() + for l in node.query( + f"SELECT column1, column2, column3 FROM {dst_table_name}_3" + ).splitlines() ] assert {tuple(v) for v in res1 + res2 + res3} == set( [tuple(i) for i in total_values] @@ -757,54 +684,41 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): # Checking that all files were processed only once time.sleep(10) assert ( - get_count("test.s3_queue_persistent") - + get_count("test.s3_queue_persistent_copy") - + get_count("test.s3_queue_persistent_copy_2") + get_count(f"{dst_table_name}_1") + + get_count(f"{dst_table_name}_2") + + get_count(f"{dst_table_name}_3") ) == files_to_generate @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): - files_to_generate = 100 + node = started_cluster.instances["instance"] + node_2 = started_cluster.instances["instance2"] + table_name = f"multiple_tables_streaming_sync_distributed_{mode}" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + files_to_generate = 300 poll_size = 2 - files_path = f"test_multiple_{mode}" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - instance_2 = started_cluster.instances["instance2"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - for inst in [instance, instance_2]: - inst.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - DROP TABLE IF EXISTS test.s3_queue_persistent; - DROP TABLE IF EXISTS test.persistent_s3_queue_mv; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = '{mode}', - keeper_path = '/clickhouse/test_multiple_consumers_{mode}', - s3queue_polling_size = {poll_size}; - - CREATE TABLE test.s3_queue_persistent ({table_format}) - ENGINE = MergeTree() - ORDER BY column1; - """ + for instance in [node, node_2]: + create_table( + started_cluster, + instance, + table_name, + mode, + files_path, + additional_settings={ + "s3queue_polling_size": poll_size, + "keeper_path": keeper_path, + }, ) - for inst in [instance, instance_2]: - inst.query( - f""" - CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS - SELECT - * - FROM test.s3_queue; - """ - ) + for instance in [node, node_2]: + create_mv(instance, table_name, dst_table_name) total_values = generate_random_files( - files_to_generate, files_path, started_cluster, bucket, row_num=1 + started_cluster, files_path, files_to_generate, row_num=1 ) def get_count(node, table_name): @@ -812,18 +726,15 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): for _ in range(150): if ( - get_count(instance, "test.s3_queue_persistent") - + get_count(instance_2, "test.s3_queue_persistent") + get_count(node, dst_table_name) + get_count(node_2, dst_table_name) ) == files_to_generate: break time.sleep(1) - get_query = f"SELECT * FROM test.s3_queue_persistent" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] + get_query = f"SELECT column1, column2, column3 FROM {dst_table_name}" + res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] res2 = [ - list(map(int, l.split())) for l in run_query(instance_2, get_query).splitlines() + list(map(int, l.split())) for l in run_query(node_2, get_query).splitlines() ] assert len(res1) + len(res2) == files_to_generate @@ -837,11 +748,51 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): # Checking that all files were processed only once time.sleep(10) assert ( - get_count(instance, "test.s3_queue_persistent") - + get_count(instance_2, "test.s3_queue_persistent") + get_count(node, dst_table_name) + get_count(node_2, dst_table_name) ) == files_to_generate +def test_max_set_age(started_cluster): + node = started_cluster.instances["instance"] + table_name = f"max_set_age" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + max_age = 1 + files_to_generate = 10 + + create_table( + started_cluster, + node, + table_name, + "unordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + "s3queue_tracked_files_limit": 10, + "s3queue_tracked_file_ttl_sec": max_age, + }, + ) + + node.wait_for_log_line("Checking node limits") + node.wait_for_log_line("Node limits check finished") + + total_values = generate_random_files( + started_cluster, files_path, files_to_generate, row_num=1 + ) + res1 = [ + list(map(int, l.split())) + for l in run_query(node, f"SELECT * FROM {table_name}").splitlines() + ] + assert res1 == total_values + time.sleep(max_age + 1) + + res1 = [ + list(map(int, l.split())) + for l in run_query(node, f"SELECT * FROM {table_name}").splitlines() + ] + assert res1 == total_values + + def test_max_set_size(started_cluster): files_to_generate = 10 files_path = f"test_multiple" From 434e2d4b57d8fb428f4764f857885aca50bc70be Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Sep 2023 18:57:41 +0000 Subject: [PATCH 004/120] Add blind test. --- .../test_merge_session_expired/__init__.py | 0 .../configs/keeper_config.xml | 5 +++ .../test_merge_session_expired/test.py | 41 +++++++++++++++++++ 3 files changed, 46 insertions(+) create mode 100644 tests/integration/test_merge_session_expired/__init__.py create mode 100644 tests/integration/test_merge_session_expired/configs/keeper_config.xml create mode 100644 tests/integration/test_merge_session_expired/test.py diff --git a/tests/integration/test_merge_session_expired/__init__.py b/tests/integration/test_merge_session_expired/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_merge_session_expired/configs/keeper_config.xml b/tests/integration/test_merge_session_expired/configs/keeper_config.xml new file mode 100644 index 00000000000..4258475f7b5 --- /dev/null +++ b/tests/integration/test_merge_session_expired/configs/keeper_config.xml @@ -0,0 +1,5 @@ + + + 3000 + + diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py new file mode 100644 index 00000000000..90a8f940e6b --- /dev/null +++ b/tests/integration/test_merge_session_expired/test.py @@ -0,0 +1,41 @@ +import logging +import pytest +import time +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/keeper_config.xml"], user_configs=["configs/timeouts.xml"], stay_alive=True, with_zookeeper=True +) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_merge_session_expired(started_cluster): + node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple()") + node1.query("insert into tab select number from numbers(10)") + node1.query("alter table tab delete where x = 2 settings mutations_sync=2") + node1.query("alter table tab delete where x = 4 settings mutations_sync=2") + node1.query("alter table tab delete where x = 6 settings mutations_sync=2") + node1.query("system stop merges") + node1.query("insert into tab select number + 10 from numbers(10)") + node1.query("optimize table tab final settings alter_sync=0") + + with PartitionManager() as pm: + #logging.info(pm.dump_rules()) + pm.drop_instance_zk_connections(node1) + node1.query("system start merges") + node1.query("system sync replica tab") + node1.restart_clickhouse() + + node1.query("system sync replica tab") + assert node1.query("select count() from tab") == '17' From 0b661188e7d9e51e7e040ee065a559487539bb9a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Sep 2023 19:00:37 +0000 Subject: [PATCH 005/120] Add blind test. --- tests/integration/test_merge_session_expired/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py index 90a8f940e6b..e6208203584 100644 --- a/tests/integration/test_merge_session_expired/test.py +++ b/tests/integration/test_merge_session_expired/test.py @@ -34,7 +34,7 @@ def test_merge_session_expired(started_cluster): #logging.info(pm.dump_rules()) pm.drop_instance_zk_connections(node1) node1.query("system start merges") - node1.query("system sync replica tab") + node1.query("select sleep(1)") node1.restart_clickhouse() node1.query("system sync replica tab") From 466fee196bd1b8bc31046f8de9ee27f0ad32a655 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Sep 2023 11:41:42 +0000 Subject: [PATCH 006/120] Update test. --- .../test_merge_session_expired/test.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py index e6208203584..f2122016365 100644 --- a/tests/integration/test_merge_session_expired/test.py +++ b/tests/integration/test_merge_session_expired/test.py @@ -7,7 +7,7 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/keeper_config.xml"], user_configs=["configs/timeouts.xml"], stay_alive=True, with_zookeeper=True + "node1", main_configs=["configs/keeper_config.xml"], stay_alive=True, with_zookeeper=True ) @pytest.fixture(scope="module") @@ -21,13 +21,14 @@ def started_cluster(): def test_merge_session_expired(started_cluster): - node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple()") + node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3") node1.query("insert into tab select number from numbers(10)") - node1.query("alter table tab delete where x = 2 settings mutations_sync=2") - node1.query("alter table tab delete where x = 4 settings mutations_sync=2") - node1.query("alter table tab delete where x = 6 settings mutations_sync=2") - node1.query("system stop merges") node1.query("insert into tab select number + 10 from numbers(10)") + node1.query("alter table tab delete where x = 12 settings mutations_sync=2") + node1.query("alter table tab delete where x = 14 settings mutations_sync=2") + node1.query("alter table tab delete where x = 16 settings mutations_sync=2") + node1.query("system stop merges") + # node1.query("insert into tab select number + 20 from numbers(10)") node1.query("optimize table tab final settings alter_sync=0") with PartitionManager() as pm: @@ -36,6 +37,8 @@ def test_merge_session_expired(started_cluster): node1.query("system start merges") node1.query("select sleep(1)") node1.restart_clickhouse() + pm.restore_instance_zk_connections(node1) + node1.query("system restart replica tab") node1.query("system sync replica tab") - assert node1.query("select count() from tab") == '17' + assert node1.query("select count() from tab") == '17\n' From 6846fe3c589b0388a039e668e494350ecdff94ab Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Sep 2023 14:21:08 +0200 Subject: [PATCH 007/120] Fxi --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 34 ++++-- .../integration/test_storage_s3_queue/test.py | 114 +++++++++++------- 2 files changed, 97 insertions(+), 51 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index d4c2c116a47..0e9a121339f 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,3 +1,4 @@ +#include #include "Common/Exception.h" #include "Common/ZooKeeper/Types.h" #include "Interpreters/Context_fwd.h" @@ -253,7 +254,10 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) Coordination::Responses responses; auto code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) + { + LOG_TEST(log, "Moved file `{}` to processed", path); return; + } /// TODO this could be because of the expired session. if (responses[0]->error != Coordination::Error::ZOK) @@ -433,11 +437,16 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() }; auto node_cmp = [](const Node & a, const Node & b) { - return a.metadata.last_processed_timestamp < b.metadata.last_processed_timestamp; + if (a.metadata.last_processed_timestamp == b.metadata.last_processed_timestamp) + return a.metadata.file_path < b.metadata.file_path; + else + return a.metadata.last_processed_timestamp < b.metadata.last_processed_timestamp; }; /// Ordered in ascending order of timestamps. - std::set sorted_nodes(node_cmp); + std::multiset sorted_nodes(node_cmp); + + LOG_TRACE(log, "Found {} nodes", nodes.size()); for (const auto & node : nodes) { @@ -446,9 +455,11 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() std::string metadata_str; if (zk_client->tryGet(zookeeper_processed_path / node, metadata_str)) { - bool inserted = sorted_nodes.emplace(node, NodeMetadata::fromString(metadata_str)).second; - chassert(inserted); + sorted_nodes.emplace(node, NodeMetadata::fromString(metadata_str)); + LOG_TEST(log, "Fetched metadata for node {}", node); } + else + LOG_TEST(log, "Failed to fetch node metadata {}", node); } catch (...) { @@ -458,7 +469,14 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() /// TODO add a zookeeper lock for cleanup - LOG_TRACE(log, "Checking node limits"); + auto get_nodes_str = [&]() + { + WriteBufferFromOwnString wb; + for (const auto & [node, metadata] : sorted_nodes) + wb << fmt::format("Node: {}, path: {}, timestamp: {};\n", node, metadata.file_path, metadata.last_processed_timestamp); + return wb.str(); + }; + LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", max_set_size, max_set_age_sec, get_nodes_str()); size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes.size() - max_set_size : 0; for (const auto & node : sorted_nodes) @@ -466,7 +484,8 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() if (nodes_to_remove) { auto path = zookeeper_processed_path / node.name; - LOG_TEST(log, "Removing node at path `{}` because max files limit is reached", path.string()); + LOG_TEST(log, "Removing node at path {} ({}) because max files limit is reached", + node.metadata.file_path, path.string()); auto code = zk_client->tryRemove(path); if (code == Coordination::Error::ZOK) @@ -480,7 +499,8 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() if (node_age >= max_set_age_sec) { auto path = zookeeper_processed_path / node.name; - LOG_TEST(log, "Removing node at path `{}` because file ttl is reached", path.string()); + LOG_TEST(log, "Removing node at path {} ({}) because file is reached", + node.metadata.file_path, path.string()); auto code = zk_client->tryRemove(path); if (code != Coordination::Error::ZOK) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index dd59138d935..d26afcc3aa3 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -159,6 +159,7 @@ def generate_random_files( values_csv = ( "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" ).encode() + print(f"File {filename}, content: {total_values}") put_s3_file_content(started_cluster, filename, values_csv) return total_values @@ -755,9 +756,10 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): def test_max_set_age(started_cluster): node = started_cluster.instances["instance"] table_name = f"max_set_age" + dst_table_name = f"{table_name}_dst" keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" - max_age = 1 + max_age = 10 files_to_generate = 10 create_table( @@ -768,68 +770,92 @@ def test_max_set_age(started_cluster): files_path, additional_settings={ "keeper_path": keeper_path, - "s3queue_tracked_files_limit": 10, "s3queue_tracked_file_ttl_sec": max_age, }, ) - - node.wait_for_log_line("Checking node limits") - node.wait_for_log_line("Node limits check finished") + create_mv(node, table_name, dst_table_name) total_values = generate_random_files( started_cluster, files_path, files_to_generate, row_num=1 ) - res1 = [ - list(map(int, l.split())) - for l in run_query(node, f"SELECT * FROM {table_name}").splitlines() - ] - assert res1 == total_values + + expected_rows = 10 + + node.wait_for_log_line("Checking node limits") + node.wait_for_log_line("Node limits check finished") + + def get_count(): + return int(node.query(f"SELECT count() FROM {dst_table_name}")) + + for _ in range(20): + if expected_rows == get_count(): + break + time.sleep(1) + + assert expected_rows == get_count() + assert 10 == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + time.sleep(max_age + 1) - res1 = [ - list(map(int, l.split())) - for l in run_query(node, f"SELECT * FROM {table_name}").splitlines() + expected_rows = 20 + + for _ in range(20): + if expected_rows == get_count(): + break + time.sleep(1) + + assert expected_rows == get_count() + assert 10 == int(node.query(f"SELECT uniq(_path) from {dst_table_name}")) + + paths_count = [ + int(x) + for x in node.query( + f"SELECT count() from {dst_table_name} GROUP BY _path" + ).splitlines() ] - assert res1 == total_values + assert 10 == len(paths_count) + for path_count in paths_count: + assert 2 == path_count def test_max_set_size(started_cluster): + node = started_cluster.instances["instance"] + table_name = f"max_set_size" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + max_age = 10 files_to_generate = 10 - files_path = f"test_multiple" - bucket = started_cluster.minio_restricted_bucket - instance = started_cluster.instances["instance"] - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - instance.query( - f""" - DROP TABLE IF EXISTS test.s3_queue; - - CREATE TABLE test.s3_queue ({table_format}) - ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{files_path}/*', {AUTH}'CSV') - SETTINGS - mode = 'unordered', - keeper_path = '/clickhouse/test_set_size', - s3queue_tracked_files_limit = {files_to_generate - 1}; - """ + create_table( + started_cluster, + node, + table_name, + "unordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + "s3queue_tracked_files_limit": 9, + }, ) - total_values = generate_random_files( - files_to_generate, files_path, started_cluster, bucket, start_ind=0, row_num=1 + started_cluster, files_path, files_to_generate, start_ind=0, row_num=1 ) - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] - assert res1 == total_values - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] + get_query = f"SELECT * FROM {table_name}" + res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] + assert res1 == total_values + print(total_values) + + time.sleep(10) + + zk = started_cluster.get_kazoo_client("zoo1") + processed_nodes = zk.get_children(f"{keeper_path}/processed/") + assert len(processed_nodes) == 9 + + res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] assert res1 == [total_values[0]] - get_query = f"SELECT * FROM test.s3_queue" - res1 = [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] + time.sleep(10) + res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] assert res1 == [total_values[1]] From d4027d835e0d12afcd751ceb82d7636a3f5c2069 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Sep 2023 18:40:05 +0000 Subject: [PATCH 008/120] Add test with zero copy replication. --- .../configs/disks.xml | 21 ++++++++++ .../test_merge_session_expired/test.py | 41 +++++++++++++++++-- 2 files changed, 58 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_merge_session_expired/configs/disks.xml diff --git a/tests/integration/test_merge_session_expired/configs/disks.xml b/tests/integration/test_merge_session_expired/configs/disks.xml new file mode 100644 index 00000000000..94ac83b32ac --- /dev/null +++ b/tests/integration/test_merge_session_expired/configs/disks.xml @@ -0,0 +1,21 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + +
+ s3 +
+
+
+
+
+
diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py index f2122016365..db8eb954d3a 100644 --- a/tests/integration/test_merge_session_expired/test.py +++ b/tests/integration/test_merge_session_expired/test.py @@ -7,7 +7,7 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/keeper_config.xml"], stay_alive=True, with_zookeeper=True + "node1", main_configs=["configs/keeper_config.xml", "configs/disks.xml"], stay_alive=True, with_zookeeper=True, with_minio=True ) @pytest.fixture(scope="module") @@ -21,6 +21,7 @@ def started_cluster(): def test_merge_session_expired(started_cluster): + node1.query("drop table if exists tab") node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3") node1.query("insert into tab select number from numbers(10)") node1.query("insert into tab select number + 10 from numbers(10)") @@ -28,11 +29,9 @@ def test_merge_session_expired(started_cluster): node1.query("alter table tab delete where x = 14 settings mutations_sync=2") node1.query("alter table tab delete where x = 16 settings mutations_sync=2") node1.query("system stop merges") - # node1.query("insert into tab select number + 20 from numbers(10)") node1.query("optimize table tab final settings alter_sync=0") with PartitionManager() as pm: - #logging.info(pm.dump_rules()) pm.drop_instance_zk_connections(node1) node1.query("system start merges") node1.query("select sleep(1)") @@ -40,5 +39,39 @@ def test_merge_session_expired(started_cluster): pm.restore_instance_zk_connections(node1) node1.query("system restart replica tab") - node1.query("system sync replica tab") assert node1.query("select count() from tab") == '17\n' + + +def test_merge_session_expired_zero_copy(started_cluster): + node1.query("drop table if exists tab") + node1.query(""" + create table tab (x UInt64, y UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab2/', '0') order by tuple() + settings old_parts_lifetime=1, storage_policy='s3', allow_remote_fs_zero_copy_replication=1, replicated_max_ratio_of_wrong_parts=1, min_bytes_for_wide_part=1 + """) + + node1.query("insert into tab select number, number from numbers(10)") + node1.query("insert into tab select number + 10, number + 10 from numbers(10)") + node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") + node1.query("select * from tab") + node1.query("alter table tab update x = x + 1, y = y + 1 where 1 settings mutations_sync=2") + node1.query("select * from tab") + node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") + node1.query("select * from tab") + + node1.query("alter table tab add column z UInt64 materialized x + sleepEachRow(0.05) settings mutations_sync=2") + node1.query("optimize table tab final settings alter_sync=0") + + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node1) + # Wait some time for merge to start + # Part should be merged and stayed on disk, but not commited into zk + node1.query("select sleep(2)") + node1.restart_clickhouse() + pm.restore_instance_zk_connections(node1) + + node1.query("system restart replica tab") + # Wait for outdated parts to be removed + node1.query("select sleep(3)") + node1.query("select * from tab") + node1.query("system sync replica tab") + assert node1.query("select count() from tab") == '20\n' From 9460cd2503ae291e6e57c0206b3621d12e83ca68 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 15 Sep 2023 18:55:55 +0000 Subject: [PATCH 009/120] Automatic style fix --- .../test_merge_session_expired/test.py | 29 ++++++++++++++----- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py index db8eb954d3a..61e8ff3c627 100644 --- a/tests/integration/test_merge_session_expired/test.py +++ b/tests/integration/test_merge_session_expired/test.py @@ -7,9 +7,14 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/keeper_config.xml", "configs/disks.xml"], stay_alive=True, with_zookeeper=True, with_minio=True + "node1", + main_configs=["configs/keeper_config.xml", "configs/disks.xml"], + stay_alive=True, + with_zookeeper=True, + with_minio=True, ) + @pytest.fixture(scope="module") def started_cluster(): try: @@ -22,7 +27,9 @@ def started_cluster(): def test_merge_session_expired(started_cluster): node1.query("drop table if exists tab") - node1.query("create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3") + node1.query( + "create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3" + ) node1.query("insert into tab select number from numbers(10)") node1.query("insert into tab select number + 10 from numbers(10)") node1.query("alter table tab delete where x = 12 settings mutations_sync=2") @@ -39,26 +46,32 @@ def test_merge_session_expired(started_cluster): pm.restore_instance_zk_connections(node1) node1.query("system restart replica tab") - assert node1.query("select count() from tab") == '17\n' + assert node1.query("select count() from tab") == "17\n" def test_merge_session_expired_zero_copy(started_cluster): node1.query("drop table if exists tab") - node1.query(""" + node1.query( + """ create table tab (x UInt64, y UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab2/', '0') order by tuple() settings old_parts_lifetime=1, storage_policy='s3', allow_remote_fs_zero_copy_replication=1, replicated_max_ratio_of_wrong_parts=1, min_bytes_for_wide_part=1 - """) + """ + ) node1.query("insert into tab select number, number from numbers(10)") node1.query("insert into tab select number + 10, number + 10 from numbers(10)") node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") node1.query("select * from tab") - node1.query("alter table tab update x = x + 1, y = y + 1 where 1 settings mutations_sync=2") + node1.query( + "alter table tab update x = x + 1, y = y + 1 where 1 settings mutations_sync=2" + ) node1.query("select * from tab") node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") node1.query("select * from tab") - node1.query("alter table tab add column z UInt64 materialized x + sleepEachRow(0.05) settings mutations_sync=2") + node1.query( + "alter table tab add column z UInt64 materialized x + sleepEachRow(0.05) settings mutations_sync=2" + ) node1.query("optimize table tab final settings alter_sync=0") with PartitionManager() as pm: @@ -74,4 +87,4 @@ def test_merge_session_expired_zero_copy(started_cluster): node1.query("select sleep(3)") node1.query("select * from tab") node1.query("system sync replica tab") - assert node1.query("select count() from tab") == '20\n' + assert node1.query("select count() from tab") == "20\n" From e4256eb268ed1307a00c86dfc644750dc3411566 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Sep 2023 13:44:45 +0200 Subject: [PATCH 010/120] Better --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 56 +++++++++++++------ src/Storages/S3Queue/S3QueueFilesMetadata.h | 3 + src/Storages/S3Queue/S3QueueSettings.h | 16 +++--- src/Storages/S3Queue/StorageS3Queue.cpp | 8 +-- src/Storages/S3Queue/StorageS3Queue.h | 3 - .../integration/test_storage_s3_queue/test.py | 4 ++ 6 files changed, 57 insertions(+), 33 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 0e9a121339f..da520d9155a 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,6 +1,7 @@ #include #include "Common/Exception.h" #include "Common/ZooKeeper/Types.h" +#include "Common/scope_guard_safe.h" #include "Interpreters/Context_fwd.h" #include "Storages/S3Queue/S3QueueSettings.h" #include "config.h" @@ -35,14 +36,12 @@ namespace return std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); } - size_t generateRescheduleInterval() + size_t generateRescheduleInterval(size_t min, size_t max) { /// Use more or less random interval for unordered mode cleanup task. /// So that distributed processing cleanup tasks would not schedule cleanup at the same time. - /// TODO: make lower and upper boundary configurable by settings pcg64 rng(randomSeed()); - //return 5000 + rng() % 30000; - return rng() % 100; + return min + rng() % (max - min + 1); } } @@ -55,19 +54,19 @@ S3QueueFilesMetadata::S3QueueFilesMetadata( , max_set_size(settings_.s3queue_tracked_files_limit.value) , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) , max_loading_retries(settings_.s3queue_loading_retries.value) + , min_cleanup_interval_ms(settings_.s3queue_cleanup_interval_min_ms.value) + , max_cleanup_interval_ms(settings_.s3queue_cleanup_interval_max_ms.value) , zookeeper_processing_path(storage->getZooKeeperPath() / "processing") , zookeeper_processed_path(storage->getZooKeeperPath() / "processed") , zookeeper_failed_path(storage->getZooKeeperPath() / "failed") + , zookeeper_cleanup_lock_path(storage->getZooKeeperPath() / "cleanup_lock") , log(&Poco::Logger::get("S3QueueFilesMetadata")) { if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) { task = context->getSchedulePool().createTask("S3QueueCleanupFunc", [this] { cleanupThreadFunc(); }); task->activate(); - - auto schedule_ms = generateRescheduleInterval(); - LOG_TEST(log, "Scheduling a cleanup task in {} ms", schedule_ms); - task->scheduleAfter(schedule_ms); + task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); } } @@ -343,7 +342,7 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc { /// File is no longer retriable. /// Make a failed/node_name node and remove failed/node_name.retriable node. - /// TODO always add version for processing node. + /// TODO: always add version for processing node. Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); @@ -405,7 +404,7 @@ void S3QueueFilesMetadata::cleanupThreadFunc() if (shutdown) return; - task->scheduleAfter(generateRescheduleInterval()); + task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); } void S3QueueFilesMetadata::cleanupThreadFuncImpl() @@ -430,6 +429,33 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() return; } + /// Create a lock so that with distributed processing + /// multiple nodes do not execute cleanup in parallel. + Coordination::Error code = zk_client->tryCreate(zookeeper_cleanup_lock_path, + toString(getCurrentTime()), + zkutil::CreateMode::Ephemeral); + if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_TEST(log, "Cleanup is already being executed by another node"); + return; + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception::fromPath(code, zookeeper_cleanup_lock_path); + } + + SCOPE_EXIT_SAFE({ + try + { + zk_client->remove(zookeeper_cleanup_lock_path); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); + } + }); + struct Node { std::string name; @@ -467,8 +493,6 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() } } - /// TODO add a zookeeper lock for cleanup - auto get_nodes_str = [&]() { WriteBufferFromOwnString wb; @@ -487,11 +511,11 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Removing node at path {} ({}) because max files limit is reached", node.metadata.file_path, path.string()); - auto code = zk_client->tryRemove(path); + code = zk_client->tryRemove(path); if (code == Coordination::Error::ZOK) --nodes_to_remove; else - LOG_ERROR(log, "Failed to remove a node `{}`", path.string()); + LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", path.string(), code); } else if (check_nodes_ttl) { @@ -502,9 +526,9 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Removing node at path {} ({}) because file is reached", node.metadata.file_path, path.string()); - auto code = zk_client->tryRemove(path); + code = zk_client->tryRemove(path); if (code != Coordination::Error::ZOK) - LOG_ERROR(log, "Failed to remove a node `{}`", path.string()); + LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", path.string(), code); } else if (!nodes_to_remove) { diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index b8e172bcd88..d794adc03c5 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -36,10 +36,13 @@ private: const UInt64 max_set_size; const UInt64 max_set_age_sec; const UInt64 max_loading_retries; + const size_t min_cleanup_interval_ms; + const size_t max_cleanup_interval_ms; const fs::path zookeeper_processing_path; const fs::path zookeeper_processed_path; const fs::path zookeeper_failed_path; + const fs::path zookeeper_cleanup_lock_path; mutable std::mutex mutex; Poco::Logger * log; diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 75defc4a57f..f2c9cd8ac19 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -19,17 +19,19 @@ class ASTStorage; 0) \ M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete or keep file in S3 after successful processing", 0) \ M(String, keeper_path, "", "Zookeeper node path", 0) \ - M(UInt64, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ - M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ - M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt64, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ - M(UInt64, s3queue_tracked_files_limit, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ - M(UInt64, \ + M(UInt32, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ + M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ + M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ + M(UInt32, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ + M(UInt32, s3queue_tracked_files_limit, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ + M(UInt32, \ s3queue_tracked_file_ttl_sec, \ 0, \ "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", \ 0) \ - M(UInt64, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) + M(UInt32, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) \ + M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "Polling backoff min for cleanup for `unordered` mode", 0) \ + M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "Polling backoff max for cleanup for `unordered` mode", 0) \ #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index bbaf5ae5311..3e662946b24 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -334,13 +334,7 @@ StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(Cont zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const { - std::lock_guard lock{zk_mutex}; - if (!zk_client || zk_client->expired()) - { - zk_client = getContext()->getZooKeeper(); - zk_client->sync(zk_path); - } - return zk_client; + return getContext()->getZooKeeper(); } void StorageS3Queue::createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 07f52c434de..fde3c699142 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -67,9 +67,6 @@ private: const std::optional format_settings; NamesAndTypesList virtual_columns; - mutable zkutil::ZooKeeperPtr zk_client; - mutable std::mutex zk_mutex; - BackgroundSchedulePool::TaskHolder task; std::atomic stream_cancelled{false}; UInt64 reschedule_processing_interval_ms; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index d26afcc3aa3..ffcbdebde16 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -771,6 +771,8 @@ def test_max_set_age(started_cluster): additional_settings={ "keeper_path": keeper_path, "s3queue_tracked_file_ttl_sec": max_age, + "s3queue_cleanup_interval_min_ms": 0, + "s3queue_cleanup_interval_max_ms": 0, }, ) create_mv(node, table_name, dst_table_name) @@ -836,6 +838,8 @@ def test_max_set_size(started_cluster): additional_settings={ "keeper_path": keeper_path, "s3queue_tracked_files_limit": 9, + "s3queue_cleanup_interval_min_ms": 0, + "s3queue_cleanup_interval_max_ms": 0, }, ) total_values = generate_random_files( From 3b54b6be88b83b6f1903ea375579c1e89989d1ad Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Sep 2023 19:23:46 +0200 Subject: [PATCH 011/120] Add system tables --- .../table-engines/integrations/s3queue.md | 32 +++++++ src/Common/ProfileEvents.cpp | 13 ++- src/Common/ProfileEvents.h | 1 + src/Common/SystemLogBase.cpp | 1 + src/Common/SystemLogBase.h | 1 + src/Interpreters/Context.cpp | 9 ++ src/Interpreters/Context.h | 2 + src/Interpreters/S3QueueLog.cpp | 43 ++++++++++ src/Interpreters/S3QueueLog.h | 38 +++++++++ src/Interpreters/SystemLog.cpp | 2 + src/Interpreters/SystemLog.h | 2 + src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 65 +++++++++++++- src/Storages/S3Queue/S3QueueFilesMetadata.h | 28 ++++++- src/Storages/S3Queue/S3QueueSettings.h | 17 ++-- src/Storages/S3Queue/S3QueueSource.cpp | 68 +++++++++++++-- src/Storages/S3Queue/S3QueueSource.h | 20 ++++- src/Storages/S3Queue/StorageS3Queue.cpp | 21 ++++- src/Storages/S3Queue/StorageS3Queue.h | 2 + src/Storages/System/StorageSystemS3Queue.cpp | 84 +++++++++++++++++++ src/Storages/System/StorageSystemS3Queue.h | 22 +++++ src/Storages/System/attachSystemTables.cpp | 2 + .../integration/test_storage_s3_queue/test.py | 4 - 22 files changed, 443 insertions(+), 34 deletions(-) create mode 100644 src/Interpreters/S3QueueLog.cpp create mode 100644 src/Interpreters/S3QueueLog.h create mode 100644 src/Storages/System/StorageSystemS3Queue.cpp create mode 100644 src/Storages/System/StorageSystemS3Queue.h diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 50330962b86..76cc60ad4fd 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -200,6 +200,38 @@ Example: SELECT * FROM stats ORDER BY name; ``` +## Introspection {#introspection} + +For introspection use `system.s3queue` stateless table and `system.s3_queue_log` persistent table. + +In order to use `system.s3_queue_log` define its configuration in server config file: + +``` xml + + system + s3_queue_log
+
+``` + +Example: + +``` sql +:) select * from system.s3queue + +SELECT * +FROM system.s3queue + +Query id: bb41964e-c947-4112-be3a-0f01770a1e84 + +┌─database─┬─table───┬─file_name──────────────────────────────────────────┬─rows_processed─┬─status─────┬─processing_start_time─┬─processing_end_time─┬─ProfileEvents──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ default │ s3queue │ hits_compatible/athena_partitioned/hits_60.parquet │ 0 │ Processing │ 2023-09-25 19:56:51 │ ᴺᵁᴸᴸ │ {'S3QueueSetFileProcessingMicroseconds':516} │ +│ default │ s3queue │ hits_compatible/athena_partitioned/hits_54.parquet │ 1000000 │ Processing │ 2023-09-25 19:56:50 │ ᴺᵁᴸᴸ │ {'SelectedRows':1000000,'SelectedBytes':1284181126,'S3ReadMicroseconds':385274,'S3ReadRequestsCount':3,'S3ReadRequestsErrors':1,'S3GetObject':1,'ReadBufferFromS3Microseconds':405970,'ReadBufferFromS3InitMicroseconds':385790,'ReadBufferFromS3Bytes':65536,'ReadBufferFromS3PreservedSessions':1,'S3QueueSetFileProcessingMicroseconds':567,'S3QueuePullMicroseconds':2475045} │ +└──────────┴─────────┴────────────────────────────────────────────────────┴────────────────┴────────────┴───────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + + +SELECT * FROM system.s3_queue_log; +``` + ## Virtual columns {#virtual-columns} - `_path` — Path to the file. diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index fd7b47b4f87..dc6a3108971 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -529,6 +529,12 @@ The server successfully detected this situation and will download merged part fr M(OverflowThrow, "Number of times, data processing was cancelled by query complexity limitation with setting '*_overflow_mode' = 'throw' and exception was thrown.") \ M(OverflowAny, "Number of times approximate GROUP BY was in effect: when aggregation was performed only on top of first 'max_rows_to_group_by' unique keys and other keys were ignored due to 'group_by_overflow_mode' = 'any'.") \ \ + M(S3QueueSetFileProcessingMicroseconds, "Time spent to set file as processing")\ + M(S3QueueSetFileProcessedMicroseconds, "Time spent to set file as processed")\ + M(S3QueueSetFileFailedMicroseconds, "Time spent to set file as failed")\ + M(S3QueueCleanupMaxSetSizeOrTTLMicroseconds, "Time spent to set file as failed")\ + M(S3QueuePullMicroseconds, "Time spent to read file data")\ + \ M(ServerStartupMilliseconds, "Time elapsed from starting server to listening to sockets in milliseconds")\ M(IOUringSQEsSubmitted, "Total number of io_uring SQEs submitted") \ M(IOUringSQEsResubmits, "Total number of io_uring SQE resubmits performed") \ @@ -588,9 +594,14 @@ Timer::Timer(Counters & counters_, Event timer_event_, Event counter_event, Reso counters.increment(counter_event); } +UInt64 Timer::get() +{ + return watch.elapsedNanoseconds() / static_cast(resolution); +} + void Timer::end() { - counters.increment(timer_event, watch.elapsedNanoseconds() / static_cast(resolution)); + counters.increment(timer_event, get()); watch.reset(); } diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index 97c5ccd3731..adf2adb9808 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -41,6 +41,7 @@ namespace ProfileEvents ~Timer() { end(); } void cancel() { watch.reset(); } void end(); + UInt64 get(); private: Counters & counters; diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index 611e14fd9b3..45e08d28f2d 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 9770629e96a..6efab699b75 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -27,6 +27,7 @@ M(ZooKeeperLogElement) \ M(ProcessorProfileLogElement) \ M(TextLogElement) \ + M(S3QueueLogElement) \ M(FilesystemCacheLogElement) \ M(FilesystemReadPrefetchesLogElement) \ M(AsynchronousInsertLogElement) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 86651f31e7c..b1663e3b2b4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3413,6 +3413,15 @@ std::shared_ptr Context::getFilesystemCacheLog() const return shared->system_logs->filesystem_cache_log; } +std::shared_ptr Context::getS3QueueLog() const +{ + auto lock = getLock(); + if (!shared->system_logs) + return {}; + + return shared->system_logs->s3_queue_log; +} + std::shared_ptr Context::getFilesystemReadPrefetchesLog() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 24441ff9ab8..c705067806a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -103,6 +103,7 @@ class TransactionsInfoLog; class ProcessorsProfileLog; class FilesystemCacheLog; class FilesystemReadPrefetchesLog; +class S3QueueLog; class AsynchronousInsertLog; class BackupLog; class IAsynchronousReader; @@ -1026,6 +1027,7 @@ public: std::shared_ptr getTransactionsInfoLog() const; std::shared_ptr getProcessorsProfileLog() const; std::shared_ptr getFilesystemCacheLog() const; + std::shared_ptr getS3QueueLog() const; std::shared_ptr getFilesystemReadPrefetchesLog() const; std::shared_ptr getAsynchronousInsertLog() const; std::shared_ptr getBackupLog() const; diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp new file mode 100644 index 00000000000..963a4789d35 --- /dev/null +++ b/src/Interpreters/S3QueueLog.cpp @@ -0,0 +1,43 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList S3QueueLogElement::getNamesAndTypes() +{ + auto status_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"Processed", static_cast(S3QueueLogElement::S3QueueStatus::Processed)}, + {"Failed", static_cast(S3QueueLogElement::S3QueueStatus::Failed)}, + }); + return { + {"event_date", std::make_shared()}, + {"event_time", std::make_shared()}, + {"table_uuid", std::make_shared()}, + {"file_name", std::make_shared()}, + {"rows_processed", std::make_shared()}, + {"status", status_datatype}, + }; +} + +void S3QueueLogElement::appendToBlock(MutableColumns & columns) const +{ + size_t i = 0; + columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType()); + columns[i++]->insert(event_time); + columns[i++]->insert(table_uuid); + columns[i++]->insert(file_name); + columns[i++]->insert(rows_processed); + columns[i++]->insert(magic_enum::enum_name(status)); +} + +} diff --git a/src/Interpreters/S3QueueLog.h b/src/Interpreters/S3QueueLog.h new file mode 100644 index 00000000000..3c99221026c --- /dev/null +++ b/src/Interpreters/S3QueueLog.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct S3QueueLogElement +{ + time_t event_time{}; + std::string table_uuid; + std::string file_name; + size_t rows_processed = 0; + + enum class S3QueueStatus + { + Processed, + Failed, + }; + S3QueueStatus status; + + static std::string name() { return "S3QueueLog"; } + + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases() { return {}; } + + void appendToBlock(MutableColumns & columns) const; + static const char * getCustomColumnList() { return nullptr; } +}; + +class S3QueueLog : public SystemLog +{ + using SystemLog::SystemLog; +}; + +} diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 07ef6c33d29..d2dd4eabaec 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -289,6 +290,7 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf processors_profile_log = createSystemLog(global_context, "system", "processors_profile_log", config, "processors_profile_log"); asynchronous_insert_log = createSystemLog(global_context, "system", "asynchronous_insert_log", config, "asynchronous_insert_log"); backup_log = createSystemLog(global_context, "system", "backup_log", config, "backup_log"); + s3_queue_log = createSystemLog(global_context, "system", "s3queue_log", config, "s3queue_log"); if (query_log) logs.emplace_back(query_log.get()); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index ec04e1f4162..932afec3bad 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -50,6 +50,7 @@ class FilesystemCacheLog; class FilesystemReadPrefetchesLog; class AsynchronousInsertLog; class BackupLog; +class S3QueueLog; /// System logs should be destroyed in destructor of the last Context and before tables, /// because SystemLog destruction makes insert query while flushing data into underlying tables @@ -70,6 +71,7 @@ struct SystemLogs std::shared_ptr metric_log; /// Used to log all metrics. std::shared_ptr filesystem_cache_log; std::shared_ptr filesystem_read_prefetches_log; + std::shared_ptr s3_queue_log; /// Metrics from system.asynchronous_metrics. std::shared_ptr asynchronous_metric_log; /// OpenTelemetry trace spans. diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index da520d9155a..837d270f8fd 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -21,12 +21,21 @@ #include #include +namespace ProfileEvents +{ + extern const Event S3QueueSetFileProcessingMicroseconds; + extern const Event S3QueueSetFileProcessedMicroseconds; + extern const Event S3QueueSetFileFailedMicroseconds; + extern const Event S3QueueCleanupMaxSetSizeOrTTLMicroseconds; +}; + namespace DB { namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int FILE_DOESNT_EXIST; } namespace @@ -129,19 +138,48 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } +std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) +{ + std::lock_guard lock(file_statuses_mutex); + return file_statuses.at(path); +} + +S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::getFileStateses() const +{ + std::lock_guard lock(file_statuses_mutex); + return file_statuses; +} + bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); + + bool result; switch (mode) { case S3QueueMode::ORDERED: { - return trySetFileAsProcessingForOrderedMode(path); + result = trySetFileAsProcessingForOrderedMode(path); + break; } case S3QueueMode::UNORDERED: { - return trySetFileAsProcessingForUnorderedMode(path); + result = trySetFileAsProcessingForUnorderedMode(path); + break; } } + if (result) + { + std::lock_guard lock(file_statuses_mutex); + auto it = file_statuses.emplace(path, std::make_shared()).first; + auto & file_status = it->second; + file_status->state = FileStatus::State::Processing; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); + timer.cancel(); + if (!file_status->processing_start_time) + file_status->processing_start_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + } + return result; } bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) @@ -225,6 +263,16 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin void S3QueueFilesMetadata::setFileProcessed(const String & path) { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessedMicroseconds); + SCOPE_EXIT({ + std::lock_guard lock(file_statuses_mutex); + auto & file_status = file_statuses.at(path); + file_status->state = FileStatus::State::Processed; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessedMicroseconds, timer.get()); + timer.cancel(); + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + }); + switch (mode) { case S3QueueMode::ORDERED: @@ -301,6 +349,17 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exception_message) { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileFailedMicroseconds); + + SCOPE_EXIT_SAFE({ + std::lock_guard lock(file_statuses_mutex); + auto & file_status = file_statuses.at(path); + file_status->state = FileStatus::State::Failed; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); + timer.cancel(); + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + }); + const auto node_name = getNodeName(path); auto node_metadata = createNodeMetadata(path, exception_message); const auto zk_client = storage->getZooKeeper(); @@ -409,6 +468,8 @@ void S3QueueFilesMetadata::cleanupThreadFunc() void S3QueueFilesMetadata::cleanupThreadFuncImpl() { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueCleanupMaxSetSizeOrTTLMicroseconds); + chassert(max_set_size || max_set_age_sec); const bool check_nodes_limit = max_set_size > 0; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index d794adc03c5..079020514c6 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -28,8 +28,32 @@ public: void setFileFailed(const std::string & path, const std::string & exception_message); + using OnProgress = std::function; + void deactivateCleanupTask(); + struct FileStatus + { + size_t processed_rows = 0; + enum class State + { + Processing, + Processed, + Failed, + None + }; + State state = State::None; + ProfileEvents::Counters profile_counters; + + time_t processing_start_time = 0; + time_t processing_end_time = 0; + }; + using FileStatuses = std::unordered_map>; + + std::shared_ptr getFileStatus(const std::string & path); + + FileStatuses getFileStateses() const; + private: const StorageS3Queue * storage; const S3QueueMode mode; @@ -44,12 +68,14 @@ private: const fs::path zookeeper_failed_path; const fs::path zookeeper_cleanup_lock_path; - mutable std::mutex mutex; Poco::Logger * log; std::atomic_bool shutdown = false; BackgroundSchedulePool::TaskHolder task; + FileStatuses file_statuses; + mutable std::mutex file_statuses_mutex; + bool trySetFileAsProcessingForOrderedMode(const std::string & path); bool trySetFileAsProcessingForUnorderedMode(const std::string & path); diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index f2c9cd8ac19..6dbffecbadf 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -20,18 +20,15 @@ class ASTStorage; M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete or keep file in S3 after successful processing", 0) \ M(String, keeper_path, "", "Zookeeper node path", 0) \ M(UInt32, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ + M(UInt32, s3queue_tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ + M(UInt32, s3queue_processing_threads_num, 1, "Number of processing threads", 0) \ + M(UInt32, s3queue_enable_logging_to_s3queue_log, 0, "Enable logging to system table system.s3queue_log", 0) \ M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt32, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ - M(UInt32, s3queue_tracked_files_limit, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ - M(UInt32, \ - s3queue_tracked_file_ttl_sec, \ - 0, \ - "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", \ - 0) \ - M(UInt32, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) \ - M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "Polling backoff min for cleanup for `unordered` mode", 0) \ - M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "Polling backoff max for cleanup for `unordered` mode", 0) \ + M(UInt32, s3queue_polling_backoff_ms, 200, "Polling backoff", 0) \ + M(UInt32, s3queue_tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ + M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "For unordered mode. Polling backoff min for cleanup", 0) \ + M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "For unordered mode. Polling backoff max for cleanup", 0) \ #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 3a834dae8d0..eec5d34027a 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -11,14 +11,13 @@ namespace CurrentMetrics { -extern const Metric StorageS3Threads; -extern const Metric StorageS3ThreadsActive; + extern const Metric StorageS3Threads; + extern const Metric StorageS3ThreadsActive; } namespace ProfileEvents { -extern const Event S3DeleteObjects; -extern const Event S3ListObjects; + extern const Event S3QueuePullMicroseconds; } namespace DB @@ -50,6 +49,11 @@ StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::FileIterator::next() } } +size_t StorageS3QueueSource::FileIterator::estimatedKeysCount() +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method estimateKeysCount is not implemented"); +} + StorageS3QueueSource::StorageS3QueueSource( String name_, const Block & header_, @@ -58,7 +62,10 @@ StorageS3QueueSource::StorageS3QueueSource( const S3QueueAction & action_, RemoveFileFunc remove_file_func_, const NamesAndTypesList & requested_virtual_columns_, - ContextPtr context_) + ContextPtr context_, + const std::atomic & shutdown_called_, + std::shared_ptr s3_queue_log_, + const StorageID & storage_id_) : ISource(header_) , WithContext(context_) , name(std::move(name_)) @@ -66,12 +73,18 @@ StorageS3QueueSource::StorageS3QueueSource( , files_metadata(files_metadata_) , internal_source(std::move(internal_source_)) , requested_virtual_columns(requested_virtual_columns_) + , shutdown_called(shutdown_called_) + , s3_queue_log(s3_queue_log_) + , storage_id(storage_id_) , remove_file_func(remove_file_func_) , log(&Poco::Logger::get("StorageS3QueueSource")) { reader = std::move(internal_source->reader); if (reader) + { reader_future = std::move(internal_source->reader_future); + file_status = files_metadata->getFileStatus(reader.getFile()); + } } StorageS3QueueSource::~StorageS3QueueSource() @@ -88,19 +101,31 @@ Chunk StorageS3QueueSource::generate() { while (true) { - if (isCancelled() || !reader) + if (!reader) + break; + + if (isCancelled()) { - if (reader) - reader->cancel(); + reader->cancel(); break; } + auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); + SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); + try { + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueuePullMicroseconds); + Chunk chunk; if (reader->pull(chunk)) { LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getPath()); + + file_status->processed_rows += chunk.getNumRows(); + // file_status->profile_counters.increment(ProfileEvents::S3QueuePullMicroseconds, timer.get()); + processed_rows_from_file += chunk.getNumRows(); + VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath()); return chunk; } @@ -109,18 +134,30 @@ Chunk StorageS3QueueSource::generate() { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); files_metadata->setFileFailed(reader.getFile(), e.message()); + appendLogElement(reader.getFile(), processed_rows_from_file, false); throw; } files_metadata->setFileProcessed(reader.getFile()); applyActionAfterProcessing(reader.getFile()); + appendLogElement(reader.getFile(), processed_rows_from_file, true); + processed_rows_from_file = 0; + + if (shutdown_called) + { + LOG_INFO(log, "Shutdown was called, stopping sync"); + break; + } + chassert(reader_future.valid()); reader = reader_future.get(); if (!reader) break; + file_status = files_metadata->getFileStatus(reader.getFile()); + /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. internal_source->create_reader_pool.wait(); @@ -145,6 +182,21 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & path) } } +void StorageS3QueueSource::appendLogElement(const std::string & file_name, size_t processed_rows, bool processed) +{ + if (!s3_queue_log) + return; + + S3QueueLogElement elem + { + .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), + .file_name = file_name, + .rows_processed = processed_rows, + .status = processed ? S3QueueLogElement::S3QueueStatus::Processed : S3QueueLogElement::S3QueueStatus::Failed, + }; + s3_queue_log->add(std::move(elem)); +} + } #endif diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 1ec762d6477..bca10f790c8 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -2,10 +2,11 @@ #include "config.h" #if USE_AWS_S3 +#include #include #include #include -#include +#include namespace Poco { class Logger; } @@ -32,14 +33,14 @@ public: KeyWithInfo next() override; + size_t estimatedKeysCount() override; + private: const std::shared_ptr metadata; const std::unique_ptr glob_iterator; std::mutex mutex; }; - static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); - StorageS3QueueSource( String name_, const Block & header_, @@ -48,10 +49,15 @@ public: const S3QueueAction & action_, RemoveFileFunc remove_file_func_, const NamesAndTypesList & requested_virtual_columns_, - ContextPtr context_); + ContextPtr context_, + const std::atomic & shutdown_called_, + std::shared_ptr s3_queue_log_, + const StorageID & storage_id_); ~StorageS3QueueSource() override; + static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); + String getName() const override; Chunk generate() override; @@ -62,6 +68,9 @@ private: const std::shared_ptr files_metadata; const std::shared_ptr internal_source; const NamesAndTypesList requested_virtual_columns; + const std::atomic & shutdown_called; + const std::shared_ptr s3_queue_log; + const StorageID storage_id; RemoveFileFunc remove_file_func; Poco::Logger * log; @@ -69,8 +78,11 @@ private: using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; std::future reader_future; + size_t processed_rows_from_file = 0; + std::shared_ptr file_status; void applyActionAfterProcessing(const String & path); + void appendLogElement(const std::string & file_name, size_t processed_rows, bool processed); }; } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 3e662946b24..f72af72123e 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -163,7 +163,7 @@ Pipe StorageS3Queue::read( ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - size_t /* num_streams */) + size_t num_streams) { if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) { @@ -178,7 +178,8 @@ Pipe StorageS3Queue::read( } Pipes pipes; - pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); + for (size_t i = 0; i < num_streams; ++i) + pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); return Pipe::unitePipes(std::move(pipes)); } @@ -219,9 +220,11 @@ std::shared_ptr StorageS3Queue::createSource( LOG_TRACE(log, "Object with path {} was removed from S3", path); } }; + auto s3_queue_log = s3queue_settings->s3queue_enable_logging_to_s3queue_log ? local_context->getS3QueueLog() : nullptr; return std::make_shared( getName(), read_from_format_info.source_header, std::move(internal_source), - files_metadata, after_processing, file_deleter, read_from_format_info.requested_virtual_columns, local_context); + files_metadata, after_processing, file_deleter, read_from_format_info.requested_virtual_columns, + local_context, shutdown_called, s3_queue_log, getStorageID()); } bool StorageS3Queue::hasDependencies(const StorageID & table_id) @@ -280,6 +283,10 @@ void StorageS3Queue::threadFunc() } } } + else + { + LOG_TEST(log, "No attached dependencies"); + } } catch (...) { @@ -317,7 +324,13 @@ void StorageS3Queue::streamToViews() // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); auto block_io = interpreter.execute(); - auto pipe = Pipe(createSource(block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context)); + Pipes pipes; + for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) + { + auto source = createSource(block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + pipes.emplace_back(std::move(source)); + } + auto pipe = Pipe::unitePipes(std::move(pipes)); std::atomic_size_t rows = 0; block_io.pipeline.complete(std::move(pipe)); diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index fde3c699142..ff6871ea70f 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -54,6 +54,8 @@ public: zkutil::ZooKeeperPtr getZooKeeper() const; + S3QueueFilesMetadata::FileStatuses getFileStatuses() const { return files_metadata->getFileStateses(); } + private: using FileIterator = StorageS3QueueSource::FileIterator; diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp new file mode 100644 index 00000000000..6d409e29ee4 --- /dev/null +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -0,0 +1,84 @@ +#include "StorageSystemS3Queue.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList StorageSystemS3Queue::getNamesAndTypes() +{ + return { + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"file_name", std::make_shared()}, + {"rows_processed", std::make_shared()}, + {"status", std::make_shared()}, + {"processing_start_time", std::make_shared(std::make_shared())}, + {"processing_end_time", std::make_shared(std::make_shared())}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + }; +} + +StorageSystemS3Queue::StorageSystemS3Queue(const StorageID & table_id_) + : IStorageSystemOneBlock(table_id_) +{ +} + +void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +{ + const auto access = context->getAccess(); + const bool show_tables_granted = access->isGranted(AccessType::SHOW_TABLES); + + if (show_tables_granted) + { + auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & db : databases) + { + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + StoragePtr storage = iterator->table(); + if (auto * s3queue_table = dynamic_cast(storage.get())) + { + const auto & table_id = s3queue_table->getStorageID(); + auto file_statuses = s3queue_table->getFileStatuses(); + for (const auto & [file_name, file_status] : file_statuses) + { + size_t i = 0; + res_columns[i++]->insert(table_id.database_name); + res_columns[i++]->insert(table_id.table_name); + res_columns[i++]->insert(file_name); + res_columns[i++]->insert(file_status->processed_rows); + res_columns[i++]->insert(magic_enum::enum_name(file_status->state)); + + if (file_status->processing_start_time) + res_columns[i++]->insert(file_status->processing_start_time); + else + res_columns[i++]->insertDefault(); + if (file_status->processing_end_time) + res_columns[i++]->insert(file_status->processing_end_time); + else + res_columns[i++]->insertDefault(); + + ProfileEvents::dumpToMapColumn(file_status->profile_counters.getPartiallyAtomicSnapshot(), res_columns[i++].get(), true); + } + } + } + } + + } +} + +} diff --git a/src/Storages/System/StorageSystemS3Queue.h b/src/Storages/System/StorageSystemS3Queue.h new file mode 100644 index 00000000000..eb6b1533dce --- /dev/null +++ b/src/Storages/System/StorageSystemS3Queue.h @@ -0,0 +1,22 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class StorageSystemS3Queue final : public IStorageSystemOneBlock +{ +public: + explicit StorageSystemS3Queue(const StorageID & table_id_); + + std::string getName() const override { return "SystemS3Queue"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index e4e19ce2e06..ffa225fb929 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -84,6 +84,7 @@ #include #include #include +#include #if USE_RDKAFKA #include @@ -196,6 +197,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "named_collections"); attach(context, system_database, "user_processes"); attach(context, system_database, "jemalloc_bins"); + attach(context, system_database, "s3queue"); if (has_zookeeper) { diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index ffcbdebde16..4c3e9b1753e 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -626,7 +626,6 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 300 - poll_size = 30 for i in range(3): table = f"{table_name}_{i + 1}" @@ -638,7 +637,6 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): mode, files_path, additional_settings={ - "s3queue_polling_size": poll_size, "keeper_path": keeper_path, }, ) @@ -700,7 +698,6 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 300 - poll_size = 2 for instance in [node, node_2]: create_table( @@ -710,7 +707,6 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): mode, files_path, additional_settings={ - "s3queue_polling_size": poll_size, "keeper_path": keeper_path, }, ) From 251b0165ca33672732e8e3eeec6025c55a7a3a0b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Sep 2023 22:55:41 +0200 Subject: [PATCH 012/120] Better --- .../table-engines/integrations/s3queue.md | 83 ++++++++++++------- src/Interpreters/S3QueueLog.cpp | 15 ++++ src/Interpreters/S3QueueLog.h | 4 + src/Interpreters/SystemLog.cpp | 2 + src/Storages/S3Queue/S3QueueSettings.h | 2 +- src/Storages/S3Queue/S3QueueSource.cpp | 12 ++- src/Storages/S3Queue/S3QueueSource.h | 2 +- src/Storages/S3Queue/StorageS3Queue.cpp | 50 +++++------ src/Storages/S3Queue/StorageS3Queue.h | 2 +- 9 files changed, 106 insertions(+), 66 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 76cc60ad4fd..6ceea8d7dc9 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -200,38 +200,6 @@ Example: SELECT * FROM stats ORDER BY name; ``` -## Introspection {#introspection} - -For introspection use `system.s3queue` stateless table and `system.s3_queue_log` persistent table. - -In order to use `system.s3_queue_log` define its configuration in server config file: - -``` xml - - system - s3_queue_log
-
-``` - -Example: - -``` sql -:) select * from system.s3queue - -SELECT * -FROM system.s3queue - -Query id: bb41964e-c947-4112-be3a-0f01770a1e84 - -┌─database─┬─table───┬─file_name──────────────────────────────────────────┬─rows_processed─┬─status─────┬─processing_start_time─┬─processing_end_time─┬─ProfileEvents──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ default │ s3queue │ hits_compatible/athena_partitioned/hits_60.parquet │ 0 │ Processing │ 2023-09-25 19:56:51 │ ᴺᵁᴸᴸ │ {'S3QueueSetFileProcessingMicroseconds':516} │ -│ default │ s3queue │ hits_compatible/athena_partitioned/hits_54.parquet │ 1000000 │ Processing │ 2023-09-25 19:56:50 │ ᴺᵁᴸᴸ │ {'SelectedRows':1000000,'SelectedBytes':1284181126,'S3ReadMicroseconds':385274,'S3ReadRequestsCount':3,'S3ReadRequestsErrors':1,'S3GetObject':1,'ReadBufferFromS3Microseconds':405970,'ReadBufferFromS3InitMicroseconds':385790,'ReadBufferFromS3Bytes':65536,'ReadBufferFromS3PreservedSessions':1,'S3QueueSetFileProcessingMicroseconds':567,'S3QueuePullMicroseconds':2475045} │ -└──────────┴─────────┴────────────────────────────────────────────────────┴────────────────┴────────────┴───────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - - -SELECT * FROM system.s3_queue_log; -``` - ## Virtual columns {#virtual-columns} - `_path` — Path to the file. @@ -255,3 +223,54 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table :::note If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: +## Introspection {#introspection} + +For introspection use `system.s3queue` stateless table and `system.s3queue_log` persistent table. + +The `s3queue` system table has the following structure: + +``` sql +┌─statement──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ CREATE TABLE system.s3queue +( + `database` String, + `table` String, + `file_name` String, + `rows_processed` UInt64, + `status` String, + `processing_start_time` Nullable(DateTime), + `processing_end_time` Nullable(DateTime), + `ProfileEvents` Map(String, UInt64) +) +ENGINE = SystemS3Queue +COMMENT 'SYSTEM TABLE is built on the fly.' │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +In order to use `system.s3queue_log` define its configuration in server config file: + +``` xml + + system + s3queue_log
+
+``` + +Example: + +``` sql +:) select * from system.s3queue + +SELECT * +FROM system.s3queue + +Query id: bb41964e-c947-4112-be3a-0f01770a1e84 + +┌─database─┬─table───┬─file_name──────────────────────────────────────────┬─rows_processed─┬─status─────┬─processing_start_time─┬─processing_end_time─┬─ProfileEvents──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ default │ s3queue │ hits_compatible/athena_partitioned/hits_60.parquet │ 0 │ Processing │ 2023-09-25 19:56:51 │ ᴺᵁᴸᴸ │ {'S3QueueSetFileProcessingMicroseconds':516} │ +│ default │ s3queue │ hits_compatible/athena_partitioned/hits_54.parquet │ 1000000 │ Processing │ 2023-09-25 19:56:50 │ ᴺᵁᴸᴸ │ {'SelectedRows':1000000,'SelectedBytes':1284181126,'S3ReadMicroseconds':385274,'S3ReadRequestsCount':3,'S3ReadRequestsErrors':1,'S3GetObject':1,'ReadBufferFromS3Microseconds':405970,'ReadBufferFromS3InitMicroseconds':385790,'ReadBufferFromS3Bytes':65536,'ReadBufferFromS3PreservedSessions':1,'S3QueueSetFileProcessingMicroseconds':567,'S3QueuePullMicroseconds':2475045} │ +└──────────┴─────────┴────────────────────────────────────────────────────┴────────────────┴────────────┴───────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + + +SELECT * FROM system.s3_queue_log; +``` diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp index 963a4789d35..caccdabf503 100644 --- a/src/Interpreters/S3QueueLog.cpp +++ b/src/Interpreters/S3QueueLog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -26,6 +27,9 @@ NamesAndTypesList S3QueueLogElement::getNamesAndTypes() {"file_name", std::make_shared()}, {"rows_processed", std::make_shared()}, {"status", status_datatype}, + {"processing_start_time", std::make_shared(std::make_shared())}, + {"processing_end_time", std::make_shared(std::make_shared())}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, }; } @@ -38,6 +42,17 @@ void S3QueueLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(file_name); columns[i++]->insert(rows_processed); columns[i++]->insert(magic_enum::enum_name(status)); + + if (processing_start_time) + columns[i++]->insert(processing_start_time); + else + columns[i++]->insertDefault(); + if (processing_end_time) + columns[i++]->insert(processing_end_time); + else + columns[i++]->insertDefault(); + + ProfileEvents::dumpToMapColumn(counters_snapshot, columns[i++].get(), true); } } diff --git a/src/Interpreters/S3QueueLog.h b/src/Interpreters/S3QueueLog.h index 3c99221026c..b28ef1b82fc 100644 --- a/src/Interpreters/S3QueueLog.h +++ b/src/Interpreters/S3QueueLog.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -20,6 +21,9 @@ struct S3QueueLogElement Failed, }; S3QueueStatus status; + ProfileEvents::Counters::Snapshot counters_snapshot; + time_t processing_start_time; + time_t processing_end_time; static std::string name() { return "S3QueueLog"; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index d2dd4eabaec..9e4129219cb 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -331,6 +331,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf logs.emplace_back(asynchronous_insert_log.get()); if (backup_log) logs.emplace_back(backup_log.get()); + if (s3_queue_log) + logs.emplace_back(s3_queue_log.get()); try { diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 6dbffecbadf..6d7ab4187a5 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -25,7 +25,7 @@ class ASTStorage; M(UInt32, s3queue_enable_logging_to_s3queue_log, 0, "Enable logging to system table system.s3queue_log", 0) \ M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt32, s3queue_polling_backoff_ms, 200, "Polling backoff", 0) \ + M(UInt32, s3queue_polling_backoff_ms, 1000, "Polling backoff", 0) \ M(UInt32, s3queue_tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "For unordered mode. Polling backoff min for cleanup", 0) \ M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "For unordered mode. Polling backoff max for cleanup", 0) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index eec5d34027a..f1bce2f2dcd 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -134,14 +134,15 @@ Chunk StorageS3QueueSource::generate() { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); files_metadata->setFileFailed(reader.getFile(), e.message()); - appendLogElement(reader.getFile(), processed_rows_from_file, false); + appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); throw; } files_metadata->setFileProcessed(reader.getFile()); applyActionAfterProcessing(reader.getFile()); - appendLogElement(reader.getFile(), processed_rows_from_file, true); + appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, true); + file_status.reset(); processed_rows_from_file = 0; if (shutdown_called) @@ -182,7 +183,7 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & path) } } -void StorageS3QueueSource::appendLogElement(const std::string & file_name, size_t processed_rows, bool processed) +void StorageS3QueueSource::appendLogElement(const std::string & filename, const S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed) { if (!s3_queue_log) return; @@ -190,9 +191,12 @@ void StorageS3QueueSource::appendLogElement(const std::string & file_name, size_ S3QueueLogElement elem { .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), - .file_name = file_name, + .file_name = filename, .rows_processed = processed_rows, .status = processed ? S3QueueLogElement::S3QueueStatus::Processed : S3QueueLogElement::S3QueueStatus::Failed, + .counters_snapshot = file_status_.profile_counters.getPartiallyAtomicSnapshot(), + .processing_start_time = file_status_.processing_start_time, + .processing_end_time = file_status_.processing_end_time, }; s3_queue_log->add(std::move(elem)); } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index bca10f790c8..ce8a64022d0 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -82,7 +82,7 @@ private: std::shared_ptr file_status; void applyActionAfterProcessing(const String & path); - void appendLogElement(const std::string & file_name, size_t processed_rows, bool processed); + void appendLogElement(const std::string & filename, const S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); }; } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index f72af72123e..776d3c75ecc 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -34,8 +34,6 @@ namespace ProfileEvents namespace DB { -static const auto MAX_THREAD_WORK_DURATION_MS = 60000; - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -254,34 +252,31 @@ bool StorageS3Queue::hasDependencies(const StorageID & table_id) void StorageS3Queue::threadFunc() { - SCOPE_EXIT({ mv_attached.store(false); }); + if (shutdown_called) + return; + try { - auto table_id = getStorageID(); - size_t dependencies_count = DatabaseCatalog::instance().getDependentViews(table_id).size(); + const size_t dependencies_count = DatabaseCatalog::instance().getDependentViews(getStorageID()).size(); if (dependencies_count) { - auto start_time = std::chrono::steady_clock::now(); - /// Reset reschedule interval. - reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; - /// Disallow parallel selects while streaming to mv. mv_attached.store(true); + SCOPE_EXIT({ mv_attached.store(false); }); - /// Keep streaming as long as there are attached views and streaming is not cancelled - while (!shutdown_called && hasDependencies(table_id)) + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + + if (streamToViews()) { - LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - - streamToViews(); - - auto now = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(now - start_time); - if (duration.count() > MAX_THREAD_WORK_DURATION_MS) - { - LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); - break; - } + /// Reset the reschedule interval. + reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; } + else + { + /// Increase the reschedule interval. + reschedule_processing_interval_ms += s3queue_settings->s3queue_polling_backoff_ms; + } + + LOG_DEBUG(log, "Stopped streaming to {} attached views", dependencies_count); } else { @@ -295,15 +290,12 @@ void StorageS3Queue::threadFunc() if (!shutdown_called) { - if (reschedule_processing_interval_ms < s3queue_settings->s3queue_polling_max_timeout_ms) - reschedule_processing_interval_ms += s3queue_settings->s3queue_polling_backoff_ms; - LOG_TRACE(log, "Reschedule S3 Queue processing thread in {} ms", reschedule_processing_interval_ms); task->scheduleAfter(reschedule_processing_interval_ms); } } -void StorageS3Queue::streamToViews() +bool StorageS3Queue::streamToViews() { auto table_id = getStorageID(); auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); @@ -324,10 +316,12 @@ void StorageS3Queue::streamToViews() // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); auto block_io = interpreter.execute(); + Pipes pipes; for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) { - auto source = createSource(block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + auto source = createSource( + block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); pipes.emplace_back(std::move(source)); } auto pipe = Pipe::unitePipes(std::move(pipes)); @@ -337,6 +331,8 @@ void StorageS3Queue::streamToViews() block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); + + return rows > 0; } StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index ff6871ea70f..8ec8bfeeb0a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -92,7 +92,7 @@ private: ContextPtr local_context); bool hasDependencies(const StorageID & table_id); - void streamToViews(); + bool streamToViews(); void threadFunc(); void createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata); From 4d78dbacfe4a62089284174b34207a37f662e3cf Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Sep 2023 23:38:16 +0200 Subject: [PATCH 013/120] Fix style check, fix build without s3 --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 1 - src/Storages/S3Queue/S3QueueSource.cpp | 1 + src/Storages/S3Queue/StorageS3Queue.cpp | 1 - src/Storages/System/StorageSystemS3Queue.cpp | 5 +++++ src/Storages/System/StorageSystemS3Queue.h | 5 +++++ 5 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 837d270f8fd..a9be348405f 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -35,7 +35,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int FILE_DOESNT_EXIST; } namespace diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index f1bce2f2dcd..e9a57cbbfd4 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -26,6 +26,7 @@ namespace DB namespace ErrorCodes { extern const int S3_ERROR; + extern const int NOT_IMPLEMENTED; } StorageS3QueueSource::FileIterator::FileIterator( diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 776d3c75ecc..0bc98006463 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,6 +1,5 @@ #include "config.h" - #if USE_AWS_S3 #include #include diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index 6d409e29ee4..6f320bbd5f1 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -1,4 +1,7 @@ #include "StorageSystemS3Queue.h" + +#if USE_AWS_S3 + #include #include #include @@ -82,3 +85,5 @@ void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr con } } + +#endif diff --git a/src/Storages/System/StorageSystemS3Queue.h b/src/Storages/System/StorageSystemS3Queue.h index eb6b1533dce..208891aef42 100644 --- a/src/Storages/System/StorageSystemS3Queue.h +++ b/src/Storages/System/StorageSystemS3Queue.h @@ -1,4 +1,7 @@ #pragma once +#include "config.h" + +#if USE_AWS_S3 #include #include @@ -20,3 +23,5 @@ protected: }; } + +#endif From 14b09d3cdcbcfbe62f9780b91eed2611b48958f1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 26 Sep 2023 16:23:24 +0200 Subject: [PATCH 014/120] Add caching --- src/Common/ProfileEvents.cpp | 1 + src/Interpreters/Cache/Metadata.cpp | 1 + src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 283 +++++++++++++----- src/Storages/S3Queue/S3QueueFilesMetadata.h | 37 ++- .../S3Queue/S3QueueMetadataFactory.cpp | 29 ++ src/Storages/S3Queue/S3QueueMetadataFactory.h | 26 ++ src/Storages/S3Queue/StorageS3Queue.cpp | 3 +- src/Storages/S3Queue/StorageS3Queue.h | 2 - src/Storages/System/StorageSystemS3Queue.cpp | 56 ++-- 9 files changed, 315 insertions(+), 123 deletions(-) create mode 100644 src/Storages/S3Queue/S3QueueMetadataFactory.cpp create mode 100644 src/Storages/S3Queue/S3QueueMetadataFactory.h diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index dc6a3108971..a8a3a732784 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -534,6 +534,7 @@ The server successfully detected this situation and will download merged part fr M(S3QueueSetFileFailedMicroseconds, "Time spent to set file as failed")\ M(S3QueueCleanupMaxSetSizeOrTTLMicroseconds, "Time spent to set file as failed")\ M(S3QueuePullMicroseconds, "Time spent to read file data")\ + M(S3QueueLockLocalFileStatusesMicroseconds, "Time spent to lock local file statuses")\ \ M(ServerStartupMilliseconds, "Time elapsed from starting server to listening to sockets in milliseconds")\ M(IOUringSQEsSubmitted, "Total number of io_uring SQEs submitted") \ diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 7a9321e4215..c251cb2dec2 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index a9be348405f..009f5e3ad75 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -27,6 +27,7 @@ namespace ProfileEvents extern const Event S3QueueSetFileProcessedMicroseconds; extern const Event S3QueueSetFileFailedMicroseconds; extern const Event S3QueueCleanupMaxSetSizeOrTTLMicroseconds; + extern const Event S3QueueLockLocalFileStatusesMicroseconds; }; namespace DB @@ -35,6 +36,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } namespace @@ -53,41 +55,54 @@ namespace } } -S3QueueFilesMetadata::S3QueueFilesMetadata( - const StorageS3Queue * storage_, - const S3QueueSettings & settings_, - ContextPtr context) - : storage(storage_) - , mode(settings_.mode) - , max_set_size(settings_.s3queue_tracked_files_limit.value) - , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) - , max_loading_retries(settings_.s3queue_loading_retries.value) - , min_cleanup_interval_ms(settings_.s3queue_cleanup_interval_min_ms.value) - , max_cleanup_interval_ms(settings_.s3queue_cleanup_interval_max_ms.value) - , zookeeper_processing_path(storage->getZooKeeperPath() / "processing") - , zookeeper_processed_path(storage->getZooKeeperPath() / "processed") - , zookeeper_failed_path(storage->getZooKeeperPath() / "failed") - , zookeeper_cleanup_lock_path(storage->getZooKeeperPath() / "cleanup_lock") - , log(&Poco::Logger::get("S3QueueFilesMetadata")) +std::unique_lock S3QueueFilesMetadata::LocalFileStatuses::lock() const { - if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) + auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueLockLocalFileStatusesMicroseconds); + return std::unique_lock(mutex); +} + +S3QueueFilesMetadata::FileStatus::State S3QueueFilesMetadata::LocalFileStatuses::state(const std::string & filename) const +{ + auto lk = lock(); + if (auto it = file_statuses.find(filename); it != file_statuses.end()) + return it->second->state; + else + return FileStatus::State::None; +} + +S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::LocalFileStatuses::getAll() const +{ + auto lk = lock(); + return file_statuses; +} + +std::shared_ptr S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) +{ + auto lk = lock(); + auto it = file_statuses.find(filename); + if (it == file_statuses.end()) { - task = context->getSchedulePool().createTask("S3QueueCleanupFunc", [this] { cleanupThreadFunc(); }); - task->activate(); - task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); + if (create) + it = file_statuses.emplace(filename, std::make_shared()).first; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "File status for {} doesn't exist", filename); } + return it->second; } -S3QueueFilesMetadata::~S3QueueFilesMetadata() +bool S3QueueFilesMetadata::LocalFileStatuses::remove(const std::string & filename, bool if_exists) { - deactivateCleanupTask(); -} - -void S3QueueFilesMetadata::deactivateCleanupTask() -{ - shutdown = true; - if (task) - task->deactivate(); + auto lk = lock(); + auto it = file_statuses.find(filename); + if (it == file_statuses.end()) + { + if (if_exists) + return false; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "File status for {} doesn't exist", filename); + } + file_statuses.erase(it); + return true; } std::string S3QueueFilesMetadata::NodeMetadata::toString() const @@ -117,6 +132,49 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::NodeMetadata::fromStrin return metadata; } +S3QueueFilesMetadata::S3QueueFilesMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_) + : mode(settings_.mode) + , max_set_size(settings_.s3queue_tracked_files_limit.value) + , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) + , max_loading_retries(settings_.s3queue_loading_retries.value) + , min_cleanup_interval_ms(settings_.s3queue_cleanup_interval_min_ms.value) + , max_cleanup_interval_ms(settings_.s3queue_cleanup_interval_max_ms.value) + , zookeeper_processing_path(zookeeper_path_ / "processing") + , zookeeper_processed_path(zookeeper_path_ / "processed") + , zookeeper_failed_path(zookeeper_path_ / "failed") + , zookeeper_cleanup_lock_path(zookeeper_path_ / "cleanup_lock") + , log(&Poco::Logger::get("S3QueueFilesMetadata")) +{ + if (mode == S3QueueMode::UNORDERED && (max_set_size || max_set_age_sec)) + { + task = Context::getGlobalContextInstance()->getSchedulePool().createTask("S3QueueCleanupFunc", [this] { cleanupThreadFunc(); }); + task->activate(); + task->scheduleAfter(generateRescheduleInterval(min_cleanup_interval_ms, max_cleanup_interval_ms)); + } +} + +S3QueueFilesMetadata::~S3QueueFilesMetadata() +{ + deactivateCleanupTask(); +} + +void S3QueueFilesMetadata::deactivateCleanupTask() +{ + shutdown = true; + if (task) + task->deactivate(); +} + +zkutil::ZooKeeperPtr S3QueueFilesMetadata::getZooKeeper() const +{ + return Context::getGlobalContextInstance()->getZooKeeper(); +} + +std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) +{ + return local_file_statuses.get(path, /* create */false); +} + std::string S3QueueFilesMetadata::getNodeName(const std::string & path) { SipHash path_hash; @@ -137,23 +195,39 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } -std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) -{ - std::lock_guard lock(file_statuses_mutex); - return file_statuses.at(path); -} - -S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::getFileStateses() const -{ - std::lock_guard lock(file_statuses_mutex); - return file_statuses; -} - bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); - bool result; + /// Check locally cached file status. + switch (local_file_statuses.state(path)) + { + case FileStatus::State::Processing: [[fallthrough]]; + case FileStatus::State::Processed: + { + /// File is already processes or processing by current server. + return false; + } + case FileStatus::State::Failed: + { + if (!max_loading_retries) + { + /// File was processes by current server and failed, + /// retries are disabled. + return false; + } + /// TODO save information if file is still retriable. + break; + } + case FileStatus::State::None: + { + /// The file was not processed by current server, + /// check metadata in zookeeper. + break; + } + } + + SetFileProcessingResult result; switch (mode) { case S3QueueMode::ORDERED: @@ -167,21 +241,42 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) break; } } - if (result) + switch (result) { - std::lock_guard lock(file_statuses_mutex); - auto it = file_statuses.emplace(path, std::make_shared()).first; - auto & file_status = it->second; - file_status->state = FileStatus::State::Processing; - file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); - timer.cancel(); - if (!file_status->processing_start_time) - file_status->processing_start_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + case SetFileProcessingResult::Success: + { + auto file_status = local_file_statuses.get(path, /* create */true); + file_status->state = FileStatus::State::Processing; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); + timer.cancel(); + if (!file_status->processing_start_time) + file_status->processing_start_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + break; + } + case SetFileProcessingResult::AlreadyProcessed: + { + /// Cache the state. + auto file_status = local_file_statuses.get(path, /* create */true); + file_status->state = FileStatus::State::Processed; + break; + } + case SetFileProcessingResult::AlreadyFailed: + { + /// Cache the state. + auto file_status = local_file_statuses.get(path, /* create */true); + file_status->state = FileStatus::State::Failed; + break; + } + case SetFileProcessingResult::ProcessingByOtherNode: + { + /// We cannot save any local state. + break; + } } - return result; + return result == SetFileProcessingResult::Success; } -bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) +S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { /// Create an ephemenral node in /processing /// if corresponding node does not exist in failed/, processed/ and processing/. @@ -189,7 +284,7 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::str const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); Coordination::Requests requests; zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processed_path / node_name); @@ -198,10 +293,26 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::str Coordination::Responses responses; auto code = zk_client->tryMulti(requests, responses); - return code == Coordination::Error::ZOK; + + if (code == Coordination::Error::ZOK) + { + return SetFileProcessingResult::Success; + } + else if (responses[0]->error == Coordination::Error::ZOK) + { + if (responses[1]->error == Coordination::Error::ZOK) + { + chassert(responses[2]->error != Coordination::Error::ZOK); + return SetFileProcessingResult::ProcessingByOtherNode; + } + else + return SetFileProcessingResult::AlreadyFailed; + } + else + return SetFileProcessingResult::AlreadyProcessed; } -bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) +S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { /// Create an ephemenral node in /processing /// if corresponding it does not exist in failed/, processing/ and satisfied max processed file check. @@ -209,7 +320,7 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); while (true) { @@ -221,9 +332,16 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin auto code = zk_client->tryMulti(requests, responses); if (code != Coordination::Error::ZOK) { - LOG_TEST(log, "Skipping file `{}`: {}", - path, responses[0]->error != Coordination::Error::ZOK ? "failed" : "processing"); - return false; + if (responses[0]->error == Coordination::Error::ZOK) + { + LOG_TEST(log, "Skipping file `{}`: already processing", path); + return SetFileProcessingResult::ProcessingByOtherNode; + } + else + { + LOG_TEST(log, "Skipping file `{}`: failed", path); + return SetFileProcessingResult::AlreadyFailed; + } } Coordination::Stat processed_node_stat; @@ -234,7 +352,7 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin auto max_processed_file_path = processed_node_metadata.file_path; if (!max_processed_file_path.empty() && path <= max_processed_file_path) - return false; + return SetFileProcessingResult::AlreadyProcessed; requests.clear(); responses.clear(); @@ -244,14 +362,17 @@ bool S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::strin code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) - return true; + return SetFileProcessingResult::Success; - if (responses[0]->error != Coordination::Error::ZOK - || responses[1]->error != Coordination::Error::ZOK) + if (responses[0]->error != Coordination::Error::ZOK) { - LOG_TEST(log, "Skipping file `{}`: {}", - path, responses[0]->error != Coordination::Error::ZOK ? "failed" : "processing"); - return false; + LOG_TEST(log, "Skipping file `{}`: failed", path); + return SetFileProcessingResult::AlreadyFailed; + } + else if (responses[1]->error != Coordination::Error::ZOK) + { + LOG_TEST(log, "Skipping file `{}`: already processing", path); + return SetFileProcessingResult::ProcessingByOtherNode; } else { @@ -264,8 +385,7 @@ void S3QueueFilesMetadata::setFileProcessed(const String & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessedMicroseconds); SCOPE_EXIT({ - std::lock_guard lock(file_statuses_mutex); - auto & file_status = file_statuses.at(path); + auto file_status = local_file_statuses.get(path, /* create */false); file_status->state = FileStatus::State::Processed; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessedMicroseconds, timer.get()); timer.cancel(); @@ -291,7 +411,7 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); @@ -316,7 +436,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) { const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); while (true) { @@ -351,8 +471,7 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileFailedMicroseconds); SCOPE_EXIT_SAFE({ - std::lock_guard lock(file_statuses_mutex); - auto & file_status = file_statuses.at(path); + auto file_status = local_file_statuses.get(path, /* create */false); file_status->state = FileStatus::State::Failed; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); timer.cancel(); @@ -361,7 +480,7 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc const auto node_name = getNodeName(path); auto node_metadata = createNodeMetadata(path, exception_message); - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); if (max_loading_retries == 0) { @@ -474,7 +593,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() const bool check_nodes_limit = max_set_size > 0; const bool check_nodes_ttl = max_set_age_sec > 0; - const auto zk_client = storage->getZooKeeper(); + const auto zk_client = getZooKeeper(); auto nodes = zk_client->getChildren(zookeeper_processed_path); if (nodes.empty()) { @@ -571,6 +690,8 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Removing node at path {} ({}) because max files limit is reached", node.metadata.file_path, path.string()); + local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); + code = zk_client->tryRemove(path); if (code == Coordination::Error::ZOK) --nodes_to_remove; @@ -586,6 +707,8 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Removing node at path {} ({}) because file is reached", node.metadata.file_path, path.string()); + local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); + code = zk_client->tryRemove(path); if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", path.string(), code); @@ -608,6 +731,16 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TRACE(log, "Node limits check finished"); } +bool S3QueueFilesMetadata::checkSettings(const S3QueueSettings & settings) const +{ + return mode == settings.mode + && max_set_size == settings.s3queue_tracked_files_limit.value + && max_set_age_sec == settings.s3queue_tracked_file_ttl_sec.value + && max_loading_retries == settings.s3queue_loading_retries.value + && min_cleanup_interval_ms == settings.s3queue_cleanup_interval_min_ms.value + && max_cleanup_interval_ms == settings.s3queue_cleanup_interval_max_ms.value; +} + } #endif diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 079020514c6..5f3a6448e49 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace fs = std::filesystem; namespace Poco { class Logger; } @@ -18,7 +19,7 @@ class StorageS3Queue; class S3QueueFilesMetadata { public: - S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_, ContextPtr context); + S3QueueFilesMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_); ~S3QueueFilesMetadata(); @@ -52,10 +53,11 @@ public: std::shared_ptr getFileStatus(const std::string & path); - FileStatuses getFileStateses() const; + FileStatuses getFileStateses() const { return local_file_statuses.getAll(); } + + bool checkSettings(const S3QueueSettings & settings) const; private: - const StorageS3Queue * storage; const S3QueueMode mode; const UInt64 max_set_size; const UInt64 max_set_age_sec; @@ -73,16 +75,22 @@ private: std::atomic_bool shutdown = false; BackgroundSchedulePool::TaskHolder task; - FileStatuses file_statuses; - mutable std::mutex file_statuses_mutex; + std::string getNodeName(const std::string & path); - bool trySetFileAsProcessingForOrderedMode(const std::string & path); - bool trySetFileAsProcessingForUnorderedMode(const std::string & path); + zkutil::ZooKeeperPtr getZooKeeper() const; void setFileProcessedForOrderedMode(const std::string & path); void setFileProcessedForUnorderedMode(const std::string & path); - std::string getNodeName(const std::string & path); + enum class SetFileProcessingResult + { + Success, + ProcessingByOtherNode, + AlreadyProcessed, + AlreadyFailed, + }; + SetFileProcessingResult trySetFileAsProcessingForOrderedMode(const std::string & path); + SetFileProcessingResult trySetFileAsProcessingForUnorderedMode(const std::string & path); struct NodeMetadata { @@ -99,6 +107,19 @@ private: void cleanupThreadFunc(); void cleanupThreadFuncImpl(); + + struct LocalFileStatuses + { + FileStatuses file_statuses; + mutable std::mutex mutex; + + FileStatuses getAll() const; + std::shared_ptr get(const std::string & filename, bool create); + bool remove(const std::string & filename, bool if_exists); + FileStatus::State state(const std::string & filename) const; + std::unique_lock lock() const; + }; + LocalFileStatuses local_file_statuses; }; } diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp new file mode 100644 index 00000000000..b1b0c4aef23 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp @@ -0,0 +1,29 @@ +#include + +namespace DB +{ + +S3QueueMetadataFactory & S3QueueMetadataFactory::instance() +{ + static S3QueueMetadataFactory ret; + return ret; +} + +S3QueueMetadataFactory::MetadataPtr +S3QueueMetadataFactory::getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings) +{ + std::lock_guard lock(mutex); + auto it = metadata_by_path.find(zookeeper_path); + if (it == metadata_by_path.end()) + { + it = metadata_by_path.emplace(zookeeper_path, std::make_shared(fs::path(zookeeper_path), settings)).first; + } + else if (!it->second->checkSettings(settings)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Metadata with the same `s3queue_zookeeper_path` " + "was already created but with different settings"); + } + return it->second; +} + +} diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.h b/src/Storages/S3Queue/S3QueueMetadataFactory.h new file mode 100644 index 00000000000..e7a473d863d --- /dev/null +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.h @@ -0,0 +1,26 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class S3QueueMetadataFactory final : private boost::noncopyable +{ +public: + using MetadataPtr = std::shared_ptr; + using MetadataByPath = std::unordered_map; + + static S3QueueMetadataFactory & instance(); + + MetadataPtr getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings); + + MetadataByPath getAll() { return metadata_by_path; } + +private: + MetadataByPath metadata_by_path; + std::mutex mutex; +}; + +} diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 0bc98006463..a15cc1bea9b 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -85,7 +86,7 @@ StorageS3Queue::StorageS3Queue( , s3queue_settings(std::move(s3queue_settings_)) , zk_path(chooseZooKeeperPath(table_id_, context_->getSettingsRef(), *s3queue_settings)) , after_processing(s3queue_settings->after_processing) - , files_metadata(std::make_shared(this, *s3queue_settings, context_)) + , files_metadata(S3QueueMetadataFactory::instance().getOrCreate(zk_path, *s3queue_settings)) , configuration{configuration_} , format_settings(format_settings_) , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 8ec8bfeeb0a..2a62078fcca 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -54,8 +54,6 @@ public: zkutil::ZooKeeperPtr getZooKeeper() const; - S3QueueFilesMetadata::FileStatuses getFileStatuses() const { return files_metadata->getFileStateses(); } - private: using FileIterator = StorageS3QueueSource::FileIterator; diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index 6f320bbd5f1..faba5bc671b 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -24,8 +25,7 @@ namespace DB NamesAndTypesList StorageSystemS3Queue::getNamesAndTypes() { return { - {"database", std::make_shared()}, - {"table", std::make_shared()}, + {"zookeeper_path", std::make_shared()}, {"file_name", std::make_shared()}, {"rows_processed", std::make_shared()}, {"status", std::make_shared()}, @@ -40,47 +40,29 @@ StorageSystemS3Queue::StorageSystemS3Queue(const StorageID & table_id_) { } -void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { - const auto access = context->getAccess(); - const bool show_tables_granted = access->isGranted(AccessType::SHOW_TABLES); - - if (show_tables_granted) + for (const auto & [zookeeper_path, metadata] : S3QueueMetadataFactory::instance().getAll()) { - auto databases = DatabaseCatalog::instance().getDatabases(); - for (const auto & db : databases) + for (const auto & [file_name, file_status] : metadata->getFileStateses()) { - for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) - { - StoragePtr storage = iterator->table(); - if (auto * s3queue_table = dynamic_cast(storage.get())) - { - const auto & table_id = s3queue_table->getStorageID(); - auto file_statuses = s3queue_table->getFileStatuses(); - for (const auto & [file_name, file_status] : file_statuses) - { - size_t i = 0; - res_columns[i++]->insert(table_id.database_name); - res_columns[i++]->insert(table_id.table_name); - res_columns[i++]->insert(file_name); - res_columns[i++]->insert(file_status->processed_rows); - res_columns[i++]->insert(magic_enum::enum_name(file_status->state)); + size_t i = 0; + res_columns[i++]->insert(zookeeper_path); + res_columns[i++]->insert(file_name); + res_columns[i++]->insert(file_status->processed_rows); + res_columns[i++]->insert(magic_enum::enum_name(file_status->state)); - if (file_status->processing_start_time) - res_columns[i++]->insert(file_status->processing_start_time); - else - res_columns[i++]->insertDefault(); - if (file_status->processing_end_time) - res_columns[i++]->insert(file_status->processing_end_time); - else - res_columns[i++]->insertDefault(); + if (file_status->processing_start_time) + res_columns[i++]->insert(file_status->processing_start_time); + else + res_columns[i++]->insertDefault(); + if (file_status->processing_end_time) + res_columns[i++]->insert(file_status->processing_end_time); + else + res_columns[i++]->insertDefault(); - ProfileEvents::dumpToMapColumn(file_status->profile_counters.getPartiallyAtomicSnapshot(), res_columns[i++].get(), true); - } - } - } + ProfileEvents::dumpToMapColumn(file_status->profile_counters.getPartiallyAtomicSnapshot(), res_columns[i++].get(), true); } - } } From 6c778d4b121f560e53253f260cc28a35875767b5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 12:05:13 +0200 Subject: [PATCH 015/120] Fix build without s3 --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 11 ++--------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 3 --- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 009f5e3ad75..f61a200f8f0 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,12 +1,6 @@ #include -#include "Common/Exception.h" -#include "Common/ZooKeeper/Types.h" -#include "Common/scope_guard_safe.h" -#include "Interpreters/Context_fwd.h" -#include "Storages/S3Queue/S3QueueSettings.h" #include "config.h" -#if USE_AWS_S3 #include #include #include @@ -14,13 +8,13 @@ #include #include #include -#include #include #include #include #include #include + namespace ProfileEvents { extern const Event S3QueueSetFileProcessingMicroseconds; @@ -226,6 +220,7 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) break; } } + /// TODO lock file token not to go to keeper simultaneously from this server. SetFileProcessingResult result; switch (mode) @@ -742,5 +737,3 @@ bool S3QueueFilesMetadata::checkSettings(const S3QueueSettings & settings) const } } - -#endif diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 5f3a6448e49..b109f584f5e 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -1,7 +1,6 @@ #pragma once #include "config.h" -#if USE_AWS_S3 #include #include #include @@ -123,5 +122,3 @@ private: }; } - -#endif From d77452c5616ff10f25f6c26ef3e75656c435d612 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 12:06:41 +0200 Subject: [PATCH 016/120] Fix style check --- src/Storages/S3Queue/S3QueueMetadataFactory.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp index b1b0c4aef23..0a2856776e2 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp @@ -2,6 +2,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} S3QueueMetadataFactory & S3QueueMetadataFactory::instance() { From 6b191a1afeb95006c8cf1bc7dcaa05b020f1a823 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 14:54:31 +0200 Subject: [PATCH 017/120] Better --- programs/copier/ClusterCopier.cpp | 2 +- src/Common/ZooKeeper/ZooKeeper.h | 29 +++- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 141 ++++++++---------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 22 ++- src/Storages/S3Queue/S3QueueSource.cpp | 27 +++- src/Storages/S3Queue/S3QueueSource.h | 22 ++- src/Storages/StorageS3.cpp | 74 ++++----- src/Storages/StorageS3.h | 19 ++- src/Storages/StorageS3Cluster.cpp | 2 +- 9 files changed, 197 insertions(+), 141 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 556eca808f6..ccd8caf1c5a 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -391,7 +391,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) - return std::make_shared(current_worker_path, *zookeeper, false, false, description); + return zkutil::EphemeralNodeHolder::existing(current_worker_path, *zookeeper); if (code == Coordination::Error::ZBADVERSION) { diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 4b598147301..d61156b31a9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -644,11 +644,18 @@ class EphemeralNodeHolder public: using Ptr = std::shared_ptr; - EphemeralNodeHolder(const std::string & path_, ZooKeeper & zookeeper_, bool create, bool sequential, const std::string & data) + EphemeralNodeHolder(const std::string & path_, ZooKeeper & zookeeper_, bool create, bool try_create, bool sequential, const std::string & data) : path(path_), zookeeper(zookeeper_) { if (create) + { path = zookeeper.create(path, data, sequential ? CreateMode::EphemeralSequential : CreateMode::Ephemeral); + need_remove = created = true; + } + else if (try_create) + { + need_remove = created = Coordination::Error::ZOK == zookeeper.tryCreate(path, data, sequential ? CreateMode::EphemeralSequential : CreateMode::Ephemeral); + } } std::string getPath() const @@ -656,19 +663,32 @@ public: return path; } + bool isCreated() const + { + return created; + } + static Ptr create(const std::string & path, ZooKeeper & zookeeper, const std::string & data = "") { - return std::make_shared(path, zookeeper, true, false, data); + return std::make_shared(path, zookeeper, true, false, false, data); + } + + static Ptr tryCreate(const std::string & path, ZooKeeper & zookeeper, const std::string & data = "") + { + auto node = std::make_shared(path, zookeeper, false, true, false, data); + if (node->isCreated()) + return node; + return nullptr; } static Ptr createSequential(const std::string & path, ZooKeeper & zookeeper, const std::string & data = "") { - return std::make_shared(path, zookeeper, true, true, data); + return std::make_shared(path, zookeeper, true, false, true, data); } static Ptr existing(const std::string & path, ZooKeeper & zookeeper) { - return std::make_shared(path, zookeeper, false, false, ""); + return std::make_shared(path, zookeeper, false, false, false, ""); } void setAlreadyRemoved() @@ -702,6 +722,7 @@ private: ZooKeeper & zookeeper; CurrentMetrics::Increment metric_increment{CurrentMetrics::EphemeralNode}; bool need_remove = true; + bool created = false; }; using EphemeralNodeHolderPtr = EphemeralNodeHolder::Ptr; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index f61a200f8f0..dbcc085575b 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -4,10 +4,13 @@ #include #include #include +#include #include #include #include +#include #include +#include #include #include #include @@ -55,15 +58,6 @@ std::unique_lock S3QueueFilesMetadata::LocalFileStatuses::lock() con return std::unique_lock(mutex); } -S3QueueFilesMetadata::FileStatus::State S3QueueFilesMetadata::LocalFileStatuses::state(const std::string & filename) const -{ - auto lk = lock(); - if (auto it = file_statuses.find(filename); it != file_statuses.end()) - return it->second->state; - else - return FileStatus::State::None; -} - S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::LocalFileStatuses::getAll() const { auto lk = lock(); @@ -106,6 +100,7 @@ std::string S3QueueFilesMetadata::NodeMetadata::toString() const json.set("last_processed_timestamp", getCurrentTime()); json.set("last_exception", last_exception); json.set("retries", retries); + json.set("processing_id", processing_id); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); @@ -123,6 +118,7 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::NodeMetadata::fromStrin metadata.last_processed_timestamp = json->getValue("last_processed_timestamp"); metadata.last_exception = json->getValue("last_exception"); metadata.retries = json->getValue("retries"); + metadata.processing_id = json->getValue("processing_id"); return metadata; } @@ -189,28 +185,28 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } -bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) +S3QueueFilesMetadata::ProcessingHolderPtr S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); + auto file_status = local_file_statuses.get(path, /* create */false); /// Check locally cached file status. - switch (local_file_statuses.state(path)) + switch (file_status->state) { case FileStatus::State::Processing: [[fallthrough]]; case FileStatus::State::Processed: { /// File is already processes or processing by current server. - return false; + return nullptr; } case FileStatus::State::Failed: { - if (!max_loading_retries) - { - /// File was processes by current server and failed, - /// retries are disabled. - return false; - } - /// TODO save information if file is still retriable. + /// max_loading_retries == 0 => file is not retriable. + /// file_status->retries is a cached value, so in case file_status->retries >= max_loading retries + /// we can fully rely that it is true, but in other case the value might be outdated, + /// but this is ok, we will recheck with zookeeper. + if (!max_loading_retries || file_status->retries >= max_loading_retries) + return nullptr; break; } case FileStatus::State::None: @@ -220,19 +216,25 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) break; } } - /// TODO lock file token not to go to keeper simultaneously from this server. + std::unique_lock lock(file_status->processing_lock, std::defer_lock); + if (!lock.try_lock()) + { + /// Another thread is already trying to set file as processing. + return nullptr; + } SetFileProcessingResult result; + ProcessingHolderPtr processing_holder; switch (mode) { case S3QueueMode::ORDERED: { - result = trySetFileAsProcessingForOrderedMode(path); + std::tie(result, processing_holder) = trySetFileAsProcessingForOrderedMode(path); break; } case S3QueueMode::UNORDERED: { - result = trySetFileAsProcessingForUnorderedMode(path); + std::tie(result, processing_holder) = trySetFileAsProcessingForUnorderedMode(path); break; } } @@ -240,7 +242,6 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { case SetFileProcessingResult::Success: { - auto file_status = local_file_statuses.get(path, /* create */true); file_status->state = FileStatus::State::Processing; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); timer.cancel(); @@ -251,14 +252,12 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) case SetFileProcessingResult::AlreadyProcessed: { /// Cache the state. - auto file_status = local_file_statuses.get(path, /* create */true); file_status->state = FileStatus::State::Processed; break; } case SetFileProcessingResult::AlreadyFailed: { /// Cache the state. - auto file_status = local_file_statuses.get(path, /* create */true); file_status->state = FileStatus::State::Failed; break; } @@ -268,54 +267,60 @@ bool S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) break; } } - return result == SetFileProcessingResult::Success; + + if (result != SetFileProcessingResult::Success) + return nullptr; + + return processing_holder; } -S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) +std::pair +S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { /// Create an ephemenral node in /processing /// if corresponding node does not exist in failed/, processed/ and processing/. /// Return false otherwise. const auto node_name = getNodeName(path); - const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = getZooKeeper(); + auto node_metadata = createNodeMetadata(path); + node_metadata.processing_id = getRandomASCIIString(10); Coordination::Requests requests; zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processed_path / node_name); zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); - requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); Coordination::Responses responses; auto code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) { - return SetFileProcessingResult::Success; + auto holder = std::make_unique(node_metadata.processing_id, zookeeper_processing_path / node_name, zk_client); + return std::pair{SetFileProcessingResult::Success, std::move(holder)}; } - else if (responses[0]->error == Coordination::Error::ZOK) - { - if (responses[1]->error == Coordination::Error::ZOK) - { - chassert(responses[2]->error != Coordination::Error::ZOK); - return SetFileProcessingResult::ProcessingByOtherNode; - } - else - return SetFileProcessingResult::AlreadyFailed; - } - else - return SetFileProcessingResult::AlreadyProcessed; + + if (responses[0]->error != Coordination::Error::ZOK) + return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; + + if (responses[1]->error != Coordination::Error::ZOK) + return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; + + chassert(responses[2]->error != Coordination::Error::ZOK); + return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } -S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) +std::pair +S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { /// Create an ephemenral node in /processing /// if corresponding it does not exist in failed/, processing/ and satisfied max processed file check. /// Return false otherwise. const auto node_name = getNodeName(path); - const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = getZooKeeper(); + auto node_metadata = createNodeMetadata(path); + node_metadata.processing_id = getRandomASCIIString(10); while (true) { @@ -330,12 +335,12 @@ S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAs if (responses[0]->error == Coordination::Error::ZOK) { LOG_TEST(log, "Skipping file `{}`: already processing", path); - return SetFileProcessingResult::ProcessingByOtherNode; + return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } else { LOG_TEST(log, "Skipping file `{}`: failed", path); - return SetFileProcessingResult::AlreadyFailed; + return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; } } @@ -347,27 +352,30 @@ S3QueueFilesMetadata::SetFileProcessingResult S3QueueFilesMetadata::trySetFileAs auto max_processed_file_path = processed_node_metadata.file_path; if (!max_processed_file_path.empty() && path <= max_processed_file_path) - return SetFileProcessingResult::AlreadyProcessed; + return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; requests.clear(); responses.clear(); zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); - requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata, zkutil::CreateMode::Ephemeral)); + requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) - return SetFileProcessingResult::Success; + { + auto holder = std::make_unique(node_metadata.processing_id, zookeeper_processing_path / node_name, zk_client); + return std::pair{SetFileProcessingResult::Success, std::move(holder)}; + } if (responses[0]->error != Coordination::Error::ZOK) { LOG_TEST(log, "Skipping file `{}`: failed", path); - return SetFileProcessingResult::AlreadyFailed; + return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; } else if (responses[1]->error != Coordination::Error::ZOK) { LOG_TEST(log, "Skipping file `{}`: already processing", path); - return SetFileProcessingResult::ProcessingByOtherNode; + return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } else { @@ -465,8 +473,8 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileFailedMicroseconds); - SCOPE_EXIT_SAFE({ - auto file_status = local_file_statuses.get(path, /* create */false); + auto file_status = local_file_statuses.get(path, /* create */false); + SCOPE_EXIT({ file_status->state = FileStatus::State::Failed; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); timer.cancel(); @@ -505,6 +513,7 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc { auto failed_node_metadata = NodeMetadata::fromString(res); node_metadata.retries = failed_node_metadata.retries + 1; + file_status->retries = node_metadata.retries; } LOG_TEST(log, "File `{}` failed to process, try {}/{} (Error: {})", @@ -605,30 +614,12 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() /// Create a lock so that with distributed processing /// multiple nodes do not execute cleanup in parallel. - Coordination::Error code = zk_client->tryCreate(zookeeper_cleanup_lock_path, - toString(getCurrentTime()), - zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNODEEXISTS) + auto ephemeral_node = zkutil::EphemeralNodeHolder::create(zookeeper_cleanup_lock_path, *zk_client, toString(getCurrentTime())); + if (!ephemeral_node) { LOG_TEST(log, "Cleanup is already being executed by another node"); return; } - else if (code != Coordination::Error::ZOK) - { - throw Coordination::Exception::fromPath(code, zookeeper_cleanup_lock_path); - } - - SCOPE_EXIT_SAFE({ - try - { - zk_client->remove(zookeeper_cleanup_lock_path); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - chassert(false); - } - }); struct Node { @@ -687,7 +678,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); - code = zk_client->tryRemove(path); + auto code = zk_client->tryRemove(path); if (code == Coordination::Error::ZOK) --nodes_to_remove; else @@ -704,7 +695,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() local_file_statuses.remove(node.metadata.file_path, /* if_exists */true); - code = zk_client->tryRemove(path); + auto code = zk_client->tryRemove(path); if (code != Coordination::Error::ZOK) LOG_ERROR(log, "Failed to remove a node `{}` (code: {})", path.string(), code); } diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index b109f584f5e..7dec4c11383 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -22,7 +22,17 @@ public: ~S3QueueFilesMetadata(); - bool trySetFileAsProcessing(const std::string & path); + struct ProcessingHolder + { + ProcessingHolder(const std::string & processing_id_, const std::string & zk_node_path_, zkutil::ZooKeeperPtr zk_client_) + : zk_client(zk_client_), zk_node_path(zk_node_path_), processing_id(processing_id_) {} + + zkutil::ZooKeeperPtr zk_client; + std::string zk_node_path; + std::string processing_id; + }; + using ProcessingHolderPtr = std::unique_ptr; + ProcessingHolderPtr trySetFileAsProcessing(const std::string & path); void setFileProcessed(const std::string & path); @@ -47,6 +57,10 @@ public: time_t processing_start_time = 0; time_t processing_end_time = 0; + + size_t retries = 0; + + std::mutex processing_lock; }; using FileStatuses = std::unordered_map>; @@ -88,8 +102,8 @@ private: AlreadyProcessed, AlreadyFailed, }; - SetFileProcessingResult trySetFileAsProcessingForOrderedMode(const std::string & path); - SetFileProcessingResult trySetFileAsProcessingForUnorderedMode(const std::string & path); + std::pair trySetFileAsProcessingForOrderedMode(const std::string & path); + std::pair trySetFileAsProcessingForUnorderedMode(const std::string & path); struct NodeMetadata { @@ -97,6 +111,7 @@ private: UInt64 last_processed_timestamp = 0; std::string last_exception; UInt64 retries = 0; + std::string processing_id; /// For ephemeral processing node. std::string toString() const; static NodeMetadata fromString(const std::string & metadata_str); @@ -115,7 +130,6 @@ private: FileStatuses getAll() const; std::shared_ptr get(const std::string & filename, bool create); bool remove(const std::string & filename, bool if_exists); - FileStatus::State state(const std::string & filename) const; std::unique_lock lock() const; }; LocalFileStatuses local_file_statuses; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index e9a57cbbfd4..ff1a8c86b64 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -29,24 +30,39 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( + const std::string & key_, + std::optional info_, + std::unique_ptr processing_holder_, + std::shared_ptr file_status_) + : StorageS3Source::KeyWithInfo(key_, info_) + , processing_holder(std::move(processing_holder_)) + , file_status(file_status_) +{ +} + StorageS3QueueSource::FileIterator::FileIterator( std::shared_ptr metadata_, std::unique_ptr glob_iterator_) : metadata(metadata_) , glob_iterator(std::move(glob_iterator_)) { } -StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::FileIterator::next() +StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next() { /// List results in s3 are always returned in UTF-8 binary order. /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) while (true) { - KeyWithInfo val = glob_iterator->next(); - if (val.key.empty()) + KeyWithInfoPtr val = glob_iterator->next(); + + if (!val) return {}; - if (metadata->trySetFileAsProcessing(val.key)) - return val; + + if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder) + { + return std::make_shared(val->key, val->info, std::move(processing_holder), nullptr); + } } } @@ -77,6 +93,7 @@ StorageS3QueueSource::StorageS3QueueSource( , shutdown_called(shutdown_called_) , s3_queue_log(s3_queue_log_) , storage_id(storage_id_) + , s3_queue_user_id(fmt::format("{}:{}", CurrentThread::getQueryId(), getRandomASCIIString(8))) , remove_file_func(remove_file_func_) , log(&Poco::Logger::get("StorageS3QueueSource")) { diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index ce8a64022d0..634c0803465 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -19,19 +19,30 @@ class StorageS3QueueSource : public ISource, WithContext { public: using IIterator = StorageS3Source::IIterator; + using KeyWithInfoPtr = StorageS3Source::KeyWithInfoPtr; using GlobIterator = StorageS3Source::DisclosedGlobIterator; - using KeyWithInfo = StorageS3Source::KeyWithInfo; using ZooKeeperGetter = std::function; using RemoveFileFunc = std::function; + using Metadata = S3QueueFilesMetadata; + + struct S3QueueKeyWithInfo : public StorageS3Source::KeyWithInfo + { + S3QueueKeyWithInfo( + const std::string & key_, + std::optional info_, + std::unique_ptr processing_holder_, + std::shared_ptr file_status_); + + std::unique_ptr processing_holder; + std::shared_ptr file_status; + }; class FileIterator : public IIterator { public: - FileIterator( - std::shared_ptr metadata_, - std::unique_ptr glob_iterator_); + FileIterator(std::shared_ptr metadata_, std::unique_ptr glob_iterator_); - KeyWithInfo next() override; + KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; @@ -71,6 +82,7 @@ private: const std::atomic & shutdown_called; const std::shared_ptr s3_queue_log; const StorageID storage_id; + const std::string s3_queue_user_id; RemoveFileFunc remove_file_func; Poco::Logger * log; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 3deb22bd32d..3f8919d0f05 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -161,7 +161,7 @@ public: /// We don't have to list bucket, because there is no asterisks. if (key_prefix.size() == globbed_uri.key.size()) { - buffer.emplace_back(globbed_uri.key, std::nullopt); + buffer.emplace_back(std::make_shared(globbed_uri.key, std::nullopt)); buffer_iter = buffer.begin(); is_finished = true; return; @@ -182,7 +182,7 @@ public: fillInternalBufferAssumeLocked(); } - KeyWithInfo next() + KeyWithInfoPtr next() { std::lock_guard lock(mutex); return nextAssumeLocked(); @@ -201,7 +201,7 @@ public: private: using ListObjectsOutcome = Aws::S3::Model::ListObjectsV2Outcome; - KeyWithInfo nextAssumeLocked() + KeyWithInfoPtr nextAssumeLocked() { if (buffer_iter != buffer.end()) { @@ -277,7 +277,7 @@ private: .last_modification_time = row.GetLastModified().Millis() / 1000, }; - temp_buffer.emplace_back(std::move(key), std::move(info)); + temp_buffer.emplace_back(std::make_shared(std::move(key), std::move(info))); } } @@ -289,7 +289,7 @@ private: if (!is_initialized) { - filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(globbed_uri.bucket) / temp_buffer.front().key, getContext()); + filter_ast = VirtualColumnUtils::createPathAndFileFilterAst(query, virtual_columns, fs::path(globbed_uri.bucket) / temp_buffer.front()->key, getContext()); is_initialized = true; } @@ -298,7 +298,7 @@ private: std::vector paths; paths.reserve(temp_buffer.size()); for (const auto & key_with_info : temp_buffer) - paths.push_back(fs::path(globbed_uri.bucket) / key_with_info.key); + paths.push_back(fs::path(globbed_uri.bucket) / key_with_info->key); VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, query, virtual_columns, getContext(), filter_ast); } @@ -307,8 +307,8 @@ private: if (file_progress_callback) { - for (const auto & [_, info] : buffer) - file_progress_callback(FileProgress(0, info->size)); + for (const auto & key_with_info : buffer) + file_progress_callback(FileProgress(0, key_with_info->info->size)); } /// Set iterator only after the whole batch is processed @@ -371,7 +371,7 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( { } -StorageS3Source::KeyWithInfo StorageS3Source::DisclosedGlobIterator::next() +StorageS3Source::KeyWithInfoPtr StorageS3Source::DisclosedGlobIterator::next() { return pimpl->next(); } @@ -422,11 +422,11 @@ public: if (read_keys_) { for (const auto & key : keys) - read_keys_->push_back({key, {}}); + read_keys_->push_back(std::make_shared(key)); } } - KeyWithInfo next() + KeyWithInfoPtr next() { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); if (current_index >= keys.size()) @@ -439,7 +439,7 @@ public: file_progress_callback(FileProgress(0, info->size)); } - return {key, info}; + return std::make_shared(key, info); } size_t objectsCount() @@ -476,7 +476,7 @@ StorageS3Source::KeysIterator::KeysIterator( { } -StorageS3Source::KeyWithInfo StorageS3Source::KeysIterator::next() +StorageS3Source::KeyWithInfoPtr StorageS3Source::KeysIterator::next() { return pimpl->next(); } @@ -502,14 +502,14 @@ StorageS3Source::ReadTaskIterator::ReadTaskIterator( pool.wait(); buffer.reserve(max_threads_count); for (auto & key_future : keys) - buffer.emplace_back(key_future.get(), std::nullopt); + buffer.emplace_back(std::make_shared(key_future.get(), std::nullopt)); } -StorageS3Source::KeyWithInfo StorageS3Source::ReadTaskIterator::next() +StorageS3Source::KeyWithInfoPtr StorageS3Source::ReadTaskIterator::next() { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); if (current_index >= buffer.size()) - return {callback(), {}}; + return std::make_shared(callback()); return buffer[current_index]; } @@ -566,22 +566,22 @@ StorageS3Source::StorageS3Source( StorageS3Source::ReaderHolder StorageS3Source::createReader() { - KeyWithInfo key_with_info; + KeyWithInfoPtr key_with_info; do { key_with_info = (*file_iterator)(); - if (key_with_info.key.empty()) + if (!key_with_info) return {}; - if (!key_with_info.info) - key_with_info.info = S3::getObjectInfo(*client, bucket, key_with_info.key, version_id, request_settings); + if (!key_with_info->info) + key_with_info->info = S3::getObjectInfo(*client, bucket, key_with_info->key, version_id, request_settings); } - while (getContext()->getSettingsRef().s3_skip_empty_files && key_with_info.info->size == 0); + while (getContext()->getSettingsRef().s3_skip_empty_files && key_with_info->info->size == 0); QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(key_with_info) : std::nullopt; + std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(*key_with_info) : std::nullopt; if (num_rows_from_cache) { /// We should not return single chunk with all number of rows, @@ -594,8 +594,8 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() } else { - auto compression_method = chooseCompressionMethod(key_with_info.key, compression_hint); - read_buf = createS3ReadBuffer(key_with_info.key, key_with_info.info->size); + auto compression_method = chooseCompressionMethod(key_with_info->key, compression_hint); + read_buf = createS3ReadBuffer(key_with_info->key, key_with_info->info->size); auto input_format = FormatFactory::instance().getInput( format, @@ -639,7 +639,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - return ReaderHolder{key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{*key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() @@ -1494,7 +1494,7 @@ namespace { current_key_with_info = (*file_iterator)(); - if (current_key_with_info.key.empty()) + if (!current_key_with_info) { if (first) throw Exception( @@ -1506,6 +1506,8 @@ namespace return nullptr; } + chassert(!current_key_with_info->key.empty()); + /// S3 file iterator could get new keys after new iteration, check them in schema cache. if (getContext()->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size) { @@ -1515,15 +1517,15 @@ namespace return nullptr; } - if (getContext()->getSettingsRef().s3_skip_empty_files && current_key_with_info.info && current_key_with_info.info->size == 0) + if (getContext()->getSettingsRef().s3_skip_empty_files && current_key_with_info->info && current_key_with_info->info->size == 0) continue; int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - auto impl = std::make_unique(configuration.client, configuration.url.bucket, current_key_with_info.key, configuration.url.version_id, configuration.request_settings, getContext()->getReadSettings()); + auto impl = std::make_unique(configuration.client, configuration.url.bucket, current_key_with_info->key, configuration.url.version_id, configuration.request_settings, getContext()->getReadSettings()); if (!getContext()->getSettingsRef().s3_skip_empty_files || !impl->eof()) { first = false; - return wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info.key, configuration.compression_method), zstd_window_log_max); + return wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info->key, configuration.compression_method), zstd_window_log_max); } } } @@ -1538,7 +1540,7 @@ namespace if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) return; - String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket / current_key_with_info.key; + String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket / current_key_with_info->key; auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); StorageS3::getSchemaCache(getContext()).addNumRows(key, num_rows); } @@ -1549,7 +1551,7 @@ namespace const StorageS3::Configuration & configuration; const std::optional & format_settings; std::optional columns_from_cache; - StorageS3Source::KeyWithInfo current_key_with_info; + StorageS3Source::KeyWithInfoPtr current_key_with_info; size_t prev_read_keys_size; bool first = true; }; @@ -1689,9 +1691,9 @@ std::optional StorageS3::tryGetColumnsFromCache( auto get_last_mod_time = [&] { time_t last_modification_time = 0; - if (it->info) + if ((*it)->info) { - last_modification_time = it->info->last_modification_time; + last_modification_time = (*it)->info->last_modification_time; } else { @@ -1701,7 +1703,7 @@ std::optional StorageS3::tryGetColumnsFromCache( last_modification_time = S3::getObjectInfo( *configuration.client, configuration.url.bucket, - it->key, + (*it)->key, configuration.url.version_id, configuration.request_settings, /*with_metadata=*/ false, @@ -1712,7 +1714,7 @@ std::optional StorageS3::tryGetColumnsFromCache( return last_modification_time ? std::make_optional(last_modification_time) : std::nullopt; }; - String path = fs::path(configuration.url.bucket) / it->key; + String path = fs::path(configuration.url.bucket) / (*it)->key; String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / path; auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, ctx); auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); @@ -1734,7 +1736,7 @@ void StorageS3::addColumnsToCache( auto host_and_bucket = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket; Strings sources; sources.reserve(keys.size()); - std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem.key; }); + std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem->key; }); auto cache_keys = getKeysForSchemaCache(sources, format_name, format_settings, ctx); auto & schema_cache = getSchemaCache(ctx); schema_cache.addManyColumns(cache_keys, columns); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 088f9000ce8..38cf3a5f65b 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -43,22 +43,21 @@ public: struct KeyWithInfo { KeyWithInfo() = default; - KeyWithInfo(String key_, std::optional info_) - : key(std::move(key_)), info(std::move(info_)) - { - } + explicit KeyWithInfo(String key_, std::optional info_ = std::nullopt) + : key(std::move(key_)), info(std::move(info_)) {} String key; std::optional info; }; + using KeyWithInfoPtr = std::shared_ptr; - using KeysWithInfo = std::vector; + using KeysWithInfo = std::vector; class IIterator { public: virtual ~IIterator() = default; - virtual KeyWithInfo next() = 0; + virtual KeyWithInfoPtr next() = 0; /// Estimates how many streams we need to process all files. /// If keys count >= max_threads_count, the returned number may not represent the actual number of the keys. @@ -66,7 +65,7 @@ public: /// fixme: May underestimate if the glob has a strong filter, so there are few matches among the first 1000 ListObjects results. virtual size_t estimatedKeysCount() = 0; - KeyWithInfo operator ()() { return next(); } + KeyWithInfoPtr operator ()() { return next(); } }; class DisclosedGlobIterator : public IIterator @@ -82,7 +81,7 @@ public: const S3Settings::RequestSettings & request_settings_ = {}, std::function progress_callback_ = {}); - KeyWithInfo next() override; + KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; private: @@ -106,7 +105,7 @@ public: KeysWithInfo * read_keys = nullptr, std::function progress_callback_ = {}); - KeyWithInfo next() override; + KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; private: @@ -120,7 +119,7 @@ public: public: explicit ReadTaskIterator(const ReadTaskCallback & callback_, const size_t max_threads_count); - KeyWithInfo next() override; + KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; private: diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index c8715938c6f..1dd1a410595 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -82,7 +82,7 @@ RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr { auto iterator = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); - auto callback = std::make_shared>([iterator]() mutable -> String { return iterator->next().key; }); + auto callback = std::make_shared>([iterator]() mutable -> String { return iterator->next()->key; }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } From f753b91a3b64d71794b9d15a0fde1dd473e47e49 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 17:17:52 +0200 Subject: [PATCH 018/120] Better maintenance of processing node --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 132 ++++++++++++++---- src/Storages/S3Queue/S3QueueFilesMetadata.h | 52 ++++--- .../S3Queue/S3QueueMetadataFactory.cpp | 43 +++++- src/Storages/S3Queue/S3QueueMetadataFactory.h | 18 ++- src/Storages/S3Queue/S3QueueSource.cpp | 16 ++- src/Storages/S3Queue/S3QueueSource.h | 6 +- src/Storages/S3Queue/StorageS3Queue.cpp | 12 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3.h | 12 +- 9 files changed, 216 insertions(+), 77 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index dbcc085575b..0492a8c2f4d 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -1,4 +1,3 @@ -#include #include "config.h" #include @@ -25,6 +24,7 @@ namespace ProfileEvents extern const Event S3QueueSetFileFailedMicroseconds; extern const Event S3QueueCleanupMaxSetSizeOrTTLMicroseconds; extern const Event S3QueueLockLocalFileStatusesMicroseconds; + extern const Event CannotRemoveEphemeralNode; }; namespace DB @@ -118,7 +118,7 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::NodeMetadata::fromStrin metadata.last_processed_timestamp = json->getValue("last_processed_timestamp"); metadata.last_exception = json->getValue("last_exception"); metadata.retries = json->getValue("retries"); - metadata.processing_id = json->getValue("processing_id"); + metadata.processing_id = json->getValue("processing_id"); return metadata; } @@ -185,10 +185,10 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } -S3QueueFilesMetadata::ProcessingHolderPtr S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) +S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); - auto file_status = local_file_statuses.get(path, /* create */false); + auto file_status = local_file_statuses.get(path, /* create */true); /// Check locally cached file status. switch (file_status->state) @@ -224,7 +224,7 @@ S3QueueFilesMetadata::ProcessingHolderPtr S3QueueFilesMetadata::trySetFileAsProc } SetFileProcessingResult result; - ProcessingHolderPtr processing_holder; + ProcessingNodeHolderPtr processing_holder; switch (mode) { case S3QueueMode::ORDERED: @@ -274,7 +274,7 @@ S3QueueFilesMetadata::ProcessingHolderPtr S3QueueFilesMetadata::trySetFileAsProc return processing_holder; } -std::pair +std::pair S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { /// Create an ephemenral node in /processing @@ -296,7 +296,7 @@ S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & if (code == Coordination::Error::ZOK) { - auto holder = std::make_unique(node_metadata.processing_id, zookeeper_processing_path / node_name, zk_client); + auto holder = std::make_unique(node_metadata.processing_id, path, zookeeper_processing_path / node_name, zk_client); return std::pair{SetFileProcessingResult::Success, std::move(holder)}; } @@ -310,7 +310,7 @@ S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } -std::pair +std::pair S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { /// Create an ephemenral node in /processing @@ -356,6 +356,7 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p requests.clear(); responses.clear(); + zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); @@ -363,7 +364,7 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) { - auto holder = std::make_unique(node_metadata.processing_id, zookeeper_processing_path / node_name, zk_client); + auto holder = std::make_unique(node_metadata.processing_id, path, zookeeper_processing_path / node_name, zk_client); return std::pair{SetFileProcessingResult::Success, std::move(holder)}; } @@ -384,9 +385,11 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p } } -void S3QueueFilesMetadata::setFileProcessed(const String & path) +void S3QueueFilesMetadata::setFileProcessed(ProcessingNodeHolderPtr holder) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessedMicroseconds); + const auto & path = holder->path; + SCOPE_EXIT({ auto file_status = local_file_statuses.get(path, /* create */false); file_status->state = FileStatus::State::Processed; @@ -399,44 +402,44 @@ void S3QueueFilesMetadata::setFileProcessed(const String & path) { case S3QueueMode::ORDERED: { - return setFileProcessedForOrderedMode(path); + return setFileProcessedForOrderedMode(holder); } case S3QueueMode::UNORDERED: { - return setFileProcessedForUnorderedMode(path); + return setFileProcessedForUnorderedMode(holder); } } } -void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(const String & path) +void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolderPtr holder) { /// Create a persistent node in /processed and remove ephemeral node from /processing. + const auto & path = holder->path; const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = getZooKeeper(); Coordination::Requests requests; - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); requests.push_back(zkutil::makeCreateRequest(zookeeper_processed_path / node_name, node_metadata, zkutil::CreateMode::Persistent)); Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) + if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); return; } /// TODO this could be because of the expired session. - if (responses[0]->error != Coordination::Error::ZOK) + if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is not processing"); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is already processed"); } -void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) +void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder) { + const auto & path = holder->path; const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); const auto zk_client = getZooKeeper(); @@ -463,15 +466,22 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(const String & path) } Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) + if (holder->remove(&requests, &responses)) return; + + if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) + continue; + + LOG_ERROR(log, "Cannot set file {} as failed - failed to remove ephemeral processing node", path); + chassert(false); + return; } } -void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exception_message) +void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const String & exception_message) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileFailedMicroseconds); + const auto & path = holder->path; auto file_status = local_file_statuses.get(path, /* create */false); SCOPE_EXIT({ @@ -488,14 +498,12 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc if (max_loading_retries == 0) { Coordination::Requests requests; - requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, node_metadata.toString(), zkutil::CreateMode::Persistent)); Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code == Coordination::Error::ZOK) + if (holder->remove(&requests, &responses)) { LOG_TEST(log, "File `{}` failed to process and will not be retried. " "Error: {}", path, exception_message); @@ -565,6 +573,82 @@ void S3QueueFilesMetadata::setFileFailed(const String & path, const String & exc } } +S3QueueFilesMetadata::ProcessingNodeHolder::ProcessingNodeHolder( + const std::string & processing_id_, + const std::string & path_, + const std::string & zk_node_path_, + zkutil::ZooKeeperPtr zk_client_) + : zk_client(zk_client_) + , path(path_) + , zk_node_path(zk_node_path_) + , processing_id(processing_id_) + , log(&Poco::Logger::get("ProcessingNodeHolder")) +{ +} + +S3QueueFilesMetadata::ProcessingNodeHolder::~ProcessingNodeHolder() +{ + if (!removed) + remove(); +} + +bool S3QueueFilesMetadata::ProcessingNodeHolder::remove(Coordination::Requests * requests, Coordination::Responses * responses) +{ + if (removed) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing file holder is already released"); + + removed = true; + + try + { + if (!zk_client->expired()) + { + /// Is is possible that we created an ephemeral processing node + /// but session expired and someone other created an ephemeral processing node. + /// To avoid deleting this new node, check processing_id. + std::string res; + Coordination::Stat stat; + if (zk_client->tryGet(zk_node_path, res, &stat)) + { + auto node_metadata = NodeMetadata::fromString(res); + if (node_metadata.processing_id == processing_id) + { + if (requests) + { + requests->push_back(zkutil::makeRemoveRequest(zk_node_path, stat.version)); + auto code = zk_client->tryMulti(*requests, *responses); + return code == Coordination::Error::ZOK; + } + else + zk_client->remove(zk_node_path); + return true; + } + else + LOG_WARNING(log, "Cannot remove {} since precessing id changed: {} -> {}", + zk_node_path, processing_id, node_metadata.processing_id); + } + else + LOG_DEBUG(log, "Cannot remove {}, node doesn't exist, " + "probably because of session expiration", zk_node_path); + + /// TODO: this actually would mean that we already processed (or partially processed) + /// the data but another thread will try processing it again and data can be duplicated. + /// This can be solved via persistenly saving last processed offset in the file. + } + else + { + ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode); + LOG_DEBUG(log, "Cannot remove {} since session has been expired", zk_node_path); + } + } + catch (...) + { + ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode); + DB::tryLogCurrentException(__PRETTY_FUNCTION__, "Cannot remove " + zk_node_path); + } + return false; +} + void S3QueueFilesMetadata::cleanupThreadFunc() { /// A background task is responsible for maintaining diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index 7dec4c11383..dd760b29299 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -18,29 +18,18 @@ class StorageS3Queue; class S3QueueFilesMetadata { public: + class ProcessingNodeHolder; + using ProcessingNodeHolderPtr = std::shared_ptr; + S3QueueFilesMetadata(const fs::path & zookeeper_path_, const S3QueueSettings & settings_); ~S3QueueFilesMetadata(); - struct ProcessingHolder - { - ProcessingHolder(const std::string & processing_id_, const std::string & zk_node_path_, zkutil::ZooKeeperPtr zk_client_) - : zk_client(zk_client_), zk_node_path(zk_node_path_), processing_id(processing_id_) {} + ProcessingNodeHolderPtr trySetFileAsProcessing(const std::string & path); - zkutil::ZooKeeperPtr zk_client; - std::string zk_node_path; - std::string processing_id; - }; - using ProcessingHolderPtr = std::unique_ptr; - ProcessingHolderPtr trySetFileAsProcessing(const std::string & path); + void setFileProcessed(ProcessingNodeHolderPtr holder); - void setFileProcessed(const std::string & path); - - void setFileFailed(const std::string & path, const std::string & exception_message); - - using OnProgress = std::function; - - void deactivateCleanupTask(); + void setFileFailed(ProcessingNodeHolderPtr holder, const std::string & exception_message); struct FileStatus { @@ -70,6 +59,8 @@ public: bool checkSettings(const S3QueueSettings & settings) const; + void deactivateCleanupTask(); + private: const S3QueueMode mode; const UInt64 max_set_size; @@ -92,8 +83,8 @@ private: zkutil::ZooKeeperPtr getZooKeeper() const; - void setFileProcessedForOrderedMode(const std::string & path); - void setFileProcessedForUnorderedMode(const std::string & path); + void setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder); + void setFileProcessedForUnorderedMode(ProcessingNodeHolderPtr holder); enum class SetFileProcessingResult { @@ -102,8 +93,8 @@ private: AlreadyProcessed, AlreadyFailed, }; - std::pair trySetFileAsProcessingForOrderedMode(const std::string & path); - std::pair trySetFileAsProcessingForUnorderedMode(const std::string & path); + std::pair trySetFileAsProcessingForOrderedMode(const std::string & path); + std::pair trySetFileAsProcessingForUnorderedMode(const std::string & path); struct NodeMetadata { @@ -135,4 +126,23 @@ private: LocalFileStatuses local_file_statuses; }; +class S3QueueFilesMetadata::ProcessingNodeHolder +{ + friend class S3QueueFilesMetadata; +public: + ProcessingNodeHolder(const std::string & processing_id_, const std::string & path_, const std::string & zk_node_path_, zkutil::ZooKeeperPtr zk_client_); + + ~ProcessingNodeHolder(); + +private: + bool remove(Coordination::Requests * requests = nullptr, Coordination::Responses * responses = nullptr); + + zkutil::ZooKeeperPtr zk_client; + std::string path; + std::string zk_node_path; + std::string processing_id; + bool removed = false; + Poco::Logger * log; +}; + } diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp index 0a2856776e2..bd01bd52425 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.cpp +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -13,7 +14,7 @@ S3QueueMetadataFactory & S3QueueMetadataFactory::instance() return ret; } -S3QueueMetadataFactory::MetadataPtr +S3QueueMetadataFactory::FilesMetadataPtr S3QueueMetadataFactory::getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings) { std::lock_guard lock(mutex); @@ -22,12 +23,48 @@ S3QueueMetadataFactory::getOrCreate(const std::string & zookeeper_path, const S3 { it = metadata_by_path.emplace(zookeeper_path, std::make_shared(fs::path(zookeeper_path), settings)).first; } - else if (!it->second->checkSettings(settings)) + else if (it->second.metadata->checkSettings(settings)) + { + it->second.ref_count += 1; + } + else { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Metadata with the same `s3queue_zookeeper_path` " "was already created but with different settings"); } - return it->second; + return it->second.metadata; +} + +void S3QueueMetadataFactory::remove(const std::string & zookeeper_path) +{ + std::lock_guard lock(mutex); + auto it = metadata_by_path.find(zookeeper_path); + + if (it == metadata_by_path.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Metadata with zookeeper path {} does not exist", zookeeper_path); + + if (--it->second.ref_count == 0) + { + try + { + auto zk_client = Context::getGlobalContextInstance()->getZooKeeper(); + zk_client->tryRemove(it->first); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + metadata_by_path.erase(it); + } +} + +std::unordered_map S3QueueMetadataFactory::getAll() +{ + std::unordered_map result; + for (const auto & [zk_path, metadata_and_ref_count] : metadata_by_path) + result.emplace(zk_path, metadata_and_ref_count.metadata); + return result; } } diff --git a/src/Storages/S3Queue/S3QueueMetadataFactory.h b/src/Storages/S3Queue/S3QueueMetadataFactory.h index e7a473d863d..55c2dfad5dd 100644 --- a/src/Storages/S3Queue/S3QueueMetadataFactory.h +++ b/src/Storages/S3Queue/S3QueueMetadataFactory.h @@ -9,16 +9,26 @@ namespace DB class S3QueueMetadataFactory final : private boost::noncopyable { public: - using MetadataPtr = std::shared_ptr; - using MetadataByPath = std::unordered_map; + using FilesMetadataPtr = std::shared_ptr; static S3QueueMetadataFactory & instance(); - MetadataPtr getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings); + FilesMetadataPtr getOrCreate(const std::string & zookeeper_path, const S3QueueSettings & settings); - MetadataByPath getAll() { return metadata_by_path; } + void remove(const std::string & zookeeper_path); + + std::unordered_map getAll(); private: + struct Metadata + { + explicit Metadata(std::shared_ptr metadata_) : metadata(metadata_), ref_count(1) {} + + std::shared_ptr metadata; + size_t ref_count = 0; + }; + using MetadataByPath = std::unordered_map; + MetadataByPath metadata_by_path; std::mutex mutex; }; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index ff1a8c86b64..4171597b619 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -33,11 +33,9 @@ namespace ErrorCodes StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( const std::string & key_, std::optional info_, - std::unique_ptr processing_holder_, - std::shared_ptr file_status_) + Metadata::ProcessingNodeHolderPtr processing_holder_) : StorageS3Source::KeyWithInfo(key_, info_) - , processing_holder(std::move(processing_holder_)) - , file_status(file_status_) + , processing_holder(processing_holder_) { } @@ -61,7 +59,7 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next() if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder) { - return std::make_shared(val->key, val->info, std::move(processing_holder), nullptr); + return std::make_shared(val->key, val->info, processing_holder); } } } @@ -151,12 +149,16 @@ Chunk StorageS3QueueSource::generate() catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); - files_metadata->setFileFailed(reader.getFile(), e.message()); + + const StorageS3QueueSource::S3QueueKeyWithInfo * key_with_info = assert_cast(&reader.getKeyWithInfo()); + files_metadata->setFileFailed(key_with_info->processing_holder, e.message()); + appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); throw; } - files_metadata->setFileProcessed(reader.getFile()); + const StorageS3QueueSource::S3QueueKeyWithInfo * key_with_info = assert_cast(&reader.getKeyWithInfo()); + files_metadata->setFileProcessed(key_with_info->processing_holder); applyActionAfterProcessing(reader.getFile()); appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, true); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 634c0803465..304b7d3b661 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -30,11 +30,9 @@ public: S3QueueKeyWithInfo( const std::string & key_, std::optional info_, - std::unique_ptr processing_holder_, - std::shared_ptr file_status_); + Metadata::ProcessingNodeHolderPtr processing_holder_); - std::unique_ptr processing_holder; - std::shared_ptr file_status; + Metadata::ProcessingNodeHolderPtr processing_holder; }; class FileIterator : public IIterator diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index a15cc1bea9b..553211f7772 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -149,6 +149,11 @@ void StorageS3Queue::shutdown() } } +void StorageS3Queue::drop() +{ + S3QueueMetadataFactory::instance().remove(zk_path); +} + bool StorageS3Queue::supportsSubsetOfColumns(const ContextPtr & context_) const { return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context_, format_settings); @@ -422,13 +427,6 @@ std::shared_ptr StorageS3Queue::createFileIterator return std::make_shared(files_metadata, std::move(glob_iterator)); } -void StorageS3Queue::drop() -{ - auto zookeeper = getZooKeeper(); - if (zookeeper->exists(zk_path)) - zookeeper->removeRecursive(zk_path); -} - void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 3f8919d0f05..1bb2dccbccc 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -639,7 +639,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - return ReaderHolder{*key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 38cf3a5f65b..c053c6f9bd2 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -175,13 +175,13 @@ private: { public: ReaderHolder( - KeyWithInfo key_with_info_, + KeyWithInfoPtr key_with_info_, String bucket_, std::unique_ptr read_buf_, std::shared_ptr source_, std::unique_ptr pipeline_, std::unique_ptr reader_) - : key_with_info(std::move(key_with_info_)) + : key_with_info(key_with_info_) , bucket(std::move(bucket_)) , read_buf(std::move(read_buf_)) , source(std::move(source_)) @@ -215,14 +215,14 @@ private: explicit operator bool() const { return reader != nullptr; } PullingPipelineExecutor * operator->() { return reader.get(); } const PullingPipelineExecutor * operator->() const { return reader.get(); } - String getPath() const { return fs::path(bucket) / key_with_info.key; } - const String & getFile() const { return key_with_info.key; } - const KeyWithInfo & getKeyWithInfo() const { return key_with_info; } + String getPath() const { return fs::path(bucket) / key_with_info->key; } + const String & getFile() const { return key_with_info->key; } + const KeyWithInfo & getKeyWithInfo() const { return *key_with_info; } const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } private: - KeyWithInfo key_with_info; + KeyWithInfoPtr key_with_info; String bucket; std::unique_ptr read_buf; std::shared_ptr source; From e0ff76a7f968c8fa0511f55f81907e8b97880984 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 18:44:53 +0200 Subject: [PATCH 019/120] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 24 +++++++++++-------- src/Storages/S3Queue/S3QueueSource.cpp | 1 - src/Storages/S3Queue/StorageS3Queue.cpp | 9 ++++++- .../integration/test_storage_s3_queue/test.py | 7 +++--- 4 files changed, 26 insertions(+), 15 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 0492a8c2f4d..e82345dd4fb 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -430,11 +430,12 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder return; } - /// TODO this could be because of the expired session. if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is not processing"); - else throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is already processed"); + + LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " + "does not exist with expected processing id does not exist, " + "this could be a result of expired zookeeper session", path); } void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder) @@ -472,8 +473,9 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) continue; - LOG_ERROR(log, "Cannot set file {} as failed - failed to remove ephemeral processing node", path); - chassert(false); + LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " + "does not exist with expected processing id does not exist, " + "this could be a result of expired zookeeper session", path); return; } } @@ -531,7 +533,6 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S { /// File is no longer retriable. /// Make a failed/node_name node and remove failed/node_name.retriable node. - /// TODO: always add version for processing node. Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); @@ -595,9 +596,9 @@ S3QueueFilesMetadata::ProcessingNodeHolder::~ProcessingNodeHolder() bool S3QueueFilesMetadata::ProcessingNodeHolder::remove(Coordination::Requests * requests, Coordination::Responses * responses) { if (removed) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing file holder is already released"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing node is already removed"); - removed = true; + LOG_TEST(log, "Removing processing node {} ({})", zk_node_path, path); try { @@ -617,11 +618,14 @@ bool S3QueueFilesMetadata::ProcessingNodeHolder::remove(Coordination::Requests * { requests->push_back(zkutil::makeRemoveRequest(zk_node_path, stat.version)); auto code = zk_client->tryMulti(*requests, *responses); - return code == Coordination::Error::ZOK; + removed = code == Coordination::Error::ZOK; } else + { zk_client->remove(zk_node_path); - return true; + removed = true; + } + return removed; } else LOG_WARNING(log, "Cannot remove {} since precessing id changed: {} -> {}", diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 4171597b619..11415a40302 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -139,7 +139,6 @@ Chunk StorageS3QueueSource::generate() LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getPath()); file_status->processed_rows += chunk.getNumRows(); - // file_status->profile_counters.increment(ProfileEvents::S3QueuePullMicroseconds, timer.get()); processed_rows_from_file += chunk.getNumRows(); VirtualColumnUtils::addRequestedPathAndFileVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath()); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 553211f7772..6bed73b0315 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -101,6 +101,12 @@ StorageS3Queue::StorageS3Queue( throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); } + if (s3queue_settings->mode == S3QueueMode::ORDERED && s3queue_settings->s3queue_processing_threads_num > 1) + { + LOG_WARNING(log, "Parallel processing is not yet supported for Ordered mode"); + s3queue_settings->s3queue_processing_threads_num = 1; + } + configuration.update(context_); FormatFactory::instance().checkFormatName(configuration.format); context_->getRemoteHostFilter().checkURL(configuration.url.uri); @@ -181,7 +187,8 @@ Pipe StorageS3Queue::read( } Pipes pipes; - for (size_t i = 0; i < num_streams; ++i) + const size_t adjusted_num_streams = std::min(num_streams, s3queue_settings->s3queue_processing_threads_num); + for (size_t i = 0; i < adjusted_num_streams; ++i) pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); return Pipe::unitePipes(std::move(pipes)); } diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 4c3e9b1753e..7d8cfd68dd2 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -558,7 +558,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): }, ) except QueryRuntimeException as e: - assert "Existing table metadata in ZooKeeper differs in engine mode" in str(e) + assert "Metadata with the same `s3queue_zookeeper_path` was already created but with different settings" in str(e) failed = True assert failed is True @@ -836,15 +836,16 @@ def test_max_set_size(started_cluster): "s3queue_tracked_files_limit": 9, "s3queue_cleanup_interval_min_ms": 0, "s3queue_cleanup_interval_max_ms": 0, + "s3queue_processing_threads_num": 1, }, ) total_values = generate_random_files( started_cluster, files_path, files_to_generate, start_ind=0, row_num=1 ) - get_query = f"SELECT * FROM {table_name}" + get_query = f"SELECT * FROM {table_name} ORDER BY column1, column2, column3" res1 = [list(map(int, l.split())) for l in run_query(node, get_query).splitlines()] - assert res1 == total_values + assert res1 == sorted(total_values, key=lambda x: (x[0], x[1], x[2])) print(total_values) time.sleep(10) From 42783894b75661fde201806543dafddc00a387a6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 20:07:19 +0200 Subject: [PATCH 020/120] Add comments --- src/Interpreters/S3QueueLog.cpp | 2 + src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 179 ++++++++++++------ src/Storages/S3Queue/S3QueueFilesMetadata.h | 22 ++- src/Storages/S3Queue/S3QueueSource.cpp | 1 - src/Storages/S3Queue/S3QueueSource.h | 1 - 5 files changed, 141 insertions(+), 64 deletions(-) diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp index caccdabf503..78c3c8dadf7 100644 --- a/src/Interpreters/S3QueueLog.cpp +++ b/src/Interpreters/S3QueueLog.cpp @@ -33,6 +33,8 @@ NamesAndTypesList S3QueueLogElement::getNamesAndTypes() }; } +/// TODO add last_exception column + void S3QueueLogElement::appendToBlock(MutableColumns & columns) const { size_t i = 0; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index e82345dd4fb..c2a74e45dab 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -64,7 +64,8 @@ S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::LocalFileStatuses::getA return file_statuses; } -std::shared_ptr S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) +std::shared_ptr +S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) { auto lk = lock(); auto it = file_statuses.find(filename); @@ -162,11 +163,16 @@ zkutil::ZooKeeperPtr S3QueueFilesMetadata::getZooKeeper() const std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) { + /// Return a locally cached file status. return local_file_statuses.get(path, /* create */false); } std::string S3QueueFilesMetadata::getNodeName(const std::string & path) { + /// Since with are dealing with paths in s3 which can have "/", + /// we cannot create a zookeeper node with the name equal to path. + /// Therefore we use a hash of the path as a node name. + SipHash path_hash; path_hash.update(path); return toString(path_hash.get64()); @@ -177,6 +183,13 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( const std::string & exception, size_t retries) { + /// Create a metadata which will be stored in a node named as getNodeName(path). + + /// Since node name is just a hash we want to know to which file it corresponds, + /// so we keep "file_path" in nodes data. + /// "last_processed_timestamp" is needed for TTL metadata nodes enabled by s3queue_tracked_file_ttl_sec. + /// "last_exception" is kept for introspection, should also be visible in system.s3queue_log if it is enabled. + /// "retries" is kept for retrying the processing enabled by s3queue_loading_retries. NodeMetadata metadata; metadata.file_path = path; metadata.last_processed_timestamp = getCurrentTime(); @@ -191,95 +204,111 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs auto file_status = local_file_statuses.get(path, /* create */true); /// Check locally cached file status. + /// Processed or Failed state is always cached. + /// Processing state is cached only if processing is being done by current clickhouse server + /// (because If another server is doing the processing, + /// we cannot know if state changes without checking with zookeeper so there is no point in cache here). switch (file_status->state) { case FileStatus::State::Processing: [[fallthrough]]; case FileStatus::State::Processed: { - /// File is already processes or processing by current server. return nullptr; } case FileStatus::State::Failed: { - /// max_loading_retries == 0 => file is not retriable. - /// file_status->retries is a cached value, so in case file_status->retries >= max_loading retries - /// we can fully rely that it is true, but in other case the value might be outdated, - /// but this is ok, we will recheck with zookeeper. - if (!max_loading_retries || file_status->retries >= max_loading_retries) + /// If max_loading_retries == 0, file is not retriable. + if (max_loading_retries == 0) return nullptr; + + /// Otherwise file_status->retries is also cached. + /// In case file_status->retries >= max_loading_retries we can fully rely that it is true + /// and will not attempt processing it. + /// But in case file_status->retries < max_loading_retries we cannot be sure + /// (another server could have done a try after we cached retries value), + /// so check with zookeeper here. + if (file_status->retries >= max_loading_retries) + return nullptr; + break; } case FileStatus::State::None: { - /// The file was not processed by current server, + /// The file was not processed by current server and file status was not cached, /// check metadata in zookeeper. break; } } + + /// Another thread could already be trying to set file as processing. + /// So there is no need to attempt the same, better to continue with the next file. std::unique_lock lock(file_status->processing_lock, std::defer_lock); if (!lock.try_lock()) { - /// Another thread is already trying to set file as processing. return nullptr; } + /// Let's go and check metadata in zookeeper and try to create a /processing ephemeral node. + /// If successful, return result with processing node holder. SetFileProcessingResult result; - ProcessingNodeHolderPtr processing_holder; + ProcessingNodeHolderPtr processing_node_holder; + switch (mode) { case S3QueueMode::ORDERED: { - std::tie(result, processing_holder) = trySetFileAsProcessingForOrderedMode(path); + std::tie(result, processing_node_holder) = trySetFileAsProcessingForOrderedMode(path); break; } case S3QueueMode::UNORDERED: { - std::tie(result, processing_holder) = trySetFileAsProcessingForUnorderedMode(path); + std::tie(result, processing_node_holder) = trySetFileAsProcessingForUnorderedMode(path); break; } } + + /// Cache file status, save some statistics. switch (result) { case SetFileProcessingResult::Success: { file_status->state = FileStatus::State::Processing; + file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); timer.cancel(); + if (!file_status->processing_start_time) file_status->processing_start_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + break; } case SetFileProcessingResult::AlreadyProcessed: { - /// Cache the state. file_status->state = FileStatus::State::Processed; break; } case SetFileProcessingResult::AlreadyFailed: { - /// Cache the state. file_status->state = FileStatus::State::Failed; break; } case SetFileProcessingResult::ProcessingByOtherNode: { - /// We cannot save any local state. + /// We cannot save any local state here, see comment above. break; } } - if (result != SetFileProcessingResult::Success) - return nullptr; - - return processing_holder; + return result == SetFileProcessingResult::Success ? processing_node_holder : nullptr; } -std::pair -S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) +std::pair S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & path) { - /// Create an ephemenral node in /processing - /// if corresponding node does not exist in failed/, processed/ and processing/. - /// Return false otherwise. + /// In one zookeeper transaction do the following: + /// 1. check that corresponding persistent nodes do not exist in processed/ and failed/; + /// 2. create an ephemenral node in /processing if it does not exist; + /// Return corresponding status if any of the step failed. const auto node_name = getNodeName(path); const auto zk_client = getZooKeeper(); @@ -310,12 +339,14 @@ S3QueueFilesMetadata::trySetFileAsProcessingForUnorderedMode(const std::string & return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; } -std::pair -S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) +std::pair S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & path) { - /// Create an ephemenral node in /processing - /// if corresponding it does not exist in failed/, processing/ and satisfied max processed file check. - /// Return false otherwise. + /// Same as for Unordered mode. + /// The only difference is the check if the file is already processed. + /// For Ordered mode we do not keep a separate /processed/hash_node for each file + /// but instead we only keep a maximum processed file + /// (since all files are ordered and new files have a lexically bigger name, it makes sense). const auto node_name = getNodeName(path); const auto zk_client = getZooKeeper(); @@ -324,25 +355,13 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p while (true) { - Coordination::Requests requests; - zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); - zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_processing_path / node_name); - - Coordination::Responses responses; - auto code = zk_client->tryMulti(requests, responses); - if (code != Coordination::Error::ZOK) - { - if (responses[0]->error == Coordination::Error::ZOK) - { - LOG_TEST(log, "Skipping file `{}`: already processing", path); - return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; - } - else - { - LOG_TEST(log, "Skipping file `{}`: failed", path); - return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; - } - } + /// Get a /processed node content - max_processed path. + /// Compare our path to it. + /// If file is not yet processed, check corresponding /failed node and try create /processing node + /// and in the same zookeeper transaction als check that /processed node did not change + /// in between, e.g. that stat.version remained the same. + /// If the version did change - retry (since we cannot do Get and Create requests + /// in the same zookeeper transaction, so we use a while loop with tries). Coordination::Stat processed_node_stat; auto data = zk_client->get(zookeeper_processed_path, &processed_node_stat); @@ -354,14 +373,13 @@ S3QueueFilesMetadata::trySetFileAsProcessingForOrderedMode(const std::string & p if (!max_processed_file_path.empty() && path <= max_processed_file_path) return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; - requests.clear(); - responses.clear(); - + Coordination::Requests requests; zkutil::addCheckNotExistsRequest(requests, *zk_client, zookeeper_failed_path / node_name); requests.push_back(zkutil::makeCreateRequest(zookeeper_processing_path / node_name, node_metadata.toString(), zkutil::CreateMode::Ephemeral)); requests.push_back(zkutil::makeCheckRequest(zookeeper_processed_path, processed_node_stat.version)); - code = zk_client->tryMulti(requests, responses); + Coordination::Responses responses; + auto code = zk_client->tryMulti(requests, responses); if (code == Coordination::Error::ZOK) { auto holder = std::make_unique(node_metadata.processing_id, path, zookeeper_processing_path / node_name, zk_client); @@ -431,15 +449,21 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder } if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to set file as processed but it is already processed"); + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot create a persistent node in /processed since it already exists"); + } - LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " - "does not exist with expected processing id does not exist, " + LOG_WARNING(log, + "Cannot set file ({}) as processed since ephemeral node in /processing" + "does not exist with expected id, " "this could be a result of expired zookeeper session", path); } void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPtr holder) { + /// Update a presistent node in /processed and remove ephemeral node from /processing. + const auto & path = holder->path; const auto node_name = getNodeName(path); const auto node_metadata = createNodeMetadata(path).toString(); @@ -457,7 +481,20 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt { auto metadata = NodeMetadata::fromString(res); if (metadata.file_path >= path) + { + /// Here we get in the case that maximum processed file is bigger than ours. + /// This is possible to achive in case of parallel processing + /// but for local processing we explicitly disable parallel mode and do everything in a single thread + /// (see constructor of StorageS3Queue where s3queue_processing_threads_num is explicitly set to 1 in case of Ordered mode). + /// Nevertheless, in case of distributed processing we cannot do anything with parallelism. + /// What this means? + /// It means that in scenario "distributed processing + Ordered mode" + /// a setting s3queue_loading_retries will not work. It is possible to fix, it is in TODO. + + /// Return because there is nothing to change, + /// the max processed file is already bigger than ours. return; + } } requests.push_back(zkutil::makeSetRequest(zookeeper_processed_path, node_metadata, stat.version)); } @@ -470,6 +507,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (holder->remove(&requests, &responses)) return; + /// Failed to update max processed node, retry. if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) continue; @@ -497,13 +535,16 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S auto node_metadata = createNodeMetadata(path, exception_message); const auto zk_client = getZooKeeper(); + /// Is file retriable? if (max_loading_retries == 0) { + /// File is not retriable, + /// just create a node in /failed and remove a node from /processing. + Coordination::Requests requests; requests.push_back(zkutil::makeCreateRequest(zookeeper_failed_path / node_name, node_metadata.toString(), zkutil::CreateMode::Persistent)); - Coordination::Responses responses; if (holder->remove(&requests, &responses)) { @@ -512,13 +553,30 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S return; } - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to set file as failed"); + if (responses[0]->error != Coordination::Error::ZOK) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot create a persistent node in /failed since it already exists"); + } + + LOG_WARNING(log, "Cannot set file ({}) as processed since processing node " + "does not exist with expected processing id does not exist, " + "this could be a result of expired zookeeper session", path); + return; } - const auto node_name_with_retriable_suffix = node_name + ".retriable"; + /// So file is retriable. + /// Let's do an optimization here. + /// Instead of creating a persistent /failed/node_hash node + /// we create a persistent /failed/node_hash.retriable node. + /// This allows us to make less zookeeper requests as we avoid checking + /// the number of already done retries in trySetFileAsProcessing. + const auto node_name_with_retriable_suffix = node_name + ".retriable"; Coordination::Stat stat; std::string res; + + /// Extract the number of already done retries from node_hash.retriable node if it exists. if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) { auto failed_node_metadata = NodeMetadata::fromString(res); @@ -529,10 +587,11 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S LOG_TEST(log, "File `{}` failed to process, try {}/{} (Error: {})", path, node_metadata.retries, max_loading_retries, exception_message); + /// Check if file can be retried futher or not. if (node_metadata.retries >= max_loading_retries) { /// File is no longer retriable. - /// Make a failed/node_name node and remove failed/node_name.retriable node. + /// Make a persistent node /failed/node_hash, remove /failed/node_hash.retriable node and node in /processing. Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); @@ -551,6 +610,8 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S } else { + /// File is still retriable, update retries count and remove node from /processing. + Coordination::Requests requests; requests.push_back(zkutil::makeRemoveRequest(zookeeper_processing_path / node_name, -1)); if (node_metadata.retries == 0) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index dd760b29299..e9870292348 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -15,6 +15,22 @@ namespace DB struct S3QueueSettings; class StorageS3Queue; +/** + * A class for managing S3Queue metadata in zookeeper, e.g. + * the following folders: + * - /processing + * - /processed + * - /failed + * + * Depending on S3Queue processing mode (ordered or unordered) + * we can differently store metadata in /processed node. + * + * Implements caching of zookeeper metadata for faster responses. + * Cached part is located in LocalFileStatuses. + * + * In case of Unordered mode - if files TTL is enabled or maximum tracked files limit is set + * starts a background cleanup thread which is responsible for maintaining them. + */ class S3QueueFilesMetadata { public: @@ -25,6 +41,7 @@ public: ~S3QueueFilesMetadata(); + /// Set file as processing, if it is not alreaty processed, failed or processing. ProcessingNodeHolderPtr trySetFileAsProcessing(const std::string & path); void setFileProcessed(ProcessingNodeHolderPtr holder); @@ -33,7 +50,6 @@ public: struct FileStatus { - size_t processed_rows = 0; enum class State { Processing, @@ -42,12 +58,12 @@ public: None }; State state = State::None; - ProfileEvents::Counters profile_counters; + size_t processed_rows = 0; time_t processing_start_time = 0; time_t processing_end_time = 0; - size_t retries = 0; + ProfileEvents::Counters profile_counters; std::mutex processing_lock; }; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 11415a40302..a1868238bc3 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -91,7 +91,6 @@ StorageS3QueueSource::StorageS3QueueSource( , shutdown_called(shutdown_called_) , s3_queue_log(s3_queue_log_) , storage_id(storage_id_) - , s3_queue_user_id(fmt::format("{}:{}", CurrentThread::getQueryId(), getRandomASCIIString(8))) , remove_file_func(remove_file_func_) , log(&Poco::Logger::get("StorageS3QueueSource")) { diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 304b7d3b661..fa21b6cdd59 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -80,7 +80,6 @@ private: const std::atomic & shutdown_called; const std::shared_ptr s3_queue_log; const StorageID storage_id; - const std::string s3_queue_user_id; RemoveFileFunc remove_file_func; Poco::Logger * log; From c579f5b32127e5939292c6d8c90dc4d7b38ec765 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 20:10:05 +0200 Subject: [PATCH 021/120] Fix style check --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 2 +- tests/integration/test_storage_s3_queue/test.py | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index c2a74e45dab..aff4a91bf7f 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -546,7 +546,7 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S node_metadata.toString(), zkutil::CreateMode::Persistent)); Coordination::Responses responses; - if (holder->remove(&requests, &responses)) + if (holder->remove(&requests, &responses)) { LOG_TEST(log, "File `{}` failed to process and will not be retried. " "Error: {}", path, exception_message); diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 7d8cfd68dd2..079311422cc 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -558,7 +558,10 @@ def test_multiple_tables_meta_mismatch(started_cluster): }, ) except QueryRuntimeException as e: - assert "Metadata with the same `s3queue_zookeeper_path` was already created but with different settings" in str(e) + assert ( + "Metadata with the same `s3queue_zookeeper_path` was already created but with different settings" + in str(e) + ) failed = True assert failed is True From 57cfb88cb6e79e41476002773bc49c18675e8772 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Sep 2023 22:22:49 +0200 Subject: [PATCH 022/120] Fix typos check, fix build wihtout s3 --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 10 +++++----- src/Storages/System/StorageSystemS3Queue.cpp | 4 ---- src/Storages/System/StorageSystemS3Queue.h | 4 ---- 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index aff4a91bf7f..5d2ac9514bc 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -358,7 +358,7 @@ std::pairpath; const auto node_name = getNodeName(path); @@ -483,7 +483,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (metadata.file_path >= path) { /// Here we get in the case that maximum processed file is bigger than ours. - /// This is possible to achive in case of parallel processing + /// This is possible to achieve in case of parallel processing /// but for local processing we explicitly disable parallel mode and do everything in a single thread /// (see constructor of StorageS3Queue where s3queue_processing_threads_num is explicitly set to 1 in case of Ordered mode). /// Nevertheless, in case of distributed processing we cannot do anything with parallelism. @@ -587,7 +587,7 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S LOG_TEST(log, "File `{}` failed to process, try {}/{} (Error: {})", path, node_metadata.retries, max_loading_retries, exception_message); - /// Check if file can be retried futher or not. + /// Check if file can be retried further or not. if (node_metadata.retries >= max_loading_retries) { /// File is no longer retriable. @@ -689,7 +689,7 @@ bool S3QueueFilesMetadata::ProcessingNodeHolder::remove(Coordination::Requests * return removed; } else - LOG_WARNING(log, "Cannot remove {} since precessing id changed: {} -> {}", + LOG_WARNING(log, "Cannot remove {} since processing id changed: {} -> {}", zk_node_path, processing_id, node_metadata.processing_id); } else diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index faba5bc671b..ce76469b72a 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -1,7 +1,5 @@ #include "StorageSystemS3Queue.h" -#if USE_AWS_S3 - #include #include #include @@ -67,5 +65,3 @@ void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, co } } - -#endif diff --git a/src/Storages/System/StorageSystemS3Queue.h b/src/Storages/System/StorageSystemS3Queue.h index 208891aef42..1bb4e3694d2 100644 --- a/src/Storages/System/StorageSystemS3Queue.h +++ b/src/Storages/System/StorageSystemS3Queue.h @@ -1,8 +1,6 @@ #pragma once #include "config.h" -#if USE_AWS_S3 - #include #include @@ -23,5 +21,3 @@ protected: }; } - -#endif From 1749874e7bdbd4436bc2d8d49eccd556745861da Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Sep 2023 13:51:07 +0200 Subject: [PATCH 023/120] Fxi --- src/Storages/StorageS3Cluster.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 1dd1a410595..8649c3f1baa 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -82,7 +82,12 @@ RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr { auto iterator = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); - auto callback = std::make_shared>([iterator]() mutable -> String { return iterator->next()->key; }); + auto callback = std::make_shared>([iterator]() mutable -> String + { + if (auto next = iterator->next()) + return next->key; + return ""; + }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } From 7d91ba109ae1147048bee73b4353292739a7fbe4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Sep 2023 13:58:59 +0200 Subject: [PATCH 024/120] Update doc --- .../table-engines/integrations/s3queue.md | 36 +++++++++++++------ src/Storages/S3Queue/S3QueueSettings.h | 2 +- 2 files changed, 26 insertions(+), 12 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 6ceea8d7dc9..7edc12145e8 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -17,14 +17,17 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) [after_processing = 'keep',] [keeper_path = '',] [s3queue_loading_retries = 0,] + [s3queue_processing_threads_num = 1,] + [s3queue_enable_logging_to_s3queue_log = 0,] [s3queue_polling_min_timeout_ms = 1000,] [s3queue_polling_max_timeout_ms = 10000,] [s3queue_polling_backoff_ms = 0,] - [s3queue_tracked_files_limit = 1000,] [s3queue_tracked_file_ttl_sec = 0,] - [s3queue_polling_size = 50,] + [s3queue_tracked_files_limit = 1000,] + [s3queue_cleanup_interval_min_ms = 10000,] + [s3queue_cleanup_interval_max_ms = 30000,] ``` - + * [ ] **Engine parameters** - `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). @@ -39,7 +42,7 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) CREATE TABLE s3queue_engine_table (name String, value UInt32) ENGINE=S3Queue('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*', 'CSV', 'gzip') SETTINGS - mode = 'ordered'; + mode = 'unordered'; ``` Using named collections: @@ -102,6 +105,18 @@ Possible values: Default value: `0`. +### s3queue_processing_threads_num {#processing_threads_num} + +Number of threads to perform processing. Applies only for `Unordered` mode. + +Default value: `1`. + +### s3queue_enable_logging_to_s3queue_log {#enable_logging_to_s3queue_log} + +Enable logging to `system.s3queue_log`. + +Default value: `0`. + ### s3queue_polling_min_timeout_ms {#polling_min_timeout_ms} Minimal timeout before next polling (in milliseconds). @@ -154,18 +169,17 @@ Possible values: Default value: `0`. -### s3queue_polling_size {#polling_size} +### s3queue_cleanup_interval_min_ms {#cleanup_interval_min_ms} -Maximum files to fetch from S3 with SELECT or in background task. -Engine takes files for processing from S3 in batches. -We limit the batch size to increase concurrency if multiple table engines with the same `keeper_path` consume files from the same path. +For 'Ordered' mode. Defines a minimum boundary for reschedule interval for a background task, which is responsible for maintaining tracked file TTL and maximum tracked files set. -Possible values: +Default value: `10000`. -- Positive integer. +### s3queue_cleanup_interval_max_ms {#cleanup_interval_max_ms} -Default value: `50`. +For 'Ordered' mode. Defines a maximum boundary for reschedule interval for a background task, which is responsible for maintaining tracked file TTL and maximum tracked files set. +Default value: `30000`. ## S3-related Settings {#s3-settings} diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 6d7ab4187a5..e6a85a834fc 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -20,9 +20,9 @@ class ASTStorage; M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete or keep file in S3 after successful processing", 0) \ M(String, keeper_path, "", "Zookeeper node path", 0) \ M(UInt32, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ - M(UInt32, s3queue_tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ M(UInt32, s3queue_processing_threads_num, 1, "Number of processing threads", 0) \ M(UInt32, s3queue_enable_logging_to_s3queue_log, 0, "Enable logging to system table system.s3queue_log", 0) \ + M(UInt32, s3queue_tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ M(UInt32, s3queue_polling_backoff_ms, 1000, "Polling backoff", 0) \ From 4a7922507ba2475bb4c3c5162b3952dba0173e93 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Sep 2023 16:18:00 +0200 Subject: [PATCH 025/120] Minor changes --- docs/en/engines/table-engines/integrations/s3queue.md | 2 +- src/Interpreters/Cache/Metadata.cpp | 1 - src/Interpreters/S3QueueLog.cpp | 5 +++-- src/Interpreters/S3QueueLog.h | 1 + src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 8 +++++--- src/Storages/S3Queue/S3QueueFilesMetadata.h | 1 + src/Storages/S3Queue/S3QueueSource.cpp | 5 +++-- src/Storages/StorageS3.h | 3 +++ 8 files changed, 17 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 7edc12145e8..7c8c35d1b5c 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -27,7 +27,7 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) [s3queue_cleanup_interval_min_ms = 10000,] [s3queue_cleanup_interval_max_ms = 30000,] ``` - * [ ] + **Engine parameters** - `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index c251cb2dec2..7a9321e4215 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include namespace fs = std::filesystem; diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp index 78c3c8dadf7..af40d1dd1e8 100644 --- a/src/Interpreters/S3QueueLog.cpp +++ b/src/Interpreters/S3QueueLog.cpp @@ -30,11 +30,10 @@ NamesAndTypesList S3QueueLogElement::getNamesAndTypes() {"processing_start_time", std::make_shared(std::make_shared())}, {"processing_end_time", std::make_shared(std::make_shared())}, {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + {"exception", std::make_shared()}, }; } -/// TODO add last_exception column - void S3QueueLogElement::appendToBlock(MutableColumns & columns) const { size_t i = 0; @@ -55,6 +54,8 @@ void S3QueueLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertDefault(); ProfileEvents::dumpToMapColumn(counters_snapshot, columns[i++].get(), true); + + columns[i++]->insert(exception); } } diff --git a/src/Interpreters/S3QueueLog.h b/src/Interpreters/S3QueueLog.h index b28ef1b82fc..76ff5ca0cdc 100644 --- a/src/Interpreters/S3QueueLog.h +++ b/src/Interpreters/S3QueueLog.h @@ -24,6 +24,7 @@ struct S3QueueLogElement ProfileEvents::Counters::Snapshot counters_snapshot; time_t processing_start_time; time_t processing_end_time; + std::string exception; static std::string name() { return "S3QueueLog"; } diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 5d2ac9514bc..dbbf7e2ee1d 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -524,11 +524,13 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S const auto & path = holder->path; auto file_status = local_file_statuses.get(path, /* create */false); + file_status->state = FileStatus::State::Failed; + file_status->last_exception = exception_message; + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + SCOPE_EXIT({ - file_status->state = FileStatus::State::Failed; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); timer.cancel(); - file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); }); const auto node_name = getNodeName(path); @@ -575,11 +577,11 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S const auto node_name_with_retriable_suffix = node_name + ".retriable"; Coordination::Stat stat; std::string res; + auto failed_node_metadata = NodeMetadata::fromString(res); /// Extract the number of already done retries from node_hash.retriable node if it exists. if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) { - auto failed_node_metadata = NodeMetadata::fromString(res); node_metadata.retries = failed_node_metadata.retries + 1; file_status->retries = node_metadata.retries; } diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index e9870292348..f6d98d13aaf 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -63,6 +63,7 @@ public: time_t processing_start_time = 0; time_t processing_end_time = 0; size_t retries = 0; + std::string last_exception; ProfileEvents::Counters profile_counters; std::mutex processing_lock; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index a1868238bc3..09713e77f37 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -128,6 +128,8 @@ Chunk StorageS3QueueSource::generate() auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); + const auto * key_with_info = dynamic_cast(&reader.getKeyWithInfo()); + try { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueuePullMicroseconds); @@ -148,14 +150,12 @@ Chunk StorageS3QueueSource::generate() { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); - const StorageS3QueueSource::S3QueueKeyWithInfo * key_with_info = assert_cast(&reader.getKeyWithInfo()); files_metadata->setFileFailed(key_with_info->processing_holder, e.message()); appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); throw; } - const StorageS3QueueSource::S3QueueKeyWithInfo * key_with_info = assert_cast(&reader.getKeyWithInfo()); files_metadata->setFileProcessed(key_with_info->processing_holder); applyActionAfterProcessing(reader.getFile()); @@ -215,6 +215,7 @@ void StorageS3QueueSource::appendLogElement(const std::string & filename, const .counters_snapshot = file_status_.profile_counters.getPartiallyAtomicSnapshot(), .processing_start_time = file_status_.processing_start_time, .processing_end_time = file_status_.processing_end_time, + .exception = file_status_.last_exception, }; s3_queue_log->add(std::move(elem)); } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index c053c6f9bd2..adcfa6ebdb9 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -43,9 +43,12 @@ public: struct KeyWithInfo { KeyWithInfo() = default; + explicit KeyWithInfo(String key_, std::optional info_ = std::nullopt) : key(std::move(key_)), info(std::move(info_)) {} + virtual ~KeyWithInfo() = default; + String key; std::optional info; }; From d6449921921e0b432dd8f246f376df155d47c55d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Sep 2023 16:25:04 +0200 Subject: [PATCH 026/120] Fxi --- src/Storages/StorageS3.cpp | 6 ++---- src/Storages/StorageS3Cluster.cpp | 1 + 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 1bb2dccbccc..8d391a02d30 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -570,7 +570,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() do { key_with_info = (*file_iterator)(); - if (!key_with_info) + if (!key_with_info || key_with_info->key.empty()) return {}; if (!key_with_info->info) @@ -1494,7 +1494,7 @@ namespace { current_key_with_info = (*file_iterator)(); - if (!current_key_with_info) + if (!current_key_with_info || current_key_with_info->key.empty()) { if (first) throw Exception( @@ -1506,8 +1506,6 @@ namespace return nullptr; } - chassert(!current_key_with_info->key.empty()); - /// S3 file iterator could get new keys after new iteration, check them in schema cache. if (getContext()->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size) { diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 8649c3f1baa..824dae6bc3e 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -82,6 +82,7 @@ RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(ASTPtr { auto iterator = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); + auto callback = std::make_shared>([iterator]() mutable -> String { if (auto next = iterator->next()) From 1ef21bab3d7f64c0594a61bb73b640c2651baf64 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Sep 2023 16:02:51 +0200 Subject: [PATCH 027/120] Fix data race --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 81 +++++++++++-------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 3 +- src/Storages/S3Queue/S3QueueSource.cpp | 26 +++--- src/Storages/S3Queue/S3QueueSource.h | 2 +- src/Storages/System/StorageSystemS3Queue.cpp | 5 +- 5 files changed, 71 insertions(+), 46 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index dbbf7e2ee1d..1856490c92e 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -208,42 +208,46 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs /// Processing state is cached only if processing is being done by current clickhouse server /// (because If another server is doing the processing, /// we cannot know if state changes without checking with zookeeper so there is no point in cache here). - switch (file_status->state) + { - case FileStatus::State::Processing: [[fallthrough]]; - case FileStatus::State::Processed: + std::lock_guard lock(file_status->metadata_lock); + switch (file_status->state) { - return nullptr; - } - case FileStatus::State::Failed: - { - /// If max_loading_retries == 0, file is not retriable. - if (max_loading_retries == 0) + case FileStatus::State::Processing: [[fallthrough]]; + case FileStatus::State::Processed: + { return nullptr; + } + case FileStatus::State::Failed: + { + /// If max_loading_retries == 0, file is not retriable. + if (max_loading_retries == 0) + return nullptr; - /// Otherwise file_status->retries is also cached. - /// In case file_status->retries >= max_loading_retries we can fully rely that it is true - /// and will not attempt processing it. - /// But in case file_status->retries < max_loading_retries we cannot be sure - /// (another server could have done a try after we cached retries value), - /// so check with zookeeper here. - if (file_status->retries >= max_loading_retries) - return nullptr; + /// Otherwise file_status->retries is also cached. + /// In case file_status->retries >= max_loading_retries we can fully rely that it is true + /// and will not attempt processing it. + /// But in case file_status->retries < max_loading_retries we cannot be sure + /// (another server could have done a try after we cached retries value), + /// so check with zookeeper here. + if (file_status->retries >= max_loading_retries) + return nullptr; - break; - } - case FileStatus::State::None: - { - /// The file was not processed by current server and file status was not cached, - /// check metadata in zookeeper. - break; + break; + } + case FileStatus::State::None: + { + /// The file was not processed by current server and file status was not cached, + /// check metadata in zookeeper. + break; + } } } /// Another thread could already be trying to set file as processing. /// So there is no need to attempt the same, better to continue with the next file. - std::unique_lock lock(file_status->processing_lock, std::defer_lock); - if (!lock.try_lock()) + std::unique_lock processing_lock(file_status->processing_lock, std::defer_lock); + if (!processing_lock.try_lock()) { return nullptr; } @@ -272,6 +276,7 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs { case SetFileProcessingResult::Success: { + std::lock_guard lock(file_status->metadata_lock); file_status->state = FileStatus::State::Processing; file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessingMicroseconds, timer.get()); @@ -284,11 +289,13 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs } case SetFileProcessingResult::AlreadyProcessed: { + std::lock_guard lock(file_status->metadata_lock); file_status->state = FileStatus::State::Processed; break; } case SetFileProcessingResult::AlreadyFailed: { + std::lock_guard lock(file_status->metadata_lock); file_status->state = FileStatus::State::Failed; break; } @@ -408,12 +415,16 @@ void S3QueueFilesMetadata::setFileProcessed(ProcessingNodeHolderPtr holder) auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessedMicroseconds); const auto & path = holder->path; - SCOPE_EXIT({ - auto file_status = local_file_statuses.get(path, /* create */false); + auto file_status = local_file_statuses.get(path, /* create */false); + { + std::lock_guard lock(file_status->metadata_lock); file_status->state = FileStatus::State::Processed; + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + } + + SCOPE_EXIT({ file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileProcessedMicroseconds, timer.get()); timer.cancel(); - file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); }); switch (mode) @@ -524,9 +535,12 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S const auto & path = holder->path; auto file_status = local_file_statuses.get(path, /* create */false); - file_status->state = FileStatus::State::Failed; - file_status->last_exception = exception_message; - file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + { + std::lock_guard lock(file_status->metadata_lock); + file_status->state = FileStatus::State::Failed; + file_status->last_exception = exception_message; + file_status->processing_end_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); + } SCOPE_EXIT({ file_status->profile_counters.increment(ProfileEvents::S3QueueSetFileFailedMicroseconds, timer.get()); @@ -583,6 +597,8 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) { node_metadata.retries = failed_node_metadata.retries + 1; + + std::lock_guard lock(file_status->metadata_lock); file_status->retries = node_metadata.retries; } @@ -771,6 +787,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Cleanup is already being executed by another node"); return; } + /// TODO because of this lock we might not update local file statuses on time on one of the nodes. struct Node { diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index f6d98d13aaf..7eb5c40eb26 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -59,7 +59,7 @@ public: }; State state = State::None; - size_t processed_rows = 0; + std::atomic processed_rows = 0; time_t processing_start_time = 0; time_t processing_end_time = 0; size_t retries = 0; @@ -67,6 +67,7 @@ public: ProfileEvents::Counters profile_counters; std::mutex processing_lock; + std::mutex metadata_lock; }; using FileStatuses = std::unordered_map>; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 09713e77f37..d86fc6fe1ce 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -201,22 +201,26 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & path) } } -void StorageS3QueueSource::appendLogElement(const std::string & filename, const S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed) +void StorageS3QueueSource::appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed) { if (!s3_queue_log) return; - S3QueueLogElement elem + S3QueueLogElement elem{}; { - .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), - .file_name = filename, - .rows_processed = processed_rows, - .status = processed ? S3QueueLogElement::S3QueueStatus::Processed : S3QueueLogElement::S3QueueStatus::Failed, - .counters_snapshot = file_status_.profile_counters.getPartiallyAtomicSnapshot(), - .processing_start_time = file_status_.processing_start_time, - .processing_end_time = file_status_.processing_end_time, - .exception = file_status_.last_exception, - }; + std::lock_guard lock(file_status_.metadata_lock); + elem = S3QueueLogElement + { + .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), + .file_name = filename, + .rows_processed = processed_rows, + .status = processed ? S3QueueLogElement::S3QueueStatus::Processed : S3QueueLogElement::S3QueueStatus::Failed, + .counters_snapshot = file_status_.profile_counters.getPartiallyAtomicSnapshot(), + .processing_start_time = file_status_.processing_start_time, + .processing_end_time = file_status_.processing_end_time, + .exception = file_status_.last_exception, + }; + } s3_queue_log->add(std::move(elem)); } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index fa21b6cdd59..9bd2cad9c72 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -91,7 +91,7 @@ private: std::shared_ptr file_status; void applyActionAfterProcessing(const String & path); - void appendLogElement(const std::string & filename, const S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); + void appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); }; } diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index ce76469b72a..235f4ff6f8f 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -47,7 +47,10 @@ void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, co size_t i = 0; res_columns[i++]->insert(zookeeper_path); res_columns[i++]->insert(file_name); - res_columns[i++]->insert(file_status->processed_rows); + + std::lock_guard lock(file_status->metadata_lock); + + res_columns[i++]->insert(file_status->processed_rows.load()); res_columns[i++]->insert(magic_enum::enum_name(file_status->state)); if (file_status->processing_start_time) From d1f688811933af46b709815a860162e607898f0e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Sep 2023 16:17:02 +0200 Subject: [PATCH 028/120] Allow unique consumer identifier --- .../database-engines/materialized-postgresql.md | 5 +++++ .../PostgreSQL/DatabaseMaterializedPostgreSQL.cpp | 12 +++++++++++- .../PostgreSQL/MaterializedPostgreSQLSettings.h | 1 + .../PostgreSQL/StorageMaterializedPostgreSQL.cpp | 11 ++++++++++- 4 files changed, 27 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 33d75dc9582..4e978947e36 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -197,6 +197,11 @@ Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.htm ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_block_size = ; ``` +### `materialized_postgresql_use_unique_replication_consumer_identifier` {#materialized_postgresql_use_unique_replication_consumer_identifier} + +Use a unique replication consumer identifier for replication. Default: `0`. +If set to `1`, allows to setup several `MaterializedPostgreSQL` tables pointing to the same `PostgreSQL` table. + ## Notes {#notes} ### Failover of the logical replication slot {#logical-replication-slot-failover} diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index f2b970a39af..71a508e818b 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -63,8 +63,18 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() if (shutdown_called) return; + String replication_identifier; + if (settings->materialized_postgresql_use_unique_replication_consumer_identifier) + { + replication_identifier = fmt::format("{}_{}", getUUID(), TSA_SUPPRESS_WARNING_FOR_READ(database_name)); + } + else + { + replication_identifier = TSA_SUPPRESS_WARNING_FOR_READ(database_name); + } + replication_handler = std::make_unique( - /* replication_identifier */ TSA_SUPPRESS_WARNING_FOR_READ(database_name), /// FIXME + replication_identifier, remote_database_name, TSA_SUPPRESS_WARNING_FOR_READ(database_name), /// FIXME connection_info, diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h index d3d2faba497..18473e1bb26 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h @@ -24,6 +24,7 @@ namespace DB M(UInt64, materialized_postgresql_backoff_min_ms, 200, "Poll backoff start point", 0) \ M(UInt64, materialized_postgresql_backoff_max_ms, 10000, "Poll backoff max point", 0) \ M(UInt64, materialized_postgresql_backoff_factor, 2, "Poll backoff factor", 0) \ + M(Bool, materialized_postgresql_use_unique_replication_consumer_identifier, false, "Should a unique consumer be registered for table replication", 0) \ DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS) diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index e6389da32fd..d871b846ece 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -74,7 +74,16 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( setInMemoryMetadata(storage_metadata); - String replication_identifier = remote_database_name + "_" + remote_table_name_; + String replication_identifier; + if (replication_settings->materialized_postgresql_use_unique_replication_consumer_identifier) + { + replication_identifier = fmt::format("{}_{}_{}", table_id_.uuid, remote_database_name, remote_table_name_); + } + else + { + replication_identifier = fmt::format("{}_{}", remote_database_name, remote_table_name_); + } + replication_settings->materialized_postgresql_tables_list = remote_table_name_; replication_handler = std::make_unique( From 1917a882b77f9361b3ab1401ce8a056ba513c103 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Oct 2023 16:32:56 +0200 Subject: [PATCH 029/120] Better --- .../DatabaseMaterializedPostgreSQL.cpp | 13 +-- .../MaterializedPostgreSQLConsumer.cpp | 5 +- .../PostgreSQLReplicationHandler.cpp | 92 ++++++++++++++++--- .../PostgreSQL/PostgreSQLReplicationHandler.h | 14 +-- .../StorageMaterializedPostgreSQL.cpp | 13 +-- tests/integration/helpers/postgres_utility.py | 19 +++- .../test.py | 43 +++++++++ 7 files changed, 153 insertions(+), 46 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 71a508e818b..c03add1a96f 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -63,20 +63,11 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() if (shutdown_called) return; - String replication_identifier; - if (settings->materialized_postgresql_use_unique_replication_consumer_identifier) - { - replication_identifier = fmt::format("{}_{}", getUUID(), TSA_SUPPRESS_WARNING_FOR_READ(database_name)); - } - else - { - replication_identifier = TSA_SUPPRESS_WARNING_FOR_READ(database_name); - } - replication_handler = std::make_unique( - replication_identifier, remote_database_name, + /* table_name */"", TSA_SUPPRESS_WARNING_FOR_READ(database_name), /// FIXME + toString(getUUID()), connection_info, getContext(), is_attach, diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 502ced12e75..cfd1771aa46 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -574,6 +574,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl void MaterializedPostgreSQLConsumer::syncTables() { + size_t synced_tables = 0; while (!tables_to_sync.empty()) { auto table_name = *tables_to_sync.begin(); @@ -604,6 +605,7 @@ void MaterializedPostgreSQLConsumer::syncTables() CompletedPipelineExecutor executor(io.pipeline); executor.execute(); + ++synced_tables; } } catch (...) @@ -616,7 +618,8 @@ void MaterializedPostgreSQLConsumer::syncTables() tables_to_sync.erase(tables_to_sync.begin()); } - LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", tables_to_sync.size(), current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); + LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", + synced_tables, current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); updateLsn(); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index f57a6a26a62..ee38dcb44d4 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -17,12 +17,14 @@ #include #include #include +#include namespace DB { static const auto CLEANUP_RESCHEDULE_MS = 600000 * 3; /// 30 min +static constexpr size_t replication_slot_name_max_size = 64; namespace ErrorCodes { @@ -56,10 +58,70 @@ private: }; +namespace +{ + /// There can be several replication slots per publication, but one publication per table/database replication. + /// Replication slot might be unique (contain uuid) to allow have multiple replicas for the same PostgreSQL table/database. + + String getPublicationName(const String & postgres_database, const String & postgres_table) + { + return fmt::format( + "{}_ch_publication", + postgres_table.empty() ? postgres_database : fmt::format("{}_{}", postgres_database, postgres_table)); + } + + void checkReplicationSlot(String name) + { + for (const auto & c : name) + { + const bool ok = (std::isalpha(c) && std::islower(c)) || std::isdigit(c) || c == '_'; + if (!ok) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Replication slot can contain lower-case letters, numbers, and the underscore character. " + "Got: {}", name); + } + } + + if (name.size() > replication_slot_name_max_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Too big replication slot size: {}", name); + } + + String normalizeReplicationSlot(String name) + { + name = Poco::toLower(name); + for (auto & c : name) + if (c == '-') + c = '_'; + return name; + } + + String getReplicationSlotName( + const String & postgres_database, + const String & postgres_table, + const String & clickhouse_uuid, + const MaterializedPostgreSQLSettings & replication_settings) + { + String slot_name = replication_settings.materialized_postgresql_replication_slot; + if (slot_name.empty()) + { + if (replication_settings.materialized_postgresql_use_unique_replication_consumer_identifier) + slot_name = clickhouse_uuid; + else + slot_name = postgres_table.empty() ? postgres_database : fmt::format("{}_{}_ch_replication_slot", postgres_database, postgres_table); + + slot_name = normalizeReplicationSlot(slot_name); + } + return slot_name; + } +} + PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( - const String & replication_identifier, const String & postgres_database_, - const String & current_database_name_, + const String & postgres_table_, + const String & clickhouse_database_, + const String & clickhouse_uuid_, const postgres::ConnectionInfo & connection_info_, ContextPtr context_, bool is_attach_, @@ -70,14 +132,18 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , is_attach(is_attach_) , postgres_database(postgres_database_) , postgres_schema(replication_settings.materialized_postgresql_schema) - , current_database_name(current_database_name_) + , current_database_name(clickhouse_database_) , connection_info(connection_info_) , max_block_size(replication_settings.materialized_postgresql_max_block_size) , is_materialized_postgresql_database(is_materialized_postgresql_database_) , tables_list(replication_settings.materialized_postgresql_tables_list) , schema_list(replication_settings.materialized_postgresql_schema_list) , schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings.materialized_postgresql_tables_list_with_schema) + , user_managed_slot(!replication_settings.materialized_postgresql_replication_slot.value.empty()) , user_provided_snapshot(replication_settings.materialized_postgresql_snapshot) + , replication_slot(getReplicationSlotName(postgres_database_, postgres_table_, clickhouse_uuid_, replication_settings)) + , tmp_replication_slot(replication_slot + "_tmp") + , publication_name(getPublicationName(postgres_database_, postgres_table_)) , reschedule_backoff_min_ms(replication_settings.materialized_postgresql_backoff_min_ms) , reschedule_backoff_max_ms(replication_settings.materialized_postgresql_backoff_max_ms) , reschedule_backoff_factor(replication_settings.materialized_postgresql_backoff_factor) @@ -89,13 +155,9 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( if (!schema_list.empty() && !postgres_schema.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have schema list and common schema at the same time"); - replication_slot = replication_settings.materialized_postgresql_replication_slot; - if (replication_slot.empty()) - { - user_managed_slot = false; - replication_slot = fmt::format("{}_ch_replication_slot", replication_identifier); - } - publication_name = fmt::format("{}_ch_publication", replication_identifier); + checkReplicationSlot(replication_slot); + + LOG_INFO(log, "Using replication slot {} and publication {}", replication_slot, publication_name); startup_task = getContext()->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ checkConnectionAndStart(); }); consumer_task = getContext()->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); @@ -496,7 +558,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::nontransactio throw Exception(ErrorCodes::LOGICAL_ERROR, "No table found to be replicated"); /// 'ONLY' means just a table, without descendants. - std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, tables_list); + std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", doubleQuoteString(publication_name), tables_list); try { tx.exec(query_str); @@ -519,7 +581,7 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & { String slot_name; if (temporary) - slot_name = replication_slot + "_tmp"; + slot_name = tmp_replication_slot; else slot_name = replication_slot; @@ -546,11 +608,11 @@ void PostgreSQLReplicationHandler::createReplicationSlot( String query_str, slot_name; if (temporary) - slot_name = replication_slot + "_tmp"; + slot_name = tmp_replication_slot; else slot_name = replication_slot; - query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", slot_name); + query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", doubleQuoteString(slot_name)); try { @@ -573,7 +635,7 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(pqxx::nontransaction & tx std::string slot_name; if (temporary) - slot_name = replication_slot + "_tmp"; + slot_name = tmp_replication_slot; else slot_name = replication_slot; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 4c16ff95692..5d426b3c512 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -21,9 +21,10 @@ public: using ConsumerPtr = std::shared_ptr; PostgreSQLReplicationHandler( - const String & replication_identifier, const String & postgres_database_, - const String & current_database_name_, + const String & postgres_table_, + const String & clickhouse_database_, + const String & clickhouse_uuid_, const postgres::ConnectionInfo & connection_info_, ContextPtr context_, bool is_attach_, @@ -128,10 +129,11 @@ private: /// This is possible to allow replicating tables from multiple schemas in the same MaterializedPostgreSQL database engine. mutable bool schema_as_a_part_of_table_name = false; - bool user_managed_slot = true; - String user_provided_snapshot; - - String replication_slot, publication_name; + const bool user_managed_slot; + const String user_provided_snapshot; + const String replication_slot; + const String tmp_replication_slot; + const String publication_name; /// Replication consumer. Manages decoding of replication stream and syncing into tables. ConsumerPtr consumer; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index d871b846ece..d83722dba6c 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -74,22 +74,13 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( setInMemoryMetadata(storage_metadata); - String replication_identifier; - if (replication_settings->materialized_postgresql_use_unique_replication_consumer_identifier) - { - replication_identifier = fmt::format("{}_{}_{}", table_id_.uuid, remote_database_name, remote_table_name_); - } - else - { - replication_identifier = fmt::format("{}_{}", remote_database_name, remote_table_name_); - } - replication_settings->materialized_postgresql_tables_list = remote_table_name_; replication_handler = std::make_unique( - replication_identifier, remote_database_name, + remote_table_name_, table_id_.database_name, + toString(table_id_.uuid), connection_info, getContext(), is_attach, diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 3c8a23b15a2..77dc72f8020 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -113,11 +113,19 @@ class PostgresManager: self.created_materialized_postgres_db_list = set() self.created_ch_postgres_db_list = set() - def init(self, instance, ip, port, default_database="postgres_database"): + def init( + self, + instance, + ip, + port, + default_database="postgres_database", + postgres_db_exists=False, + ): self.instance = instance self.ip = ip self.port = port self.default_database = default_database + self.postgres_db_exists = postgres_db_exists self.prepare() def get_default_database(self): @@ -138,7 +146,8 @@ class PostgresManager: self.conn = get_postgres_conn(ip=self.ip, port=self.port) self.cursor = self.conn.cursor() if self.default_database != "": - self.create_postgres_db(self.default_database) + if not self.postgres_db_exists: + self.create_postgres_db(self.default_database) self.conn = get_postgres_conn( ip=self.ip, port=self.port, @@ -364,6 +373,12 @@ def check_tables_are_synchronized( time.sleep(1) result = instance.query(result_query) + if result != expected: + count = int(instance.query(f"select count() from {table_path}")) + expected_count = int( + instance.query(f"select count() from {postgres_database}.{table_name}") + ) + print(f"Having {count}, expected {expected_count}") assert result == expected diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3f2ec74180b..903d246e1c5 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -719,6 +719,49 @@ def test_too_many_parts(started_cluster): pg_manager2.drop_materialized_db() +def test_replica_consumer(started_cluster): + table = "test_replica_consumer" + + pg_manager_replica = PostgresManager() + pg_manager_replica.init( + instance2, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres_database", + postgres_db_exists=True + ) + + for pm in [pg_manager, pg_manager_replica]: + pm.create_and_fill_postgres_table(table) + pm.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100", + "materialized_postgresql_use_unique_replication_consumer_identifier = 1" + ], + ) + + assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 50 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + instance.query(f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)") + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance2, table, postgres_database=pg_manager_replica.get_default_database() + ) + + assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + pg_manager_replica.clear() + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 34915a8a5e383b402c79960f2fb021b239582300 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 12 Oct 2023 19:47:39 +0200 Subject: [PATCH 030/120] Fix black check --- .../test_postgresql_replica_database_engine_2/test.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 903d246e1c5..21da5bd3ed0 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -728,7 +728,7 @@ def test_replica_consumer(started_cluster): cluster.postgres_ip, cluster.postgres_port, default_database="postgres_database", - postgres_db_exists=True + postgres_db_exists=True, ) for pm in [pg_manager, pg_manager_replica]: @@ -740,14 +740,16 @@ def test_replica_consumer(started_cluster): f"materialized_postgresql_tables_list = '{table}'", "materialized_postgresql_backoff_min_ms = 100", "materialized_postgresql_backoff_max_ms = 100", - "materialized_postgresql_use_unique_replication_consumer_identifier = 1" + "materialized_postgresql_use_unique_replication_consumer_identifier = 1", ], ) assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) assert 50 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) - instance.query(f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)") + instance.query( + f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)" + ) check_tables_are_synchronized( instance, table, postgres_database=pg_manager.get_default_database() From bfe174f71b575c8831eee86e57e6307fd74a59c8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 12:57:57 +0200 Subject: [PATCH 031/120] Fix test --- src/Interpreters/Context.cpp | 2 +- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 2 +- src/Storages/S3Queue/S3QueueSource.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 658fb9adbb2..dfa18694858 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -3590,7 +3590,7 @@ std::shared_ptr Context::getFilesystemCacheLog() const std::shared_ptr Context::getS3QueueLog() const { - auto lock = getLock(); + auto lock = getGlobalSharedLock(); if (!shared->system_logs) return {}; diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 1856490c92e..535ced7f6df 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -591,11 +591,11 @@ void S3QueueFilesMetadata::setFileFailed(ProcessingNodeHolderPtr holder, const S const auto node_name_with_retriable_suffix = node_name + ".retriable"; Coordination::Stat stat; std::string res; - auto failed_node_metadata = NodeMetadata::fromString(res); /// Extract the number of already done retries from node_hash.retriable node if it exists. if (zk_client->tryGet(zookeeper_failed_path / node_name_with_retriable_suffix, res, &stat)) { + auto failed_node_metadata = NodeMetadata::fromString(res); node_metadata.retries = failed_node_metadata.retries + 1; std::lock_guard lock(file_status->metadata_lock); diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index d86fc6fe1ce..7140ec9d4f0 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -148,7 +148,7 @@ Chunk StorageS3QueueSource::generate() } catch (const Exception & e) { - LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); + LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getFile(), e.displayText()); files_metadata->setFileFailed(key_with_info->processing_holder, e.message()); From a9c0c20cad0db7366b1cd5a9d780f73d3c58e38a Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 13:21:50 +0200 Subject: [PATCH 032/120] Update documentation --- .../table-engines/integrations/s3queue.md | 77 +++++++++++++++---- src/Interpreters/S3QueueLog.cpp | 3 +- src/Storages/System/StorageSystemS3Queue.cpp | 3 + 3 files changed, 69 insertions(+), 14 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 7ea848b53b0..4769f573b54 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -248,7 +248,7 @@ If the listing of files contains number ranges with leading zeros, use the const For introspection use `system.s3queue` stateless table and `system.s3queue_log` persistent table. -The `s3queue` system table has the following structure: +1. `system.s3queue`. This table is not persistent and shows in-memory state of `S3Queue`: which files are currently being processed, which files are processed or failed. ``` sql ┌─statement──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ @@ -262,12 +262,62 @@ The `s3queue` system table has the following structure: `processing_start_time` Nullable(DateTime), `processing_end_time` Nullable(DateTime), `ProfileEvents` Map(String, UInt64) + `exception` String ) ENGINE = SystemS3Queue COMMENT 'SYSTEM TABLE is built on the fly.' │ └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +Example: + +``` sql + +SELECT * +FROM system.s3queue + +Row 1: +────── +zookeeper_path: /clickhouse/s3queue/25ea5621-ae8c-40c7-96d0-cec959c5ab88/3b3f66a1-9866-4c2e-ba78-b6bfa154207e +file_name: wikistat/original/pageviews-20150501-030000.gz +rows_processed: 5068534 +status: Processed +processing_start_time: 2023-10-13 13:09:48 +processing_end_time: 2023-10-13 13:10:31 +ProfileEvents: {'ZooKeeperTransactions':3,'ZooKeeperGet':2,'ZooKeeperMulti':1,'SelectedRows':5068534,'SelectedBytes':198132283,'ContextLock':1,'S3QueueSetFileProcessingMicroseconds':2480,'S3QueueSetFileProcessedMicroseconds':9985,'S3QueuePullMicroseconds':273776,'LogTest':17} +exception: +``` + +2. `system.s3queue_log`. Persistent table. Has the same information as `system.s3queue`, but for `processed` and `failed` files. + +The table has the following structure: + +``` sql +SHOW CREATE TABLE system.s3queue_log + +Query id: 0ad619c3-0f2a-4ee4-8b40-c73d86e04314 + +┌─statement──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ CREATE TABLE system.s3queue_log +( + `event_date` Date, + `event_time` DateTime, + `table_uuid` String, + `file_name` String, + `rows_processed` UInt64, + `status` Enum8('Processed' = 0, 'Failed' = 1), + `processing_start_time` Nullable(DateTime), + `processing_end_time` Nullable(DateTime), + `ProfileEvents` Map(String, UInt64), + `exception` String +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(event_date) +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192 │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + In order to use `system.s3queue_log` define its configuration in server config file: ``` xml @@ -280,18 +330,19 @@ In order to use `system.s3queue_log` define its configuration in server config f Example: ``` sql -:) select * from system.s3queue - SELECT * -FROM system.s3queue +FROM system.s3queue_log -Query id: bb41964e-c947-4112-be3a-0f01770a1e84 - -┌─database─┬─table───┬─file_name──────────────────────────────────────────┬─rows_processed─┬─status─────┬─processing_start_time─┬─processing_end_time─┬─ProfileEvents──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ default │ s3queue │ hits_compatible/athena_partitioned/hits_60.parquet │ 0 │ Processing │ 2023-09-25 19:56:51 │ ᴺᵁᴸᴸ │ {'S3QueueSetFileProcessingMicroseconds':516} │ -│ default │ s3queue │ hits_compatible/athena_partitioned/hits_54.parquet │ 1000000 │ Processing │ 2023-09-25 19:56:50 │ ᴺᵁᴸᴸ │ {'SelectedRows':1000000,'SelectedBytes':1284181126,'S3ReadMicroseconds':385274,'S3ReadRequestsCount':3,'S3ReadRequestsErrors':1,'S3GetObject':1,'ReadBufferFromS3Microseconds':405970,'ReadBufferFromS3InitMicroseconds':385790,'ReadBufferFromS3Bytes':65536,'ReadBufferFromS3PreservedSessions':1,'S3QueueSetFileProcessingMicroseconds':567,'S3QueuePullMicroseconds':2475045} │ -└──────────┴─────────┴────────────────────────────────────────────────────┴────────────────┴────────────┴───────────────────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ - - -SELECT * FROM system.s3_queue_log; +Row 1: +────── +event_date: 2023-10-13 +event_time: 2023-10-13 13:10:12 +table_uuid: +file_name: wikistat/original/pageviews-20150501-020000.gz +rows_processed: 5112621 +status: Processed +processing_start_time: 2023-10-13 13:09:48 +processing_end_time: 2023-10-13 13:10:12 +ProfileEvents: {'ZooKeeperTransactions':3,'ZooKeeperGet':2,'ZooKeeperMulti':1,'SelectedRows':5112621,'SelectedBytes':198577687,'ContextLock':1,'S3QueueSetFileProcessingMicroseconds':1934,'S3QueueSetFileProcessedMicroseconds':17063,'S3QueuePullMicroseconds':5841972,'LogTest':17} +exception: ``` diff --git a/src/Interpreters/S3QueueLog.cpp b/src/Interpreters/S3QueueLog.cpp index af40d1dd1e8..4e43aae6b0d 100644 --- a/src/Interpreters/S3QueueLog.cpp +++ b/src/Interpreters/S3QueueLog.cpp @@ -42,12 +42,13 @@ void S3QueueLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(table_uuid); columns[i++]->insert(file_name); columns[i++]->insert(rows_processed); - columns[i++]->insert(magic_enum::enum_name(status)); + columns[i++]->insert(status); if (processing_start_time) columns[i++]->insert(processing_start_time); else columns[i++]->insertDefault(); + if (processing_end_time) columns[i++]->insert(processing_end_time); else diff --git a/src/Storages/System/StorageSystemS3Queue.cpp b/src/Storages/System/StorageSystemS3Queue.cpp index 235f4ff6f8f..33b3dce4a83 100644 --- a/src/Storages/System/StorageSystemS3Queue.cpp +++ b/src/Storages/System/StorageSystemS3Queue.cpp @@ -30,6 +30,7 @@ NamesAndTypesList StorageSystemS3Queue::getNamesAndTypes() {"processing_start_time", std::make_shared(std::make_shared())}, {"processing_end_time", std::make_shared(std::make_shared())}, {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + {"exception", std::make_shared()}, }; } @@ -63,6 +64,8 @@ void StorageSystemS3Queue::fillData(MutableColumns & res_columns, ContextPtr, co res_columns[i++]->insertDefault(); ProfileEvents::dumpToMapColumn(file_status->profile_counters.getPartiallyAtomicSnapshot(), res_columns[i++].get(), true); + + res_columns[i++]->insert(file_status->last_exception); } } } From b6b124f5a9a8b93ec21445273e9886818b73fd3b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 13:54:20 +0200 Subject: [PATCH 033/120] Usability improvement --- src/Core/Settings.h | 1 + src/Storages/S3Queue/StorageS3Queue.cpp | 32 ++++++++++++++++--- .../test_storage_s3_queue/configs/users.xml | 1 + 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c08425c03fd..93a15bb80fd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -108,6 +108,7 @@ class IColumn; M(UInt64, s3_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. Only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ M(String, s3queue_default_zookeeper_path, "/clickhouse/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \ + M(Bool, s3queue_enable_logging_to_s3queue_log, false, "Enable writing to system.s3queue_log. The value can be overwritten per table with table settings", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 6bed73b0315..c280b909d1a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -70,6 +70,32 @@ namespace } return zkutil::extractZooKeeperPath(result_zk_path, true); } + + void checkAndAdjustSettings(S3QueueSettings & s3queue_settings, const Settings & settings, Poco::Logger * log) + { + if (s3queue_settings.mode == S3QueueMode::ORDERED && s3queue_settings.s3queue_processing_threads_num > 1) + { + LOG_WARNING(log, "Parallel processing is not yet supported for Ordered mode"); + s3queue_settings.s3queue_processing_threads_num = 1; + } + + if (!s3queue_settings.s3queue_processing_threads_num) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Setting `s3queue_processing_threads_num` cannot be set to zero"); + } + + if (!s3queue_settings.s3queue_enable_logging_to_s3queue_log.changed) + { + s3queue_settings.s3queue_enable_logging_to_s3queue_log = settings.s3queue_enable_logging_to_s3queue_log; + } + + if (s3queue_settings.s3queue_cleanup_interval_min_ms > s3queue_settings.s3queue_cleanup_interval_max_ms) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Setting `s3queue_cleanup_interval_min_ms` ({}) must be less or equal to `s3queue_cleanup_interval_max_ms` ({})", + s3queue_settings.s3queue_cleanup_interval_min_ms, s3queue_settings.s3queue_cleanup_interval_max_ms); + } + } } StorageS3Queue::StorageS3Queue( @@ -101,11 +127,7 @@ StorageS3Queue::StorageS3Queue( throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); } - if (s3queue_settings->mode == S3QueueMode::ORDERED && s3queue_settings->s3queue_processing_threads_num > 1) - { - LOG_WARNING(log, "Parallel processing is not yet supported for Ordered mode"); - s3queue_settings->s3queue_processing_threads_num = 1; - } + checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef(), log); configuration.update(context_); FormatFactory::instance().checkFormatName(configuration.format); diff --git a/tests/integration/test_storage_s3_queue/configs/users.xml b/tests/integration/test_storage_s3_queue/configs/users.xml index 3118ec43654..8e447278752 100644 --- a/tests/integration/test_storage_s3_queue/configs/users.xml +++ b/tests/integration/test_storage_s3_queue/configs/users.xml @@ -3,6 +3,7 @@ 1 1 + 1 From 44ea61e57fa5e6898c15c67f51430024d519a8f8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 15:20:54 +0200 Subject: [PATCH 034/120] Improve shutdown --- src/Storages/S3Queue/S3QueueSettings.h | 4 +-- src/Storages/S3Queue/S3QueueSource.cpp | 33 +++++++++++++++++++------ src/Storages/S3Queue/S3QueueSource.h | 7 ++++-- src/Storages/S3Queue/StorageS3Queue.cpp | 2 +- 4 files changed, 33 insertions(+), 13 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index cc9f97182bc..351245e24be 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -27,8 +27,8 @@ class ASTStorage; M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ M(UInt32, s3queue_polling_backoff_ms, 1000, "Polling backoff", 0) \ M(UInt32, s3queue_tracked_files_limit, 1000, "For unordered mode. Max set size for tracking processed files in ZooKeeper", 0) \ - M(UInt32, s3queue_cleanup_interval_min_ms, 10000, "For unordered mode. Polling backoff min for cleanup", 0) \ - M(UInt32, s3queue_cleanup_interval_max_ms, 30000, "For unordered mode. Polling backoff max for cleanup", 0) \ + M(UInt32, s3queue_cleanup_interval_min_ms, 60000, "For unordered mode. Polling backoff min for cleanup", 0) \ + M(UInt32, s3queue_cleanup_interval_max_ms, 60000, "For unordered mode. Polling backoff max for cleanup", 0) \ #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 7140ec9d4f0..2e2435b0d94 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -40,28 +40,30 @@ StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( } StorageS3QueueSource::FileIterator::FileIterator( - std::shared_ptr metadata_, std::unique_ptr glob_iterator_) - : metadata(metadata_) , glob_iterator(std::move(glob_iterator_)) + std::shared_ptr metadata_, + std::unique_ptr glob_iterator_, + std::atomic & shutdown_called_) + : metadata(metadata_) + , glob_iterator(std::move(glob_iterator_)) + , shutdown_called(shutdown_called_) { } StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next() { - /// List results in s3 are always returned in UTF-8 binary order. - /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) - - while (true) + while (!shutdown_called) { KeyWithInfoPtr val = glob_iterator->next(); - if (!val) + if (!val || shutdown_called) return {}; - if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder) + if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder && !shutdown_called) { return std::make_shared(val->key, val->info, processing_holder); } } + return {}; } size_t StorageS3QueueSource::FileIterator::estimatedKeysCount() @@ -125,6 +127,21 @@ Chunk StorageS3QueueSource::generate() break; } + if (shutdown_called) + { + if (processed_rows_from_file) + { + /// We could delay shutdown until files, which already started processing before the shutdown, finished. + /// But if files are big and `s3queue_processing_threads_num` is not small, it can take a significant time. + /// Anyway we cannot do anything in case of SIGTERM, so destination table must anyway support deduplication, + /// so here we will rely on it here as well. + LOG_WARNING( + log, "Shutdown called, {} rows are already processed, but file is not fully processed", + processed_rows_from_file); + } + break; + } + auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 9bd2cad9c72..a52bfb5a34f 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -14,7 +14,6 @@ namespace Poco { class Logger; } namespace DB { - class StorageS3QueueSource : public ISource, WithContext { public: @@ -38,8 +37,11 @@ public: class FileIterator : public IIterator { public: - FileIterator(std::shared_ptr metadata_, std::unique_ptr glob_iterator_); + FileIterator(std::shared_ptr metadata_, std::unique_ptr glob_iterator_, std::atomic & shutdown_called_); + /// Note: + /// List results in s3 are always returned in UTF-8 binary order. + /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) KeyWithInfoPtr next() override; size_t estimatedKeysCount() override; @@ -47,6 +49,7 @@ public: private: const std::shared_ptr metadata; const std::unique_ptr glob_iterator; + std::atomic & shutdown_called; std::mutex mutex; }; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index c280b909d1a..37389eb1bd0 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -453,7 +453,7 @@ std::shared_ptr StorageS3Queue::createFileIterator auto glob_iterator = std::make_unique( *configuration.client, configuration.url, query, virtual_columns, local_context, /* read_keys */nullptr, configuration.request_settings); - return std::make_shared(files_metadata, std::move(glob_iterator)); + return std::make_shared(files_metadata, std::move(glob_iterator), shutdown_called); } void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) From 6fde98a33f40b8ef3d5b75578c6ee6b64a609b9c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 13 Oct 2023 15:56:05 +0200 Subject: [PATCH 035/120] Minor improvement --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 21 +++++++++++-------- src/Storages/S3Queue/S3QueueFilesMetadata.h | 13 ++++++------ src/Storages/S3Queue/S3QueueSource.cpp | 15 +++++++------ src/Storages/S3Queue/S3QueueSource.h | 6 ++++-- 4 files changed, 32 insertions(+), 23 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 535ced7f6df..ac12e8fe70f 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -64,8 +64,7 @@ S3QueueFilesMetadata::FileStatuses S3QueueFilesMetadata::LocalFileStatuses::getA return file_statuses; } -std::shared_ptr -S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) +S3QueueFilesMetadata::FileStatusPtr S3QueueFilesMetadata::LocalFileStatuses::get(const std::string & filename, bool create) { auto lk = lock(); auto it = file_statuses.find(filename); @@ -161,7 +160,7 @@ zkutil::ZooKeeperPtr S3QueueFilesMetadata::getZooKeeper() const return Context::getGlobalContextInstance()->getZooKeeper(); } -std::shared_ptr S3QueueFilesMetadata::getFileStatus(const std::string & path) +S3QueueFilesMetadata::FileStatusPtr S3QueueFilesMetadata::getFileStatus(const std::string & path) { /// Return a locally cached file status. return local_file_statuses.get(path, /* create */false); @@ -198,7 +197,8 @@ S3QueueFilesMetadata::NodeMetadata S3QueueFilesMetadata::createNodeMetadata( return metadata; } -S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) +std::pair S3QueueFilesMetadata::trySetFileAsProcessing(const std::string & path) { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueueSetFileProcessingMicroseconds); auto file_status = local_file_statuses.get(path, /* create */true); @@ -216,13 +216,13 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs case FileStatus::State::Processing: [[fallthrough]]; case FileStatus::State::Processed: { - return nullptr; + return {}; } case FileStatus::State::Failed: { /// If max_loading_retries == 0, file is not retriable. if (max_loading_retries == 0) - return nullptr; + return {}; /// Otherwise file_status->retries is also cached. /// In case file_status->retries >= max_loading_retries we can fully rely that it is true @@ -231,7 +231,7 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs /// (another server could have done a try after we cached retries value), /// so check with zookeeper here. if (file_status->retries >= max_loading_retries) - return nullptr; + return {}; break; } @@ -249,7 +249,7 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs std::unique_lock processing_lock(file_status->processing_lock, std::defer_lock); if (!processing_lock.try_lock()) { - return nullptr; + return {}; } /// Let's go and check metadata in zookeeper and try to create a /processing ephemeral node. @@ -306,7 +306,10 @@ S3QueueFilesMetadata::ProcessingNodeHolderPtr S3QueueFilesMetadata::trySetFileAs } } - return result == SetFileProcessingResult::Success ? processing_node_holder : nullptr; + if (result == SetFileProcessingResult::Success) + return std::pair(processing_node_holder, file_status); + + return {}; } std::pair>; + using FileStatusPtr = std::shared_ptr; + using FileStatuses = std::unordered_map; - std::shared_ptr getFileStatus(const std::string & path); + /// Set file as processing, if it is not alreaty processed, failed or processing. + std::pair trySetFileAsProcessing(const std::string & path); + + FileStatusPtr getFileStatus(const std::string & path); FileStatuses getFileStateses() const { return local_file_statuses.getAll(); } @@ -137,7 +138,7 @@ private: mutable std::mutex mutex; FileStatuses getAll() const; - std::shared_ptr get(const std::string & filename, bool create); + FileStatusPtr get(const std::string & filename, bool create); bool remove(const std::string & filename, bool if_exists); std::unique_lock lock() const; }; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 2e2435b0d94..b3f8f0cf1c3 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -33,9 +33,11 @@ namespace ErrorCodes StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( const std::string & key_, std::optional info_, - Metadata::ProcessingNodeHolderPtr processing_holder_) + Metadata::ProcessingNodeHolderPtr processing_holder_, + FileStatusPtr file_status_) : StorageS3Source::KeyWithInfo(key_, info_) , processing_holder(processing_holder_) + , file_status(file_status_) { } @@ -58,9 +60,10 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next() if (!val || shutdown_called) return {}; - if (auto processing_holder = metadata->trySetFileAsProcessing(val->key); processing_holder && !shutdown_called) + if (auto [processing_holder, processing_file_status] = metadata->trySetFileAsProcessing(val->key); + processing_holder && !shutdown_called) { - return std::make_shared(val->key, val->info, processing_holder); + return std::make_shared(val->key, val->info, processing_holder, processing_file_status); } } return {}; @@ -100,7 +103,6 @@ StorageS3QueueSource::StorageS3QueueSource( if (reader) { reader_future = std::move(internal_source->reader_future); - file_status = files_metadata->getFileStatus(reader.getFile()); } } @@ -142,11 +144,12 @@ Chunk StorageS3QueueSource::generate() break; } + const auto * key_with_info = dynamic_cast(&reader.getKeyWithInfo()); + auto file_status = key_with_info->file_status; + auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); - const auto * key_with_info = dynamic_cast(&reader.getKeyWithInfo()); - try { auto timer = DB::CurrentThread::getProfileEvents().timer(ProfileEvents::S3QueuePullMicroseconds); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index a52bfb5a34f..db3015f129e 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -22,6 +22,7 @@ public: using GlobIterator = StorageS3Source::DisclosedGlobIterator; using ZooKeeperGetter = std::function; using RemoveFileFunc = std::function; + using FileStatusPtr = S3QueueFilesMetadata::FileStatusPtr; using Metadata = S3QueueFilesMetadata; struct S3QueueKeyWithInfo : public StorageS3Source::KeyWithInfo @@ -29,9 +30,11 @@ public: S3QueueKeyWithInfo( const std::string & key_, std::optional info_, - Metadata::ProcessingNodeHolderPtr processing_holder_); + Metadata::ProcessingNodeHolderPtr processing_holder_, + FileStatusPtr file_status_); Metadata::ProcessingNodeHolderPtr processing_holder; + FileStatusPtr file_status; }; class FileIterator : public IIterator @@ -91,7 +94,6 @@ private: ReaderHolder reader; std::future reader_future; size_t processed_rows_from_file = 0; - std::shared_ptr file_status; void applyActionAfterProcessing(const String & path); void appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); From ba5cd016dd661cfc332534664ed17e335d1e9f7c Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 13 Oct 2023 14:02:31 +0000 Subject: [PATCH 036/120] Fix crash in QueryNormalizer with cyclic aliases --- src/Interpreters/QueryNormalizer.cpp | 6 ++++++ .../02896_cyclic_aliases_crash.reference | 1 + .../0_stateless/02896_cyclic_aliases_crash.sql | 17 +++++++++++++++++ 3 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02896_cyclic_aliases_crash.reference create mode 100644 tests/queries/0_stateless/02896_cyclic_aliases_crash.sql diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 56b81b3d224..6b6ead1f463 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -68,6 +68,10 @@ private: void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) { + /// We do handle cycles via tracking current_asts + /// but in case of bug in that tricky logic we need to prevent stack overflow + checkStackSize(); + auto & current_asts = data.current_asts; String & current_alias = data.current_alias; @@ -116,6 +120,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a. /// Check size of the alias before cloning too large alias AST alias_node->checkSize(data.settings.max_expanded_ast_elements); + current_asts.insert(alias_node.get()); ast = alias_node->clone(); ast->setAlias(node_alias); @@ -134,6 +139,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// Check size of the alias before cloning too large alias AST alias_node->checkSize(data.settings.max_expanded_ast_elements); auto alias_name = ast->getAliasOrColumnName(); + current_asts.insert(alias_node.get()); ast = alias_node->clone(); ast->setAlias(alias_name); diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference b/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference new file mode 100644 index 00000000000..d43017edcc5 --- /dev/null +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.reference @@ -0,0 +1 @@ +1 2 3 diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql new file mode 100644 index 00000000000..af54f5df7a7 --- /dev/null +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql @@ -0,0 +1,17 @@ + +SET max_ast_depth = 10_000_000; + +SELECT + val, + val + 1 as prev, + val + prev as val +FROM ( SELECT 1 as val ) +; -- { serverError CYCLIC_ALIASES } + + +SELECT + val, + val + 1 as prev, + val + prev as val2 +FROM ( SELECT 1 as val ) +; From 25545d504d932f23b466a001bd3567b7483e76de Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Fri, 13 Oct 2023 10:31:21 -0700 Subject: [PATCH 037/120] Replace setting density to max_rows_in postings_list for full text search --- src/Interpreters/GinFilter.cpp | 17 +++-- src/Interpreters/GinFilter.h | 8 ++- .../MergeTree/MergeTreeIndexInverted.cpp | 21 +++--- .../MergeTree/MergeTreeIndexInverted.h | 2 +- .../0_stateless/02346_full_text_search.sql | 71 ------------------- 5 files changed, 27 insertions(+), 92 deletions(-) diff --git a/src/Interpreters/GinFilter.cpp b/src/Interpreters/GinFilter.cpp index 4662128e8ab..e60d54026eb 100644 --- a/src/Interpreters/GinFilter.cpp +++ b/src/Interpreters/GinFilter.cpp @@ -21,14 +21,18 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -GinFilterParameters::GinFilterParameters(size_t ngrams_, Float64 density_) +GinFilterParameters::GinFilterParameters(size_t ngrams_, UInt64 max_rows_) : ngrams(ngrams_) - , density(density_) + , max_rows_in_postings_list(max_rows_) { + /// 0 indicates no limitation of postings list's size + if (max_rows_in_postings_list == 0) + max_rows_in_postings_list = std::numeric_limits::max(); + if (ngrams > 8) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of inverted index filter cannot be greater than 8"); - if (density <= 0 || density > 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The density inverted index gin filter must be between 0 and 1"); + if (max_rows_in_postings_list < MIN_ROWS_IN_POSTINGS_LIST) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The maximum rows in postings list must be no less than {}", MIN_ROWS_IN_POSTINGS_LIST); } GinFilter::GinFilter(const GinFilterParameters & params_) @@ -36,7 +40,7 @@ GinFilter::GinFilter(const GinFilterParameters & params_) { } -void GinFilter::add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store, UInt64 limit) const +void GinFilter::add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store) const { if (len > FST::MAX_TERM_LENGTH) return; @@ -51,8 +55,7 @@ void GinFilter::add(const char * data, size_t len, UInt32 rowID, GinIndexStorePt } else { - UInt64 size_limit = std::lround(limit * params.density); - auto builder = std::make_shared(size_limit); + auto builder = std::make_shared(params.max_rows_in_postings_list); builder->add(rowID); store->setPostingsBuilder(term, builder); diff --git a/src/Interpreters/GinFilter.h b/src/Interpreters/GinFilter.h index 8985d84f215..3e57a07ecd4 100644 --- a/src/Interpreters/GinFilter.h +++ b/src/Interpreters/GinFilter.h @@ -8,13 +8,15 @@ namespace DB { static inline constexpr auto INVERTED_INDEX_NAME = "inverted"; +static inline constexpr UInt64 MIN_ROWS_IN_POSTINGS_LIST = 8 * 1024; +static inline constexpr UInt64 DEFAULT_ROWS_IN_POSTINGS_LIST = 64 * 1024; struct GinFilterParameters { - GinFilterParameters(size_t ngrams_, Float64 density_); + GinFilterParameters(size_t ngrams_, UInt64 max_rows_); size_t ngrams; - Float64 density; + UInt64 max_rows_in_postings_list; }; struct GinSegmentWithRowIdRange @@ -42,7 +44,7 @@ public: /// Add term (located at 'data' with length 'len') and its row ID to the postings list builder /// for building inverted index for the given store. - void add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store, UInt64 limit) const; + void add(const char * data, size_t len, UInt32 rowID, GinIndexStorePtr & store) const; /// Accumulate (segmentID, RowIDStart, RowIDEnd) for building skipping index void addRowRangeToGinFilter(UInt32 segmentID, UInt32 rowIDStart, UInt32 rowIDEnd); diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 04ef7339a0e..169ae768b31 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -109,14 +109,14 @@ MergeTreeIndexGranulePtr MergeTreeIndexAggregatorInverted::getGranuleAndReset() return new_granule; } -void MergeTreeIndexAggregatorInverted::addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter, UInt64 limit) +void MergeTreeIndexAggregatorInverted::addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter) { size_t cur = 0; size_t token_start = 0; size_t token_len = 0; while (cur < length && token_extractor->nextInStringPadded(data, length, &cur, &token_start, &token_len)) - gin_filter.add(data + token_start, token_len, rowID, store, limit); + gin_filter.add(data + token_start, token_len, rowID, store); } void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, size_t limit) @@ -150,7 +150,7 @@ void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, for (size_t row_num = 0; row_num < elements_size; ++row_num) { auto ref = column_key.getDataAt(element_start_row + row_num); - addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col], rows_read); + addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col]); store->incrementCurrentSizeBy(ref.size); } current_position += 1; @@ -165,7 +165,7 @@ void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, for (size_t i = 0; i < rows_read; ++i) { auto ref = column->getDataAt(current_position + i); - addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col], rows_read); + addToGinFilter(row_id, ref.data, ref.size, granule->gin_filters[col]); store->incrementCurrentSizeBy(ref.size); row_id++; if (store->needToWrite()) @@ -735,8 +735,8 @@ MergeTreeIndexPtr invertedIndexCreator( const IndexDescription & index) { size_t n = index.arguments.empty() ? 0 : index.arguments[0].get(); - Float64 density = index.arguments.size() < 2 ? 1.0 : index.arguments[1].get(); - GinFilterParameters params(n, density); + UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_ROWS_IN_POSTINGS_LIST : index.arguments[1].get(); + GinFilterParameters params(n, max_rows); /// Use SplitTokenExtractor when n is 0, otherwise use NgramTokenExtractor if (n > 0) @@ -780,13 +780,14 @@ void invertedIndexValidator(const IndexDescription & index, bool /*attach*/) if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INCORRECT_QUERY, "The first Inverted index argument must be positive integer."); - if (index.arguments.size() == 2 && (index.arguments[1].getType() != Field::Types::Float64 || index.arguments[1].get() <= 0 || index.arguments[1].get() > 1)) - throw Exception(ErrorCodes::INCORRECT_QUERY, "The second Inverted index argument must be a float between 0 and 1."); + if (index.arguments.size() == 2 && (index.arguments[1].getType() != Field::Types::UInt64 || + (index.arguments[1].get() != 0 && index.arguments[1].get() < MIN_ROWS_IN_POSTINGS_LIST))) + throw Exception(ErrorCodes::INCORRECT_QUERY, "The maximum rows in postings list must be no less than {}", MIN_ROWS_IN_POSTINGS_LIST); /// Just validate size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get(); - Float64 density = index.arguments.size() < 2 ? 1.0 : index.arguments[1].get(); - GinFilterParameters params(ngrams, density); + UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_ROWS_IN_POSTINGS_LIST : index.arguments[1].get(); + GinFilterParameters params(ngrams, max_rows); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.h b/src/Storages/MergeTree/MergeTreeIndexInverted.h index 96a87c2e2ba..96d12128bb4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.h +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.h @@ -48,7 +48,7 @@ struct MergeTreeIndexAggregatorInverted final : IMergeTreeIndexAggregator void update(const Block & block, size_t * pos, size_t limit) override; - void addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter, UInt64 limit); + void addToGinFilter(UInt32 rowID, const char * data, size_t length, GinFilter & gin_filter); GinIndexStorePtr store; Names index_columns; diff --git a/tests/queries/0_stateless/02346_full_text_search.sql b/tests/queries/0_stateless/02346_full_text_search.sql index c8536976377..7d3337b9407 100644 --- a/tests/queries/0_stateless/02346_full_text_search.sql +++ b/tests/queries/0_stateless/02346_full_text_search.sql @@ -277,74 +277,3 @@ SELECT * FROM tab WHERE str == 'b' AND 1.0; -- AND result_rows==1 -- LIMIT 1; -- --- ---------------------------------------------------- --- SELECT 'Test density==1'; --- --- DROP TABLE IF EXISTS tab; --- --- CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 1.0)) --- Engine=MergeTree --- ORDER BY (k) --- SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512 --- AS --- SELECT number, if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number))) --- FROM numbers(1024); --- --- -- check inverted index was created --- SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; --- --- -- search inverted index, no row has 'happy birthday' --- SELECT count() == 0 FROM tab WHERE s =='happy birthday'; --- --- -- check the query only skip all granules (0 row total; each granule has 512 rows) --- SYSTEM FLUSH LOGS; --- SELECT read_rows==0 from system.query_log --- WHERE query_kind ='Select' --- AND current_database = currentDatabase() --- AND endsWith(trimRight(query), 'SELECT count() == 0 FROM tab WHERE s ==\'happy birthday\';') --- AND type='QueryFinish' --- AND result_rows==1 --- LIMIT 1; --- --- ---------------------------------------------------- --- SELECT 'Test density==0.1'; --- --- DROP TABLE IF EXISTS tab; --- --- CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 0.1)) --- Engine=MergeTree --- ORDER BY (k) --- SETTINGS max_digestion_size_per_segment = 1, index_granularity = 512 --- AS --- SELECT number, if(number==1023, 'happy new year', if(number%2, format('happy {}', hex(number)), format('birthday {}', hex(number)))) --- FROM numbers(1024); --- --- -- check inverted index was created --- --- SELECT name, type FROM system.data_skipping_indices WHERE table == 'tab' AND database = currentDatabase() LIMIT 1; --- --- -- search inverted index, no row has 'happy birthday' --- SELECT count() == 0 FROM tab WHERE s == 'happy birthday'; --- --- -- check the query does not skip any of the 2 granules(1024 rows total; each granule has 512 rows) --- SYSTEM FLUSH LOGS; --- SELECT read_rows==1024 from system.query_log --- WHERE query_kind ='Select' --- AND current_database = currentDatabase() --- AND endsWith(trimRight(query), 'SELECT count() == 0 FROM tab WHERE s == \'happy birthday\';') --- AND type='QueryFinish' --- AND result_rows==1 --- LIMIT 1; --- --- -- search inverted index, no row has 'happy new year' --- SELECT count() == 1 FROM tab WHERE s == 'happy new year'; --- --- -- check the query only read 1 granule because of density (1024 rows total; each granule has 512 rows) --- SYSTEM FLUSH LOGS; --- SELECT read_rows==512 from system.query_log --- WHERE query_kind ='Select' --- AND current_database = currentDatabase() --- AND endsWith(trimRight(query), 'SELECT count() == 1 FROM tab WHERE s == \'happy new year\';') --- AND type='QueryFinish' --- AND result_rows==1 --- LIMIT 1; From 447631976e2e2364760be25494c87e1bff10e573 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 13 Oct 2023 20:36:14 +0200 Subject: [PATCH 038/120] Add MySQL String BLOB vs TEXT configuration --- src/Core/Settings.h | 4 +- .../InterpreterShowColumnsQuery.cpp | 52 ++--- ...show_columns_mysql_compatibility.reference | 183 ++++++++++++++++++ ...02775_show_columns_mysql_compatibility.sql | 9 + 4 files changed, 224 insertions(+), 24 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c08425c03fd..41a99ff1319 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -205,7 +205,9 @@ class IColumn; M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ - M(Bool, use_mysql_types_in_show_columns, false, "Show MySQL types in SHOW COLUMNS and system.columns", 0) \ + M(Bool, use_mysql_types_in_show_columns, false, "Show native MySQL types in SHOW [FULL] COLUMNS", 0) \ + M(Bool, mysql_remap_string_as_text_in_show_columns, false, "If enabled, String type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Only works if use_mysql_types_in_show_columns is enabled too", 0) \ + M(Bool, mysql_remap_fixed_string_as_text_in_show_columns, false, "If enabled, FixedString type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Only works if use_mysql_types_in_show_columns is enabled too", 0) \ \ M(UInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \ \ diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index fe8ac63c214..a213f27972a 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -24,7 +24,10 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() { const auto & query = query_ptr->as(); - const bool use_mysql_types = getContext()->getSettingsRef().use_mysql_types_in_show_columns; + const auto & settings = getContext()->getSettingsRef(); + const bool use_mysql_types = settings.use_mysql_types_in_show_columns; + const bool remap_string_as_text = settings.mysql_remap_string_as_text_in_show_columns; + const bool remap_fixed_string_as_text = settings.mysql_remap_fixed_string_as_text_in_show_columns; WriteBufferFromOwnString buf_database; String resolved_database = getContext()->resolveDatabase(query.database); @@ -41,28 +44,29 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() /// Only used with setting 'use_mysql_types_in_show_columns = 1' /// Known issues: /// - Enums are translated to TEXT - rewritten_query += R"( + rewritten_query += fmt::format(R"( WITH map( - 'Int8', 'TINYINT', - 'Int16', 'SMALLINT', - 'Int32', 'INTEGER', - 'Int64', 'BIGINT', - 'UInt8', 'TINYINT UNSIGNED', - 'UInt16', 'SMALLINT UNSIGNED', - 'UInt32', 'INTEGER UNSIGNED', - 'UInt64', 'BIGINT UNSIGNED', - 'Float32', 'FLOAT', - 'Float64', 'DOUBLE', - 'String', 'BLOB', - 'UUID', 'CHAR', - 'Bool', 'TINYINT', - 'Date', 'DATE', - 'Date32', 'DATE', - 'DateTime', 'DATETIME', - 'DateTime64', 'DATETIME', - 'Map', 'JSON', - 'Tuple', 'JSON', - 'Object', 'JSON') AS native_to_mysql_mapping, + 'Int8', 'TINYINT', + 'Int16', 'SMALLINT', + 'Int32', 'INTEGER', + 'Int64', 'BIGINT', + 'UInt8', 'TINYINT UNSIGNED', + 'UInt16', 'SMALLINT UNSIGNED', + 'UInt32', 'INTEGER UNSIGNED', + 'UInt64', 'BIGINT UNSIGNED', + 'Float32', 'FLOAT', + 'Float64', 'DOUBLE', + 'String', '{}', + 'FixedString', '{}', + 'UUID', 'CHAR', + 'Bool', 'TINYINT', + 'Date', 'DATE', + 'Date32', 'DATE', + 'DateTime', 'DATETIME', + 'DateTime64', 'DATETIME', + 'Map', 'JSON', + 'Tuple', 'JSON', + 'Object', 'JSON') AS native_to_mysql_mapping, splitByRegexp('\(|\)', type_) AS split, multiIf(startsWith(type_, 'LowCardinality(Nullable'), split[3], startsWith(type_, 'LowCardinality'), split[2], @@ -72,7 +76,9 @@ WITH map( multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'), mapContains(native_to_mysql_mapping, inner_type) = true, native_to_mysql_mapping[inner_type], 'TEXT') AS mysql_type - )"; + )", + remap_string_as_text ? "TEXT" : "BLOB", + remap_fixed_string_as_text ? "TEXT" : "BLOB"); rewritten_query += R"( SELECT diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference index 6613e4f170e..29db3e1b73b 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference @@ -84,6 +84,128 @@ dt_tz2 DATETIME NO \N enm TEXT NO \N f32 FLOAT NO \N f64 DOUBLE NO \N +fs BLOB NO \N +i128 TEXT NO \N +i16 SMALLINT NO \N +i256 TEXT NO \N +i32 INTEGER NO \N +i64 BIGINT NO \N +i8 TINYINT NO \N +ip4 TEXT NO \N +ip6 TEXT NO \N +lfs BLOB NO \N +lnfs BLOB YES \N +lns BLOB YES \N +ls BLOB NO \N +m JSON NO \N +m_complex JSON NO \N +mpg TEXT NO \N +ndt64 DATETIME YES \N +ndt64_tz DATETIME YES \N +nested.col1 TEXT NO \N +nested.col2 TEXT NO \N +nfs BLOB YES \N +ns BLOB YES \N +o JSON NO \N +p TEXT NO \N +pg TEXT NO \N +r TEXT NO \N +s BLOB NO \N +sagg TEXT NO \N +t JSON NO \N +ui128 TEXT NO \N +ui16 SMALLINT UNSIGNED NO \N +ui256 TEXT NO \N +ui32 INTEGER UNSIGNED NO \N +ui64 BIGINT UNSIGNED NO \N +ui8 TINYINT UNSIGNED NO \N +uuid CHAR NO \N +-- SHOW COLUMNS with mysql_remap_string_as_text_in_show_columns = 1 +a TEXT NO \N +agg TEXT NO \N +b TINYINT NO \N +d DATE NO \N +d32 DATE NO \N +dec128 DECIMAL(38, 2) NO \N +dec128_native DECIMAL(35, 30) NO \N +dec128_text TEXT NO \N +dec256 TEXT NO \N +dec256_native DECIMAL(65, 2) NO \N +dec256_text TEXT NO \N +dec32 DECIMAL(9, 2) NO \N +dec64 DECIMAL(18, 2) NO \N +dt DATETIME NO \N +dt64 DATETIME NO \N +dt64_3_tz1 DATETIME NO \N +dt64_3_tz2 DATETIME NO \N +dt64_6 DATETIME NO \N +dt64_9 DATETIME NO \N +dt_tz1 DATETIME NO \N +dt_tz2 DATETIME NO \N +enm TEXT NO \N +f32 FLOAT NO \N +f64 DOUBLE NO \N +fs BLOB NO \N +i128 TEXT NO \N +i16 SMALLINT NO \N +i256 TEXT NO \N +i32 INTEGER NO \N +i64 BIGINT NO \N +i8 TINYINT NO \N +ip4 TEXT NO \N +ip6 TEXT NO \N +lfs BLOB NO \N +lnfs BLOB YES \N +lns TEXT YES \N +ls TEXT NO \N +m JSON NO \N +m_complex JSON NO \N +mpg TEXT NO \N +ndt64 DATETIME YES \N +ndt64_tz DATETIME YES \N +nested.col1 TEXT NO \N +nested.col2 TEXT NO \N +nfs BLOB YES \N +ns TEXT YES \N +o JSON NO \N +p TEXT NO \N +pg TEXT NO \N +r TEXT NO \N +s TEXT NO \N +sagg TEXT NO \N +t JSON NO \N +ui128 TEXT NO \N +ui16 SMALLINT UNSIGNED NO \N +ui256 TEXT NO \N +ui32 INTEGER UNSIGNED NO \N +ui64 BIGINT UNSIGNED NO \N +ui8 TINYINT UNSIGNED NO \N +uuid CHAR NO \N +-- SHOW COLUMNS with mysql_remap_fixed_string_as_text_in_show_columns = 1 +a TEXT NO \N +agg TEXT NO \N +b TINYINT NO \N +d DATE NO \N +d32 DATE NO \N +dec128 DECIMAL(38, 2) NO \N +dec128_native DECIMAL(35, 30) NO \N +dec128_text TEXT NO \N +dec256 TEXT NO \N +dec256_native DECIMAL(65, 2) NO \N +dec256_text TEXT NO \N +dec32 DECIMAL(9, 2) NO \N +dec64 DECIMAL(18, 2) NO \N +dt DATETIME NO \N +dt64 DATETIME NO \N +dt64_3_tz1 DATETIME NO \N +dt64_3_tz2 DATETIME NO \N +dt64_6 DATETIME NO \N +dt64_9 DATETIME NO \N +dt_tz1 DATETIME NO \N +dt_tz2 DATETIME NO \N +enm TEXT NO \N +f32 FLOAT NO \N +f64 DOUBLE NO \N fs TEXT NO \N i128 TEXT NO \N i16 SMALLINT NO \N @@ -120,3 +242,64 @@ ui32 INTEGER UNSIGNED NO \N ui64 BIGINT UNSIGNED NO \N ui8 TINYINT UNSIGNED NO \N uuid CHAR NO \N +-- SHOW COLUMNS with both mysql_remap_string_as_text_in_show_columns = 1 and mysql_remap_fixed_string_as_text_in_show_columns = 1 +a TEXT NO \N +agg TEXT NO \N +b TINYINT NO \N +d DATE NO \N +d32 DATE NO \N +dec128 DECIMAL(38, 2) NO \N +dec128_native DECIMAL(35, 30) NO \N +dec128_text TEXT NO \N +dec256 TEXT NO \N +dec256_native DECIMAL(65, 2) NO \N +dec256_text TEXT NO \N +dec32 DECIMAL(9, 2) NO \N +dec64 DECIMAL(18, 2) NO \N +dt DATETIME NO \N +dt64 DATETIME NO \N +dt64_3_tz1 DATETIME NO \N +dt64_3_tz2 DATETIME NO \N +dt64_6 DATETIME NO \N +dt64_9 DATETIME NO \N +dt_tz1 DATETIME NO \N +dt_tz2 DATETIME NO \N +enm TEXT NO \N +f32 FLOAT NO \N +f64 DOUBLE NO \N +fs TEXT NO \N +i128 TEXT NO \N +i16 SMALLINT NO \N +i256 TEXT NO \N +i32 INTEGER NO \N +i64 BIGINT NO \N +i8 TINYINT NO \N +ip4 TEXT NO \N +ip6 TEXT NO \N +lfs TEXT NO \N +lnfs TEXT YES \N +lns TEXT YES \N +ls TEXT NO \N +m JSON NO \N +m_complex JSON NO \N +mpg TEXT NO \N +ndt64 DATETIME YES \N +ndt64_tz DATETIME YES \N +nested.col1 TEXT NO \N +nested.col2 TEXT NO \N +nfs TEXT YES \N +ns TEXT YES \N +o JSON NO \N +p TEXT NO \N +pg TEXT NO \N +r TEXT NO \N +s TEXT NO \N +sagg TEXT NO \N +t JSON NO \N +ui128 TEXT NO \N +ui16 SMALLINT UNSIGNED NO \N +ui256 TEXT NO \N +ui32 INTEGER UNSIGNED NO \N +ui64 BIGINT UNSIGNED NO \N +ui8 TINYINT UNSIGNED NO \N +uuid CHAR NO \N diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql index 34c034fa77f..5f7d1881702 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql @@ -78,4 +78,13 @@ SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 0; SELECT '-- SHOW COLUMNS with use_mysql_types_in_show_columns = 1'; SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 1; +SELECT '-- SHOW COLUMNS with mysql_remap_string_as_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS mysql_remap_string_as_text_in_show_columns=1; + +SELECT '-- SHOW COLUMNS with mysql_remap_fixed_string_as_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS mysql_remap_fixed_string_as_text_in_show_columns=1; + +SELECT '-- SHOW COLUMNS with both mysql_remap_string_as_text_in_show_columns = 1 and mysql_remap_fixed_string_as_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS mysql_remap_string_as_text_in_show_columns=1, mysql_remap_fixed_string_as_text_in_show_columns=1; + DROP TABLE tab; From 42fc670c2242977219eb541ea1423ab7c3711b9a Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Fri, 13 Oct 2023 20:56:12 +0200 Subject: [PATCH 039/120] Add new settings docs entries --- docs/en/operations/settings/settings.md | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2cb85a61be5..f22badb8c59 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3290,6 +3290,28 @@ Possible values: Default value: `0`. +## mysql_remap_string_as_text_in_show_columns {#mysql_remap_string_as_text_in_show_columns} + +When enabled, [String](../../sql-reference/data-types/string.md) ClickHouse data type will be displayed as `TEXT` in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns). + +Has effect only when [use_mysql_types_in_show_columns](#use_mysql_types_in_show_columns) is enabled. + +- 0 - Use `BLOB`. +- 1 - Use `TEXT`. + +Default value: `0`. + +## mysql_remap_fixed_string_as_text_in_show_columns {#mysql_remap_fixed_string_as_text_in_show_columns} + +When enabled, [FixedString](../../sql-reference/data-types/fixedstring.md) ClickHouse data type will be displayed as `TEXT` in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns). + +Has effect only when [use_mysql_types_in_show_columns](#use_mysql_types_in_show_columns) is enabled. + +- 0 - Use `BLOB`. +- 1 - Use `TEXT`. + +Default value: `0`. + ## execute_merges_on_single_replica_time_threshold {#execute-merges-on-single-replica-time-threshold} Enables special logic to perform merges on replicas. From bbd67d262a5c51afea7690b9b56bf64636d64ab1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 14 Oct 2023 02:52:53 +0200 Subject: [PATCH 040/120] Merging #52352 --- src/Storages/StorageSet.cpp | 69 +++++++++++++++++-- src/Storages/StorageSet.h | 6 +- .../02867_storage_set_tsan.reference | 0 .../0_stateless/02867_storage_set_tsan.sh | 42 +++++++++++ 4 files changed, 108 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02867_storage_set_tsan.reference create mode 100755 tests/queries/0_stateless/02867_storage_set_tsan.sh diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 79369ab4bcb..c8b38186dfb 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -156,12 +156,62 @@ StorageSet::StorageSet( } -void StorageSet::insertBlock(const Block & block, ContextPtr) { set->insertFromBlock(block.getColumnsWithTypeAndName()); } -void StorageSet::finishInsert() { set->finishInsert(); } +SetPtr StorageSet::getSet() const +{ + std::lock_guard lock(mutex); + return set; +} -size_t StorageSet::getSize(ContextPtr) const { return set->getTotalRowCount(); } -std::optional StorageSet::totalRows(const Settings &) const { return set->getTotalRowCount(); } -std::optional StorageSet::totalBytes(const Settings &) const { return set->getTotalByteCount(); } + +void StorageSet::insertBlock(const Block & block, ContextPtr) +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + current_set->insertFromBlock(block.getColumnsWithTypeAndName()); +} + +void StorageSet::finishInsert() +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + current_set->finishInsert(); +} + +size_t StorageSet::getSize(ContextPtr) const +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + return current_set->getTotalRowCount(); +} + +std::optional StorageSet::totalRows(const Settings &) const +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + return current_set->getTotalRowCount(); +} + +std::optional StorageSet::totalBytes(const Settings &) const +{ + SetPtr current_set; + { + std::lock_guard lock(mutex); + current_set = set; + } + return current_set->getTotalRowCount(); +} void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) { @@ -176,8 +226,13 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn Block header = metadata_snapshot->getSampleBlock(); increment = 0; - set = std::make_shared(SizeLimits(), 0, true); - set->setHeader(header.getColumnsWithTypeAndName()); + + auto new_set = std::make_shared(SizeLimits(), 0, true); + new_set->setHeader(header.getColumnsWithTypeAndName()); + { + std::lock_guard lock(mutex); + set = new_set; + } } diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index b310f817eb9..67a9528ff5e 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -79,7 +79,7 @@ public: String getName() const override { return "Set"; } /// Access the insides. - SetPtr & getSet() { return set; } + SetPtr getSet() const; void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; @@ -87,7 +87,9 @@ public: std::optional totalBytes(const Settings & settings) const override; private: - SetPtr set; + /// Allows to concurrently truncate the set and work (read/fill) the existing set. + mutable std::mutex mutex; + SetPtr set TSA_GUARDED_BY(mutex); void insertBlock(const Block & block, ContextPtr) override; void finishInsert() override; diff --git a/tests/queries/0_stateless/02867_storage_set_tsan.reference b/tests/queries/0_stateless/02867_storage_set_tsan.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02867_storage_set_tsan.sh b/tests/queries/0_stateless/02867_storage_set_tsan.sh new file mode 100755 index 00000000000..81ae5f0bda8 --- /dev/null +++ b/tests/queries/0_stateless/02867_storage_set_tsan.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# Tags: race, no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -mn -q """ +DROP TABLE IF EXISTS t1_02867; +CREATE TABLE t1_02867 (x UInt64) ENGINE=Set(); +""" + +function repeat_select() { + n=0 + while [ "$n" -lt 20 ]; + do + n=$(( n + 1 )) + $CLICKHOUSE_CLIENT -q "SELECT count() as a FROM numbers(10) WHERE number IN t1_02867" > /dev/null 2> /dev/null || exit + done +} + +function repeat_truncate_insert() { + n=0 + while [ "$n" -lt 20 ]; + do + n=$(( n + 1 )) + $CLICKHOUSE_CLIENT -q "TRUNCATE t1_02867;" > /dev/null 2> /dev/null || exit + done +} + +repeat_select & +repeat_truncate_insert & +repeat_select & +repeat_truncate_insert & +repeat_select & +repeat_truncate_insert & +repeat_select & +repeat_truncate_insert & + +sleep 10 + +$CLICKHOUSE_CLIENT -mn -q "DROP TABLE IF EXISTS t1_02867;" From e3976a5e5008fad5c8689a87cc1bba17ab648256 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 5 Oct 2023 18:52:20 +0200 Subject: [PATCH 041/120] Do not allow tests with state ERROR be overwritten by PASSED In the ERROR case there can be sanitizers issues, that should not be hidden, like right now it is doing for `test_s3_table_functions/test.py::test_s3_table_functions_timeouts` test [1], but I was lucky enough to trigger this in [2]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/55247/e99b0f46961733fa8ba10e490279dbcb0cdd67ad/integration_tests__asan__[5_6].html [2]: https://s3.amazonaws.com/clickhouse-test-reports/55245/918d65d6707c69ab541cdb56a076cdb83845d3ed/integration_tests__asan__[5_6].html Though there could be also other issues, like with dependent services, but still, let's try. Signed-off-by: Azat Khuzhin --- tests/integration/ci-runner.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 5c3a7695119..177a43cc591 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -492,8 +492,6 @@ class ClickhouseIntegrationTestsRunner: if test not in main_counters["PASSED"]: if test in main_counters["FAILED"]: main_counters["FAILED"].remove(test) - if test in main_counters["ERROR"]: - main_counters["ERROR"].remove(test) if test in main_counters["BROKEN"]: main_counters["BROKEN"].remove(test) @@ -506,7 +504,6 @@ class ClickhouseIntegrationTestsRunner: for test in current_counters[state]: if test in main_counters["PASSED"]: main_counters["PASSED"].remove(test) - continue if test not in broken_tests: if test not in main_counters[state]: main_counters[state].append(test) From 395a54915dc94c3a50126ec145055d4d59ddb393 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sun, 15 Oct 2023 10:40:37 +0200 Subject: [PATCH 042/120] Fix totalBytes() --- src/Storages/StorageSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index c8b38186dfb..1b0db1da800 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -210,7 +210,7 @@ std::optional StorageSet::totalBytes(const Settings &) const std::lock_guard lock(mutex); current_set = set; } - return current_set->getTotalRowCount(); + return current_set->getTotalByteCount(); } void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) From e0668e9ea01c879ac175786006ddba2eaf26844b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 15 Oct 2023 14:19:44 +0200 Subject: [PATCH 043/120] Update test.py --- .../test.py | 97 ++++++++++--------- 1 file changed, 52 insertions(+), 45 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 2c1a8a0a3b6..7bdec76e4eb 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -719,51 +719,6 @@ def test_too_many_parts(started_cluster): pg_manager2.drop_materialized_db() -def test_replica_consumer(started_cluster): - table = "test_replica_consumer" - - pg_manager_replica = PostgresManager() - pg_manager_replica.init( - instance2, - cluster.postgres_ip, - cluster.postgres_port, - default_database="postgres_database", - postgres_db_exists=True, - ) - - for pm in [pg_manager, pg_manager_replica]: - pm.create_and_fill_postgres_table(table) - pm.create_materialized_db( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - settings=[ - f"materialized_postgresql_tables_list = '{table}'", - "materialized_postgresql_backoff_min_ms = 100", - "materialized_postgresql_backoff_max_ms = 100", - "materialized_postgresql_use_unique_replication_consumer_identifier = 1", - ], - ) - - assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) - assert 50 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) - - instance.query( - f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)" - ) - - check_tables_are_synchronized( - instance, table, postgres_database=pg_manager.get_default_database() - ) - check_tables_are_synchronized( - instance2, table, postgres_database=pg_manager_replica.get_default_database() - ) - - assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) - assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) - - pg_manager_replica.clear() - - def test_toast(started_cluster): table = "test_toast" pg_manager.execute( @@ -794,6 +749,58 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) ) +def test_replica_consumer(started_cluster): + table = "test_replica_consumer" + + pg_manager_replica = PostgresManager() + pg_manager_replica.init( + instance2, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres_database", + postgres_db_exists=True, + ) + + for pm in [pg_manager, pg_manager_replica]: + pm.create_and_fill_postgres_table(table) + pm.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100", + "materialized_postgresql_use_unique_replication_consumer_identifier = 1", + ], + ) + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance2, table, postgres_database=pg_manager_replica.get_default_database() + ) + + assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 50 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + instance.query( + f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(1000, 1000)" + ) + + check_tables_are_synchronized( + instance, table, postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance2, table, postgres_database=pg_manager_replica.get_default_database() + ) + + assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) + assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + + pg_manager_replica.clear() + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 0ddee6ad6c2d201534761944ce182ac529194837 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 15 Oct 2023 14:44:57 +0200 Subject: [PATCH 044/120] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 33 ++++++++++++++----- src/Storages/S3Queue/S3QueueSource.cpp | 1 + 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index ac12e8fe70f..150174aabcb 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -326,8 +326,13 @@ std::pairerror != Coordination::Error::ZOK) + { return std::pair{SetFileProcessingResult::AlreadyProcessed, nullptr}; - - if (responses[1]->error != Coordination::Error::ZOK) + } + else if (responses[2]->error != Coordination::Error::ZOK) + { return std::pair{SetFileProcessingResult::AlreadyFailed, nullptr}; - - chassert(responses[2]->error != Coordination::Error::ZOK); - return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; + } + else if (responses[4]->error != Coordination::Error::ZOK) + { + return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state of zookeeper transaction: {}", magic_enum::enum_name(code)); + } } std::pairerror != Coordination::Error::ZOK) + else if (responses[2]->error != Coordination::Error::ZOK) { LOG_TEST(log, "Skipping file `{}`: already processing", path); return std::pair{SetFileProcessingResult::ProcessingByOtherNode, nullptr}; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index b3f8f0cf1c3..1a44fe9cff8 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -149,6 +149,7 @@ Chunk StorageS3QueueSource::generate() auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); SCOPE_EXIT({ CurrentThread::get().attachProfileCountersScope(prev_scope); }); + /// FIXME: if files are compressed, profile counters update does not work fully (s3 related counters are not saved). Why? try { From 41479b21e60f85f953b4b5b52697efd1238ead31 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 15:33:33 +0200 Subject: [PATCH 045/120] Restore only top-level parts instead of unexpected --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 74 +++++++++-- src/Storages/MergeTree/ActiveDataPartSet.h | 11 ++ src/Storages/MergeTree/MergeTreeData.cpp | 130 +++++++++---------- 3 files changed, 136 insertions(+), 79 deletions(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index 5b7965bc3a0..aac1705c3b9 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -21,18 +21,46 @@ ActiveDataPartSet::ActiveDataPartSet(MergeTreeDataFormatVersion format_version_, add(name); } -bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) +ActiveDataPartSet::AddPartOutcome ActiveDataPartSet::tryAddPart(const MergeTreePartInfo & part_info, String * out_reason) { - auto part_info = MergeTreePartInfo::fromPartName(name, format_version); - return add(part_info, name, out_replaced_parts); + return addImpl(part_info, part_info.getPartNameAndCheckFormat(format_version), nullptr, out_reason); } bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts) +{ + String out_reason; + AddPartOutcome outcome = addImpl(part_info, name, out_replaced_parts, &out_reason); + if (outcome == AddPartOutcome::HasIntersectingPart) + { + chassert(out_reason.empty()); + throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); + } + + return outcome == AddPartOutcome::Added; +} + + +bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) +{ + auto part_info = MergeTreePartInfo::fromPartName(name, format_version); + String out_reason; + AddPartOutcome outcome = addImpl(part_info, name, out_replaced_parts, &out_reason); + if (outcome == AddPartOutcome::HasIntersectingPart) + { + chassert(out_reason.empty()); + throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); + } + + return outcome == AddPartOutcome::Added; +} + + +ActiveDataPartSet::AddPartOutcome ActiveDataPartSet::addImpl(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts, String * out_reason) { /// TODO make it exception safe (out_replaced_parts->push_back(...) may throw) if (getContainingPartImpl(part_info) != part_info_to_name.end()) - return false; + return AddPartOutcome::HasCovering; /// Parts contained in `part` are located contiguously in `part_info_to_name`, overlapping with the place where the part itself would be inserted. auto it = part_info_to_name.lower_bound(part_info); @@ -47,10 +75,15 @@ bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, const String & if (!part_info.contains(it->first)) { if (!part_info.isDisjoint(it->first)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Part {} intersects previous part {}. " - "It is a bug or a result of manual intervention in the ZooKeeper data.", - part_info.getPartNameForLogs(), it->first.getPartNameForLogs()); + { + if (out_reason != nullptr) + *out_reason = fmt::format( + "Part {} intersects previous part {}. " + "It is a bug or a result of manual intervention in the ZooKeeper data.", + part_info.getPartNameForLogs(), + it->first.getPartNameForLogs()); + return AddPartOutcome::HasIntersectingPart; + } ++it; break; } @@ -73,18 +106,33 @@ bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, const String & } if (it != part_info_to_name.end() && !part_info.isDisjoint(it->first)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Part {} intersects part {}. It is a bug or a result of manual intervention " - "in the ZooKeeper data.", name, it->first.getPartNameForLogs()); + { + if (out_reason != nullptr) + *out_reason = fmt::format( + "Part {} intersects part {}. It is a bug or a result of manual intervention " + "in the ZooKeeper data.", + name, + it->first.getPartNameForLogs()); + + return AddPartOutcome::HasIntersectingPart; + } part_info_to_name.emplace(part_info, name); - return true; + return AddPartOutcome::Added; } bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, Strings * out_replaced_parts) { - return add(part_info, part_info.getPartNameAndCheckFormat(format_version), out_replaced_parts); + String out_reason; + AddPartOutcome outcome = addImpl(part_info, part_info.getPartNameAndCheckFormat(format_version), out_replaced_parts, &out_reason); + if (outcome == AddPartOutcome::HasIntersectingPart) + { + chassert(out_reason.empty()); + throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); + } + + return outcome == AddPartOutcome::Added; } diff --git a/src/Storages/MergeTree/ActiveDataPartSet.h b/src/Storages/MergeTree/ActiveDataPartSet.h index f3cd6b0019d..662309f64cf 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.h +++ b/src/Storages/MergeTree/ActiveDataPartSet.h @@ -22,6 +22,13 @@ using Strings = std::vector; class ActiveDataPartSet { public: + enum class AddPartOutcome + { + Added, + HasCovering, + HasIntersectingPart, + }; + explicit ActiveDataPartSet(MergeTreeDataFormatVersion format_version_) : format_version(format_version_) {} ActiveDataPartSet(MergeTreeDataFormatVersion format_version_, const Strings & names); @@ -43,6 +50,8 @@ public: bool add(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts = nullptr); bool add(const MergeTreePartInfo & part_info, Strings * out_replaced_parts = nullptr); + AddPartOutcome tryAddPart(const MergeTreePartInfo & part_info, String * out_reason = nullptr); + bool remove(const MergeTreePartInfo & part_info) { return part_info_to_name.erase(part_info) > 0; @@ -97,6 +106,8 @@ public: MergeTreeDataFormatVersion getFormatVersion() const { return format_version; } private: + + AddPartOutcome addImpl(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts = nullptr, String * out_reason = nullptr); MergeTreeDataFormatVersion format_version; std::map part_info_to_name; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 889dcfa537f..fcfbe27c741 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -73,6 +73,7 @@ #include #include #include +#include #include #include #include @@ -3912,25 +3913,17 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT return; } + /// Let's restore some parts covered by unexpected to avoid partial data if (restore_covered) { Strings restored; - bool error = false; - String error_parts; - - Int64 pos = part->info.min_block; + Strings error_parts; auto is_appropriate_state = [] (DataPartState state) { return state == DataPartState::Active || state == DataPartState::Outdated; }; - auto update_error = [&] (DataPartIteratorByInfo it) - { - error = true; - error_parts += (*it)->getNameWithState() + " "; - }; - auto activate_part = [this, &restored_active_part](auto it) { /// It's not clear what to do if we try to activate part that was removed in transaction. @@ -3948,68 +3941,73 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT restored_active_part = true; }; - auto it_middle = data_parts_by_info.lower_bound(part->info); - - /// Restore the leftmost part covered by the part - if (it_middle != data_parts_by_info.begin()) + /// ActiveDataPartSet allows to restore most top-level parts instead of unexpected. + /// It can be important in case of assigned merges. If unexpected part is result of some + /// finished, but not committed merge we should restore closest ancestors for the + /// unexpected part to be able to execute it. + ActiveDataPartSet parts_for_replacement(format_version); + for (const auto & part_candidate_in_partition : getDataPartsPartitionRange(part->info.partition_id)) { - auto it = std::prev(it_middle); - - if (part->contains(**it) && is_appropriate_state((*it)->getState())) + if (part->info.contains(part_candidate_in_partition->info) + && is_appropriate_state(part_candidate_in_partition->getState())) { - /// Maybe, we must consider part level somehow - if ((*it)->info.min_block != part->info.min_block) - update_error(it); - - if ((*it)->getState() != DataPartState::Active) - activate_part(it); - - pos = (*it)->info.max_block + 1; - restored.push_back((*it)->name); + String out_reason; + /// Outdated parts can itersect legally (because of DROP_PART) here it's okay, we + /// are trying to do out best to restore covered parts. + auto outcome = parts_for_replacement.tryAddPart(part_candidate_in_partition->info, &out_reason); + if (outcome == ActiveDataPartSet::AddPartOutcome::HasIntersectingPart) + { + error_parts.push_back(part->name); + LOG_ERROR(log, "Failed to restore part {}, because of intersection reason '{}'", part->name, out_reason); + } + } + } + + if (parts_for_replacement.size() > 0) + { + std::vector> holes_list; + auto part_infos = parts_for_replacement.getPartInfos(); + int64_t current_right_block = part_infos[0].min_block; + for (const auto & top_level_part_to_replace : part_infos) + { + auto data_part_it = data_parts_by_info.find(top_level_part_to_replace); + if (data_part_it == data_parts_by_info.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find part {} in own set", top_level_part_to_replace.getPartNameForLogs()); + activate_part(data_part_it); + restored.push_back((*data_part_it)->name); + if (top_level_part_to_replace.min_block - current_right_block > 1) + holes_list.emplace_back(current_right_block, top_level_part_to_replace.min_block); + current_right_block = top_level_part_to_replace.max_block; + } + if (part->info.max_block != current_right_block) + holes_list.emplace_back(current_right_block, part->info.max_block); + + for (const String & name : restored) + LOG_INFO(log, "Activated part {} in place of unexpected {}", name, part->name); + + if (!error_parts.empty() || !holes_list.empty()) + { + std::string error_parts_message, holes_list_message; + if (!error_parts.empty()) + error_parts_message = fmt::format(" Parts failed to restore because of intersection: [{}]", fmt::join(error_parts, ", ")); + if (!holes_list.empty()) + { + if (!error_parts.empty()) + holes_list_message = "."; + + Strings holes_list_pairs; + for (const auto & [left_side, right_side] : holes_list) + holes_list_pairs.push_back(fmt::format("({}, {})", left_side + 1, right_side - 1)); + holes_list_message += fmt::format(" Block ranges failed to restore: [{}]", fmt::join(holes_list_pairs, ", ")); + } + LOG_WARNING(log, "The set of parts restored in place of {} looks incomplete. " + "SELECT queries may observe gaps in data until this replica is synchronized with other replicas.{}{}", + part->name, error_parts_message, holes_list_message); } - else if ((*it)->info.partition_id == part->info.partition_id) - update_error(it); - else - error = true; } else - error = true; - - /// Restore "right" parts - for (auto it = it_middle; it != data_parts_by_info.end() && part->contains(**it); ++it) { - if ((*it)->info.min_block < pos) - continue; - - if (!is_appropriate_state((*it)->getState())) - { - update_error(it); - continue; - } - - if ((*it)->info.min_block > pos) - update_error(it); - - if ((*it)->getState() != DataPartState::Active) - activate_part(it); - - pos = (*it)->info.max_block + 1; - restored.push_back((*it)->name); - } - - if (pos != part->info.max_block + 1) - error = true; - - for (const String & name : restored) - { - LOG_INFO(log, "Activated part {}", name); - } - - if (error) - { - LOG_WARNING(log, "The set of parts restored in place of {} looks incomplete. " - "SELECT queries may observe gaps in data until this replica is synchronized with other replicas.{}", - part->name, (error_parts.empty() ? "" : " Suspicious parts: " + error_parts)); + LOG_INFO(log, "Don't find any parts for replacement instead of unexpected {}", part->name); } } From 17ce951ed79debc5757de6807d71111b629d54a3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 15 Oct 2023 18:12:01 +0200 Subject: [PATCH 046/120] Update test.py --- .../test_postgresql_replica_database_engine_2/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 7bdec76e4eb..82730c34664 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -798,6 +798,8 @@ def test_replica_consumer(started_cluster): assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) + for pm in [pg_manager, pg_manager_replica]: + pm.drop_materialized_db() pg_manager_replica.clear() From 6ac790e91831ece6b34718cc717aef7bf645f7e4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 20:17:46 +0200 Subject: [PATCH 047/120] Update src/Storages/MergeTree/ActiveDataPartSet.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index aac1705c3b9..c192e3484b9 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -128,7 +128,7 @@ bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, Strings * out_r AddPartOutcome outcome = addImpl(part_info, part_info.getPartNameAndCheckFormat(format_version), out_replaced_parts, &out_reason); if (outcome == AddPartOutcome::HasIntersectingPart) { - chassert(out_reason.empty()); + chassert(!out_reason.empty()); throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); } From 13ca29649706d396c1b595f1fab2e41c05315819 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 20:17:50 +0200 Subject: [PATCH 048/120] Update src/Storages/MergeTree/ActiveDataPartSet.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index c192e3484b9..67f7004cd70 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -47,7 +47,7 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) AddPartOutcome outcome = addImpl(part_info, name, out_replaced_parts, &out_reason); if (outcome == AddPartOutcome::HasIntersectingPart) { - chassert(out_reason.empty()); + chassert(!out_reason.empty()); throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); } From bdafe17e9a486caac402067869375636fb1bf7b7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 15 Oct 2023 20:17:55 +0200 Subject: [PATCH 049/120] Update src/Storages/MergeTree/ActiveDataPartSet.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/ActiveDataPartSet.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index 67f7004cd70..a1746cc1746 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -32,7 +32,7 @@ bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, const String & AddPartOutcome outcome = addImpl(part_info, name, out_replaced_parts, &out_reason); if (outcome == AddPartOutcome::HasIntersectingPart) { - chassert(out_reason.empty()); + chassert(!out_reason.empty()); throw Exception(ErrorCodes::LOGICAL_ERROR, fmt::runtime(out_reason)); } From 750d45bad381388400f6994295f69a97c51e5775 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 16 Oct 2023 05:25:59 +0200 Subject: [PATCH 050/120] Add support for touch devices --- programs/server/dashboard.html | 144 +++++++++++++++++---------------- 1 file changed, 76 insertions(+), 68 deletions(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 555d039cec3..0c311fb8250 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -817,77 +817,85 @@ function insertChart(i) { let move_text = document.createTextNode('✥'); move.appendChild(move_text); - let is_dragging = false; - move.addEventListener('mousedown', e => { - const idx = getCurrentIndex(); - is_dragging = true; + let drag_state = { + is_dragging: false, + idx: null, + offset_x: null, + offset_y: null, + displace_idx: null, + displace_chart: null + }; + + function dragStop(e) { + drag_state.is_dragging = false; + chart.className = 'chart'; + chart.style.left = null; + chart.style.top = null; + + if (drag_state.displace_idx !== null) { + const elem = queries[drag_state.idx]; + queries.splice(drag_state.idx, 1); + queries.splice(drag_state.displace_idx, 0, elem); + + drag_state.displace_chart.className = 'chart'; + drawAll(); + } + } + + function dragMove(e) { + if (!drag_state.is_dragging) return; + + let x = e.clientX - drag_state.offset_x; + let y = e.clientY - drag_state.offset_y; + + chart.style.left = `${x}px`; + chart.style.top = `${y}px`; + + drag_state.displace_idx = null; + drag_state.displace_chart = null; + let current_idx = -1; + for (const elem of charts.querySelectorAll('.chart')) { + ++current_idx; + if (current_idx == drag_state.idx) { + continue; + } + + const this_rect = chart.getBoundingClientRect(); + const this_center_x = this_rect.left + this_rect.width / 2; + const this_center_y = this_rect.top + this_rect.height / 2; + + const elem_rect = elem.getBoundingClientRect(); + + if (this_center_x >= elem_rect.left && this_center_x <= elem_rect.right + && this_center_y >= elem_rect.top && this_center_y <= elem_rect.bottom) { + + elem.className = 'chart chart-displaced'; + drag_state.displace_idx = current_idx; + drag_state.displace_chart = elem; + } else { + elem.className = 'chart'; + } + } + } + + function dragStart(e) { + if (e.button !== 0) return; /// left button only + move.setPointerCapture(e.pointerId); + + drag_state.is_dragging = true; + drag_state.idx = getCurrentIndex(); chart.className = 'chart chart-moving'; - let offset_x = e.clientX; - let offset_y = e.clientY; + drag_state.offset_x = e.clientX; + drag_state.offset_y = e.clientY; + } - let displace_idx = null; - let displace_chart = null; - - function mouseup(e) { - is_dragging = false; - chart.className = 'chart'; - chart.style.left = null; - chart.style.top = null; - - if (displace_idx !== null) { - const elem = queries[idx]; - queries.splice(idx, 1); - queries.splice(displace_idx, 0, elem); - - displace_chart.className = 'chart'; - drawAll(); - } - } - - function mousemove(e) { - if (!is_dragging) { - document.body.removeEventListener('mousemove', mousemove); - document.body.removeEventListener('mouseup', mouseup); - return; - } - - let x = e.clientX - offset_x; - let y = e.clientY - offset_y; - - chart.style.left = `${x}px`; - chart.style.top = `${y}px`; - - displace_idx = null; - displace_chart = null; - let current_idx = -1; - for (const elem of charts.querySelectorAll('.chart')) { - ++current_idx; - if (current_idx == idx) { - continue; - } - - const this_rect = chart.getBoundingClientRect(); - const this_center_x = this_rect.left + this_rect.width / 2; - const this_center_y = this_rect.top + this_rect.height / 2; - - const elem_rect = elem.getBoundingClientRect(); - - if (this_center_x >= elem_rect.left && this_center_x <= elem_rect.right - && this_center_y >= elem_rect.top && this_center_y <= elem_rect.bottom) { - - elem.className = 'chart chart-displaced'; - displace_idx = current_idx; - displace_chart = elem; - } else { - elem.className = 'chart'; - } - } - } - - document.body.addEventListener('mouseup', mouseup); - document.body.addEventListener('mousemove', mousemove); - }); + /// Read https://www.redblobgames.com/making-of/draggable/ + move.addEventListener('pointerdown', dragStart); + move.addEventListener('pointermove', dragMove); + move.addEventListener('pointerup', dragStop); + move.addEventListener('pointerancel', dragStop); + move.addEventListener('touchstart', (e) => e.preventDefault()); let maximize = document.createElement('a'); let maximize_text = document.createTextNode('🗖'); From d7c77420b6539e73946fd58f4591ae93a3cc1a9f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 16 Oct 2023 10:33:04 +0000 Subject: [PATCH 051/120] QueryNormalizer only checks stack size --- src/Interpreters/QueryNormalizer.cpp | 2 -- tests/queries/0_stateless/02896_cyclic_aliases_crash.sql | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 6b6ead1f463..f47635a3c3f 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -120,7 +120,6 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a. /// Check size of the alias before cloning too large alias AST alias_node->checkSize(data.settings.max_expanded_ast_elements); - current_asts.insert(alias_node.get()); ast = alias_node->clone(); ast->setAlias(node_alias); @@ -139,7 +138,6 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// Check size of the alias before cloning too large alias AST alias_node->checkSize(data.settings.max_expanded_ast_elements); auto alias_name = ast->getAliasOrColumnName(); - current_asts.insert(alias_node.get()); ast = alias_node->clone(); ast->setAlias(alias_name); diff --git a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql index af54f5df7a7..76eff95cf31 100644 --- a/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql +++ b/tests/queries/0_stateless/02896_cyclic_aliases_crash.sql @@ -6,7 +6,7 @@ SELECT val + 1 as prev, val + prev as val FROM ( SELECT 1 as val ) -; -- { serverError CYCLIC_ALIASES } +; -- { serverError CYCLIC_ALIASES, TOO_DEEP_RECURSION } SELECT From cf1deb7bd5dc7fa1e36e18ac66e8d11149d54e47 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 12 Oct 2023 14:40:41 +0000 Subject: [PATCH 052/120] Fix 'Invalid cursor state' in odbc interacting with MS SQL Server --- programs/odbc-bridge/ColumnInfoHandler.cpp | 8 ++++++++ programs/odbc-bridge/ODBCPooledConnectionFactory.h | 13 +++++++------ 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 147ba43a51d..77ca46c9c88 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -146,6 +146,10 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ { catalog_name = tables.table_catalog(); LOG_TRACE(log, "Will fetch info for table '{}.{}'", catalog_name, table_name); + while (tables.next()) + { + /// drain the iterator before next operation to avoid "Invalid cursor state" + } return catalog.find_columns(/* column = */ "", table_name, /* schema = */ "", catalog_name); } @@ -154,6 +158,10 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ { catalog_name = tables.table_catalog(); LOG_TRACE(log, "Will fetch info for table '{}.{}.{}'", catalog_name, schema_name, table_name); + while (tables.next()) + { + /// drain the iterator before next operation to avoid "Invalid cursor state" + } return catalog.find_columns(/* column = */ "", table_name, schema_name, catalog_name); } diff --git a/programs/odbc-bridge/ODBCPooledConnectionFactory.h b/programs/odbc-bridge/ODBCPooledConnectionFactory.h index e425dea47f7..a10055c6659 100644 --- a/programs/odbc-bridge/ODBCPooledConnectionFactory.h +++ b/programs/odbc-bridge/ODBCPooledConnectionFactory.h @@ -91,16 +91,17 @@ T execute(nanodbc::ConnectionHolderPtr connection_holder, std::functionupdateConnection(); return query_func(connection_holder->get()); From aae3894c23ff8bccac282a40cdadb71808fb3127 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 13 Oct 2023 10:50:36 +0000 Subject: [PATCH 053/120] Throw an exception in odbc-bridge if more than one table matched a single query --- programs/odbc-bridge/ColumnInfoHandler.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/programs/odbc-bridge/ColumnInfoHandler.cpp b/programs/odbc-bridge/ColumnInfoHandler.cpp index 77ca46c9c88..434abf0bf14 100644 --- a/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -145,11 +145,11 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ if (tables.next()) { catalog_name = tables.table_catalog(); + /// `tables.next()` call is mandatory to drain the iterator before next operation and avoid "Invalid cursor state" + if (tables.next()) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Driver returned more than one table for '{}': '{}' and '{}'", + table_name, catalog_name, tables.table_schema()); LOG_TRACE(log, "Will fetch info for table '{}.{}'", catalog_name, table_name); - while (tables.next()) - { - /// drain the iterator before next operation to avoid "Invalid cursor state" - } return catalog.find_columns(/* column = */ "", table_name, /* schema = */ "", catalog_name); } @@ -157,11 +157,11 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ if (tables.next()) { catalog_name = tables.table_catalog(); + /// `tables.next()` call is mandatory to drain the iterator before next operation and avoid "Invalid cursor state" + if (tables.next()) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Driver returned more than one table for '{}': '{}' and '{}'", + table_name, catalog_name, tables.table_schema()); LOG_TRACE(log, "Will fetch info for table '{}.{}.{}'", catalog_name, schema_name, table_name); - while (tables.next()) - { - /// drain the iterator before next operation to avoid "Invalid cursor state" - } return catalog.find_columns(/* column = */ "", table_name, schema_name, catalog_name); } From 3cd175d11d355dbbae64665f9693ff3dfa8096fc Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 13:01:36 +0200 Subject: [PATCH 054/120] Fix test --- .../DatabaseMaterializedPostgreSQL.cpp | 33 ++++++++++++------ .../DatabaseMaterializedPostgreSQL.h | 1 + tests/integration/helpers/postgres_utility.py | 2 +- .../test.py | 34 ++++++++++--------- 4 files changed, 42 insertions(+), 28 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index c03add1a96f..6400d1e45f9 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -52,10 +52,29 @@ DatabaseMaterializedPostgreSQL::DatabaseMaterializedPostgreSQL( , remote_database_name(postgres_database_name) , connection_info(connection_info_) , settings(std::move(settings_)) - , startup_task(getContext()->getSchedulePool().createTask("MaterializedPostgreSQLDatabaseStartup", [this]{ startSynchronization(); })) + , startup_task(getContext()->getSchedulePool().createTask("MaterializedPostgreSQLDatabaseStartup", [this]{ tryStartSynchronization(); })) { } +void DatabaseMaterializedPostgreSQL::tryStartSynchronization() +{ + if (shutdown_called) + return; + + try + { + startSynchronization(); + LOG_INFO(log, "Successfully loaded tables from PostgreSQL and started replication"); + } + catch (...) + { + LOG_ERROR(log, "Failed to start replication from PostgreSQL, " + "will retry. Error: {}", getCurrentExceptionMessage(true)); + + if (!shutdown_called) + startup_task->scheduleAfter(5000); + } +} void DatabaseMaterializedPostgreSQL::startSynchronization() { @@ -115,15 +134,7 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size()); - try - { - replication_handler->startup(/* delayed */false); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - throw; - } + replication_handler->startup(/* delayed */false); } @@ -402,6 +413,7 @@ void DatabaseMaterializedPostgreSQL::detachTablePermanently(ContextPtr, const St void DatabaseMaterializedPostgreSQL::shutdown() { + shutdown_called = true; startup_task->deactivate(); stopReplication(); DatabaseAtomic::shutdown(); @@ -414,7 +426,6 @@ void DatabaseMaterializedPostgreSQL::stopReplication() if (replication_handler) replication_handler->shutdown(); - shutdown_called = true; /// Clear wrappers over nested, all access is not done to nested tables directly. materialized_tables.clear(); } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index edbef281da4..838516fa6f6 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -73,6 +73,7 @@ protected: ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const override; private: + void tryStartSynchronization(); void startSynchronization(); ASTPtr createAlterSettingsQuery(const SettingChange & new_setting); diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 77dc72f8020..9b431317b4a 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -193,7 +193,7 @@ class PostgresManager: def drop_postgres_db(self, database_name=""): database_name = self.database_or_default(database_name) - self.cursor.execute(f"DROP DATABASE IF EXISTS {database_name}") + self.cursor.execute(f"DROP DATABASE IF EXISTS {database_name} WITH (FORCE)") if database_name in self.created_postgres_db_list: self.created_postgres_db_list.remove(database_name) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 82730c34664..e72e5f179f5 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -57,6 +57,7 @@ instance2 = cluster.add_instance( pg_manager = PostgresManager() pg_manager2 = PostgresManager() +pg_manager_instance2 = PostgresManager() @pytest.fixture(scope="module") @@ -69,9 +70,17 @@ def started_cluster(): cluster.postgres_port, default_database="postgres_database", ) + pg_manager_instance2.init( + instance2, + cluster.postgres_ip, + cluster.postgres_port, + default_database="postgres_database", + postgres_db_exists=True, + ) pg_manager2.init( instance2, cluster.postgres_ip, cluster.postgres_port, "postgres_database2" ) + yield cluster finally: @@ -721,8 +730,10 @@ def test_too_many_parts(started_cluster): def test_toast(started_cluster): table = "test_toast" - pg_manager.execute( - f"CREATE TABLE {table} (id integer PRIMARY KEY, txt text, other text)" + pg_manager.create_postgres_table( + table, + "", + """CREATE TABLE "{}" (id integer PRIMARY KEY, txt text, other text)""", ) pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, @@ -752,16 +763,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) def test_replica_consumer(started_cluster): table = "test_replica_consumer" - pg_manager_replica = PostgresManager() - pg_manager_replica.init( - instance2, - cluster.postgres_ip, - cluster.postgres_port, - default_database="postgres_database", - postgres_db_exists=True, - ) - - for pm in [pg_manager, pg_manager_replica]: + for pm in [pg_manager, pg_manager_instance2]: pm.create_and_fill_postgres_table(table) pm.create_materialized_db( ip=started_cluster.postgres_ip, @@ -778,7 +780,7 @@ def test_replica_consumer(started_cluster): instance, table, postgres_database=pg_manager.get_default_database() ) check_tables_are_synchronized( - instance2, table, postgres_database=pg_manager_replica.get_default_database() + instance2, table, postgres_database=pg_manager_instance2.get_default_database() ) assert 50 == int(instance.query(f"SELECT count() FROM test_database.{table}")) @@ -792,15 +794,15 @@ def test_replica_consumer(started_cluster): instance, table, postgres_database=pg_manager.get_default_database() ) check_tables_are_synchronized( - instance2, table, postgres_database=pg_manager_replica.get_default_database() + instance2, table, postgres_database=pg_manager_instance2.get_default_database() ) assert 1050 == int(instance.query(f"SELECT count() FROM test_database.{table}")) assert 1050 == int(instance2.query(f"SELECT count() FROM test_database.{table}")) - for pm in [pg_manager, pg_manager_replica]: + for pm in [pg_manager, pg_manager_instance2]: pm.drop_materialized_db() - pg_manager_replica.clear() + pg_manager_instance2.clear() if __name__ == "__main__": From 31dc46a8898e3db905c2d64cc168f3e71f144ff0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 13:04:45 +0200 Subject: [PATCH 055/120] Better test --- src/Storages/MergeTree/MergeTreeData.cpp | 25 +++++- .../test_merge_session_expired/__init__.py | 0 .../configs/disks.xml | 21 ----- .../configs/keeper_config.xml | 5 -- .../test_merge_session_expired/test.py | 90 ------------------- ...tore_parts_replicated_merge_tree.reference | 6 ++ ...899_restore_parts_replicated_merge_tree.sh | 54 +++++++++++ 7 files changed, 82 insertions(+), 119 deletions(-) delete mode 100644 tests/integration/test_merge_session_expired/__init__.py delete mode 100644 tests/integration/test_merge_session_expired/configs/disks.xml delete mode 100644 tests/integration/test_merge_session_expired/configs/keeper_config.xml delete mode 100644 tests/integration/test_merge_session_expired/test.py create mode 100644 tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference create mode 100755 tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fcfbe27c741..f9935671d9c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -94,6 +95,7 @@ #include #include #include +#include #include #include #include @@ -3943,10 +3945,26 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT /// ActiveDataPartSet allows to restore most top-level parts instead of unexpected. /// It can be important in case of assigned merges. If unexpected part is result of some - /// finished, but not committed merge we should restore closest ancestors for the - /// unexpected part to be able to execute it. + /// finished, but not committed merge then we should restore (at least try to restore) + /// closest ancestors for the unexpected part to be able to execute it. + /// However it's not guaranteed because outdated parts can intersect ActiveDataPartSet parts_for_replacement(format_version); - for (const auto & part_candidate_in_partition : getDataPartsPartitionRange(part->info.partition_id)) + auto range = getDataPartsPartitionRange(part->info.partition_id); + DataPartsVector parts_candidates(range.begin(), range.end()); + + /// In case of intersecting outdated parts we want to add bigger parts (with higher level) first + auto comparator = [] (const DataPartPtr left, const DataPartPtr right) -> bool + { + if (left->info.level < right->info.level) + return true; + else if (left->info.level > right->info.level) + return false; + else + return left->info.mutation < right->info.mutation; + }; + std::sort(parts_candidates.begin(), parts_candidates.end(), comparator); + /// From larger to smaller parts + for (const auto & part_candidate_in_partition : parts_candidates | std::views::reverse) { if (part->info.contains(part_candidate_in_partition->info) && is_appropriate_state(part_candidate_in_partition->getState())) @@ -3966,6 +3984,7 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT if (parts_for_replacement.size() > 0) { std::vector> holes_list; + /// Most part of the code bellow is just to write pretty message auto part_infos = parts_for_replacement.getPartInfos(); int64_t current_right_block = part_infos[0].min_block; for (const auto & top_level_part_to_replace : part_infos) diff --git a/tests/integration/test_merge_session_expired/__init__.py b/tests/integration/test_merge_session_expired/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_merge_session_expired/configs/disks.xml b/tests/integration/test_merge_session_expired/configs/disks.xml deleted file mode 100644 index 94ac83b32ac..00000000000 --- a/tests/integration/test_merge_session_expired/configs/disks.xml +++ /dev/null @@ -1,21 +0,0 @@ - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - - - - - -
- s3 -
-
-
-
-
-
diff --git a/tests/integration/test_merge_session_expired/configs/keeper_config.xml b/tests/integration/test_merge_session_expired/configs/keeper_config.xml deleted file mode 100644 index 4258475f7b5..00000000000 --- a/tests/integration/test_merge_session_expired/configs/keeper_config.xml +++ /dev/null @@ -1,5 +0,0 @@ - - - 3000 - - diff --git a/tests/integration/test_merge_session_expired/test.py b/tests/integration/test_merge_session_expired/test.py deleted file mode 100644 index 61e8ff3c627..00000000000 --- a/tests/integration/test_merge_session_expired/test.py +++ /dev/null @@ -1,90 +0,0 @@ -import logging -import pytest -import time -from helpers.cluster import ClickHouseCluster -from helpers.network import PartitionManager -from helpers.client import QueryRuntimeException - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - main_configs=["configs/keeper_config.xml", "configs/disks.xml"], - stay_alive=True, - with_zookeeper=True, - with_minio=True, -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -def test_merge_session_expired(started_cluster): - node1.query("drop table if exists tab") - node1.query( - "create table tab (x UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab/', '0') order by tuple() settings old_parts_lifetime=3" - ) - node1.query("insert into tab select number from numbers(10)") - node1.query("insert into tab select number + 10 from numbers(10)") - node1.query("alter table tab delete where x = 12 settings mutations_sync=2") - node1.query("alter table tab delete where x = 14 settings mutations_sync=2") - node1.query("alter table tab delete where x = 16 settings mutations_sync=2") - node1.query("system stop merges") - node1.query("optimize table tab final settings alter_sync=0") - - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node1) - node1.query("system start merges") - node1.query("select sleep(1)") - node1.restart_clickhouse() - pm.restore_instance_zk_connections(node1) - - node1.query("system restart replica tab") - assert node1.query("select count() from tab") == "17\n" - - -def test_merge_session_expired_zero_copy(started_cluster): - node1.query("drop table if exists tab") - node1.query( - """ - create table tab (x UInt64, y UInt64) engine = ReplicatedMergeTree('/clickhouse/tables/tab2/', '0') order by tuple() - settings old_parts_lifetime=1, storage_policy='s3', allow_remote_fs_zero_copy_replication=1, replicated_max_ratio_of_wrong_parts=1, min_bytes_for_wide_part=1 - """ - ) - - node1.query("insert into tab select number, number from numbers(10)") - node1.query("insert into tab select number + 10, number + 10 from numbers(10)") - node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") - node1.query("select * from tab") - node1.query( - "alter table tab update x = x + 1, y = y + 1 where 1 settings mutations_sync=2" - ) - node1.query("select * from tab") - node1.query("alter table tab update x = x + 1 where 1 settings mutations_sync=2") - node1.query("select * from tab") - - node1.query( - "alter table tab add column z UInt64 materialized x + sleepEachRow(0.05) settings mutations_sync=2" - ) - node1.query("optimize table tab final settings alter_sync=0") - - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node1) - # Wait some time for merge to start - # Part should be merged and stayed on disk, but not commited into zk - node1.query("select sleep(2)") - node1.restart_clickhouse() - pm.restore_instance_zk_connections(node1) - - node1.query("system restart replica tab") - # Wait for outdated parts to be removed - node1.query("select sleep(3)") - node1.query("select * from tab") - node1.query("system sync replica tab") - assert node1.query("select count() from tab") == "20\n" diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference new file mode 100644 index 00000000000..3f4a9dbc3a9 --- /dev/null +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference @@ -0,0 +1,6 @@ +all_0_1_11_5 +all_4_4_0_5 +all_0_1_11_5 +all_4_4_0_5 +all_0_1_11_5 +all_4_4_0_5 diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh new file mode 100755 index 00000000000..c40996851f5 --- /dev/null +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_with_unsuccessful_commits" + +$CLICKHOUSE_CLIENT --query "CREATE TABLE table_with_unsuccessful_commits (key UInt64, value String) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/unsuccessful', '1') ORDER BY tuple() SETTINGS cleanup_delay_period=1000, max_cleanup_delay_period=1000, old_parts_lifetime = 1697460529, remove_rolled_back_parts_immediately=0, replicated_max_ratio_of_wrong_parts=1, max_suspicious_broken_parts=1000000, max_suspicious_broken_parts_bytes=10000000000" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT rand(), toString(rand()) FROM numbers(10)" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT rand(), toString(rand()) FROM numbers(10)" + + +for i in {0..10}; do + $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE table_with_unsuccessful_commits FINAL" +done + +$CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits DELETE WHERE value = 'hello' SETTINGS mutations_sync=2" +$CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits DELETE WHERE value = 'hello' SETTINGS mutations_sync=2" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT rand(), toString(rand()) FROM numbers(10)" + +$CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits DELETE WHERE value = 'hello' SETTINGS mutations_sync=2" + +$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" + +$CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits MODIFY SETTING fault_probability_before_part_commit=1" + +$CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE table_with_unsuccessful_commits FINAL SETTINGS alter_sync=0" + +i=0 retries=300 + +while [[ $i -lt $retries ]]; do + result=$($CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.replication_queue WHERE table = 'table_with_unsuccessful_commits' and database=currentDatabase()") + + if [[ $result ]]; then + break + fi + + ((++i)) +done + +$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" + +$CLICKHOUSE_CLIENT --query "DETACH TABLE table_with_unsuccessful_commits" + +$CLICKHOUSE_CLIENT --query "ATTACH TABLE table_with_unsuccessful_commits" + +$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" + +$CLICKHOUSE_CLIENT --query "DROP TABLE table_with_unsuccessful_commits" From 2eb1a058990dc2133e3c3d731c4b46ffff245e45 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 13:07:41 +0200 Subject: [PATCH 056/120] Fix --- .../0_stateless/02899_restore_parts_replicated_merge_tree.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh index c40996851f5..c496eed3b53 100755 --- a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh @@ -7,7 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_with_unsuccessful_commits" -$CLICKHOUSE_CLIENT --query "CREATE TABLE table_with_unsuccessful_commits (key UInt64, value String) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/unsuccessful', '1') ORDER BY tuple() SETTINGS cleanup_delay_period=1000, max_cleanup_delay_period=1000, old_parts_lifetime = 1697460529, remove_rolled_back_parts_immediately=0, replicated_max_ratio_of_wrong_parts=1, max_suspicious_broken_parts=1000000, max_suspicious_broken_parts_bytes=10000000000" +# will be flaky in 2031 +$CLICKHOUSE_CLIENT --query "CREATE TABLE table_with_unsuccessful_commits (key UInt64, value String) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/unsuccessful', '1') ORDER BY tuple() SETTINGS cleanup_delay_period=1000, max_cleanup_delay_period=1000, old_parts_lifetime = 1949748529, remove_rolled_back_parts_immediately=0, replicated_max_ratio_of_wrong_parts=1, max_suspicious_broken_parts=1000000, max_suspicious_broken_parts_bytes=10000000000" $CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT rand(), toString(rand()) FROM numbers(10)" From 9a925a0802f86e8bebfd0681d234eb72c07b1b9e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 16 Oct 2023 13:35:20 +0200 Subject: [PATCH 057/120] Fix typo --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f9935671d9c..e5f7903948c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3984,7 +3984,7 @@ void MergeTreeData::forcefullyMovePartToDetachedAndRemoveFromMemory(const MergeT if (parts_for_replacement.size() > 0) { std::vector> holes_list; - /// Most part of the code bellow is just to write pretty message + /// Most part of the code below is just to write pretty message auto part_infos = parts_for_replacement.getPartInfos(); int64_t current_right_block = part_infos[0].min_block; for (const auto & top_level_part_to_replace : part_infos) From 96f2a46a66aa0f63998d8a6a3d6e38d57bf8fbbf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 9 Oct 2023 16:08:22 +0200 Subject: [PATCH 058/120] Fix filtering by virtual columns with OR filter in query The problem with the initial implementation #52653 was: - OR can have multiple arguments - It simply not correct to assume that if there are two arguments this is OK. Consider the following example: "WHERE (column_not_from_partition_by = 1) OR false OR false" Will be converted to: "WHERE false OR false" And it will simply read nothing. Yes, we could apply some optimization for bool, but this will not always work, since to optimize things like "0 = 1" we need to execute it. And the only way to make handle this correctly (with ability to ignore some commands during filtering) is to make is_constant() function return has it use something from the input block, so that we can be sure, that we have some sensible, and not just "false". Plus we cannot simply ignore the difference of the input and output arguments of handling OR, we need to add always-true (1/true) if the size is different, since otherwise it could break invariants (see comment in the code). This includes (but not limited to): - _part* filtering for MergeTree - _path/_file for various File/HDFS/... engines - _table for Merge - ... P.S. analyzer does not have this bug, since it execute expression as whole, and this is what filterBlockWithQuery() should do actually instead, but this will be a more complex patch. Signed-off-by: Azat Khuzhin (cherry picked from commit b107712e0cf8e2671da7c1641b268df57e54bd7a) --- src/Storages/VirtualColumnUtils.cpp | 55 +++++++++++++++---- .../02840_merge__table_or_filter.sql.j2 | 5 ++ .../0_stateless/02896_multiple_OR.reference | 14 +++++ .../queries/0_stateless/02896_multiple_OR.sql | 28 ++++++++++ 4 files changed, 92 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02896_multiple_OR.reference create mode 100644 tests/queries/0_stateless/02896_multiple_OR.sql diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index d0d6233728e..219043f25c6 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -81,14 +82,33 @@ bool extractFunctions(const ASTPtr & expression, const std::functionname == "or") { - bool ret = true; + bool ret = false; ASTs or_args; for (const auto & child : function->arguments->children) - ret &= extractFunctions(child, is_constant, or_args); - /// We can keep condition only if it still OR condition (i.e. we - /// have dependent conditions for columns at both sides) - if (or_args.size() == 2) + ret |= extractFunctions(child, is_constant, or_args); + + if (!or_args.empty()) + { + /// In case of there are less number of arguments for which + /// is_constant() == true, we need to add always-true + /// implicitly to avoid breaking AND invariant. + /// + /// Consider the following: + /// + /// ((value = 10) OR (_table = 'v2')) AND ((_table = 'v1') OR (value = 20)) + /// + /// Without implicit always-true: + /// + /// (_table = 'v2') AND (_table = 'v1') + /// + /// With: + /// + /// (_table = 'v2' OR 1) AND (_table = 'v1' OR 1) -> (_table = 'v2') OR (_table = 'v1') + /// + if (or_args.size() != function->arguments->children.size()) + or_args.push_back(std::make_shared(Field(1))); result.push_back(makeASTForLogicalOr(std::move(or_args))); + } return ret; } } @@ -165,8 +185,10 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block if (!select.where() && !select.prewhere()) return unmodified; - // Provide input columns as constant columns to check if an expression is constant. - std::function is_constant = [&block, &context](const ASTPtr & node) + // Provide input columns as constant columns to check if an expression is + // constant and depends on the columns from provided block (the last is + // required to allow skipping some conditions for handling OR). + std::function is_constant = [&block, &context](const ASTPtr & expr) { auto actions = std::make_shared(block.getColumnsWithTypeAndName()); PreparedSetsPtr prepared_sets = std::make_shared(); @@ -178,13 +200,26 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block context, SizeLimits{}, 1, source_columns, std::move(actions), prepared_sets, true, true, true, { aggregation_keys, grouping_set_keys, GroupByKind::NONE }); - ActionsVisitor(visitor_data).visit(node); + ActionsVisitor(visitor_data).visit(expr); actions = visitor_data.getActions(); + auto expr_column_name = expr->getColumnName(); + + const auto * expr_const_node = actions->tryFindInOutputs(expr_column_name); + if (!expr_const_node) + return false; + auto filter_actions = ActionsDAG::buildFilterActionsDAG({expr_const_node}, {}, context); + const auto & nodes = filter_actions->getNodes(); + bool has_dependent_columns = std::any_of(nodes.begin(), nodes.end(), [&](const auto & node) + { + return block.has(node.result_name); + }); + if (!has_dependent_columns) + return false; + auto expression_actions = std::make_shared(actions); auto block_with_constants = block; expression_actions->execute(block_with_constants); - auto column_name = node->getColumnName(); - return block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column); + return block_with_constants.has(expr_column_name) && isColumnConst(*block_with_constants.getByName(expr_column_name).column); }; /// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. diff --git a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 index a87ef7302c6..286e4545ef7 100644 --- a/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 +++ b/tests/queries/0_stateless/02840_merge__table_or_filter.sql.j2 @@ -18,6 +18,11 @@ create view v2 as select * from d2; create table m as v1 engine=Merge(currentDatabase(), '^(v1|v2)$'); +{# -- FIXME: +select _table, key from m where (value = 10 and _table = 'v1') or (value = 20 and _table = 'v1') or 0 or 0 settings {{ settings }}; +select _table, key from m where (value = 10 and _table = 'v3') or (value = 20 and _table = 'v3') or 0 or 0 settings {{ settings }}; +#} + -- avoid reorder set max_threads=1; -- { echoOn } diff --git a/tests/queries/0_stateless/02896_multiple_OR.reference b/tests/queries/0_stateless/02896_multiple_OR.reference new file mode 100644 index 00000000000..96480a75d11 --- /dev/null +++ b/tests/queries/0_stateless/02896_multiple_OR.reference @@ -0,0 +1,14 @@ +-- { echoOn } +SELECT * FROM or_bug WHERE (key = 1) OR false OR false; +1 +SELECT * FROM or_bug WHERE (key = 1) OR false; +1 +SELECT * FROM or_bug WHERE (key = 1); +1 +-- { echoOn } +select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1; +5840ead423829c1eab29fa97 this is a test +select * from forms where text_field like '%this%' or 0 = 1; +5840ead423829c1eab29fa97 this is a test +select * from forms where text_field like '%this%'; +5840ead423829c1eab29fa97 this is a test diff --git a/tests/queries/0_stateless/02896_multiple_OR.sql b/tests/queries/0_stateless/02896_multiple_OR.sql new file mode 100644 index 00000000000..653ddebca1f --- /dev/null +++ b/tests/queries/0_stateless/02896_multiple_OR.sql @@ -0,0 +1,28 @@ +-- https://github.com/ClickHouse/ClickHouse/pull/52653 +DROP TABLE IF EXISTS or_bug; +CREATE TABLE or_bug (key UInt8) ENGINE=MergeTree ORDER BY key; +INSERT INTO TABLE or_bug VALUES (0), (1); + +-- { echoOn } +SELECT * FROM or_bug WHERE (key = 1) OR false OR false; +SELECT * FROM or_bug WHERE (key = 1) OR false; +SELECT * FROM or_bug WHERE (key = 1); +-- { echoOff } + +-- https://github.com/ClickHouse/ClickHouse/issues/55288 +DROP TABLE IF EXISTS forms; +CREATE TABLE forms +( + `form_id` FixedString(24), + `text_field` String +) +ENGINE = MergeTree +PRIMARY KEY form_id +ORDER BY form_id; +insert into forms values ('5840ead423829c1eab29fa97','this is a test'); + +-- { echoOn } +select * from forms where text_field like '%this%' or 0 = 1 or 0 = 1; +select * from forms where text_field like '%this%' or 0 = 1; +select * from forms where text_field like '%this%'; +-- { echoOff } From b0fbd798089033f03208822a0b750dc1406b039b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 16 Oct 2023 15:57:34 +0200 Subject: [PATCH 059/120] Fix test_system_merges after optimization for "x OR 1" conditions After the previous patch "x OR 1" will not execute "x", and because of this test_system_merges::test_mutation_simple started to fail since "sleep" function did not executed. Signed-off-by: Azat Khuzhin --- tests/integration/test_system_merges/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index d0fa7a1d426..6dbe6c891f2 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -188,7 +188,7 @@ def test_mutation_simple(started_cluster, replicated): # ALTER will sleep for 9s def alter(): node1.query( - f"ALTER TABLE {name} UPDATE a = 42 WHERE sleep(9) OR 1", + f"ALTER TABLE {name} UPDATE a = 42 WHERE sleep(9) = 0", settings=settings, ) From 0b2003cf2716064f04ccf2f6ce7a30566a46731d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 16 Oct 2023 16:11:55 +0200 Subject: [PATCH 060/120] Suppress data-race in rd_kafka_broker_set_nodename Refs: https://github.com/ClickHouse/ClickHouse/issues/55629 Signed-off-by: Azat Khuzhin --- tests/tsan_suppressions.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/tsan_suppressions.txt b/tests/tsan_suppressions.txt index 6a55155e330..67c7eae08f3 100644 --- a/tests/tsan_suppressions.txt +++ b/tests/tsan_suppressions.txt @@ -1 +1,2 @@ -# We have no suppressions! +# https://github.com/ClickHouse/ClickHouse/issues/55629 +race:rd_kafka_broker_set_nodename From c9a2a4922da09d6f31f36fa94cba32d0107475d6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 16 Oct 2023 15:17:55 +0000 Subject: [PATCH 061/120] Fix StorageMaterializedView::isRemote --- src/Storages/StorageMaterializedView.cpp | 7 +++++++ src/Storages/StorageMaterializedView.h | 1 + 2 files changed, 8 insertions(+) diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 78dc89857b7..c28fa90b118 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -478,6 +478,13 @@ ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type) return ActionLock{}; } +bool StorageMaterializedView::isRemote() const +{ + if (auto table = tryGetTargetTable()) + return table->isRemote(); + return false; +} + void registerStorageMaterializedView(StorageFactory & factory) { factory.registerStorage("MaterializedView", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 95bedcd9ade..0f6a6fd3db7 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -22,6 +22,7 @@ public: std::string getName() const override { return "MaterializedView"; } bool isView() const override { return true; } + bool isRemote() const override; bool hasInnerTable() const { return has_inner_table; } From 4bc32c638768c07eceb97ee6823a9400b8e40b9a Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 17:31:30 +0200 Subject: [PATCH 062/120] Fix uncaught exception in background tasj --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index e90dcfcd8ad..dea05ce19d8 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -322,8 +322,19 @@ void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, Load void DatabasePostgreSQL::removeOutdatedTables() { std::lock_guard lock{mutex}; - auto connection_holder = pool->get(); - auto actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema); + + std::set actual_tables; + try + { + auto connection_holder = pool->get(); + actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + cleaner_task->scheduleAfter(cleaner_reschedule_ms); + return; + } if (cache_tables) { From 0a6a4b389496423f82b47aa2dd41398807d5551c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 16 Oct 2023 18:15:11 +0200 Subject: [PATCH 063/120] Review fixes --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 15 +++++++++------ src/Storages/S3Queue/S3QueueFilesMetadata.h | 6 +++++- src/Storages/S3Queue/S3QueueSource.cpp | 7 ++++--- src/Storages/S3Queue/S3QueueSource.h | 2 +- src/Storages/S3Queue/StorageS3Queue.cpp | 11 ++++++----- src/Storages/S3Queue/StorageS3Queue.h | 2 +- 6 files changed, 26 insertions(+), 17 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 150174aabcb..963a64e257e 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -474,6 +474,7 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); + zk_client->tryRemove(node_name + ".retriable", -1); return; } @@ -534,7 +535,11 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt Coordination::Responses responses; if (holder->remove(&requests, &responses)) + { + LOG_TEST(log, "Moved file `{}` to processed", path); + zk_client->tryRemove(node_name + ".retriable", -1); return; + } /// Failed to update max processed node, retry. if (!responses.empty() && responses[0]->error != Coordination::Error::ZOK) @@ -814,14 +819,12 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() }; auto node_cmp = [](const Node & a, const Node & b) { - if (a.metadata.last_processed_timestamp == b.metadata.last_processed_timestamp) - return a.metadata.file_path < b.metadata.file_path; - else - return a.metadata.last_processed_timestamp < b.metadata.last_processed_timestamp; + return std::tie(a.metadata.last_processed_timestamp, a.metadata.file_path) + < std::tie(b.metadata.last_processed_timestamp, b.metadata.file_path); }; /// Ordered in ascending order of timestamps. - std::multiset sorted_nodes(node_cmp); + std::set sorted_nodes(node_cmp); LOG_TRACE(log, "Found {} nodes", nodes.size()); @@ -854,7 +857,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() LOG_TEST(log, "Checking node limits (max size: {}, max age: {}) for {}", max_set_size, max_set_age_sec, get_nodes_str()); size_t nodes_to_remove = check_nodes_limit && nodes_limit_exceeded ? nodes.size() - max_set_size : 0; - for (const auto & node : sorted_nodes) + for (const auto & node : sorted_nodes) { if (nodes_to_remove) { diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h index a2fd1fb6699..df9db87a621 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -149,7 +149,11 @@ class S3QueueFilesMetadata::ProcessingNodeHolder { friend class S3QueueFilesMetadata; public: - ProcessingNodeHolder(const std::string & processing_id_, const std::string & path_, const std::string & zk_node_path_, zkutil::ZooKeeperPtr zk_client_); + ProcessingNodeHolder( + const std::string & processing_id_, + const std::string & path_, + const std::string & zk_node_path_, + zkutil::ZooKeeperPtr zk_client_); ~ProcessingNodeHolder(); diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 1a44fe9cff8..6ea222df71f 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -167,11 +167,12 @@ Chunk StorageS3QueueSource::generate() return chunk; } } - catch (const Exception & e) + catch (...) { - LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getFile(), e.displayText()); + const auto message = getCurrentExceptionMessage(true); + LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getFile(), message); - files_metadata->setFileFailed(key_with_info->processing_holder, e.message()); + files_metadata->setFileFailed(key_with_info->processing_holder, message); appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); throw; diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index db3015f129e..7c8eb3eeb74 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -65,7 +65,7 @@ public: RemoveFileFunc remove_file_func_, const NamesAndTypesList & requested_virtual_columns_, ContextPtr context_, - const std::atomic & shutdown_called_, + const std::atomic & shutdown_called_, std::shared_ptr s3_queue_log_, const StorageID & storage_id_); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 37389eb1bd0..92f15aed62f 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -210,20 +210,21 @@ Pipe StorageS3Queue::read( Pipes pipes; const size_t adjusted_num_streams = std::min(num_streams, s3queue_settings->s3queue_processing_threads_num); + + auto file_iterator = createFileIterator(local_context, query_info.query); for (size_t i = 0; i < adjusted_num_streams; ++i) - pipes.emplace_back(createSource(column_names, storage_snapshot, query_info.query, max_block_size, local_context)); + pipes.emplace_back(createSource(file_iterator, column_names, storage_snapshot, max_block_size, local_context)); return Pipe::unitePipes(std::move(pipes)); } std::shared_ptr StorageS3Queue::createSource( + std::shared_ptr file_iterator, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - ASTPtr query, size_t max_block_size, ContextPtr local_context) { auto configuration_snapshot = updateConfigurationAndGetCopy(local_context); - auto file_iterator = createFileIterator(local_context, query); auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); auto internal_source = std::make_unique( @@ -350,12 +351,12 @@ bool StorageS3Queue::streamToViews() // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); auto block_io = interpreter.execute(); + auto file_iterator = createFileIterator(s3queue_context, nullptr); Pipes pipes; for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) { - auto source = createSource( - block_io.pipeline.getHeader().getNames(), storage_snapshot, nullptr, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + auto source = createSource(file_iterator, block_io.pipeline.getHeader().getNames(), storage_snapshot, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); pipes.emplace_back(std::move(source)); } auto pipe = Pipe::unitePipes(std::move(pipes)); diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 2a62078fcca..000015951ea 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -83,9 +83,9 @@ private: std::shared_ptr createFileIterator(ContextPtr local_context, ASTPtr query); std::shared_ptr createSource( + std::shared_ptr file_iterator, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - ASTPtr query, size_t max_block_size, ContextPtr local_context); From f85e82931d009100bbd1dfe1998810c930aee765 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 16 Oct 2023 18:36:45 +0200 Subject: [PATCH 064/120] Make test less flaky --- ...9_restore_parts_replicated_merge_tree.reference | 7 +------ .../02899_restore_parts_replicated_merge_tree.sh | 14 +++++++++++--- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference index 3f4a9dbc3a9..7326d960397 100644 --- a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.reference @@ -1,6 +1 @@ -all_0_1_11_5 -all_4_4_0_5 -all_0_1_11_5 -all_4_4_0_5 -all_0_1_11_5 -all_4_4_0_5 +Ok diff --git a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh index c496eed3b53..c6165c1e983 100755 --- a/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh +++ b/tests/queries/0_stateless/02899_restore_parts_replicated_merge_tree.sh @@ -26,7 +26,7 @@ $CLICKHOUSE_CLIENT --query "INSERT INTO table_with_unsuccessful_commits SELECT r $CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits DELETE WHERE value = 'hello' SETTINGS mutations_sync=2" -$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" +original_parts=$($CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name") $CLICKHOUSE_CLIENT --query "ALTER TABLE table_with_unsuccessful_commits MODIFY SETTING fault_probability_before_part_commit=1" @@ -44,12 +44,20 @@ while [[ $i -lt $retries ]]; do ((++i)) done -$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" +parts_after_mutation=$($CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name") $CLICKHOUSE_CLIENT --query "DETACH TABLE table_with_unsuccessful_commits" $CLICKHOUSE_CLIENT --query "ATTACH TABLE table_with_unsuccessful_commits" -$CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name" +parts_after_detach_attach=$($CLICKHOUSE_CLIENT --query "SELECT name FROM system.parts where table = 'table_with_unsuccessful_commits' and database = currentDatabase() and active order by name") + +if [[ "$parts_after_detach_attach" == "$parts_after_mutation" && "$parts_after_mutation" == "$original_parts" ]]; then + echo "Ok" +else + echo "Original parts $original_parts" + echo "Parts after mutation $parts_after_mutation" + echo "Parts after detach attach $parts_after_detach_attach" +fi $CLICKHOUSE_CLIENT --query "DROP TABLE table_with_unsuccessful_commits" From 31ad4a98c8b0c79fb7d071c5e078d4048efd6640 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 16 Oct 2023 18:38:29 +0000 Subject: [PATCH 065/120] Bump gRPC 1.34 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 3f975ecab37..e5276ec3693 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 3f975ecab377cd5f739af780566596128f17bb74 +Subproject commit e5276ec369342e0069f71c3354df9eac69ae3f07 From fb8cd298d1837870566cdfc1f408f3e6d6e516d3 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Mon, 16 Oct 2023 17:52:39 -0700 Subject: [PATCH 066/120] Code refactoring according to code review --- .../mergetree-family/invertedindexes.md | 6 +++++ src/Interpreters/GinFilter.cpp | 13 ++++------- src/Interpreters/GinFilter.h | 9 ++++---- .../MergeTree/MergeTreeIndexInverted.cpp | 18 ++++++++------- .../02346_full_text_search.reference | 2 ++ .../0_stateless/02346_full_text_search.sql | 23 +++++++++++++++++++ 6 files changed, 51 insertions(+), 20 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md index db3d6d0a479..7e5140b4c4f 100644 --- a/docs/en/engines/table-engines/mergetree-family/invertedindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/invertedindexes.md @@ -58,6 +58,12 @@ where `N` specifies the tokenizer: - `inverted(0)` (or shorter: `inverted()`) set the tokenizer to "tokens", i.e. split strings along spaces, - `inverted(N)` with `N` between 2 and 8 sets the tokenizer to "ngrams(N)" +The maximum rows per postings list can be specified as the second parameter. This parameter can be used to control postings list sizes to avoid generating huge postings list files. The following variants exist: + +- `inverted(ngrams, max_rows_per_postings_list)`: Use given max_rows_per_postings_list (assuming it is not 0) +- `inverted(ngrams, 0)`: No limitation of maximum rows per postings list +- `inverted(ngrams)`: Use a default maximum rows which is 64K. + Being a type of skipping index, inverted indexes can be dropped or added to a column after table creation: ``` sql diff --git a/src/Interpreters/GinFilter.cpp b/src/Interpreters/GinFilter.cpp index e60d54026eb..5d823318313 100644 --- a/src/Interpreters/GinFilter.cpp +++ b/src/Interpreters/GinFilter.cpp @@ -21,18 +21,15 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -GinFilterParameters::GinFilterParameters(size_t ngrams_, UInt64 max_rows_) +GinFilterParameters::GinFilterParameters(size_t ngrams_, UInt64 max_rows_per_postings_list_) : ngrams(ngrams_) - , max_rows_in_postings_list(max_rows_) + , max_rows_per_postings_list(max_rows_per_postings_list_) { - /// 0 indicates no limitation of postings list's size - if (max_rows_in_postings_list == 0) - max_rows_in_postings_list = std::numeric_limits::max(); + if (max_rows_per_postings_list == UNLIMITED_ROWS_PER_POSTINGS_LIST) + max_rows_per_postings_list = std::numeric_limits::max(); if (ngrams > 8) throw Exception(ErrorCodes::BAD_ARGUMENTS, "The size of inverted index filter cannot be greater than 8"); - if (max_rows_in_postings_list < MIN_ROWS_IN_POSTINGS_LIST) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The maximum rows in postings list must be no less than {}", MIN_ROWS_IN_POSTINGS_LIST); } GinFilter::GinFilter(const GinFilterParameters & params_) @@ -55,7 +52,7 @@ void GinFilter::add(const char * data, size_t len, UInt32 rowID, GinIndexStorePt } else { - auto builder = std::make_shared(params.max_rows_in_postings_list); + auto builder = std::make_shared(params.max_rows_per_postings_list); builder->add(rowID); store->setPostingsBuilder(term, builder); diff --git a/src/Interpreters/GinFilter.h b/src/Interpreters/GinFilter.h index 3e57a07ecd4..7924bd4ce3d 100644 --- a/src/Interpreters/GinFilter.h +++ b/src/Interpreters/GinFilter.h @@ -8,15 +8,16 @@ namespace DB { static inline constexpr auto INVERTED_INDEX_NAME = "inverted"; -static inline constexpr UInt64 MIN_ROWS_IN_POSTINGS_LIST = 8 * 1024; -static inline constexpr UInt64 DEFAULT_ROWS_IN_POSTINGS_LIST = 64 * 1024; +static inline constexpr UInt64 UNLIMITED_ROWS_PER_POSTINGS_LIST = 0; +static inline constexpr UInt64 MIN_ROWS_PER_POSTINGS_LIST = 8 * 1024; +static inline constexpr UInt64 DEFAULT_MAX_ROWS_PER_POSTINGS_LIST = 64 * 1024; struct GinFilterParameters { - GinFilterParameters(size_t ngrams_, UInt64 max_rows_); + GinFilterParameters(size_t ngrams_, UInt64 max_rows_per_postings_list_); size_t ngrams; - UInt64 max_rows_in_postings_list; + UInt64 max_rows_per_postings_list; }; struct GinSegmentWithRowIdRange diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 169ae768b31..f04c20dc50e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -735,7 +735,7 @@ MergeTreeIndexPtr invertedIndexCreator( const IndexDescription & index) { size_t n = index.arguments.empty() ? 0 : index.arguments[0].get(); - UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_ROWS_IN_POSTINGS_LIST : index.arguments[1].get(); + UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].get(); GinFilterParameters params(n, max_rows); /// Use SplitTokenExtractor when n is 0, otherwise use NgramTokenExtractor @@ -780,14 +780,16 @@ void invertedIndexValidator(const IndexDescription & index, bool /*attach*/) if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64) throw Exception(ErrorCodes::INCORRECT_QUERY, "The first Inverted index argument must be positive integer."); - if (index.arguments.size() == 2 && (index.arguments[1].getType() != Field::Types::UInt64 || - (index.arguments[1].get() != 0 && index.arguments[1].get() < MIN_ROWS_IN_POSTINGS_LIST))) - throw Exception(ErrorCodes::INCORRECT_QUERY, "The maximum rows in postings list must be no less than {}", MIN_ROWS_IN_POSTINGS_LIST); - + if (index.arguments.size() == 2) + { + if (index.arguments[1].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "The second Inverted index argument must be UInt64"); + if (index.arguments[1].get() != UNLIMITED_ROWS_PER_POSTINGS_LIST && index.arguments[1].get() < MIN_ROWS_PER_POSTINGS_LIST) + throw Exception(ErrorCodes::INCORRECT_QUERY, "The maximum rows per postings list must be no less than {}", MIN_ROWS_PER_POSTINGS_LIST); + } /// Just validate size_t ngrams = index.arguments.empty() ? 0 : index.arguments[0].get(); - UInt64 max_rows = index.arguments.size() < 2 ? DEFAULT_ROWS_IN_POSTINGS_LIST : index.arguments[1].get(); - GinFilterParameters params(ngrams, max_rows); + UInt64 max_rows_per_postings_list = index.arguments.size() < 2 ? DEFAULT_MAX_ROWS_PER_POSTINGS_LIST : index.arguments[1].get(); + GinFilterParameters params(ngrams, max_rows_per_postings_list); } - } diff --git a/tests/queries/0_stateless/02346_full_text_search.reference b/tests/queries/0_stateless/02346_full_text_search.reference index 0cf74e14427..73fb0d04157 100644 --- a/tests/queries/0_stateless/02346_full_text_search.reference +++ b/tests/queries/0_stateless/02346_full_text_search.reference @@ -47,3 +47,5 @@ af inverted 102 clickhouse你好 1 AST Fuzzer crash, issue #54541 +Test max_rows_per_postings_list +1 diff --git a/tests/queries/0_stateless/02346_full_text_search.sql b/tests/queries/0_stateless/02346_full_text_search.sql index 7d3337b9407..be56f24d5da 100644 --- a/tests/queries/0_stateless/02346_full_text_search.sql +++ b/tests/queries/0_stateless/02346_full_text_search.sql @@ -277,3 +277,26 @@ SELECT * FROM tab WHERE str == 'b' AND 1.0; -- AND result_rows==1 -- LIMIT 1; -- +SELECT 'Test max_rows_per_postings_list'; +DROP TABLE IF EXISTS tab; +-- create table 'tab' with inverted index parameter (ngrams, max_rows_per_most_list) which is (0, 10240) +CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(0, 12040)) + Engine=MergeTree + ORDER BY (k) + AS + SELECT + number, + format('{},{},{},{}', hex(12345678), hex(87654321), hex(number/17 + 5), hex(13579012)) as s + FROM numbers(1024); +SELECT count(s) FROM tab WHERE hasToken(s, '4C4B4B4B4B4B5040'); +DROP TABLE IF EXISTS tab; +-- create table 'tab' with inverted index parameter (ngrams, max_rows_per_most_list) which is (0, 123) +-- it should throw exception since max_rows_per_most_list(123) is less than its minimum value(8196) +CREATE TABLE tab(k UInt64, s String, INDEX af(s) TYPE inverted(3, 123)) + Engine=MergeTree + ORDER BY (k) + AS + SELECT + number, + format('{},{},{},{}', hex(12345678), hex(87654321), hex(number/17 + 5), hex(13579012)) as s + FROM numbers(1024); -- { serverError 80 } From fc9a5a725c6cd661ea51bd7b4a9aa257ec60d0c8 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 17 Oct 2023 08:18:17 +0000 Subject: [PATCH 067/120] Revert "Revert "Integration check script fix ups"" This reverts commit 67b9407530bb15f7e6d49cd1c2bde7b6c441389b. --- docs/en/development/continuous-integration.md | 58 +++++++++++++++++++ tests/README.md | 1 + tests/ci/build_download_helper.py | 42 +++++++++----- tests/ci/pr_info.py | 36 +++++------- tests/ci/worker/prepare-ci-ami.sh | 1 + 5 files changed, 102 insertions(+), 36 deletions(-) create mode 100644 tests/README.md diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index eec5ccbb9dc..46a30f56f11 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -67,6 +67,48 @@ This check means that the CI system started to process the pull request. When it Performs some simple regex-based checks of code style, using the [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check-style) binary (note that it can be run locally). If it fails, fix the style errors following the [code style guide](style.md). +#### Running style check locally: +```sh +mkdir -p /tmp/test_output +# running all checks +docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE clickhouse/style-test + +# run specified check script (e.g.: ./check-mypy) +docker run --rm --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output -u $(id -u ${USER}):$(id -g ${USER}) --cap-add=SYS_PTRACE --entrypoint= -w/ClickHouse/utils/check-style clickhouse/style-test ./check-mypy + +# find all style check scripts under the directory: +cd ./utils/check-style + +# Check duplicate includes +./check-duplicate-includes.sh + +# Check c++ formatiing +./check-style + +# Check python formatting with black +./check-black + +# Check python type hinting with mypy +./check-mypy + +# Check code with codespell +./check-typos + +# Check docs spelling +./check-doc-aspell + +# Check whitespaces +./check-whitespaces + +# Check github actions workflows +./check-workflows + +# Check submodules +./check-submodules + +# Check shell scripts with shellcheck +./shellcheck-run.sh +``` ## Fast Test Normally this is the first check that is ran for a PR. It builds ClickHouse and @@ -75,6 +117,15 @@ some. If it fails, further checks are not started until it is fixed. Look at the report to see which tests fail, then reproduce the failure locally as described [here](tests.md#functional-test-locally). +#### Running Fast Test locally: +```sh +mkdir -p /tmp/test_output +mkdir -p /tmp/fasttest-workspace +cd ClickHouse +# this docker command performs minimal ClickHouse build and run FastTests against it +docker run --rm --cap-add=SYS_PTRACE -u $(id -u ${USER}):$(id -g ${USER}) --network=host -e FASTTEST_WORKSPACE=/fasttest-workspace -e FASTTEST_OUTPUT=/test_output -e FASTTEST_SOURCE=/ClickHouse --cap-add=SYS_PTRACE -e stage=clone_submodules --volume=/tmp/fasttest-workspace:/fasttest-workspace --volume=.:/ClickHouse --volume=/tmp/test_output:/test_output clickhouse/fasttest +``` + #### Status Page Files - `runlog.out.log` is the general log that includes all other logs. @@ -122,6 +173,13 @@ Builds ClickHouse in various configurations for use in further steps. You have t ## Special Build Check Performs static analysis and code style checks using `clang-tidy`. The report is similar to the [build check](#build-check). Fix the errors found in the build log. +#### Running clang-tidy locally: +There is a convenience `packager` script that runs the clang-tidy build in docker +```sh +mkdir build_tidy +./docker/packager/packager --output-dir=./build_tidy --package-type=binary --compiler=clang-17 --debug-build --clang-tidy +``` + ## Functional Stateless Tests Runs [stateless functional tests](tests.md#functional-tests) for ClickHouse diff --git a/tests/README.md b/tests/README.md new file mode 100644 index 00000000000..a1fc0f530f2 --- /dev/null +++ b/tests/README.md @@ -0,0 +1 @@ +Find CI documents and instructions on running CI checks localy [here](https://clickhouse.com/docs/en/development/continuous-integration). \ No newline at end of file diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index b76c5433142..21012f6337d 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -51,9 +51,9 @@ def get_gh_api( sleep: int = 3, **kwargs: Any, ) -> requests.Response: - """It's a wrapper around get_with_retries that requests GH api w/o auth by - default, and falls back to the get_best_robot_token in case of receiving - "403 rate limit exceeded" error + """ + Request GH api w/o auth by default, and failover to the get_best_robot_token in case of receiving + "403 rate limit exceeded" or "404 not found" error It sets auth automatically when ROBOT_TOKEN is already set by get_best_robot_token """ @@ -71,27 +71,39 @@ def get_gh_api( if grt.ROBOT_TOKEN is not None: set_auth_header() - need_retry = False - for _ in range(retries): + token_is_set = "Authorization" in kwargs.get("headers", {}) + exc = Exception("A placeholder to satisfy typing and avoid nesting") + try_cnt = 0 + while try_cnt < retries: + try_cnt += 1 try: - response = get_with_retries(url, 1, sleep, **kwargs) + response = requests.get(url, **kwargs) response.raise_for_status() return response - except requests.HTTPError as exc: - if ( - exc.response.status_code == 403 + except requests.HTTPError as e: + exc = e + ratelimit_exceeded = ( + e.response.status_code == 403 and b"rate limit exceeded" - in exc.response._content # pylint:disable=protected-access - ): + in e.response._content # pylint:disable=protected-access + ) + try_auth = e.response.status_code == 404 + if (ratelimit_exceeded or try_auth) and not token_is_set: logging.warning( "Received rate limit exception, setting the auth header and retry" ) set_auth_header() - need_retry = True - break + token_is_set = True + try_cnt = 0 + continue + except Exception as e: + exc = e - if need_retry: - return get_with_retries(url, retries, sleep, **kwargs) + if try_cnt < retries: + logging.info("Exception '%s' while getting, retry %i", exc, try_cnt) + time.sleep(sleep) + + raise exc def get_build_name_for_check(check_name: str) -> str: diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 830aa936bea..7dbfe124760 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -4,8 +4,6 @@ import logging import os from typing import Dict, List, Set, Union, Literal -from unidiff import PatchSet # type: ignore - from build_download_helper import get_gh_api from env_helper import ( GITHUB_REPOSITORY, @@ -171,7 +169,11 @@ class PRInfo: response_json = user_orgs_response.json() self.user_orgs = set(org["id"] for org in response_json) - self.diff_urls.append(github_event["pull_request"]["diff_url"]) + self.diff_urls.append( + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/master...{self.head_ref}" + ) + elif "commits" in github_event: # `head_commit` always comes with `commits` commit_message = github_event["head_commit"]["message"] # type: str @@ -215,12 +217,12 @@ class PRInfo: # files changed in upstream AND master...{self.head_ref} # to get files, changed in current HEAD self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}.diff" + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/master...{self.head_ref}" ) self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master.diff" + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master" ) # Get release PR number. self.release_pr = get_pr_for_commit(self.base_ref, self.base_ref)[ @@ -232,8 +234,8 @@ class PRInfo: # For release PRs we must get not only files changed in the PR # itself, but as well files changed since we branched out self.diff_urls.append( - f"https://github.com/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master.diff" + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" + f"compare/{self.head_ref}...master" ) else: print("event.json does not match pull_request or push:") @@ -261,19 +263,11 @@ class PRInfo: raise TypeError("The event does not have diff URLs") for diff_url in self.diff_urls: - response = get_gh_api( - diff_url, - sleep=RETRY_SLEEP, - ) + response = get_gh_api(diff_url, sleep=RETRY_SLEEP) response.raise_for_status() - if "commits" in self.event and self.number == 0: - diff = response.json() - - if "files" in diff: - self.changed_files = {f["filename"] for f in diff["files"]} - else: - diff_object = PatchSet(response.text) - self.changed_files.update({f.path for f in diff_object}) + diff = response.json() + if "files" in diff: + self.changed_files = {f["filename"] for f in diff["files"]} print(f"Fetched info about {len(self.changed_files)} changed files") def get_dict(self): diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index 20e7e3fd53e..f3e11b6d00a 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -90,6 +90,7 @@ systemctl restart docker sudo -u ubuntu docker buildx version sudo -u ubuntu docker buildx create --use --name default-builder +# FIXME: remove unidiff as soon as no old PRs could use it, here and in Dockerfile pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory mkdir -p $RUNNER_HOME && cd $RUNNER_HOME From ad38c433a1390d4402d7b53577a9693070d2e194 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Tue, 17 Oct 2023 08:48:25 +0000 Subject: [PATCH 068/120] fix commit labels for diff --- tests/ci/pr_info.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 7dbfe124760..c02c71d8761 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -171,7 +171,7 @@ class PRInfo: self.diff_urls.append( f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}" + f"compare/master...{github_event['pull_request']['head']['label']}" ) elif "commits" in github_event: @@ -218,11 +218,11 @@ class PRInfo: # to get files, changed in current HEAD self.diff_urls.append( f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/master...{self.head_ref}" + f"compare/master...{pull_request['head']['label']}" ) self.diff_urls.append( f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master" + f"compare/{pull_request['head']['label']}...master" ) # Get release PR number. self.release_pr = get_pr_for_commit(self.base_ref, self.base_ref)[ @@ -235,7 +235,7 @@ class PRInfo: # itself, but as well files changed since we branched out self.diff_urls.append( f"https://api.github.com/repos/{GITHUB_REPOSITORY}/" - f"compare/{self.head_ref}...master" + f"compare/{pull_request['head']['label']}...master" ) else: print("event.json does not match pull_request or push:") From f56804e6e72b256d847417aecb0b2085d9a46438 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Tue, 17 Oct 2023 12:13:59 +0300 Subject: [PATCH 069/120] Use pool for proxified S3 http sessions --- src/IO/HTTPCommon.cpp | 8 ++++++++ src/IO/ReadBufferFromS3.cpp | 13 +++++++++++-- src/IO/S3/PocoHTTPClient.cpp | 2 +- 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index ef03acdde5d..718ae53bb7a 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -324,6 +324,14 @@ namespace auto retry_timeout = timeouts.connection_timeout.totalMilliseconds(); auto session = pool_ptr->second->get(retry_timeout); + const auto & session_data = session->sessionData(); + if (session_data.empty() || !Poco::AnyCast(&session_data)) + { + /// Reset session if it is not reusable. See comment for HTTPSessionReuseTag. + session->reset(); + } + session->attachSessionData({}); + setTimeouts(*session, timeouts); return session; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index f346b6fb3f9..f19978ccb47 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -74,8 +74,17 @@ void resetSessionIfNeeded(bool read_all_range_successfully, std::optionalgetProxyHost().empty()) + { + /// Reset proxified sessions because proxy can change for every request. See ProxyConfigurationResolver. + resetSession(*read_result); + ProfileEvents::increment(ProfileEvents::ReadBufferFromS3ResetSessions); + } + else + { + DB::markSessionForReuse(session); + ProfileEvents::increment(ProfileEvents::ReadBufferFromS3PreservedSessions); + } } } } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index d7c5b4539ae..15a9ce637e7 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -276,7 +276,7 @@ void PocoHTTPClient::makeRequestInternal( { /// Most sessions in pool are already connected and it is not possible to set proxy host/port to a connected session. const auto request_configuration = per_request_configuration(); - if (http_connection_pool_size && request_configuration.host.empty()) + if (http_connection_pool_size) makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); else makeRequestInternalImpl(request, request_configuration, response, readLimiter, writeLimiter); From 8744cd9c0dde99484bfc0356c768d70c44db2bac Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 11:15:53 +0200 Subject: [PATCH 070/120] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 963a64e257e..7d498dcb287 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -474,7 +474,7 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(node_name + ".retriable", -1); + zk_client->tryRemove(zookeeper_failed_path / node_name + ".retriable", -1); return; } @@ -537,7 +537,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(node_name + ".retriable", -1); + zk_client->tryRemove(zookeeper_failed_path / node_name + ".retriable", -1); return; } From c220cc65d6145f15c5b47b2d04f1e351c7f93d58 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 17 Oct 2023 11:35:15 +0200 Subject: [PATCH 071/120] Update test.py --- .../test_postgresql_replica_database_engine_2/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index e72e5f179f5..2a1c7634698 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -762,6 +762,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) def test_replica_consumer(started_cluster): table = "test_replica_consumer" + pg_manager_instance2.restart() for pm in [pg_manager, pg_manager_instance2]: pm.create_and_fill_postgres_table(table) From 42ed24995473ea4932c48fd89dd160be0a004546 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 12:03:49 +0200 Subject: [PATCH 072/120] Fix build --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 4 ++-- src/Storages/StorageS3.cpp | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 7d498dcb287..c64330e9df1 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -474,7 +474,7 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(zookeeper_failed_path / node_name + ".retriable", -1); + zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); return; } @@ -537,7 +537,7 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(zookeeper_failed_path / node_name + ".retriable", -1); + zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); return; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2d3ff786abc..4f54a938ada 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -210,11 +210,11 @@ private: /// If url doesn't contain globs, we didn't list s3 bucket and didn't get object info for the key. /// So we get object info lazily here on 'next()' request. - if (!answer.info) + if (!answer->info) { - answer.info = S3::getObjectInfo(*client, globbed_uri.bucket, answer.key, globbed_uri.version_id, request_settings); + answer->info = S3::getObjectInfo(*client, globbed_uri.bucket, answer->key, globbed_uri.version_id, request_settings); if (file_progress_callback) - file_progress_callback(FileProgress(0, answer.info->size)); + file_progress_callback(FileProgress(0, answer->info->size)); } return answer; From f20d055e1ecd24019ee1ea6fece98a5d634a338d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 17 Oct 2023 10:06:57 +0000 Subject: [PATCH 073/120] do not re-create serialization for each row Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/EmbeddedRocksDBSink.cpp | 3 ++- src/Storages/RocksDB/EmbeddedRocksDBSink.h | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index 47d036c943d..b79c7284843 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -26,6 +26,7 @@ EmbeddedRocksDBSink::EmbeddedRocksDBSink( break; ++primary_key_pos; } + serializations = getHeader().getSerializations(); } void EmbeddedRocksDBSink::consume(Chunk chunk) @@ -46,7 +47,7 @@ void EmbeddedRocksDBSink::consume(Chunk chunk) size_t idx = 0; for (const auto & elem : block) { - elem.type->getDefaultSerialization()->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); + serializations[idx]->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); ++idx; } status = batch.Put(wb_key.str(), wb_value.str()); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.h b/src/Storages/RocksDB/EmbeddedRocksDBSink.h index e9e98c7df50..011322df829 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.h @@ -24,6 +24,7 @@ private: StorageEmbeddedRocksDB & storage; StorageMetadataPtr metadata_snapshot; size_t primary_key_pos = 0; + Serializations serializations; }; } From 9a8734fa5ef2158ec7fa7b5a7a87d3ea1a4beffa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 17 Oct 2023 10:22:30 +0000 Subject: [PATCH 074/120] Remove submodules --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index e5276ec3693..cb6b724ac49 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit e5276ec369342e0069f71c3354df9eac69ae3f07 +Subproject commit cb6b724ac49fe94098e30e3e0d9fd6901007ff38 From 4d2cf5206fcacc005d24c4b3617708dd604fb0a7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 14:12:15 +0200 Subject: [PATCH 075/120] Update documentation --- docs/en/engines/table-engines/integrations/s3queue.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 4769f573b54..60bc48f8bf9 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -241,6 +241,16 @@ For more information about virtual columns see [here](../../../engines/table-eng Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function. +## Limitations {#limitations} + +Duplicated rows can be as a result of: + +- an exception happens during parsing in the middle of file processing and retries are enabled via `s3queue_loading_retries`, duplicate rows are inevitabe; + +- `S3Queue` is configured on multiple servers pointing to the same path in zookeeper and keeper session expires before one server managed to commit processed file, which could lead to another server taking processing of the file, which could be partially or fully processed by the first server; + +- abnormal server termination (with SIGABRT, for example). + :::note If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: From e500dc22e4fc06f1eb3606a71dc688d318d9fa02 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 17 Oct 2023 14:14:58 +0200 Subject: [PATCH 076/120] Respect default format when using http_write_exception_in_output_format --- src/Formats/FormatFactory.cpp | 2 +- src/Interpreters/executeQuery.cpp | 49 +++++++++++++++++-- ...default_format_on_http_exception.reference | 25 ++++++++++ ...99_use_default_format_on_http_exception.sh | 33 +++++++++++++ 4 files changed, 105 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02899_use_default_format_on_http_exception.reference create mode 100755 tests/queries/0_stateless/02899_use_default_format_on_http_exception.sh diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 9b391324455..2713b7cb35f 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -410,7 +410,7 @@ std::unique_ptr FormatFactory::wrapReadBufferIfNeeded( static void addExistingProgressToOutputFormat(OutputFormatPtr format, ContextPtr context) { - auto element_id = context->getProcessListElement(); + auto element_id = context->getProcessListElementSafe(); if (element_id) { /// While preparing the query there might have been progress (for example in subscalar subqueries) so add it here diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b96e05c1f9a..cda2d70714a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1332,7 +1332,46 @@ void executeQuery( BlockIO streams; OutputFormatPtr output_format; - std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr); + auto update_format_for_exception_if_needed = [&]() + { + if (!output_format) + { + try + { + String format_name = context->getDefaultFormat(); + output_format = FormatFactory::instance().getOutputFormat(format_name, ostr, {}, context, output_format_settings); + if (output_format) + { + /// Force an update of the headers before we start writing + result_details.content_type = output_format->getContentType(); + result_details.format = format_name; + set_result_details(result_details); + set_result_details = nullptr; + } + } + catch (const DB::Exception & e) + { + /// Ignore this exception and report the original one + LOG_WARNING(&Poco::Logger::get("executeQuery"), getExceptionMessageAndPattern(e, true)); + } + } + }; + + try + { + std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr); + } + catch (...) + { + if (handle_exception_in_output_format) + { + update_format_for_exception_if_needed(); + if (output_format) + handle_exception_in_output_format(*output_format); + } + throw; + } + auto & pipeline = streams.pipeline; std::unique_ptr compressed_buffer; @@ -1426,8 +1465,12 @@ void executeQuery( } catch (...) { - if (handle_exception_in_output_format && output_format) - handle_exception_in_output_format(*output_format); + if (handle_exception_in_output_format) + { + update_format_for_exception_if_needed(); + if (output_format) + handle_exception_in_output_format(*output_format); + } streams.onException(); throw; } diff --git a/tests/queries/0_stateless/02899_use_default_format_on_http_exception.reference b/tests/queries/0_stateless/02899_use_default_format_on_http_exception.reference new file mode 100644 index 00000000000..a943df06764 --- /dev/null +++ b/tests/queries/0_stateless/02899_use_default_format_on_http_exception.reference @@ -0,0 +1,25 @@ +INSERT WITH default_format=JSON +Content-Type:application/json;charset=UTF-8 +"exception":"Code:62. + +INSERT WITH default_format=XML +Content-Type:application/xml;charset=UTF-8 +Code:62.DB::Ex---tion: + +INSERT WITH default_format=BADFORMAT +Content-Type:text/plain;charset=UTF-8 +X-ClickHouse-Ex---tion-Code:62 +Code:62.DB::Ex---tion: + +INSERT WITH X-ClickHouse-Format: JSON +Content-Type:application/json;charset=UTF-8 +"exception":"Code:62. + +INSERT WITH X-ClickHouse-Format: XML +Content-Type:application/xml;charset=UTF-8 +Code:62.DB::Ex---tion: + +INSERT WITH X-ClickHouse-Format: BADFORMAT +Content-Type:text/plain;charset=UTF-8 +X-ClickHouse-Ex---tion-Code:62 +Code:62.DB::Ex---tion: diff --git a/tests/queries/0_stateless/02899_use_default_format_on_http_exception.sh b/tests/queries/0_stateless/02899_use_default_format_on_http_exception.sh new file mode 100755 index 00000000000..f92ab7db4fb --- /dev/null +++ b/tests/queries/0_stateless/02899_use_default_format_on_http_exception.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +CH_URL="$CLICKHOUSE_URL&http_write_exception_in_output_format=1" + +echo "INSERT WITH default_format=JSON" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}&default_format=JSON" -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' +echo "" +echo "INSERT WITH default_format=XML" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}&default_format=XML" -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' +echo "" +echo "INSERT WITH default_format=BADFORMAT" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}&default_format=BADFORMAT" -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' + + +echo "" +echo "INSERT WITH X-ClickHouse-Format: JSON" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}" -H 'X-ClickHouse-Format: JSON' -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' +echo "" +echo "INSERT WITH X-ClickHouse-Format: XML" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}" -H 'X-ClickHouse-Format: XML' -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' +echo "" +echo "INSERT WITH X-ClickHouse-Format: BADFORMAT" +echo "INSERT INTO system.numbers Select * from numbers(10);" \ + | ${CLICKHOUSE_CURL} -sS "${CH_URL}" -H 'X-ClickHouse-Format: BADFORMAT' -i | grep 'xception\|Content-Type' | sed 's/Exception/Ex---tion/' | awk '{ print $1 $2 $3 }' From c549083e2c8b009b447927ad669d27d882b688b0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 14:23:59 +0200 Subject: [PATCH 077/120] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index c64330e9df1..487ca7f7f4c 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -804,7 +804,7 @@ void S3QueueFilesMetadata::cleanupThreadFuncImpl() /// Create a lock so that with distributed processing /// multiple nodes do not execute cleanup in parallel. - auto ephemeral_node = zkutil::EphemeralNodeHolder::create(zookeeper_cleanup_lock_path, *zk_client, toString(getCurrentTime())); + auto ephemeral_node = zkutil::EphemeralNodeHolder::tryCreate(zookeeper_cleanup_lock_path, *zk_client, toString(getCurrentTime())); if (!ephemeral_node) { LOG_TEST(log, "Cleanup is already being executed by another node"); From fd37e25fbd11909b620c13e87c41e7c3823474e4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 15:07:58 +0200 Subject: [PATCH 078/120] Fix doc --- .../en/engines/table-engines/integrations/s3queue.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 60bc48f8bf9..2a9b25e0c1a 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -243,17 +243,17 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table ## Limitations {#limitations} -Duplicated rows can be as a result of: +1. Duplicated rows can be as a result of: -- an exception happens during parsing in the middle of file processing and retries are enabled via `s3queue_loading_retries`, duplicate rows are inevitabe; +- an exception happens during parsing in the middle of file processing and retries are enabled via `s3queue_loading_retries`; - `S3Queue` is configured on multiple servers pointing to the same path in zookeeper and keeper session expires before one server managed to commit processed file, which could lead to another server taking processing of the file, which could be partially or fully processed by the first server; -- abnormal server termination (with SIGABRT, for example). +- abnormal server termination. + +2. `S3Queue` is configured on multiple servers pointing to the same path in zookeeper and `Ordered` mode is used, then `s3queue_loading_retries` will not work. This will be fixed soon. + -:::note -If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. -::: ## Introspection {#introspection} For introspection use `system.s3queue` stateless table and `system.s3queue_log` persistent table. From b4a4c1e02e9c7417bc942ffe0400a41cfb52d78e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 17 Oct 2023 16:02:06 +0200 Subject: [PATCH 079/120] Only change headers if necessary --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cda2d70714a..734e4e68776 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1340,7 +1340,7 @@ void executeQuery( { String format_name = context->getDefaultFormat(); output_format = FormatFactory::instance().getOutputFormat(format_name, ostr, {}, context, output_format_settings); - if (output_format) + if (output_format && output_format->supportsWritingException()) { /// Force an update of the headers before we start writing result_details.content_type = output_format->getContentType(); From b382705c2d1e5ff004f20a4efaa7de8d370b4caa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 16:29:20 +0200 Subject: [PATCH 080/120] Use diff object again, since JSON API limits the files --- tests/ci/pr_info.py | 13 +++++++++---- tests/ci/worker/prepare-ci-ami.sh | 1 - 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index c02c71d8761..faa6b539a6b 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -4,6 +4,8 @@ import logging import os from typing import Dict, List, Set, Union, Literal +from unidiff import PatchSet # type: ignore + from build_download_helper import get_gh_api from env_helper import ( GITHUB_REPOSITORY, @@ -263,11 +265,14 @@ class PRInfo: raise TypeError("The event does not have diff URLs") for diff_url in self.diff_urls: - response = get_gh_api(diff_url, sleep=RETRY_SLEEP) + response = get_gh_api( + diff_url, + sleep=RETRY_SLEEP, + headers={"Accept": "application/vnd.github.v3.diff"}, + ) response.raise_for_status() - diff = response.json() - if "files" in diff: - self.changed_files = {f["filename"] for f in diff["files"]} + diff_object = PatchSet(response.text) + self.changed_files.update({f.path for f in diff_object}) print(f"Fetched info about {len(self.changed_files)} changed files") def get_dict(self): diff --git a/tests/ci/worker/prepare-ci-ami.sh b/tests/ci/worker/prepare-ci-ami.sh index f3e11b6d00a..20e7e3fd53e 100644 --- a/tests/ci/worker/prepare-ci-ami.sh +++ b/tests/ci/worker/prepare-ci-ami.sh @@ -90,7 +90,6 @@ systemctl restart docker sudo -u ubuntu docker buildx version sudo -u ubuntu docker buildx create --use --name default-builder -# FIXME: remove unidiff as soon as no old PRs could use it, here and in Dockerfile pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory mkdir -p $RUNNER_HOME && cd $RUNNER_HOME From a43ee4116283277b465d76963e28208692fbbe59 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 18 Sep 2023 20:02:46 +0200 Subject: [PATCH 081/120] increase s3_retry_attempts --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2c45108f9a5..1012e7a2d9a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -103,7 +103,7 @@ class IColumn; M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ - M(UInt64, s3_retry_attempts, 10, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ + M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 3000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ M(UInt64, s3_http_connection_pool_size, 1000, "How many reusable open connections to keep per S3 endpoint. Only applies to the S3 table engine and table function, not to S3 disks (for disks, use disk config instead). Global setting, can only be set in config, overriding it per session or per query has no effect.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ From eb4519b9d6df7abee5360d435a477fa432a97b09 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Oct 2023 19:02:04 +0200 Subject: [PATCH 082/120] Fix --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index 487ca7f7f4c..d952160491c 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -474,7 +474,8 @@ void S3QueueFilesMetadata::setFileProcessedForUnorderedMode(ProcessingNodeHolder if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); + if (max_loading_retries) + zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); return; } @@ -537,7 +538,8 @@ void S3QueueFilesMetadata::setFileProcessedForOrderedMode(ProcessingNodeHolderPt if (holder->remove(&requests, &responses)) { LOG_TEST(log, "Moved file `{}` to processed", path); - zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); + if (max_loading_retries) + zk_client->tryRemove(zookeeper_failed_path / (node_name + ".retriable"), -1); return; } From 66337c23cce377ce8233dd1ce92655b383eab8f8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 20 Sep 2023 21:22:53 +0200 Subject: [PATCH 083/120] Use treeless checkout for image building --- .github/workflows/backport_branches.yml | 1 + .github/workflows/docs_check.yml | 1 + .github/workflows/master.yml | 1 + .github/workflows/nightly.yml | 1 + .github/workflows/pull_request.yml | 1 + .github/workflows/release_branches.yml | 1 + 6 files changed, 6 insertions(+) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index c20255579e2..0d55d094990 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -77,6 +77,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 78724a4a924..203a149df65 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -73,6 +73,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 4771e584266..4bb9f785542 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -61,6 +61,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index ed8159b229e..8fabd86e2b3 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -54,6 +54,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 676fb993ca8..489203a8dcb 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -94,6 +94,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index fba56339d16..e3a75a08925 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -53,6 +53,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # to find ancestor merge commits necessary for finding proper docker tags + filter: tree:0 - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: From c19ca5c4d5a1749a8d1d7b0891ef560c1f51d660 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Tue, 17 Oct 2023 19:19:35 +0200 Subject: [PATCH 084/120] Address PR feedback --- docs/en/operations/settings/settings.md | 4 +- src/Core/Settings.h | 4 +- .../InterpreterShowColumnsQuery.cpp | 36 ++-- ...show_columns_mysql_compatibility.reference | 187 ++++++++++++------ ...02775_show_columns_mysql_compatibility.sql | 15 +- 5 files changed, 158 insertions(+), 88 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f22badb8c59..fe3f1298ff5 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3290,7 +3290,7 @@ Possible values: Default value: `0`. -## mysql_remap_string_as_text_in_show_columns {#mysql_remap_string_as_text_in_show_columns} +## mysql_map_string_to_text_in_show_columns {#mysql_map_string_to_text_in_show_columns} When enabled, [String](../../sql-reference/data-types/string.md) ClickHouse data type will be displayed as `TEXT` in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns). @@ -3301,7 +3301,7 @@ Has effect only when [use_mysql_types_in_show_columns](#use_mysql_types_in_show_ Default value: `0`. -## mysql_remap_fixed_string_as_text_in_show_columns {#mysql_remap_fixed_string_as_text_in_show_columns} +## mysql_map_fixed_string_to_text_in_show_columns {#mysql_map_fixed_string_to_text_in_show_columns} When enabled, [FixedString](../../sql-reference/data-types/fixedstring.md) ClickHouse data type will be displayed as `TEXT` in [SHOW COLUMNS](../../sql-reference/statements/show.md#show_columns). diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 41a99ff1319..537ffc75323 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -206,8 +206,8 @@ class IColumn; \ M(UInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ M(Bool, use_mysql_types_in_show_columns, false, "Show native MySQL types in SHOW [FULL] COLUMNS", 0) \ - M(Bool, mysql_remap_string_as_text_in_show_columns, false, "If enabled, String type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Only works if use_mysql_types_in_show_columns is enabled too", 0) \ - M(Bool, mysql_remap_fixed_string_as_text_in_show_columns, false, "If enabled, FixedString type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Only works if use_mysql_types_in_show_columns is enabled too", 0) \ + M(Bool, mysql_map_string_to_text_in_show_columns, false, "If enabled, String type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Will only take effect if use_mysql_types_in_show_columns is enabled too", 0) \ + M(Bool, mysql_map_fixed_string_to_text_in_show_columns, false, "If enabled, FixedString type will be mapped to TEXT in SHOW [FULL] COLUMNS, BLOB otherwise. Will only take effect if use_mysql_types_in_show_columns is enabled too", 0) \ \ M(UInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \ \ diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index a213f27972a..55891f51063 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -26,8 +26,8 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() const auto & settings = getContext()->getSettingsRef(); const bool use_mysql_types = settings.use_mysql_types_in_show_columns; - const bool remap_string_as_text = settings.mysql_remap_string_as_text_in_show_columns; - const bool remap_fixed_string_as_text = settings.mysql_remap_fixed_string_as_text_in_show_columns; + const bool remap_string_as_text = settings.mysql_map_string_to_text_in_show_columns; + const bool remap_fixed_string_as_text = settings.mysql_map_fixed_string_to_text_in_show_columns; WriteBufferFromOwnString buf_database; String resolved_database = getContext()->resolveDatabase(query.database); @@ -40,11 +40,13 @@ String InterpreterShowColumnsQuery::getRewrittenQuery() String rewritten_query; if (use_mysql_types) + { /// Cheapskate SQL-based mapping from native types to MySQL types, see https://dev.mysql.com/doc/refman/8.0/en/data-types.html /// Only used with setting 'use_mysql_types_in_show_columns = 1' /// Known issues: /// - Enums are translated to TEXT - rewritten_query += fmt::format(R"( + rewritten_query += fmt::format( + R"( WITH map( 'Int8', 'TINYINT', 'Int16', 'SMALLINT', @@ -56,8 +58,6 @@ WITH map( 'UInt64', 'BIGINT UNSIGNED', 'Float32', 'FLOAT', 'Float64', 'DOUBLE', - 'String', '{}', - 'FixedString', '{}', 'UUID', 'CHAR', 'Bool', 'TINYINT', 'Date', 'DATE', @@ -66,20 +66,26 @@ WITH map( 'DateTime64', 'DATETIME', 'Map', 'JSON', 'Tuple', 'JSON', - 'Object', 'JSON') AS native_to_mysql_mapping, - splitByRegexp('\(|\)', type_) AS split, - multiIf(startsWith(type_, 'LowCardinality(Nullable'), split[3], - startsWith(type_, 'LowCardinality'), split[2], - startsWith(type_, 'Nullable'), split[2], - split[1]) AS inner_type, - if (length(split) > 1, splitByString(', ', split[2]), []) AS decimal_scale_and_precision, - multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'), - mapContains(native_to_mysql_mapping, inner_type) = true, native_to_mysql_mapping[inner_type], - 'TEXT') AS mysql_type + 'Object', 'JSON', + 'String', '{}', + 'FixedString', '{}') AS native_to_mysql_mapping, )", remap_string_as_text ? "TEXT" : "BLOB", remap_fixed_string_as_text ? "TEXT" : "BLOB"); + rewritten_query += R"( + splitByRegexp('\(|\)', type_) AS split, + multiIf(startsWith(type_, 'LowCardinality(Nullable'), split[3], + startsWith(type_, 'LowCardinality'), split[2], + startsWith(type_, 'Nullable'), split[2], + split[1]) AS inner_type, + if (length(split) > 1, splitByString(', ', split[2]), []) AS decimal_scale_and_precision, + multiIf(inner_type = 'Decimal' AND toInt8(decimal_scale_and_precision[1]) <= 65 AND toInt8(decimal_scale_and_precision[2]) <= 30, concat('DECIMAL(', decimal_scale_and_precision[1], ', ', decimal_scale_and_precision[2], ')'), + mapContains(native_to_mysql_mapping, inner_type) = true, native_to_mysql_mapping[inner_type], + 'TEXT') AS mysql_type + )"; + } + rewritten_query += R"( SELECT name_ AS field, diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference index 29db3e1b73b..e038a3362df 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.reference @@ -120,7 +120,7 @@ ui32 INTEGER UNSIGNED NO \N ui64 BIGINT UNSIGNED NO \N ui8 TINYINT UNSIGNED NO \N uuid CHAR NO \N --- SHOW COLUMNS with mysql_remap_string_as_text_in_show_columns = 1 +-- SHOW COLUMNS with mysql_map_string_to_text_in_show_columns = 1 a TEXT NO \N agg TEXT NO \N b TINYINT NO \N @@ -181,7 +181,7 @@ ui32 INTEGER UNSIGNED NO \N ui64 BIGINT UNSIGNED NO \N ui8 TINYINT UNSIGNED NO \N uuid CHAR NO \N --- SHOW COLUMNS with mysql_remap_fixed_string_as_text_in_show_columns = 1 +-- SHOW COLUMNS with mysql_map_fixed_string_to_text_in_show_columns = 1 a TEXT NO \N agg TEXT NO \N b TINYINT NO \N @@ -242,64 +242,125 @@ ui32 INTEGER UNSIGNED NO \N ui64 BIGINT UNSIGNED NO \N ui8 TINYINT UNSIGNED NO \N uuid CHAR NO \N --- SHOW COLUMNS with both mysql_remap_string_as_text_in_show_columns = 1 and mysql_remap_fixed_string_as_text_in_show_columns = 1 -a TEXT NO \N -agg TEXT NO \N -b TINYINT NO \N -d DATE NO \N -d32 DATE NO \N -dec128 DECIMAL(38, 2) NO \N -dec128_native DECIMAL(35, 30) NO \N -dec128_text TEXT NO \N -dec256 TEXT NO \N -dec256_native DECIMAL(65, 2) NO \N -dec256_text TEXT NO \N -dec32 DECIMAL(9, 2) NO \N -dec64 DECIMAL(18, 2) NO \N -dt DATETIME NO \N -dt64 DATETIME NO \N -dt64_3_tz1 DATETIME NO \N -dt64_3_tz2 DATETIME NO \N -dt64_6 DATETIME NO \N -dt64_9 DATETIME NO \N -dt_tz1 DATETIME NO \N -dt_tz2 DATETIME NO \N -enm TEXT NO \N -f32 FLOAT NO \N -f64 DOUBLE NO \N -fs TEXT NO \N -i128 TEXT NO \N -i16 SMALLINT NO \N -i256 TEXT NO \N -i32 INTEGER NO \N -i64 BIGINT NO \N -i8 TINYINT NO \N -ip4 TEXT NO \N -ip6 TEXT NO \N -lfs TEXT NO \N -lnfs TEXT YES \N -lns TEXT YES \N -ls TEXT NO \N -m JSON NO \N -m_complex JSON NO \N -mpg TEXT NO \N -ndt64 DATETIME YES \N -ndt64_tz DATETIME YES \N -nested.col1 TEXT NO \N -nested.col2 TEXT NO \N -nfs TEXT YES \N -ns TEXT YES \N -o JSON NO \N -p TEXT NO \N -pg TEXT NO \N -r TEXT NO \N -s TEXT NO \N -sagg TEXT NO \N -t JSON NO \N -ui128 TEXT NO \N -ui16 SMALLINT UNSIGNED NO \N -ui256 TEXT NO \N -ui32 INTEGER UNSIGNED NO \N -ui64 BIGINT UNSIGNED NO \N -ui8 TINYINT UNSIGNED NO \N -uuid CHAR NO \N +-- SHOW COLUMNS with mysql_map_string_to_text_in_show_columns = 1 and without use_mysql_types_in_show_columns +a Array(String) NO \N +agg AggregateFunction(uniq, UInt64) NO \N +b Bool NO \N +d Date NO \N +d32 Date32 NO \N +dec128 Decimal(38, 2) NO \N +dec128_native Decimal(35, 30) NO \N +dec128_text Decimal(35, 31) NO \N +dec256 Decimal(76, 2) NO \N +dec256_native Decimal(65, 2) NO \N +dec256_text Decimal(66, 2) NO \N +dec32 Decimal(9, 2) NO \N +dec64 Decimal(18, 2) NO \N +dt DateTime NO \N +dt64 DateTime64(3) NO \N +dt64_3_tz1 DateTime64(3, \'UTC\') NO \N +dt64_3_tz2 DateTime64(3, \'Asia/Shanghai\') NO \N +dt64_6 DateTime64(6, \'UTC\') NO \N +dt64_9 DateTime64(9, \'UTC\') NO \N +dt_tz1 DateTime(\'UTC\') NO \N +dt_tz2 DateTime(\'Europe/Amsterdam\') NO \N +enm Enum8(\'hallo\' = 1, \'welt\' = 2) NO \N +f32 Float32 NO \N +f64 Float64 NO \N +fs FixedString(3) NO \N +i128 Int128 NO \N +i16 Int16 NO \N +i256 Int256 NO \N +i32 Int32 NO \N +i64 Int64 NO \N +i8 Int8 NO \N +ip4 IPv4 NO \N +ip6 IPv6 NO \N +lfs LowCardinality(FixedString(3)) NO \N +lnfs LowCardinality(Nullable(FixedString(3))) YES \N +lns LowCardinality(Nullable(String)) YES \N +ls LowCardinality(String) NO \N +m Map(Int32, String) NO \N +m_complex Map(Int32, Map(Int32, LowCardinality(Nullable(String)))) NO \N +mpg MultiPolygon NO \N +ndt64 Nullable(DateTime64(3)) YES \N +ndt64_tz Nullable(DateTime64(3, \'Asia/Shanghai\')) YES \N +nested.col1 Array(String) NO \N +nested.col2 Array(UInt32) NO \N +nfs Nullable(FixedString(3)) YES \N +ns Nullable(String) YES \N +o Object(\'json\') NO \N +p Point NO \N +pg Polygon NO \N +r Ring NO \N +s String NO \N +sagg SimpleAggregateFunction(sum, Float64) NO \N +t Tuple(Int32, String, Nullable(String), LowCardinality(String), LowCardinality(Nullable(String)), Tuple(Int32, String)) NO \N +ui128 UInt128 NO \N +ui16 UInt16 NO \N +ui256 UInt256 NO \N +ui32 UInt32 NO \N +ui64 UInt64 NO \N +ui8 UInt8 NO \N +uuid UUID NO \N +-- SHOW COLUMNS with mysql_map_fixed_string_to_text_in_show_columns = 1 and without use_mysql_types_in_show_columns +a Array(String) NO \N +agg AggregateFunction(uniq, UInt64) NO \N +b Bool NO \N +d Date NO \N +d32 Date32 NO \N +dec128 Decimal(38, 2) NO \N +dec128_native Decimal(35, 30) NO \N +dec128_text Decimal(35, 31) NO \N +dec256 Decimal(76, 2) NO \N +dec256_native Decimal(65, 2) NO \N +dec256_text Decimal(66, 2) NO \N +dec32 Decimal(9, 2) NO \N +dec64 Decimal(18, 2) NO \N +dt DateTime NO \N +dt64 DateTime64(3) NO \N +dt64_3_tz1 DateTime64(3, \'UTC\') NO \N +dt64_3_tz2 DateTime64(3, \'Asia/Shanghai\') NO \N +dt64_6 DateTime64(6, \'UTC\') NO \N +dt64_9 DateTime64(9, \'UTC\') NO \N +dt_tz1 DateTime(\'UTC\') NO \N +dt_tz2 DateTime(\'Europe/Amsterdam\') NO \N +enm Enum8(\'hallo\' = 1, \'welt\' = 2) NO \N +f32 Float32 NO \N +f64 Float64 NO \N +fs FixedString(3) NO \N +i128 Int128 NO \N +i16 Int16 NO \N +i256 Int256 NO \N +i32 Int32 NO \N +i64 Int64 NO \N +i8 Int8 NO \N +ip4 IPv4 NO \N +ip6 IPv6 NO \N +lfs LowCardinality(FixedString(3)) NO \N +lnfs LowCardinality(Nullable(FixedString(3))) YES \N +lns LowCardinality(Nullable(String)) YES \N +ls LowCardinality(String) NO \N +m Map(Int32, String) NO \N +m_complex Map(Int32, Map(Int32, LowCardinality(Nullable(String)))) NO \N +mpg MultiPolygon NO \N +ndt64 Nullable(DateTime64(3)) YES \N +ndt64_tz Nullable(DateTime64(3, \'Asia/Shanghai\')) YES \N +nested.col1 Array(String) NO \N +nested.col2 Array(UInt32) NO \N +nfs Nullable(FixedString(3)) YES \N +ns Nullable(String) YES \N +o Object(\'json\') NO \N +p Point NO \N +pg Polygon NO \N +r Ring NO \N +s String NO \N +sagg SimpleAggregateFunction(sum, Float64) NO \N +t Tuple(Int32, String, Nullable(String), LowCardinality(String), LowCardinality(Nullable(String)), Tuple(Int32, String)) NO \N +ui128 UInt128 NO \N +ui16 UInt16 NO \N +ui256 UInt256 NO \N +ui32 UInt32 NO \N +ui64 UInt64 NO \N +ui8 UInt8 NO \N +uuid UUID NO \N diff --git a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql index 5f7d1881702..e447dee47ed 100644 --- a/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql +++ b/tests/queries/0_stateless/02775_show_columns_mysql_compatibility.sql @@ -78,13 +78,16 @@ SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 0; SELECT '-- SHOW COLUMNS with use_mysql_types_in_show_columns = 1'; SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 1; -SELECT '-- SHOW COLUMNS with mysql_remap_string_as_text_in_show_columns = 1'; -SHOW COLUMNS FROM tab SETTINGS mysql_remap_string_as_text_in_show_columns=1; +SELECT '-- SHOW COLUMNS with mysql_map_string_to_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 1, mysql_map_string_to_text_in_show_columns=1; -SELECT '-- SHOW COLUMNS with mysql_remap_fixed_string_as_text_in_show_columns = 1'; -SHOW COLUMNS FROM tab SETTINGS mysql_remap_fixed_string_as_text_in_show_columns=1; +SELECT '-- SHOW COLUMNS with mysql_map_fixed_string_to_text_in_show_columns = 1'; +SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 1, mysql_map_fixed_string_to_text_in_show_columns=1; -SELECT '-- SHOW COLUMNS with both mysql_remap_string_as_text_in_show_columns = 1 and mysql_remap_fixed_string_as_text_in_show_columns = 1'; -SHOW COLUMNS FROM tab SETTINGS mysql_remap_string_as_text_in_show_columns=1, mysql_remap_fixed_string_as_text_in_show_columns=1; +SELECT '-- SHOW COLUMNS with mysql_map_string_to_text_in_show_columns = 1 and without use_mysql_types_in_show_columns'; +SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 0, mysql_map_string_to_text_in_show_columns=1; + +SELECT '-- SHOW COLUMNS with mysql_map_fixed_string_to_text_in_show_columns = 1 and without use_mysql_types_in_show_columns'; +SHOW COLUMNS FROM tab SETTINGS use_mysql_types_in_show_columns = 0, mysql_map_fixed_string_to_text_in_show_columns=1; DROP TABLE tab; From 9c9bc3fee243968b8743d66bbc844943cda2bb32 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 19:21:08 +0200 Subject: [PATCH 085/120] Add `filter:` for almost every checkout --- .github/workflows/backport_branches.yml | 5 +++++ .github/workflows/jepsen.yml | 2 ++ .github/workflows/master.yml | 13 +++++++++++++ .github/workflows/nightly.yml | 1 + .github/workflows/pull_request.yml | 3 +++ .github/workflows/release.yml | 1 + .github/workflows/release_branches.yml | 5 +++++ .github/workflows/tags_stable.yml | 1 + 8 files changed, 31 insertions(+) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 0d55d094990..eb4c29130c4 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -186,6 +186,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -228,6 +229,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -400,6 +402,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -449,6 +452,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -488,6 +492,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + filter: tree:0 - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index e67df15c4d3..7f1fd16aa89 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -24,6 +24,7 @@ jobs: with: clear-repository: true fetch-depth: 0 + filter: tree:0 - name: Jepsen Test run: | sudo rm -fr "$TEMP_PATH" @@ -53,6 +54,7 @@ jobs: # with: # clear-repository: true # fetch-depth: 0 + # filter: tree:0 # - name: Jepsen Test # run: | # sudo rm -fr "$TEMP_PATH" diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 4bb9f785542..a9aa7717add 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -201,6 +201,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -243,6 +244,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -284,6 +286,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -582,6 +585,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -631,6 +635,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -673,6 +678,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -715,6 +721,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -764,6 +771,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -806,6 +814,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -848,6 +857,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -890,6 +900,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -932,6 +943,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -964,6 +976,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + filter: tree:0 - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 8fabd86e2b3..6452b83fdd6 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -91,6 +91,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # Shallow clones should be disabled for a better relevancy of analysis + filter: tree:0 submodules: true - name: Set up JDK 11 uses: actions/setup-java@v1 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 489203a8dcb..45ce125f1a8 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -267,6 +267,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # for performance artifact + filter: tree:0 submodules: true - name: Build run: | @@ -351,6 +352,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # for performance artifact + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -1022,6 +1024,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + filter: tree:0 - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 0742ebfd449..051a3ea1eac 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -49,6 +49,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # otherwise we will have no version info + filter: tree:0 ref: ${{ env.GITHUB_TAG }} - name: Check docker clickhouse/clickhouse-server building run: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index e3a75a08925..5dd837c6456 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -162,6 +162,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -204,6 +205,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # For a proper version and performance artifacts + filter: tree:0 - name: Build run: | sudo rm -fr "$TEMP_PATH" @@ -457,6 +459,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -506,6 +509,7 @@ jobs: clear-repository: true submodules: true fetch-depth: 0 # otherwise we will have no info about contributors + filter: tree:0 - name: Apply sparse checkout for contrib # in order to check that it doesn't break build run: | rm -rf "$GITHUB_WORKSPACE/contrib" && echo 'removed' @@ -545,6 +549,7 @@ jobs: with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + filter: tree:0 - name: Check docker clickhouse/clickhouse-server building run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/tags_stable.yml b/.github/workflows/tags_stable.yml index f5b42e9c882..0a3945829ca 100644 --- a/.github/workflows/tags_stable.yml +++ b/.github/workflows/tags_stable.yml @@ -38,6 +38,7 @@ jobs: with: ref: master fetch-depth: 0 + filter: tree:0 - name: Update versions, docker version, changelog, security env: GITHUB_TOKEN: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} From 92260b7aedf19d3c31b3a5f48b06804843cd47f5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 19:27:58 +0200 Subject: [PATCH 086/120] Ignore docs_check.yml in pull_requests and wise versa --- .github/workflows/docs_check.yml | 1 + .github/workflows/pull_request.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 203a149df65..dada9999a68 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -18,6 +18,7 @@ on: # yamllint disable-line rule:truthy - 'docs/**' - 'utils/check-style/aspell-ignore/**' - 'tests/ci/docs_check.py' + - '.github/workflows/docs_check.yml' jobs: CheckLabels: runs-on: [self-hosted, style-checker] diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 45ce125f1a8..5937f434135 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -18,6 +18,7 @@ on: # yamllint disable-line rule:truthy - 'docs/**' - 'utils/check-style/aspell-ignore/**' - 'tests/ci/docs_check.py' + - '.github/workflows/docs_check.yml' ########################################################################################## ##################################### SMALL CHECKS ####################################### ########################################################################################## From 2c82329ef74268421db55ccdef1bd207d6ad3d6e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 17 Oct 2023 17:33:23 +0000 Subject: [PATCH 087/120] Update submodule --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index cb6b724ac49..4b3a77e6008 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit cb6b724ac49fe94098e30e3e0d9fd6901007ff38 +Subproject commit 4b3a77e600859bb3e143247f91ef75e4286b7ec9 From 610506febc8eae443ea79b7cc2fc00b4757d1e1e Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Tue, 17 Oct 2023 21:23:24 +0200 Subject: [PATCH 088/120] fix max intersection endian write --- src/AggregateFunctions/AggregateFunctionMaxIntersections.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 5074e491f60..41197c5ec0b 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -129,7 +129,10 @@ public: { writePODBinary(value[i].first, buf); writePODBinary(zero_padding, buf); - writePODBinary(value[i].second, buf); + if constexpr (std::endian::native == std::endian::little) + writePODBinary(value[i].second, buf); + else + writePODBinary(std::byteswap(value[i].second), buf); } } From 585e27009f0edcd41d88b3c2e0fb3706948c2893 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Tue, 17 Oct 2023 21:26:12 +0200 Subject: [PATCH 089/120] fix spaces --- src/AggregateFunctions/AggregateFunctionMaxIntersections.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 41197c5ec0b..fb333da3b85 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -129,9 +129,9 @@ public: { writePODBinary(value[i].first, buf); writePODBinary(zero_padding, buf); - if constexpr (std::endian::native == std::endian::little) - writePODBinary(value[i].second, buf); - else + if constexpr (std::endian::native == std::endian::little) + writePODBinary(value[i].second, buf); + else writePODBinary(std::byteswap(value[i].second), buf); } } From 4848baf302a5741e0fe31285672cc5a5c58efccf Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 21:08:55 +0200 Subject: [PATCH 090/120] Adjust performance artifacts prepare to the filtered checkout --- docker/packager/binary/build.sh | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 6a27e7adceb..cc2613cbaf5 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -172,10 +172,15 @@ then # This is why we add this repository snapshot from CI to the performance test # package. mkdir "$PERF_OUTPUT"/ch - git -C "$PERF_OUTPUT"/ch init --bare - git -C "$PERF_OUTPUT"/ch remote add origin /build - git -C "$PERF_OUTPUT"/ch fetch --no-tags --depth 50 origin HEAD:pr - git -C "$PERF_OUTPUT"/ch fetch --no-tags --depth 50 origin master:master + # Copy .git only, but skip modules, using tar + tar c -C /build/ --exclude='.git/modules/**' .git | tar x -C "$PERF_OUTPUT"/ch + # Create branch pr and origin/master to have them for the following performance comparison + git -C "$PERF_OUTPUT"/ch branch pr + git -C "$PERF_OUTPUT"/ch fetch --no-tags --depth 50 origin master:origin/master + # Clean remote, to not have it stale + git -C "$PERF_OUTPUT"/ch remote | xargs -n1 git -C "$PERF_OUTPUT"/ch remote remove + # And clean all tags + git -C "$PERF_OUTPUT"/ch tag | xargs git -C "$PERF_OUTPUT"/ch tag -d git -C "$PERF_OUTPUT"/ch reset --soft pr git -C "$PERF_OUTPUT"/ch log -5 ( From fc24e599fddeefbe4a5a136b263efd255ac9a29e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 07:26:32 +0000 Subject: [PATCH 091/120] Check if id is nullptr --- src/Parsers/ASTPartition.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTPartition.cpp b/src/Parsers/ASTPartition.cpp index 22eecd37565..d08f9ae1ca5 100644 --- a/src/Parsers/ASTPartition.cpp +++ b/src/Parsers/ASTPartition.cpp @@ -36,8 +36,9 @@ String ASTPartition::getID(char delim) const { if (value) return "Partition"; - else - return "Partition_ID" + (delim + id->getID()); + + std::string id_string = id ? id->getID() : ""; + return "Partition_ID" + (delim + id_string); } ASTPtr ASTPartition::clone() const From 70d366a8ebb8d2483ff9e8ff75d468809516e5ff Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 07:59:49 +0000 Subject: [PATCH 092/120] Control feature flag randomization with env --- docker/test/stateless/stress_tests.lib | 12 +++++++----- tests/ci/stress_check.py | 16 +++++++++++++++- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/docker/test/stateless/stress_tests.lib b/docker/test/stateless/stress_tests.lib index 11945b68f70..e1642517187 100644 --- a/docker/test/stateless/stress_tests.lib +++ b/docker/test/stateless/stress_tests.lib @@ -61,11 +61,13 @@ function configure() sudo mv /etc/clickhouse-server/config.d/keeper_port.xml.tmp /etc/clickhouse-server/config.d/keeper_port.xml } - # Randomize all Keeper feature flags - randomize_config_boolean_value filtered_list - randomize_config_boolean_value multi_read - randomize_config_boolean_value check_not_exists - randomize_config_boolean_value create_if_not_exists + if [[ -n "$RANDOMIZE_KEEPER_FEATURE_FLAGS" ]] && [[ "$RANDOMIZE_KEEPER_FEATURE_FLAGS" -eq 1 ]]; then + # Randomize all Keeper feature flags + randomize_config_boolean_value filtered_list + randomize_config_boolean_value multi_read + randomize_config_boolean_value check_not_exists + randomize_config_boolean_value create_if_not_exists + fi sudo chown clickhouse /etc/clickhouse-server/config.d/keeper_port.xml sudo chgrp clickhouse /etc/clickhouse-server/config.d/keeper_port.xml diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index b147b12f8ac..2ae734d4345 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -32,14 +32,25 @@ from tee_popen import TeePopen from upload_result_helper import upload_results +def get_additional_envs() -> List[str]: + result = [] + result.append(f"RANDOMIZE_KEEPER_FEATURE_FLAGS=1") + + return result + + def get_run_command( build_path: Path, result_path: Path, repo_tests_path: Path, server_log_path: Path, + additional_envs: List[str], ci_logs_args: str, image: DockerImage, ) -> str: + envs = [f"-e {e}" for e in additional_envs] + env_str = " ".join(envs) + cmd = ( "docker run --cap-add=SYS_PTRACE " # For dmesg and sysctl @@ -50,7 +61,7 @@ def get_run_command( f"--volume={build_path}:/package_folder " f"--volume={result_path}:/test_output " f"--volume={repo_tests_path}:/usr/share/clickhouse-test " - f"--volume={server_log_path}:/var/log/clickhouse-server {image} " + f"--volume={server_log_path}:/var/log/clickhouse-server {env_str} {image} " ) return cmd @@ -148,11 +159,14 @@ def run_stress_test(docker_image_name: str) -> None: pr_info, stopwatch.start_time_str, check_name ) + additional_envs = get_additional_envs() + run_command = get_run_command( packages_path, result_path, repo_tests_path, server_log_path, + additional_envs, ci_logs_args, docker_image, ) From be7be17352c30ed37a760ea87089e05958a24f9b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 10:22:18 +0200 Subject: [PATCH 093/120] add comment --- tests/ci/stress_check.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 2ae734d4345..75c425f6231 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -34,6 +34,9 @@ from upload_result_helper import upload_results def get_additional_envs() -> List[str]: result = [] + # some cloud-specific features require feature flags enabled + # so we need this ENV to be able to disable the randomization + # of feature flags result.append(f"RANDOMIZE_KEEPER_FEATURE_FLAGS=1") return result From 0b52d29cd81e35f266a725c0c2ad8580ff28115b Mon Sep 17 00:00:00 2001 From: huzhicheng <872237106@qq.com> Date: Wed, 18 Oct 2023 16:56:17 +0800 Subject: [PATCH 094/120] Query Cache: Only cache initial query --- src/Interpreters/executeQuery.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b96e05c1f9a..07456a31eeb 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -988,7 +988,11 @@ static std::tuple executeQueryImpl( } QueryCachePtr query_cache = context->getQueryCache(); - const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && (ast->as() || ast->as()); + const bool can_use_query_cache = query_cache != nullptr + && settings.use_query_cache + && !internal + && client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY + && (ast->as() || ast->as()); QueryCache::Usage query_cache_usage = QueryCache::Usage::None; if (!async_insert) From 71adb434e33e8c284959b53680a6a9bd89465a72 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 09:19:09 +0000 Subject: [PATCH 095/120] Remove f-string --- tests/ci/stress_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 75c425f6231..56c96dc2338 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -37,7 +37,7 @@ def get_additional_envs() -> List[str]: # some cloud-specific features require feature flags enabled # so we need this ENV to be able to disable the randomization # of feature flags - result.append(f"RANDOMIZE_KEEPER_FEATURE_FLAGS=1") + result.append("RANDOMIZE_KEEPER_FEATURE_FLAGS=1") return result From fff6a8dcc80243b38c90b4d4fd8013d4a6278a01 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 18 Oct 2023 11:35:53 +0200 Subject: [PATCH 096/120] Temporarily disable flaky test --- .../test_postgresql_replica_database_engine_2/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 2a1c7634698..15625e6d8a4 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -759,7 +759,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) order_by="id", ) - +@pytest.mark.skip(reason="flaky test, will fix soon") def test_replica_consumer(started_cluster): table = "test_replica_consumer" pg_manager_instance2.restart() From 2d638d64706ce426bcdcce3043083619df708525 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 18 Oct 2023 03:47:25 +0000 Subject: [PATCH 097/120] rocksdb: not creating block during writing Signed-off-by: Duc Canh Le --- src/Storages/RocksDB/EmbeddedRocksDBSink.cpp | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index b79c7284843..c451cfd1bf5 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -32,7 +32,7 @@ EmbeddedRocksDBSink::EmbeddedRocksDBSink( void EmbeddedRocksDBSink::consume(Chunk chunk) { auto rows = chunk.getNumRows(); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + const auto & columns = chunk.getColumns(); WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; @@ -44,12 +44,9 @@ void EmbeddedRocksDBSink::consume(Chunk chunk) wb_key.restart(); wb_value.restart(); - size_t idx = 0; - for (const auto & elem : block) - { - serializations[idx]->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value, {}); - ++idx; - } + for (size_t idx = 0; idx < columns.size(); ++idx) + serializations[idx]->serializeBinary(*columns[idx], i, idx == primary_key_pos ? wb_key : wb_value, {}); + status = batch.Put(wb_key.str(), wb_value.str()); if (!status.ok()) throw Exception(ErrorCodes::ROCKSDB_ERROR, "RocksDB write error: {}", status.ToString()); From babbdd9202466f0007d85d780097ed74733fb01f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 12:08:47 +0200 Subject: [PATCH 098/120] Fxi --- .../test_postgresql_replica_database_engine_2/test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 15625e6d8a4..7ae8c9b8713 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -759,13 +759,16 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) order_by="id", ) -@pytest.mark.skip(reason="flaky test, will fix soon") def test_replica_consumer(started_cluster): table = "test_replica_consumer" pg_manager_instance2.restart() + pg_manager.create_postgres_table(table) + instance.query( + f"INSERT INTO postgres_database.{table} SELECT number, number from numbers(0, 50)" + ) + for pm in [pg_manager, pg_manager_instance2]: - pm.create_and_fill_postgres_table(table) pm.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, From 33118442ddf248656bf2738405d6b82e85dd87c7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 13:01:35 +0200 Subject: [PATCH 099/120] Fix black check --- .../test_postgresql_replica_database_engine_2/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 7ae8c9b8713..e8053730c44 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -759,6 +759,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) order_by="id", ) + def test_replica_consumer(started_cluster): table = "test_replica_consumer" pg_manager_instance2.restart() From 663c2611eefabf34e2d1c7215eced5d02ca96610 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 13:05:29 +0200 Subject: [PATCH 100/120] Fix --- .../test_postgresql_replica_database_engine_2/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 15625e6d8a4..eee5c9624a7 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -759,6 +759,7 @@ VALUES (1, (SELECT array_to_string(ARRAY(SELECT chr((100 + round(random() * 25)) order_by="id", ) + @pytest.mark.skip(reason="flaky test, will fix soon") def test_replica_consumer(started_cluster): table = "test_replica_consumer" From 2eabb260a7f95b7d0e99cfe999cbe6531ca3ed7a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 18 Oct 2023 12:09:22 +0000 Subject: [PATCH 101/120] Correctly grep fuzzer.log --- docker/test/fuzzer/run-fuzzer.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index e56fe6ca83a..326f2a751e6 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -337,8 +337,8 @@ quit # which is confusing. task_exit_code=$fuzzer_exit_code echo "failure" > status.txt - { rg --text -o "Found error:.*" fuzzer.log \ - || rg --text -ao "Exception:.*" fuzzer.log \ + { rg -ao "Found error:.*" fuzzer.log \ + || rg -ao "Exception:.*" fuzzer.log \ || echo "Fuzzer failed ($fuzzer_exit_code). See the logs." ; } \ | tail -1 > description.txt fi From 726c370d087ad47348f504a167dac32225b109f8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 11:30:24 +0200 Subject: [PATCH 102/120] Replace temporary REPORITORY url by a proper one --- docker/keeper/Dockerfile | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 0e10068d79a..4dd231ae305 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -2,8 +2,8 @@ # If the image is built from Dockerfile.alpine, then the `-alpine` suffix is added automatically, # so the only purpose of Dockerfile.ubuntu is to push `latest`, `head` and so on w/o suffixes FROM ubuntu:20.04 AS glibc-donor - ARG TARGETARCH + RUN arch=${TARGETARCH:-amd64} \ && case $arch in \ amd64) rarch=x86_64 ;; \ @@ -31,7 +31,9 @@ RUN arch=${TARGETARCH:-amd64} \ arm64) ln -sf /lib/ld-2.31.so /lib/ld-linux-aarch64.so.1 ;; \ esac -ARG REPOSITORY="https://s3.amazonaws.com/clickhouse-builds/22.4/31c367d3cd3aefd316778601ff6565119fe36682/package_release" +# lts / testing / prestable / etc +ARG REPO_CHANNEL="stable" +ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" ARG VERSION="23.9.1.1854" ARG PACKAGES="clickhouse-keeper" From 01fa15c33fe79614479fb8a9b188a9d45f4ab25f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 18 Oct 2023 12:20:14 +0000 Subject: [PATCH 103/120] Less copying while handling annoucement --- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- .../ParallelReplicasReadingCoordinator.cpp | 13 ++++--------- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 8f4e90ea3d2..da40c197b47 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -574,7 +574,7 @@ void RemoteQueryExecutor::processMergeTreeInitialReadAnnouncement(InitialAllRang if (!extension || !extension->parallel_reading_coordinator) throw Exception(ErrorCodes::LOGICAL_ERROR, "Coordinator for parallel reading from replicas is not initialized"); - extension->parallel_reading_coordinator->handleInitialAllRangesAnnouncement(announcement); + extension->parallel_reading_coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); } void RemoteQueryExecutor::finish() diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 57cd91cc995..bae3a36a779 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -385,7 +385,7 @@ void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRang LOG_TRACE(log, "Received an announcement {}", announcement.describe()); /// To get rid of duplicates - for (const auto & part: announcement.description) + for (auto && part: announcement.description) { auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info == part.info; }); @@ -404,13 +404,8 @@ void InOrderCoordinator::handleInitialAllRangesAnnouncement(InitialAllRang if (covering_or_the_same_it != all_parts_to_read.end()) continue; - auto new_part = Part{ - .description = part, - .replicas = {announcement.replica_num} - }; - - auto insert_it = all_parts_to_read.insert(new_part); - auto & ranges = insert_it.first->description.ranges; + auto [inserted_it, _] = all_parts_to_read.emplace(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); + auto & ranges = inserted_it->description.ranges; std::sort(ranges.begin(), ranges.end()); } } @@ -517,7 +512,7 @@ void ParallelReplicasReadingCoordinator::handleInitialAllRangesAnnouncement(Init } - return pimpl->handleInitialAllRangesAnnouncement(announcement); + return pimpl->handleInitialAllRangesAnnouncement(std::move(announcement)); } ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelReadRequest request) From 8b49a9471357a0f5aa847350ecd532cb0cb7bffa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 11:34:12 +0200 Subject: [PATCH 104/120] Check sha512 sums for alpine server and keeper images --- docker/keeper/Dockerfile | 18 ++++++++---------- docker/server/Dockerfile.alpine | 19 ++++++++----------- 2 files changed, 16 insertions(+), 21 deletions(-) diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index 4dd231ae305..77a8977145a 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -48,16 +48,14 @@ ARG PACKAGES="clickhouse-keeper" ARG TARGETARCH RUN arch=${TARGETARCH:-amd64} \ && for package in ${PACKAGES}; do \ - { \ - { echo "Get ${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ - && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" -O "/tmp/${package}-${VERSION}-${arch}.tgz" \ - && tar xvzf "/tmp/${package}-${VERSION}-${arch}.tgz" --strip-components=1 -C / ; \ - } || \ - { echo "Fallback to ${REPOSITORY}/${package}-${VERSION}.tgz" \ - && wget -c -q "${REPOSITORY}/${package}-${VERSION}.tgz" -O "/tmp/${package}-${VERSION}.tgz" \ - && tar xvzf "/tmp/${package}-${VERSION}.tgz" --strip-components=2 -C / ; \ - } ; \ - } || exit 1 \ + ( \ + cd /tmp \ + && echo "Get ${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ + && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ + && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz.sha512" \ + && sed 's:/output/:/tmp/:' < "${package}-${VERSION}-${arch}.tgz.sha512" | sha512sum -c \ + && tar xvzf "${package}-${VERSION}-${arch}.tgz" --strip-components=1 -C / \ + ) \ ; done \ && rm /tmp/*.tgz /install -r \ && addgroup -S -g 101 clickhouse \ diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index afe1be75bed..de553b748a0 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -23,7 +23,6 @@ COPY docker_related_config.xml /etc/clickhouse-server/config.d/ COPY entrypoint.sh /entrypoint.sh ARG TARGETARCH - RUN arch=${TARGETARCH:-amd64} \ && case $arch in \ amd64) mkdir -p /lib64 && ln -sf /lib/ld-2.31.so /lib64/ld-linux-x86-64.so.2 ;; \ @@ -45,16 +44,14 @@ ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" RUN arch=${TARGETARCH:-amd64} \ && for package in ${PACKAGES}; do \ - { \ - { echo "Get ${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ - && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" -O "/tmp/${package}-${VERSION}-${arch}.tgz" \ - && tar xvzf "/tmp/${package}-${VERSION}-${arch}.tgz" --strip-components=1 -C / ; \ - } || \ - { echo "Fallback to ${REPOSITORY}/${package}-${VERSION}.tgz" \ - && wget -c -q "${REPOSITORY}/${package}-${VERSION}.tgz" -O "/tmp/${package}-${VERSION}.tgz" \ - && tar xvzf "/tmp/${package}-${VERSION}.tgz" --strip-components=2 -C / ; \ - } ; \ - } || exit 1 \ + ( \ + cd /tmp \ + && echo "Get ${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ + && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz" \ + && wget -c -q "${REPOSITORY}/${package}-${VERSION}-${arch}.tgz.sha512" \ + && sed 's:/output/:/tmp/:' < "${package}-${VERSION}-${arch}.tgz.sha512" | sha512sum -c \ + && tar xvzf "${package}-${VERSION}-${arch}.tgz" --strip-components=1 -C / \ + ) \ ; done \ && rm /tmp/*.tgz /install -r \ && addgroup -S -g 101 clickhouse \ From e4e1aac1548805cdfed9894b3078f690ddb7a143 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 11:34:57 +0200 Subject: [PATCH 105/120] Move comment about created user to a proper place --- docker/server/Dockerfile.ubuntu | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 402168c3e15..eef4fbf7c74 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -5,6 +5,13 @@ ARG DEBIAN_FRONTEND=noninteractive # ARG for quick switch to a given ubuntu mirror ARG apt_archive="http://archive.ubuntu.com" + +# user/group precreated explicitly with fixed uid/gid on purpose. +# It is especially important for rootless containers: in that case entrypoint +# can't do chown and owners of mounted volumes should be configured externally. +# We do that in advance at the begining of Dockerfile before any packages will be +# installed to prevent picking those uid / gid by some unrelated software. +# The same uid / gid (101) is used both for alpine and ubuntu. RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list \ && groupadd -r clickhouse --gid=101 \ && useradd -r -g clickhouse --uid=101 --home-dir=/var/lib/clickhouse --shell=/bin/bash clickhouse \ @@ -35,13 +42,6 @@ ARG deb_location_url="" # from a single binary url (useful for non-standard builds - with sanitizers, for arm64). ARG single_binary_location_url="" -# user/group precreated explicitly with fixed uid/gid on purpose. -# It is especially important for rootless containers: in that case entrypoint -# can't do chown and owners of mounted volumes should be configured externally. -# We do that in advance at the begining of Dockerfile before any packages will be -# installed to prevent picking those uid / gid by some unrelated software. -# The same uid / gid (101) is used both for alpine and ubuntu. - ARG TARGETARCH # install from a web location with deb packages From af31224ecd0c6c530ee2f38b0aed0e9b306e5135 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 17 Oct 2023 14:55:57 +0200 Subject: [PATCH 106/120] Write only filename to sha512 files for tarballs --- packages/build | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/packages/build b/packages/build index cc089dd08ad..c2285b8ee7c 100755 --- a/packages/build +++ b/packages/build @@ -112,7 +112,8 @@ EOF tar -czf "$TARBALL" -C "$OUTPUT_DIR" "$PKG_DIR" fi - sha512sum "$TARBALL" > "$TARBALL".sha512 + # Cut the $OUTPUT_DIR/ from the sha512sum output to make it universal + sha512sum "$TARBALL" | sed "s|$OUTPUT_DIR/||" > "$TARBALL".sha512 rm -r "$PKG_PATH" } From c4bad255b711a4cd32eae643fa992747c6a4e1bf Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 14:37:33 +0200 Subject: [PATCH 107/120] Fix --- src/Storages/S3Queue/StorageS3Queue.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 92f15aed62f..72e74d3c2a0 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -354,16 +354,24 @@ bool StorageS3Queue::streamToViews() auto file_iterator = createFileIterator(s3queue_context, nullptr); Pipes pipes; + pipes.reserve(s3queue_settings->s3queue_processing_threads_num); for (size_t i = 0; i < s3queue_settings->s3queue_processing_threads_num; ++i) { - auto source = createSource(file_iterator, block_io.pipeline.getHeader().getNames(), storage_snapshot, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + auto source = createSource( + file_iterator, block_io.pipeline.getHeader().getNames(), + storage_snapshot, DBMS_DEFAULT_BUFFER_SIZE, s3queue_context); + pipes.emplace_back(std::move(source)); } auto pipe = Pipe::unitePipes(std::move(pipes)); - std::atomic_size_t rows = 0; block_io.pipeline.complete(std::move(pipe)); + block_io.pipeline.setNumThreads(s3queue_settings->s3queue_processing_threads_num); + block_io.pipeline.setConcurrencyControl(s3queue_context->getSettingsRef().use_concurrency_control); + + std::atomic_size_t rows = 0; block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); From c7f78ac73a670b8e4421ae33af8d9a984930c89d Mon Sep 17 00:00:00 2001 From: Sinan Date: Wed, 18 Oct 2023 18:17:18 +0530 Subject: [PATCH 108/120] Add aggregation function for Largest-Triangle-Three-Buckets (#53145) * Added a simple lttb aggregate function * Added support for multiple datatypes * Added support for Date and Date32, updated LTTBData struct * Updated code to handle bucket size 0 and 1 * Added sort for LTTBData * Added tests and documentation * Added some code style fixes * Added function to new func ref file * Removed function from new func ref file * Apply suggestions from code review * Updated unit tests * updated LTTB data code * Minor style fixes * Updated code with std sort * updated tests * Renamed lttb to largestTriangleThreeBuckets * Added alias lttb --------- Co-authored-by: Alexey Milovidov --- .../aggregate-functions/reference/index.md | 1 + .../reference/largestTriangleThreeBuckets.md | 67 ++++ ...ateFunctionLargestTriangleThreeBuckets.cpp | 52 +++ ...egateFunctionLargestTriangleThreeBuckets.h | 327 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 2 + ...eThreeBuckets_aggregate_function.reference | 7 + ...riangleThreeBuckets_aggregate_function.sql | 47 +++ .../aspell-ignore/en/aspell-dict.txt | 3 + 8 files changed, 506 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md create mode 100644 src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h create mode 100644 tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.reference create mode 100644 tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index 6c56aefd51d..b1f2c5bacbb 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -103,4 +103,5 @@ ClickHouse-specific aggregate functions: - [quantileInterpolatedWeighted](./quantileinterpolatedweighted.md) - [sparkBar](./sparkbar.md) - [sumCount](./sumcount.md) +- [largestTriangleThreeBuckets](./largestTriangleThreeBuckets.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md b/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md new file mode 100644 index 00000000000..ca033bb48c8 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets.md @@ -0,0 +1,67 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/largestTriangleThreeBuckets +sidebar_position: 312 +sidebar_label: largestTriangleThreeBuckets +--- + +# largestTriangleThreeBuckets + +Applies the [Largest-Triangle-Three-Buckets](https://skemman.is/bitstream/1946/15343/3/SS_MSthesis.pdf) algorithm to the input data. +The algorithm is used for downsampling time series data for visualization. It is designed to operate on series sorted by x coordinate. +It works by dividing the sorted series into buckets and then finding the largest triangle in each bucket. The number of buckets is equal to the number of points in the resulting series. +the function will sort data by `x` and then apply the downsampling algorithm to the sorted data. + +**Syntax** + +``` sql +largestTriangleThreeBuckets(n)(x, y) +``` + +Alias: `lttb`. + +**Arguments** + +- `x` — x coordinate. [Integer](../../../sql-reference/data-types/int-uint.md) , [Float](../../../sql-reference/data-types/float.md) , [Decimal](../../../sql-reference/data-types/decimal.md) , [Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md). +- `y` — y coordinate. [Integer](../../../sql-reference/data-types/int-uint.md) , [Float](../../../sql-reference/data-types/float.md) , [Decimal](../../../sql-reference/data-types/decimal.md) , [Date](../../../sql-reference/data-types/date.md), [Date32](../../../sql-reference/data-types/date32.md), [DateTime](../../../sql-reference/data-types/datetime.md), [DateTime64](../../../sql-reference/data-types/datetime64.md). + +**Parameters** + +- `n` — number of points in the resulting series. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +[Array](../../../sql-reference/data-types/array.md) of [Tuple](../../../sql-reference/data-types/tuple.md) with two elements: + +**Example** + +Input table: + +``` text +┌─────x───────┬───────y──────┐ +│ 1.000000000 │ 10.000000000 │ +│ 2.000000000 │ 20.000000000 │ +│ 3.000000000 │ 15.000000000 │ +│ 8.000000000 │ 60.000000000 │ +│ 9.000000000 │ 55.000000000 │ +│ 10.00000000 │ 70.000000000 │ +│ 4.000000000 │ 30.000000000 │ +│ 5.000000000 │ 40.000000000 │ +│ 6.000000000 │ 35.000000000 │ +│ 7.000000000 │ 50.000000000 │ +└─────────────┴──────────────┘ +``` + +Query: + +``` sql +SELECT largestTriangleThreeBuckets(4)(x, y) FROM largestTriangleThreeBuckets_test; +``` + +Result: + +``` text +┌────────largestTriangleThreeBuckets(3)(x, y)───────────┐ +│ [(1,10),(3,15),(5,40),(10,70)] │ +└───────────────────────────────────────────────────────┘ +``` + diff --git a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp new file mode 100644 index 00000000000..9052f7a6661 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp @@ -0,0 +1,52 @@ +#include +#include +#include +#include + + +namespace ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +} + +namespace DB +{ +struct Settings; + +namespace +{ + + AggregateFunctionPtr + createAggregateFunctionLargestTriangleThreeBuckets(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) + { + assertBinary(name, argument_types); + + + if (!(isNumber(argument_types[0]) || isDateOrDate32(argument_types[0]) || isDateTime(argument_types[0]) + || isDateTime64(argument_types[0]))) + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the first argument", + name); + + if (!(isNumber(argument_types[1]) || isDateOrDate32(argument_types[1]) || isDateTime(argument_types[1]) + || isDateTime64(argument_types[1]))) + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the second argument", + name); + + return std::make_shared(argument_types, parameters); + } + +} + + +void registerAggregateFunctionLargestTriangleThreeBuckets(AggregateFunctionFactory & factory) +{ + factory.registerFunction(AggregateFunctionLargestTriangleThreeBuckets::name, createAggregateFunctionLargestTriangleThreeBuckets); + factory.registerAlias("lttb", AggregateFunctionLargestTriangleThreeBuckets::name); +} + + +} diff --git a/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h new file mode 100644 index 00000000000..0f2e888e967 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h @@ -0,0 +1,327 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +struct Settings; + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +struct LargestTriangleThreeBucketsData : public StatisticalSample +{ + void add(const Float64 xval, const Float64 yval, Arena * arena) + { + this->addX(xval, arena); + this->addY(yval, arena); + } + + void sort(Arena * arena) + { + // sort the this->x and this->y in ascending order of this->x using index + std::vector index(this->x.size()); + + std::iota(index.begin(), index.end(), 0); + ::sort(index.begin(), index.end(), [&](size_t i1, size_t i2) { return this->x[i1] < this->x[i2]; }); + + SampleX temp_x{}; + SampleY temp_y{}; + + for (size_t i = 0; i < this->x.size(); ++i) + { + temp_x.push_back(this->x[index[i]], arena); + temp_y.push_back(this->y[index[i]], arena); + } + + for (size_t i = 0; i < this->x.size(); ++i) + { + this->x[i] = temp_x[i]; + this->y[i] = temp_y[i]; + } + } + + PODArray> getResult(size_t total_buckets, Arena * arena) + { + // Sort the data + this->sort(arena); + + PODArray> result; + + // Handle special cases for small data list + if (this->x.size() <= total_buckets) + { + for (size_t i = 0; i < this->x.size(); ++i) + { + result.emplace_back(std::make_pair(this->x[i], this->y[i])); + } + return result; + } + + // Handle special cases for 0 or 1 or 2 buckets + if (total_buckets == 0) + return result; + if (total_buckets == 1) + { + result.emplace_back(std::make_pair(this->x.front(), this->y.front())); + return result; + } + if (total_buckets == 2) + { + result.emplace_back(std::make_pair(this->x.front(), this->y.front())); + result.emplace_back(std::make_pair(this->x.back(), this->y.back())); + return result; + } + + // Find the size of each bucket + size_t single_bucket_size = this->x.size() / total_buckets; + + // Include the first data point + result.emplace_back(std::make_pair(this->x[0], this->y[0])); + + for (size_t i = 1; i < total_buckets - 1; ++i) // Skip the first and last bucket + { + size_t start_index = i * single_bucket_size; + size_t end_index = (i + 1) * single_bucket_size; + + // Compute the average point in the next bucket + Float64 avg_x = 0; + Float64 avg_y = 0; + for (size_t j = end_index; j < (i + 2) * single_bucket_size; ++j) + { + avg_x += this->x[j]; + avg_y += this->y[j]; + } + avg_x /= single_bucket_size; + avg_y /= single_bucket_size; + + // Find the point in the current bucket that forms the largest triangle + size_t max_index = start_index; + Float64 max_area = 0.0; + for (size_t j = start_index; j < end_index; ++j) + { + Float64 area = std::abs( + 0.5 + * (result.back().first * this->y[j] + this->x[j] * avg_y + avg_x * result.back().second - result.back().first * avg_y + - this->x[j] * result.back().second - avg_x * this->y[j])); + if (area > max_area) + { + max_area = area; + max_index = j; + } + } + + // Include the selected point + result.emplace_back(std::make_pair(this->x[max_index], this->y[max_index])); + } + + // Include the last data point + result.emplace_back(std::make_pair(this->x.back(), this->y.back())); + + return result; + } +}; + +class AggregateFunctionLargestTriangleThreeBuckets final : public IAggregateFunctionDataHelper +{ +private: + UInt64 total_buckets{0}; + TypeIndex x_type; + TypeIndex y_type; + +public: + explicit AggregateFunctionLargestTriangleThreeBuckets(const DataTypes & arguments, const Array & params) + : IAggregateFunctionDataHelper({arguments}, {}, createResultType(arguments)) + { + if (params.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require one parameter", getName()); + + if (params[0].getType() != Field::Types::UInt64) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a UInt64", getName()); + + total_buckets = params[0].get(); + + this->x_type = WhichDataType(arguments[0]).idx; + this->y_type = WhichDataType(arguments[1]).idx; + } + + static constexpr auto name = "largestTriangleThreeBuckets"; + + String getName() const override { return name; } + + bool allocatesMemoryInArena() const override { return true; } + + static DataTypePtr createResultType(const DataTypes & arguments) + { + TypeIndex x_type = arguments[0]->getTypeId(); + TypeIndex y_type = arguments[1]->getTypeId(); + + UInt32 x_scale = 0; + UInt32 y_scale = 0; + + if (const auto * datetime64_type = typeid_cast(arguments[0].get())) + { + x_scale = datetime64_type->getScale(); + } + + if (const auto * datetime64_type = typeid_cast(arguments[1].get())) + { + y_scale = datetime64_type->getScale(); + } + + DataTypes types = {getDataTypeFromTypeIndex(x_type, x_scale), getDataTypeFromTypeIndex(y_type, y_scale)}; + + auto tuple = std::make_shared(std::move(types)); + + return std::make_shared(tuple); + } + + static DataTypePtr getDataTypeFromTypeIndex(TypeIndex type_index, UInt32 scale) + { + DataTypePtr data_type; + switch (type_index) + { + case TypeIndex::Date: + data_type = std::make_shared(); + break; + case TypeIndex::Date32: + data_type = std::make_shared(); + break; + case TypeIndex::DateTime: + data_type = std::make_shared(); + break; + case TypeIndex::DateTime64: + data_type = std::make_shared(scale); + break; + default: + data_type = std::make_shared>(); + } + return data_type; + } + + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + Float64 x = getFloat64DataFromColumn(columns[0], row_num, this->x_type); + Float64 y = getFloat64DataFromColumn(columns[1], row_num, this->y_type); + this->data(place).add(x, y, arena); + } + + Float64 getFloat64DataFromColumn(const IColumn * column, size_t row_num, TypeIndex type_index) const + { + switch (type_index) + { + case TypeIndex::Date: + return static_cast(*column).getData()[row_num]; + case TypeIndex::Date32: + return static_cast(*column).getData()[row_num]; + case TypeIndex::DateTime: + return static_cast(*column).getData()[row_num]; + case TypeIndex::DateTime64: + return static_cast(*column).getData()[row_num]; + default: + return column->getFloat64(row_num); + } + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + auto & a = this->data(place); + const auto & b = this->data(rhs); + + a.merge(b, arena); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + this->data(place).write(buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + { + this->data(place).read(buf, arena); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override + { + auto res = this->data(place).getResult(total_buckets, arena); + + auto & col = assert_cast(to); + auto & col_offsets = assert_cast(col.getOffsetsColumn()); + + auto column_x_adder_func = getColumnAdderFunc(x_type); + auto column_y_adder_func = getColumnAdderFunc(y_type); + + for (size_t i = 0; i < res.size(); ++i) + { + auto & column_tuple = assert_cast(col.getData()); + column_x_adder_func(column_tuple.getColumn(0), res[i].first); + column_y_adder_func(column_tuple.getColumn(1), res[i].second); + } + + col_offsets.getData().push_back(col.getData().size()); + } + + std::function getColumnAdderFunc(TypeIndex type_index) const + { + switch (type_index) + { + case TypeIndex::Date: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(static_cast(value)); + }; + case TypeIndex::Date32: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(static_cast(value)); + }; + case TypeIndex::DateTime: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(static_cast(value)); + }; + case TypeIndex::DateTime64: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(static_cast(value)); + }; + default: + return [](IColumn & column, Float64 value) + { + auto & col = assert_cast(column); + col.getData().push_back(value); + }; + } + } +}; + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 21bd783e6bc..69823be07f6 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -82,6 +82,7 @@ void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &); void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory &); void registerAggregateFunctionFlameGraph(AggregateFunctionFactory &); void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory); +void registerAggregateFunctionLargestTriangleThreeBuckets(AggregateFunctionFactory & factory); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -176,6 +177,7 @@ void registerAggregateFunctions() registerAggregateFunctionAnalysisOfVariance(factory); registerAggregateFunctionFlameGraph(factory); registerAggregateFunctionKolmogorovSmirnovTest(factory); + registerAggregateFunctionLargestTriangleThreeBuckets(factory); registerWindowFunctions(factory); } diff --git a/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.reference b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.reference new file mode 100644 index 00000000000..b4407dea13a --- /dev/null +++ b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.reference @@ -0,0 +1,7 @@ +[] +[(1,10)] +[(1,10),(10,70)] +[(1,10),(3,15),(5,40),(10,70)] +[(0.02,0.16),(0.06,0.73),(0.08,0.88),(0.09,0),(0.11,0.31),(0.15,0.09),(0.19,0.27),(0.2,0.24),(0.21,0.46),(0.23,0.23),(0.25,0.34),(0.29,0.16),(0.36,0.99),(0.37,0.86),(0.38,0.75),(0.39,0.86),(0.41,0.63),(0.45,0.86),(0.47,0.06),(0.98,0.09)] +[('2023-01-14 00:00:00.000',35.96528042030847),('2023-03-15 00:00:00.000',98.77709508458238),('2023-06-05 00:00:00.000',8.107958052612418),('2023-09-29 00:00:00.000',80.05338096818797),('2023-12-31 00:00:00.000',98.52375935588333)] +[('2023-01-14 00:00:00.000',35.96528042030847),('2023-03-15 00:00:00.000',98.77709508458238),('2023-06-05 00:00:00.000',8.107958052612418),('2023-09-29 00:00:00.000',80.05338096818797),('2023-12-31 00:00:00.000',98.52375935588333)] diff --git a/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql new file mode 100644 index 00000000000..2eb8754cb64 --- /dev/null +++ b/tests/queries/0_stateless/02842_largestTriangleThreeBuckets_aggregate_function.sql @@ -0,0 +1,47 @@ +drop table if exists largestTriangleThreeBucketsTestFloat64Float64; + +CREATE TABLE largestTriangleThreeBucketsTestFloat64Float64 +( + x Float64, + y Float64 +) ENGINE = MergeTree order by (y,x); + +INSERT INTO largestTriangleThreeBucketsTestFloat64Float64 +VALUES (1.0, 10.0),(2.0, 20.0),(3.0, 15.0),(8.0, 60.0),(9.0, 55.0),(10.0, 70.0),(4.0, 30.0),(5.0, 40.0),(6.0, 35.0),(7.0, 50.0); + +select largestTriangleThreeBuckets(0)(x, y) FROM largestTriangleThreeBucketsTestFloat64Float64; + +select largestTriangleThreeBuckets(1)(x, y) FROM largestTriangleThreeBucketsTestFloat64Float64; + +select largestTriangleThreeBuckets(2)(x, y) FROM largestTriangleThreeBucketsTestFloat64Float64; + +SELECT largestTriangleThreeBuckets(4)(x, y) AS downsampled_data +FROM largestTriangleThreeBucketsTestFloat64Float64; + +drop table largestTriangleThreeBucketsTestFloat64Float64; + +drop table if exists largestTriangleThreeBucketsTestDecimal64Decimal64; + +CREATE TABLE largestTriangleThreeBucketsTestDecimal64Decimal64 +( + x Decimal64(2), + y Decimal64(2) +) ENGINE = MergeTree order by (y,x); + +INSERT INTO largestTriangleThreeBucketsTestDecimal64Decimal64(x, y) VALUES (0.63, 0.25), (0.02, 0.16), (0.29, 0.16), (0.2, 0.24), (0.41, 0.63), (0.06, 0.73), (0.36, 0.99), (0.57, 0.18), (0.98, 0.09), (0.73, 0.95), (0.45, 0.86), (0.37, 0.86), (0.6, 0.64), (0.11, 0.31), (0.7, 0.25), (0.85, 0.15), (0.68, 0.39), (0.9, 0.3), (0.25, 0.34), (0.09, 0.0), (0.91, 0.62), (0.47, 0.06), (0.08, 0.88), (0.48, 0.57), (0.55, 0.75), (0.19, 0.27), (0.87, 0.15), (0.15, 0.09), (0.77, 0.28), (0.5, 0.2), (0.39, 0.86), (0.52, 0.11), (0.38, 0.75), (0.71, 0.44), (0.21, 0.46), (0.88, 0.15), (0.83, 0.67), (0.23, 0.23); + +select largestTriangleThreeBuckets(20)(x, y) from largestTriangleThreeBucketsTestDecimal64Decimal64; + +drop table largestTriangleThreeBucketsTestDecimal64Decimal64; + +drop table if exists largestTriangleThreeBucketsTestDateTime64Float64; + +create table largestTriangleThreeBucketsTestDateTime64Float64 (x DateTime64(3), y Float64) engine = MergeTree order by (y,x); + +INSERT INTO largestTriangleThreeBucketsTestDateTime64Float64 (x, y) VALUES ('2023-09-06 00:00:00', 14.217481939467213), ('2023-09-11 00:00:00', 30.096113766096455), ('2023-01-31 00:00:00', 91.42364224984735), ('2023-12-14 00:00:00', 42.08543753438961), ('2023-10-31 00:00:00', 29.93227107709394), ('2023-12-31 00:00:00', 98.52375935588333), ('2023-07-07 00:00:00', 79.9367415060134), ('2023-08-02 00:00:00', 55.417182033825696), ('2023-03-15 00:00:00', 98.77709508458238), ('2023-09-05 00:00:00', 2.832505232031368), ('2023-06-05 00:00:00', 8.107958052612418), ('2023-02-08 00:00:00', 62.95788480328096), ('2023-02-17 00:00:00', 76.80522155552535), ('2023-11-13 00:00:00', 24.927527306242993), ('2023-02-03 00:00:00', 7.966981342350332), ('2023-05-31 00:00:00', 44.61922229800436), ('2023-09-21 00:00:00', 65.86974701469791), ('2023-01-14 00:00:00', 35.96528042030847), ('2023-02-19 00:00:00', 16.065599678978305), ('2023-05-24 00:00:00', 17.23630978966909), ('2023-11-15 00:00:00', 15.544172190379879), ('2023-12-03 00:00:00', 13.738382187690856), ('2023-10-09 00:00:00', 16.7137129521176), ('2023-11-19 00:00:00', 12.12866001303361), ('2023-06-10 00:00:00', 95.15764263905534), ('2023-07-06 00:00:00', 18.87765798627088), ('2023-03-13 00:00:00', 44.82941460384813), ('2023-01-29 00:00:00', 36.0214717111606), ('2023-12-19 00:00:00', 90.30173319497655), ('2023-07-15 00:00:00', 12.67101467231364), ('2023-07-06 00:00:00', 88.13662733228512), ('2023-05-10 00:00:00', 34.18711141027026), ('2023-11-12 00:00:00', 75.58716684321973), ('2023-10-28 00:00:00', 35.79179186729331), ('2023-11-14 00:00:00', 0.9318182359137728), ('2023-09-29 00:00:00', 80.05338096818797), ('2023-09-13 00:00:00', 16.130217942056866), ('2023-07-28 00:00:00', 11.186638594914744), ('2023-02-12 00:00:00', 69.43690757793445), ('2023-12-18 00:00:00', 12.832032764204616), ('2023-05-21 00:00:00', 74.25002458036471), ('2023-04-03 00:00:00', 51.5662427420719), ('2023-11-27 00:00:00', 96.44359131281784), ('2023-03-29 00:00:00', 33.018594418113324), ('2023-02-07 00:00:00', 84.58945099939815), ('2023-11-16 00:00:00', 40.61531555527268), ('2023-04-21 00:00:00', 60.0545791577218), ('2023-01-31 00:00:00', 87.23185155362057), ('2023-05-19 00:00:00', 77.4095289464808), ('2023-08-26 00:00:00', 18.700816570182067); + +select largestTriangleThreeBuckets(5)(x, y) from largestTriangleThreeBucketsTestDateTime64Float64; + +select lttb(5)(x, y) from largestTriangleThreeBucketsTestDateTime64Float64; + +drop table largestTriangleThreeBucketsTestDateTime64Float64; \ No newline at end of file diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 110ca6c10ae..3a5bd023f7d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1400,6 +1400,7 @@ dmesg domainWithoutWWW dont dotProduct +downsampling dplyr dragonbox dropoff @@ -1708,6 +1709,7 @@ kurtsamp laion lang laravel +largestTriangleThreeBuckets latencies ldap leftPad @@ -1757,6 +1759,7 @@ london lowcardinality lowerUTF lowercased +lttb lzma macOS mailrugo From 38895ad2da5fa6298040b90e503cb43f9e0c46d6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 18 Oct 2023 12:59:12 +0000 Subject: [PATCH 109/120] Less copying in DefaultCoordinator --- .../ParallelReplicasReadingCoordinator.cpp | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index bae3a36a779..3834e175b78 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -134,7 +134,7 @@ public: void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) override; void markReplicaAsUnavailable(size_t replica_number) override; - void updateReadingState(const InitialAllRangesAnnouncement & announcement); + void updateReadingState(InitialAllRangesAnnouncement announcement); void finalizeReadingState(); size_t computeConsistentHash(const MergeTreePartInfo & info) const @@ -152,12 +152,12 @@ DefaultCoordinator::~DefaultCoordinator() LOG_DEBUG(log, "Coordination done: {}", toString(stats)); } -void DefaultCoordinator::updateReadingState(const InitialAllRangesAnnouncement & announcement) +void DefaultCoordinator::updateReadingState(InitialAllRangesAnnouncement announcement) { PartRefs parts_diff; /// To get rid of duplicates - for (const auto & part: announcement.description) + for (auto && part: announcement.description) { auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info.getPartNameV1() == part.info.getPartNameV1(); }); @@ -176,12 +176,7 @@ void DefaultCoordinator::updateReadingState(const InitialAllRangesAnnouncement & if (covering_or_the_same_it != all_parts_to_read.end()) continue; - auto new_part = Part{ - .description = part, - .replicas = {announcement.replica_num} - }; - - auto [insert_it, _] = all_parts_to_read.insert(new_part); + auto [insert_it, _] = all_parts_to_read.emplace(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); parts_diff.push_back(insert_it); } @@ -242,12 +237,14 @@ void DefaultCoordinator::finalizeReadingState() void DefaultCoordinator::handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { - updateReadingState(announcement); + const auto replica_num = announcement.replica_num; - if (announcement.replica_num >= stats.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", announcement.replica_num, stats.size()); + updateReadingState(std::move(announcement)); - stats[announcement.replica_num].number_of_requests +=1; + if (replica_num >= stats.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", replica_num, stats.size()); + + ++stats[replica_num].number_of_requests; ++sent_initial_requests; LOG_DEBUG(log, "Sent initial requests: {} Replicas count: {}", sent_initial_requests, replicas_count); From 3ef092f83f287cf5c6e97fa6603c5af79ad4c8af Mon Sep 17 00:00:00 2001 From: monchickey <75814968+monchickey@users.noreply.github.com> Date: Wed, 18 Oct 2023 21:04:27 +0800 Subject: [PATCH 110/120] Fix the compilation error that occurs when opening avx series instructions (#55049) * fix the compilation error that occurs when opening avx series instructions cmake command example: ```shell cmake .. -DCMAKE_BUILD_TYPE=Release -DENABLE_AVX=ON -DENABLE_AVX2=ON -DENABLE_AVX2_FOR_SPEC_OP=ON ``` * Update snappy CMakeLists.txt add SNAPPY_HAVE_X86_CRC32 parameter --- contrib/snappy-cmake/CMakeLists.txt | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/contrib/snappy-cmake/CMakeLists.txt b/contrib/snappy-cmake/CMakeLists.txt index f406de0e343..d923ddef6b6 100644 --- a/contrib/snappy-cmake/CMakeLists.txt +++ b/contrib/snappy-cmake/CMakeLists.txt @@ -24,6 +24,12 @@ else () set (SNAPPY_HAVE_SSSE3 0) endif () +if (ARCH_AMD64 AND ENABLE_SSE42) + set (SNAPPY_HAVE_X86_CRC32 1) +else () + set (SNAPPY_HAVE_X86_CRC32 0) +endif () + configure_file( "${SOURCE_DIR}/cmake/config.h.in" "${CMAKE_CURRENT_BINARY_DIR}/config.h") From f90e31eb3103547e509e02fcb74679bd9d9172e2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 18 Oct 2023 15:23:16 +0200 Subject: [PATCH 111/120] Enable log by default --- src/Storages/S3Queue/S3QueueSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 351245e24be..66fe9b4ce31 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -21,7 +21,7 @@ class ASTStorage; M(String, keeper_path, "", "Zookeeper node path", 0) \ M(UInt32, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ M(UInt32, s3queue_processing_threads_num, 1, "Number of processing threads", 0) \ - M(UInt32, s3queue_enable_logging_to_s3queue_log, 0, "Enable logging to system table system.s3queue_log", 0) \ + M(UInt32, s3queue_enable_logging_to_s3queue_log, 1, "Enable logging to system table system.s3queue_log", 0) \ M(UInt32, s3queue_tracked_file_ttl_sec, 0, "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", 0) \ M(UInt32, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt32, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ From dfdd2ee77cd9738d55a0c37a58ffe5e8b6f89a1d Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 18 Oct 2023 16:36:45 +0200 Subject: [PATCH 112/120] fix tests, to avoid timeouts do less retries --- .../test_checking_s3_blobs_paranoid/configs/setting.xml | 1 + .../integration/test_mask_sensitive_info/configs/users.xml | 5 +++++ .../test_s3_table_functions/configs/users.d/users.xml | 5 +++++ tests/integration/test_storage_s3/configs/defaultS3.xml | 5 +++++ tests/integration/test_storage_s3/configs/s3_retry.xml | 7 +++++++ tests/integration/test_storage_s3/test.py | 4 ++-- 6 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_storage_s3/configs/s3_retry.xml diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml index f8749488f57..3bc0f322cb8 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml @@ -5,6 +5,7 @@ 1 1 + 5 diff --git a/tests/integration/test_mask_sensitive_info/configs/users.xml b/tests/integration/test_mask_sensitive_info/configs/users.xml index 4b6ba057ecb..f129a5bb3e3 100644 --- a/tests/integration/test_mask_sensitive_info/configs/users.xml +++ b/tests/integration/test_mask_sensitive_info/configs/users.xml @@ -1,4 +1,9 @@ + + + 5 + + diff --git a/tests/integration/test_s3_table_functions/configs/users.d/users.xml b/tests/integration/test_s3_table_functions/configs/users.d/users.xml index 4b6ba057ecb..f129a5bb3e3 100644 --- a/tests/integration/test_s3_table_functions/configs/users.d/users.xml +++ b/tests/integration/test_s3_table_functions/configs/users.d/users.xml @@ -1,4 +1,9 @@ + + + 5 + + diff --git a/tests/integration/test_storage_s3/configs/defaultS3.xml b/tests/integration/test_storage_s3/configs/defaultS3.xml index 7dac6d9fbb5..37454ef6781 100644 --- a/tests/integration/test_storage_s3/configs/defaultS3.xml +++ b/tests/integration/test_storage_s3/configs/defaultS3.xml @@ -1,4 +1,9 @@ + + + 5 + + http://resolver:8080 diff --git a/tests/integration/test_storage_s3/configs/s3_retry.xml b/tests/integration/test_storage_s3/configs/s3_retry.xml new file mode 100644 index 00000000000..727e23273cf --- /dev/null +++ b/tests/integration/test_storage_s3/configs/s3_retry.xml @@ -0,0 +1,7 @@ + + + + 5 + + + diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index ca2ddeb78b0..bbd2d2d4be0 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -55,7 +55,7 @@ def started_cluster(): "configs/named_collections.xml", "configs/schema_cache.xml", ], - user_configs=["configs/access.xml", "configs/users.xml"], + user_configs=["configs/access.xml", "configs/users.xml", "configs/s3_retry.xml"], ) cluster.add_instance( "dummy_without_named_collections", @@ -71,7 +71,7 @@ def started_cluster(): "s3_max_redirects", with_minio=True, main_configs=["configs/defaultS3.xml"], - user_configs=["configs/s3_max_redirects.xml"], + user_configs=["configs/s3_max_redirects.xml", "configs/s3_retry.xml"], ) cluster.add_instance( "s3_non_default", From d73afed7d5118ca644392e9a5d2153190b595707 Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Wed, 18 Oct 2023 23:09:38 +0800 Subject: [PATCH 113/120] fix style --- src/Interpreters/executeQuery.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 07456a31eeb..22230c53698 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -988,11 +988,11 @@ static std::tuple executeQueryImpl( } QueryCachePtr query_cache = context->getQueryCache(); - const bool can_use_query_cache = query_cache != nullptr - && settings.use_query_cache - && !internal - && client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY - && (ast->as() || ast->as()); + const bool can_use_query_cache = query_cache != nullptr + && settings.use_query_cache + && !internal + && client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY + && (ast->as() || ast->as()); QueryCache::Usage query_cache_usage = QueryCache::Usage::None; if (!async_insert) From 6f349038bb4389403e3740abc769e9d7b678b30e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 18 Oct 2023 17:24:41 +0200 Subject: [PATCH 114/120] Enable test_mutation_simple with the analyzer --- tests/analyzer_integration_broken_tests.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index e2da429ab2f..7ad0b513900 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -39,8 +39,6 @@ test_settings_profile/test.py::test_show_profiles test_shard_level_const_function/test.py::test_remote test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view -test_system_merges/test.py::test_mutation_simple[] -test_system_merges/test.py::test_mutation_simple[replicated] test_user_defined_object_persistence/test.py::test_persistence test_wrong_db_or_table_name/test.py::test_wrong_table_name test_zookeeper_config/test.py::test_chroot_with_same_root From 1308f26cccbb66c592cb6f7cb4790b2188fefb4c Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Wed, 18 Oct 2023 17:30:59 +0200 Subject: [PATCH 115/120] Update test.py --- .../configs/inf_s3_retries.xml | 9 +++++++++ .../test_checking_s3_blobs_paranoid/test.py | 14 ++++++++++++-- tests/integration/test_storage_s3/test.py | 6 +++++- 3 files changed, 26 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml new file mode 100644 index 00000000000..206eb4f2bad --- /dev/null +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml @@ -0,0 +1,9 @@ + + + + + + 1000000 + + + diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index eb791383fde..c5b78e139bd 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -22,6 +22,17 @@ def cluster(): ], with_minio=True, ) + cluster.add_instance( + "node_with_inf_s3_retries", + main_configs=[ + "configs/storage_conf.xml", + ], + user_configs=[ + "configs/setting.xml", + "configs/inf_s3_retries.xml", + ], + with_minio=True, + ) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -468,7 +479,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): def test_query_is_canceled_with_inf_retries(cluster, broken_s3): - node = cluster.instances["node"] + node = cluster.instances["node_with_inf_s3_retries"] broken_s3.setup_at_part_upload( count=10000000, @@ -490,7 +501,6 @@ def test_query_is_canceled_with_inf_retries(cluster, broken_s3): FROM system.numbers LIMIT 1000000 SETTINGS - s3_retry_attempts=1000000, s3_max_single_part_upload_size=100, s3_min_upload_part_size=10000, s3_check_objects_after_upload=0 diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index bbd2d2d4be0..01ade1acc4d 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -55,7 +55,11 @@ def started_cluster(): "configs/named_collections.xml", "configs/schema_cache.xml", ], - user_configs=["configs/access.xml", "configs/users.xml", "configs/s3_retry.xml"], + user_configs=[ + "configs/access.xml", + "configs/users.xml", + "configs/s3_retry.xml", + ], ) cluster.add_instance( "dummy_without_named_collections", From 70711b08988958598cab9b9522f299b24bf8f7e1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 18 Oct 2023 23:05:03 -0700 Subject: [PATCH 116/120] Remove temporary debug logging in MultiplexedConnections (#55764) --- src/Client/MultiplexedConnections.cpp | 36 +-------------------------- src/Client/MultiplexedConnections.h | 9 +------ 2 files changed, 2 insertions(+), 43 deletions(-) diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 1b1ca427e05..c7d7d0143c8 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include @@ -24,14 +23,6 @@ namespace ErrorCodes } -#define MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION \ - mutex_last_locked_by.store((getThreadId() << 32) | __LINE__); \ - memcpy(mutex_memory_dump.data(), &cancel_mutex, mutex_memory_dump.size()); \ - mutex_locked += 1; \ - SCOPE_EXIT({ mutex_locked -= 1; }); -/// When you remove this macro, please also remove the clang-tidy suppressions at the beginning + end of this file. - - MultiplexedConnections::MultiplexedConnections(Connection & connection, const Settings & settings_, const ThrottlerPtr & throttler) : settings(settings_) { @@ -86,7 +77,6 @@ MultiplexedConnections::MultiplexedConnections( void MultiplexedConnections::sendScalarsData(Scalars & data) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (!sent_query) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send scalars data: query not yet sent."); @@ -102,7 +92,6 @@ void MultiplexedConnections::sendScalarsData(Scalars & data) void MultiplexedConnections::sendExternalTablesData(std::vector & data) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (!sent_query) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send external tables data: query not yet sent."); @@ -131,7 +120,6 @@ void MultiplexedConnections::sendQuery( bool with_pending_data) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (sent_query) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query already sent."); @@ -189,7 +177,6 @@ void MultiplexedConnections::sendQuery( void MultiplexedConnections::sendIgnoredPartUUIDs(const std::vector & uuids) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (sent_query) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send uuids after query is sent."); @@ -206,7 +193,6 @@ void MultiplexedConnections::sendIgnoredPartUUIDs(const std::vector & uuid void MultiplexedConnections::sendReadTaskResponse(const String & response) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (cancelled) return; current_connection->sendReadTaskResponse(response); @@ -216,7 +202,6 @@ void MultiplexedConnections::sendReadTaskResponse(const String & response) void MultiplexedConnections::sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (cancelled) return; current_connection->sendMergeTreeReadTaskResponse(response); @@ -226,29 +211,13 @@ void MultiplexedConnections::sendMergeTreeReadTaskResponse(const ParallelReadRes Packet MultiplexedConnections::receivePacket() { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION Packet packet = receivePacketUnlocked({}); return packet; } void MultiplexedConnections::disconnect() { - /// We've seen this lock mysteriously get stuck forever, without any other thread seeming to - /// hold the mutex. This is temporary code to print some extra information next time it happens. - /// std::lock_guard lock(cancel_mutex); - if (!cancel_mutex.try_lock_for(std::chrono::hours(1))) - { - UInt64 last_locked = mutex_last_locked_by.load(); - std::array new_memory_dump; - memcpy(new_memory_dump.data(), &cancel_mutex, new_memory_dump.size()); - LOG_ERROR(&Poco::Logger::get("MultiplexedConnections"), "Deadlock in MultiplexedConnections::disconnect()! Mutex was last (instrumentedly) locked by thread {} on line {}, lock balance: {}, mutex memory when last locked: {}, mutex memory now: {}", last_locked >> 32, last_locked & 0xffffffff, mutex_locked.load(), hexString(mutex_memory_dump.data(), mutex_memory_dump.size()), hexString(new_memory_dump.data(), new_memory_dump.size())); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Deadlock in MultiplexedConnections::disconnect()"); - } -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wthread-safety-analysis" - std::lock_guard lock(cancel_mutex, std::adopt_lock); -#pragma clang diagnostic pop - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION + std::lock_guard lock(cancel_mutex); for (ReplicaState & state : replica_states) { @@ -264,7 +233,6 @@ void MultiplexedConnections::disconnect() void MultiplexedConnections::sendCancel() { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (!sent_query || cancelled) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot cancel. Either no query sent or already cancelled."); @@ -282,7 +250,6 @@ void MultiplexedConnections::sendCancel() Packet MultiplexedConnections::drain() { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION if (!cancelled) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot drain connections: cancel first."); @@ -323,7 +290,6 @@ Packet MultiplexedConnections::drain() std::string MultiplexedConnections::dumpAddresses() const { std::lock_guard lock(cancel_mutex); - MUTEX_LOCK_TEMPORARY_DEBUG_INSTRUMENTATION return dumpAddressesUnlocked(); } diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index 4c776e6e168..9f7b47e0562 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -106,14 +106,7 @@ private: std::optional replica_info; /// A mutex for the sendCancel function to execute safely in separate thread. - mutable std::timed_mutex cancel_mutex; - - /// Temporary instrumentation to debug a weird deadlock on cancel_mutex. - /// TODO: Once the investigation is done, get rid of these, and of INSTRUMENTED_LOCK_MUTEX, and - /// change cancel_mutex to std::mutex. - mutable std::atomic mutex_last_locked_by{0}; - mutable std::atomic mutex_locked{0}; - mutable std::array mutex_memory_dump; + mutable std::mutex cancel_mutex; friend struct RemoteQueryExecutorRoutine; }; From 9a155ba747dbc33427334d6e75e6114a15e08277 Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Wed, 18 Oct 2023 10:48:03 +0200 Subject: [PATCH 117/120] Added test to check CapnProto cache Since current getting schema for CapnProto doesn't use cache, the integration test was for the future if someone add cache. Also, I was curious how schema cache affects performance and I compared reading binary files with Protobuf (I guess it works the same for CapnProto). ``` for i in {1..1000}; do clickhouse-client -q \ "select * from test.simple format Protobuf settings format_schema='/format_schemas/simple:KeyValuePair'" > simple-protobuf${i}.bin; done clickhouse-client --time -q "select * from file('simple-protobuf{1..999}.bin', 'Protobuf') format Null settings format_schema = 'simple:KeyValuePair'" ``` Protobuf with cache and without works with approximately the same time. --- .../test_format_schema_on_server/test.py | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/tests/integration/test_format_schema_on_server/test.py b/tests/integration/test_format_schema_on_server/test.py index 4d62dc976ce..3fa10ff1a1e 100644 --- a/tests/integration/test_format_schema_on_server/test.py +++ b/tests/integration/test_format_schema_on_server/test.py @@ -112,3 +112,66 @@ message MessageTmp { "SELECT * FROM test.simple FORMAT Protobuf SETTINGS format_schema='message_tmp:MessageTmp'" ) assert "NO_COLUMNS_SERIALIZED_TO_PROTOBUF_FIELDS)" in str(exc.value) + + +def test_drop_capn_proto_format(started_cluster): + create_simple_table() + instance.query("INSERT INTO test.simple VALUES (1, 'abc'), (2, 'def')") + capn_proto_schema = """ +@0x801f030c2b67bf19; + +struct MessageTmp { + key @0 :UInt64; + value @1 :Text; +} +""" + capn_schema_path_name = "message_tmp.capnp" + + database_path = os.path.abspath(os.path.join(instance.path, "database")) + format_schemas_path = os.path.join(database_path, "format_schemas") + with open(os.path.join(format_schemas_path, capn_schema_path_name), "w") as file: + file.write(capn_proto_schema) + + assert instance.http_query( + "SELECT * FROM test.simple FORMAT CapnProto SETTINGS format_schema='message_tmp:MessageTmp'" + ) == instance.query( + f"SELECT * FROM test.simple Format CapnProto SETTINGS format_schema='{format_schemas_path}/message_tmp:MessageTmp'" + ) + + new_schema = """ +@0x801f030c2b67bf19; + +struct MessageTmp { + key2 @0 :UInt64; + value2 @1 :Text; +} +""" + with open(os.path.join(format_schemas_path, capn_schema_path_name), "w") as file: + file.write(new_schema) + + instance.query("DROP TABLE IF EXISTS test.new_simple") + instance.query( + """ + CREATE TABLE test.new_simple (key2 UInt64, value2 String) + ENGINE = MergeTree ORDER BY tuple(); + """ + ) + instance.query("INSERT INTO test.new_simple VALUES (1, 'abc'), (2, 'def')") + + # instance.query("SYSTEM DROP FORMAT SCHEMA CACHE FOR CapnProto") + + # Tets works with new scheme + assert instance.http_query( + "SELECT * FROM test.new_simple FORMAT CapnProto SETTINGS format_schema='message_tmp:MessageTmp'" + ) == instance.query( + f"SELECT * FROM test.new_simple Format CapnProto SETTINGS format_schema='{format_schemas_path}/message_tmp:MessageTmp'" + ) + # Tests that stop working with old scheme + with pytest.raises(Exception) as exc: + instance.http_query( + "SELECT * FROM test.simple FORMAT CapnProto SETTINGS format_schema='message_tmp:MessageTmp'" + ) + assert ( + "Capnproto schema doesn't contain field with name key. (THERE_IS_NO_COLUMN)" + in str(exc.value) + ) From f18622b48b69705acd03be10aab0d9ecc656ee16 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 19 Oct 2023 09:12:14 +0000 Subject: [PATCH 118/120] Bump gRPC to 1v.36.4 --- contrib/grpc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/grpc b/contrib/grpc index 4b3a77e6008..c52656e2bfc 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 4b3a77e600859bb3e143247f91ef75e4286b7ec9 +Subproject commit c52656e2bfcda3450bd6a7c247d2d9eeb8498524 From d957d60a5267aa4713826ebe25e2631450e09eba Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 18 Oct 2023 19:36:34 +0200 Subject: [PATCH 119/120] Use publicly available play.clickhouse.com to search built images --- tests/ci/docker_manifests_merge.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index 9b61134fa75..0a81480c52a 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -209,7 +209,10 @@ def enrich_images(changed_images: Dict[str, str]) -> None: """ batch_count = 0 - ch_helper = ClickHouseHelper() + # We use always publicly available DB here intentionally + ch_helper = ClickHouseHelper( + "https://play.clickhouse.com", {"X-ClickHouse-User": "play"} + ) while ( batch_count <= MAX_COMMIT_BATCHES_TO_CHECK and len(images_to_find_tags_for) != 0 From d8b899d22bad4108a6eaac548e831032da1b3c6c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 18 Oct 2023 19:38:45 +0200 Subject: [PATCH 120/120] Use a proper tag from default.checks for enriched images --- tests/ci/docker_manifests_merge.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index 0a81480c52a..1be2a1f2e7a 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -197,8 +197,8 @@ def enrich_images(changed_images: Dict[str, str]) -> None: WITH {commit_shas:Array(String)} AS commit_shas, {images:Array(String)} AS images SELECT - substring(test_name, 1, position(test_name, ':') -1) AS image_name, - argMax(commit_sha, check_start_time) AS commit_sha + splitByChar(':', test_name)[1] AS image_name, + argMax(splitByChar(':', test_name)[2], check_start_time) AS tag FROM checks WHERE check_name == 'Push multi-arch images to Dockerhub' @@ -232,15 +232,12 @@ def enrich_images(changed_images: Dict[str, str]) -> None: "Found images for commits %s..%s:\n %s", commit_shas[0], commit_shas[-1], - "\n ".join(f"{im['image_name']}:{im['commit_sha']}" for im in result), + "\n ".join(f"{im['image_name']}:{im['tag']}" for im in result), ) for row in result: image_name = row["image_name"] - commit_sha = row["commit_sha"] - # As we only get the SHAs of merge commits from master, the PR number will be always 0 - tag = f"0-{commit_sha}" - changed_images[image_name] = tag + changed_images[image_name] = row["tag"] images_to_find_tags_for.remove(image_name) batch_count += 1